From 6e9f7ad46bcb341d37ac5d7804465dd82ef3b56e Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Tue, 28 Feb 2017 16:08:38 +0800 Subject: [PATCH 01/46] [BEAM-1517] Garbage collect user state in Flink Runner --- .../apache/beam/runners/core/DoFnRunners.java | 39 +++ .../beam/runners/core/StatefulDoFnRunner.java | 233 ++++++++++++++++ .../runners/core/StatefulDoFnRunnerTest.java | 255 ++++++++++++++++++ .../wrappers/streaming/DoFnOperator.java | 4 + 4 files changed, 531 insertions(+) create mode 100644 runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java create mode 100644 runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java 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 f3972aea545d..9455eea10008 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java @@ -19,9 +19,15 @@ import java.util.List; import org.apache.beam.runners.core.ExecutionContext.StepContext; +import org.apache.beam.runners.core.StatefulDoFnRunner.CleanupTimer; +import org.apache.beam.runners.core.StatefulDoFnRunner.StateCleaner; +import org.apache.beam.runners.core.StatefulDoFnRunner.StateInternalsStateCleaner; +import org.apache.beam.runners.core.StatefulDoFnRunner.TimeInternalsCleanupTimer; +import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; @@ -116,4 +122,37 @@ DoFnRunner, KV> lateDataDroppingRunner( stepContext.timerInternals(), droppedDueToLatenessAggregator); } + + /** + * Returns an implementation of {@link DoFnRunner} that handles + * late data dropping and garbage collection for stateful {@link DoFn DoFns}. + * + *

It registers a timer by TimeInternals, and clean all states by StateInternals. + */ + public static + DoFnRunner defaultStatefulDoFnRunner( + DoFn fn, + DoFnRunner doFnRunner, + StepContext stepContext, + AggregatorFactory aggregatorFactory, + WindowingStrategy windowingStrategy) { + Aggregator droppedDueToLateness = aggregatorFactory.createAggregatorForDoFn( + fn.getClass(), stepContext, StatefulDoFnRunner.DROPPED_DUE_TO_LATENESS_COUNTER, + Sum.ofLongs()); + + CleanupTimer cleanupTimer = + new TimeInternalsCleanupTimer(stepContext.timerInternals(), windowingStrategy); + + Coder windowCoder = (Coder) windowingStrategy.getWindowFn().windowCoder(); + StateCleaner stateCleaner = + new StateInternalsStateCleaner<>(fn, stepContext.stateInternals(), windowCoder); + + return new StatefulDoFnRunner<>( + doFnRunner, + windowingStrategy, + cleanupTimer, + stateCleaner, + droppedDueToLateness); + } + } 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 new file mode 100644 index 000000000000..154d8bcd2213 --- /dev/null +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java @@ -0,0 +1,233 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.core; + +import java.util.Map; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.Aggregator; +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; +import org.apache.beam.sdk.util.state.State; +import org.apache.beam.sdk.util.state.StateSpec; +import org.joda.time.Instant; + +/** + * A customized {@link DoFnRunner} that handles late data dropping and garbage collection for + * stateful {@link DoFn DoFns}. It registers a GC timer in {@link #processElement(WindowedValue)} + * and does cleanup in {@link #onTimer(String, BoundedWindow, Instant, TimeDomain)} + * + * @param the type of the {@link DoFn} (main) input elements + * @param the type of the {@link DoFn} (main) output elements + */ +public class StatefulDoFnRunner + implements DoFnRunner { + + public static final String GC_TIMER_ID = "__StatefulParDoGcTimerId"; + public static final String DROPPED_DUE_TO_LATENESS_COUNTER = "StatefulParDoDropped"; + + private final DoFnRunner doFnRunner; + private final WindowingStrategy windowingStrategy; + private final Aggregator droppedDueToLateness; + private final CleanupTimer cleanupTimer; + private final StateCleaner stateCleaner; + + public StatefulDoFnRunner( + DoFnRunner doFnRunner, + WindowingStrategy windowingStrategy, + CleanupTimer cleanupTimer, + StateCleaner stateCleaner, + Aggregator droppedDueToLateness) { + this.doFnRunner = doFnRunner; + this.windowingStrategy = windowingStrategy; + this.cleanupTimer = cleanupTimer; + this.stateCleaner = stateCleaner; + WindowFn windowFn = windowingStrategy.getWindowFn(); + rejectMergingWindowFn(windowFn); + this.droppedDueToLateness = droppedDueToLateness; + } + + private void rejectMergingWindowFn(WindowFn windowFn) { + if (!(windowFn instanceof NonMergingWindowFn)) { + throw new UnsupportedOperationException( + "MergingWindowFn is not supported for stateful DoFns, WindowFn is: " + + windowFn); + } + } + + @Override + public void startBundle() { + doFnRunner.startBundle(); + } + + @Override + public void processElement(WindowedValue compressedElem) { + + // StatefulDoFnRunner always observes windows, so we need to explode + for (WindowedValue value : compressedElem.explodeWindows()) { + + BoundedWindow window = value.getWindows().iterator().next(); + + if (!dropLateData(window)) { + cleanupTimer.setForWindow(window); + doFnRunner.processElement(value); + } + } + } + + private boolean dropLateData(BoundedWindow window) { + Instant gcTime = window.maxTimestamp().plus(windowingStrategy.getAllowedLateness()); + Instant inputWM = cleanupTimer.currentInputWatermarkTime(); + if (gcTime.isBefore(inputWM)) { + // The element is too late for this window. + droppedDueToLateness.addValue(1L); + WindowTracing.debug( + "StatefulDoFnRunner.processElement/onTimer: Dropping element for window:{} " + + "since too far behind inputWatermark:{}", window, inputWM); + return true; + } else { + return false; + } + } + + @Override + public void onTimer( + String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) { + boolean isEventTimer = timeDomain.equals(TimeDomain.EVENT_TIME); + Instant gcTime = window.maxTimestamp().plus(windowingStrategy.getAllowedLateness()); + if (isEventTimer && GC_TIMER_ID.equals(timerId) && gcTime.equals(timestamp)) { + stateCleaner.clearForWindow(window); + // There should invoke the onWindowExpiration of DoFn + } else { + if (isEventTimer || !dropLateData(window)) { + doFnRunner.onTimer(timerId, window, timestamp, timeDomain); + } + } + } + + @Override + public void finishBundle() { + doFnRunner.finishBundle(); + } + + /** + * A cleaner for deciding when to clean state of window. + * + *

A runner might either (a) already know that it always has a timer set + * for the expiration time or (b) not need a timer at all because it is + * a batch runner that discards state when it is done. + */ + public interface CleanupTimer { + + /** + * Return the current, local input watermark timestamp for this computation + * in the {@link TimeDomain#EVENT_TIME} time domain. + */ + Instant currentInputWatermarkTime(); + + /** + * Set the garbage collect time of the window to timer. + */ + void setForWindow(BoundedWindow window); + } + + /** + * A cleaner to clean all states of the window. + */ + public interface StateCleaner { + + void clearForWindow(W window); + } + + /** + * A {@link CleanupTimer} implemented by TimerInternals. + */ + public static class TimeInternalsCleanupTimer implements CleanupTimer { + + private final TimerInternals timerInternals; + private final WindowingStrategy windowingStrategy; + private final Coder windowCoder; + + public TimeInternalsCleanupTimer( + TimerInternals timerInternals, + WindowingStrategy windowingStrategy) { + this.windowingStrategy = windowingStrategy; + WindowFn windowFn = windowingStrategy.getWindowFn(); + windowCoder = (Coder) windowFn.windowCoder(); + this.timerInternals = timerInternals; + } + + @Override + public Instant currentInputWatermarkTime() { + return timerInternals.currentInputWatermarkTime(); + } + + @Override + public void setForWindow(BoundedWindow window) { + Instant gcTime = window.maxTimestamp().plus(windowingStrategy.getAllowedLateness()); + timerInternals.setTimer(StateNamespaces.window(windowCoder, window), + GC_TIMER_ID, gcTime, TimeDomain.EVENT_TIME); + } + + } + + /** + * A {@link StateCleaner} implemented by StateInternals. + */ + public static class StateInternalsStateCleaner + implements StateCleaner { + + private final DoFn fn; + private final DoFnSignature signature; + private final StateInternals stateInternals; + private final Coder windowCoder; + + public StateInternalsStateCleaner( + DoFn fn, + StateInternals stateInternals, + Coder windowCoder) { + this.fn = fn; + this.signature = DoFnSignatures.getSignature(fn.getClass()); + this.stateInternals = stateInternals; + this.windowCoder = windowCoder; + } + + @Override + public void clearForWindow(W window) { + for (Map.Entry entry : + signature.stateDeclarations().entrySet()) { + try { + StateSpec spec = (StateSpec) entry.getValue().field().get(fn); + State state = stateInternals.state(StateNamespaces.window(windowCoder, window), + StateTags.tagForSpec(entry.getKey(), (StateSpec) spec)); + state.clear(); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } + } + } + } + +} 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 new file mode 100644 index 000000000000..54ac77e147d1 --- /dev/null +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java @@ -0,0 +1,255 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.core; + +import static com.google.common.base.Preconditions.checkArgument; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.when; + +import com.google.common.base.MoreObjects; +import java.util.Collections; +import org.apache.beam.runners.core.BaseExecutionContext.StepContext; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Sum; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +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.TimeDomain; +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; +import org.joda.time.Instant; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +/** Tests for {@link StatefulDoFnRunnerTest}. */ +@RunWith(JUnit4.class) +public class StatefulDoFnRunnerTest { + + private static final long WINDOW_SIZE = 10; + private static final long ALLOWED_LATENESS = 1; + + private static final WindowingStrategy WINDOWING_STRATEGY = + WindowingStrategy + .of(FixedWindows.of(Duration.millis(WINDOW_SIZE))) + .withAllowedLateness(Duration.millis(ALLOWED_LATENESS)); + + private static final IntervalWindow WINDOW_1 = + new IntervalWindow(new Instant(0), new Instant(10)); + + private static final IntervalWindow WINDOW_2 = + new IntervalWindow(new Instant(10), new Instant(20)); + + @Mock StepContext mockStepContext; + + private InMemoryLongSumAggregator droppedDueToLateness; + private AggregatorFactory aggregatorFactory; + private InMemoryStateInternals stateInternals; + private InMemoryTimerInternals timerInternals; + + private static StateNamespace windowNamespace(IntervalWindow window) { + return StateNamespaces.window( + (Coder) WINDOWING_STRATEGY.getWindowFn().windowCoder(), window); + } + + @Before + public void setup() { + MockitoAnnotations.initMocks(this); + when(mockStepContext.timerInternals()).thenReturn(timerInternals); + droppedDueToLateness = new InMemoryLongSumAggregator("droppedDueToLateness"); + + aggregatorFactory = new AggregatorFactory() { + @Override + public Aggregator createAggregatorForDoFn( + Class fnClass, ExecutionContext.StepContext stepContext, String aggregatorName, + Combine.CombineFn combine) { + return (Aggregator) droppedDueToLateness; + } + }; + + stateInternals = new InMemoryStateInternals<>("hello"); + timerInternals = new InMemoryTimerInternals(); + + when(mockStepContext.stateInternals()).thenReturn((StateInternals) stateInternals); + when(mockStepContext.timerInternals()).thenReturn(timerInternals); + } + + @Test + public void testLateDropping() throws Exception { + + timerInternals.advanceInputWatermark(new Instant(BoundedWindow.TIMESTAMP_MAX_VALUE)); + timerInternals.advanceOutputWatermark(new Instant(BoundedWindow.TIMESTAMP_MAX_VALUE)); + + DoFn, Integer> fn = new MyDoFn(); + + DoFnRunner, Integer> runner = DoFnRunners.defaultStatefulDoFnRunner( + fn, getDoFnRunner(fn), mockStepContext, aggregatorFactory, WINDOWING_STRATEGY); + + runner.startBundle(); + + IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(0L + WINDOW_SIZE)); + Instant timestamp = new Instant(0); + + runner.processElement( + WindowedValue.of(KV.of("hello", 1), timestamp, window, PaneInfo.NO_FIRING)); + assertEquals(1L, droppedDueToLateness.sum); + + runner.onTimer("processTimer", window, timestamp, TimeDomain.PROCESSING_TIME); + assertEquals(2L, droppedDueToLateness.sum); + + runner.onTimer("synchronizedProcessTimer", window, timestamp, + TimeDomain.SYNCHRONIZED_PROCESSING_TIME); + assertEquals(3L, droppedDueToLateness.sum); + + runner.finishBundle(); + } + + @Test + public void testGarbageCollect() throws Exception { + timerInternals.advanceInputWatermark(new Instant(1L)); + + MyDoFn fn = new MyDoFn(); + StateTag> stateTag = StateTags.tagForSpec(fn.stateId, fn.intState); + + DoFnRunner, Integer> runner = DoFnRunners.defaultStatefulDoFnRunner( + fn, getDoFnRunner(fn), mockStepContext, aggregatorFactory, WINDOWING_STRATEGY); + + Instant elementTime = new Instant(1); + + // first element, key is hello, WINDOW_1 + runner.processElement( + WindowedValue.of(KV.of("hello", 1), elementTime, WINDOW_1, PaneInfo.NO_FIRING)); + + assertEquals( + 1, (int) stateInternals.state(windowNamespace(WINDOW_1), stateTag).read()); + + // second element, key is hello, WINDOW_2 + runner.processElement( + WindowedValue.of( + KV.of("hello", 1), elementTime.plus(WINDOW_SIZE), WINDOW_2, PaneInfo.NO_FIRING)); + + runner.processElement( + WindowedValue.of( + KV.of("hello", 1), elementTime.plus(WINDOW_SIZE), WINDOW_2, PaneInfo.NO_FIRING)); + + assertEquals( + 2, (int) stateInternals.state(windowNamespace(WINDOW_2), stateTag).read()); + + // advance watermark past end of WINDOW_1 + allowed lateness + advanceInputWatermark( + timerInternals, WINDOW_1.maxTimestamp().plus(ALLOWED_LATENESS + 1), runner); + assertTrue( + stateInternals.isEmptyForTesting( + stateInternals.state(windowNamespace(WINDOW_1), stateTag))); + + assertEquals( + 2, (int) stateInternals.state(windowNamespace(WINDOW_2), stateTag).read()); + + // advance watermark past end of WINDOW_2 + allowed lateness + advanceInputWatermark( + timerInternals, WINDOW_2.maxTimestamp().plus(ALLOWED_LATENESS + 1), runner); + assertTrue( + stateInternals.isEmptyForTesting( + stateInternals.state(windowNamespace(WINDOW_2), stateTag))); + } + + private DoFnRunner, Integer> getDoFnRunner( + DoFn, Integer> fn) { + return new SimpleDoFnRunner<>( + null, + fn, + NullSideInputReader.empty(), + null, + null, + Collections.>emptyList(), + mockStepContext, + null, + WINDOWING_STRATEGY); + } + + private static void advanceInputWatermark( + InMemoryTimerInternals timerInternals, + Instant newInputWatermark, + DoFnRunner toTrigger) throws Exception { + timerInternals.advanceInputWatermark(newInputWatermark); + TimerInternals.TimerData timer; + while ((timer = timerInternals.removeNextEventTimer()) != null) { + StateNamespace namespace = timer.getNamespace(); + checkArgument(namespace instanceof StateNamespaces.WindowNamespace); + BoundedWindow window = ((StateNamespaces.WindowNamespace) namespace).getWindow(); + toTrigger.onTimer(timer.getTimerId(), window, timer.getTimestamp(), timer.getDomain()); + } + } + + private static class MyDoFn extends DoFn, Integer> { + + public final String stateId = "foo"; + + @StateId(stateId) + public final StateSpec> intState = + StateSpecs.value(VarIntCoder.of()); + + @ProcessElement + public void processElement( + ProcessContext c, @StateId(stateId) ValueState state) { + Integer currentValue = MoreObjects.firstNonNull(state.read(), 0); + state.write(currentValue + 1); + } + }; + + private static class InMemoryLongSumAggregator implements Aggregator { + private final String name; + private long sum = 0; + + public InMemoryLongSumAggregator(String name) { + this.name = name; + } + + @Override + public void addValue(Long value) { + sum += value; + } + + @Override + public String getName() { + return name; + } + + @Override + public Combine.CombineFn getCombineFn() { + return Sum.ofLongs(); + } + } + +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index 29b6fbcf3b90..c4622baf97de 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -291,6 +291,10 @@ public Aggregator createAggregatorFor stepContext, windowingStrategy, ((GroupAlsoByWindowViaWindowSetNewDoFn) doFn).getDroppedDueToLatenessAggregator()); + } else if (keyCoder != null) { + // It is a stateful DoFn + doFnRunner = DoFnRunners.defaultStatefulDoFnRunner( + doFn, doFnRunner, stepContext, aggregatorFactory, windowingStrategy); } pushbackDoFnRunner = From d390406e27112faed31233d7daef1f650a31cd0f Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 28 Feb 2017 15:51:24 -0800 Subject: [PATCH 02/46] Inline rather than reference FunctionSpecs. --- .../src/main/proto/beam_runner_api.proto | 39 ++++++++----------- .../beam/sdk/util/WindowingStrategies.java | 18 ++------- 2 files changed, 20 insertions(+), 37 deletions(-) diff --git a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto index 58532b215b31..44ead56ca5a1 100644 --- a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto +++ b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto @@ -47,10 +47,6 @@ message Components { // (Required) A map from pipeline-scoped id to Environment. map environments = 5; - - // (Required) A map from pipeline-scoped id to FunctionSpec, - // a record for a particular user-defined function. - map function_specs = 6; } // A disjoint union of all the things that may contain references @@ -207,8 +203,8 @@ message PCollection { // The payload for the primitive ParDo transform. message ParDoPayload { - // (Required) The pipeline-scoped id of the FunctionSpec for the DoFn. - string fn_id = 1; + // (Required) The FunctionSpec of the DoFn. + FunctionSpec do_fn = 1; // (Required) Additional pieces of context the DoFn may require that // are not otherwise represented in the payload. @@ -266,9 +262,8 @@ enum IsBounded { // The payload for the primitive Read transform. message ReadPayload { - // (Required) The pipeline-scoped id of the FunctionSpec of the source for - // this Read. - string source_id = 1; + // (Required) The FunctionSpec of the source for this Read. + FunctionSpec source = 1; // (Required) Whether the source is bounded or unbounded IsBounded is_bounded = 2; @@ -279,15 +274,15 @@ message ReadPayload { // The payload for the WindowInto transform. message WindowIntoPayload { - // (Required) The pipeline-scoped id for the FunctionSpec of the WindowFn. - string fn_id = 1; + // (Required) The FunctionSpec of the WindowFn. + FunctionSpec window_fn = 1; } // The payload for the special-but-not-primitive Combine transform. message CombinePayload { - // (Required) The pipeline-scoped id of the FunctionSpec for the CombineFn. - string fn_id = 1; + // (Required) The FunctionSpec of the CombineFn. + FunctionSpec combine_fn = 1; // (Required) A reference to the Coder to use for accumulators of the CombineFn string accumulator_coder_id = 2; @@ -325,10 +320,10 @@ message Coder { // TODO: consider inlining field on PCollection message WindowingStrategy { - // (Required) The pipeline-scoped id for the FunctionSpec of the UDF that - // assigns windows, merges windows, and shifts timestamps before they are + // (Required) The FunctionSpec of the UDF that assigns windows, + // merges windows, and shifts timestamps before they are // combined according to the OutputTime. - string fn_id = 1; + FunctionSpec window_fn = 1; // (Required) Whether or not the window fn is merging. // @@ -584,20 +579,20 @@ message SideInput { // URN) UrnWithParameter access_pattern = 1; - // (Required) The pipeline-scoped id for the FunctionSpec of the UDF that - // adapts a particular access_pattern to a user-facing view type. + // (Required) The FunctionSpec of the UDF that adapts a particular + // access_pattern to a user-facing view type. // // For example, View.asSingleton() may include a `view_fn` that adapts a // specially-designed multimap to a single value per window. - string view_fn_id = 2; + FunctionSpec view_fn = 2; - // (Required) The pipeline-scoped id for the FunctionSpec of the UDF that - // maps a main input window to a side input window. + // (Required) The FunctionSpec of the UDF that maps a main input window + // to a side input window. // // For example, when the main input is in fixed windows of one hour, this // can specify that the side input should be accessed according to the day // in which that hour falls. - string window_mapping_fn_id = 3; + FunctionSpec window_mapping_fn = 3; } // An environment for executing UDFs. Generally an SDK container URL, but diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategies.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategies.java index 3047da13c286..7bc581cbc84a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategies.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategies.java @@ -195,10 +195,6 @@ public static RunnerApi.MessageWithComponents toProto(WindowFn windowFn) public static RunnerApi.MessageWithComponents toProto(WindowingStrategy windowingStrategy) throws IOException { - // TODO: have an inverted components to find the id for a thing already - // in the components - String windowFnId = UUID.randomUUID().toString(); - RunnerApi.MessageWithComponents windowFnWithComponents = toProto(windowingStrategy.getWindowFn()); @@ -209,16 +205,11 @@ public static RunnerApi.MessageWithComponents toProto(WindowingStrategy wi .setClosingBehavior(toProto(windowingStrategy.getClosingBehavior())) .setAllowedLateness(windowingStrategy.getAllowedLateness().getMillis()) .setTrigger(Triggers.toProto(windowingStrategy.getTrigger())) - .setFnId(windowFnId); + .setWindowFn(windowFnWithComponents.getFunctionSpec()); return RunnerApi.MessageWithComponents.newBuilder() .setWindowingStrategy(windowingStrategyProto) - .setComponents( - windowFnWithComponents - .getComponents() - .toBuilder() - .putFunctionSpecs(windowFnId, windowFnWithComponents.getFunctionSpec())) - .build(); + .setComponents(windowFnWithComponents.getComponents()).build(); } /** @@ -246,10 +237,7 @@ public static RunnerApi.MessageWithComponents toProto(WindowingStrategy wi RunnerApi.WindowingStrategy proto, RunnerApi.Components components) throws InvalidProtocolBufferException { - FunctionSpec windowFnSpec = - components - .getFunctionSpecsMap() - .get(proto.getFnId()); + FunctionSpec windowFnSpec = proto.getWindowFn(); checkArgument( windowFnSpec.getSpec().getUrn().equals(CUSTOM_WINDOWFN_URN), From 52e2d3a77096460ae4a10ac977b4897a1eecf3a1 Mon Sep 17 00:00:00 2001 From: Aviem Zur Date: Sun, 26 Feb 2017 22:39:28 +0200 Subject: [PATCH 03/46] [BEAM-351] Add DisplayData to KafkaIO Changes after review. --- sdks/java/io/kafka/pom.xml | 7 +++ .../org/apache/beam/sdk/io/kafka/KafkaIO.java | 49 +++++++++++++++++ .../apache/beam/sdk/io/kafka/KafkaIOTest.java | 53 ++++++++++++++++++- 3 files changed, 108 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/kafka/pom.xml b/sdks/java/io/kafka/pom.xml index 6935f1ec6c21..d5ffe637edf3 100644 --- a/sdks/java/io/kafka/pom.xml +++ b/sdks/java/io/kafka/pom.xml @@ -148,6 +148,13 @@ test + + org.apache.beam + beam-sdks-java-core + tests + test + + org.hamcrest hamcrest-all 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 5fd34b9e8361..890fb2b2633c 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -44,6 +44,7 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.Random; +import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -63,12 +64,14 @@ import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; import org.apache.beam.sdk.io.kafka.KafkaCheckpointMark.PartitionMark; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.ExposedByteArrayInputStream; import org.apache.beam.sdk.values.KV; @@ -500,6 +503,27 @@ public Consumer apply(Map config) { return new KafkaConsumer<>(config); } }; + + @SuppressWarnings("unchecked") + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + List topics = getTopics(); + List topicPartitions = getTopicPartitions(); + if (topics.size() > 0) { + builder.add(DisplayData.item("topics", Joiner.on(",").join(topics)).withLabel("Topic/s")); + } else if (topicPartitions.size() > 0) { + builder.add(DisplayData.item("topicPartitions", Joiner.on(",").join(topicPartitions)) + .withLabel("Topic Partition/s")); + } + Set ignoredConsumerPropertiesKeys = IGNORED_CONSUMER_PROPERTIES.keySet(); + for (Map.Entry conf : getConsumerConfig().entrySet()) { + String key = conf.getKey(); + if (!ignoredConsumerPropertiesKeys.contains(key)) { + builder.add(DisplayData.item(key, ValueProvider.StaticValueProvider.of(conf.getValue()))); + } + } + } } /** @@ -527,6 +551,12 @@ public void processElement(ProcessContext ctx) { } })); } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + read.populateDisplayData(builder); + } } //////////////////////////////////////////////////////////////////////////////////////////////// @@ -1222,6 +1252,19 @@ public void validate(PCollection> input) { configForKeySerializer(), "Reserved for internal serializer", configForValueSerializer(), "Reserved for internal serializer" ); + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder.addIfNotNull(DisplayData.item("topic", getTopic()).withLabel("Topic")); + Set ignoredProducerPropertiesKeys = IGNORED_PRODUCER_PROPERTIES.keySet(); + for (Map.Entry conf : getProducerConfig().entrySet()) { + String key = conf.getKey(); + if (!ignoredProducerPropertiesKeys.contains(key)) { + builder.add(DisplayData.item(key, ValueProvider.StaticValueProvider.of(conf.getValue()))); + } + } + } } /** @@ -1248,6 +1291,12 @@ public KV apply(V element) { .setCoder(KvCoder.of(new NullOnlyCoder(), kvWriteTransform.getValueCoder())) .apply(kvWriteTransform); } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + kvWriteTransform.populateDisplayData(builder); + } } private static class NullOnlyCoder extends AtomicCoder { diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java index 9d7c27b14967..1897127ac296 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.kafka; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; @@ -24,6 +25,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -58,6 +60,7 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; @@ -223,7 +226,7 @@ private static KafkaIO.Read mkKafkaReadTransform( List topics = ImmutableList.of("topic_a", "topic_b"); KafkaIO.Read reader = KafkaIO.read() - .withBootstrapServers("none") + .withBootstrapServers("myServer1:9092,myServer2:9092") .withTopics(topics) .withConsumerFactoryFn(new ConsumerFactoryFn( topics, 10, numElements, OffsetResetStrategy.EARLIEST)) // 20 partitions @@ -619,6 +622,54 @@ public void testSinkWithSendErrors() throws Throwable { } } + @Test + public void testSourceDisplayData() { + KafkaIO.Read read = mkKafkaReadTransform(10, null); + + DisplayData displayData = DisplayData.from(read); + + assertThat(displayData, hasDisplayItem("topics", "topic_a,topic_b")); + assertThat(displayData, hasDisplayItem("enable.auto.commit", false)); + assertThat(displayData, hasDisplayItem("bootstrap.servers", "myServer1:9092,myServer2:9092")); + assertThat(displayData, hasDisplayItem("auto.offset.reset", "latest")); + assertThat(displayData, hasDisplayItem("receive.buffer.bytes", 524288)); + } + + @Test + public void testSourceWithExplicitPartitionsDisplayData() { + KafkaIO.Read read = KafkaIO.read() + .withBootstrapServers("myServer1:9092,myServer2:9092") + .withTopicPartitions(ImmutableList.of(new TopicPartition("test", 5), + new TopicPartition("test", 6))) + .withConsumerFactoryFn(new ConsumerFactoryFn( + Lists.newArrayList("test"), 10, 10, OffsetResetStrategy.EARLIEST)) // 10 partitions + .withKeyCoder(ByteArrayCoder.of()) + .withValueCoder(BigEndianLongCoder.of()); + + DisplayData displayData = DisplayData.from(read); + + assertThat(displayData, hasDisplayItem("topicPartitions", "test-5,test-6")); + assertThat(displayData, hasDisplayItem("enable.auto.commit", false)); + assertThat(displayData, hasDisplayItem("bootstrap.servers", "myServer1:9092,myServer2:9092")); + assertThat(displayData, hasDisplayItem("auto.offset.reset", "latest")); + assertThat(displayData, hasDisplayItem("receive.buffer.bytes", 524288)); + } + + @Test + public void testSinkDisplayData() { + KafkaIO.Write write = KafkaIO.write() + .withBootstrapServers("myServerA:9092,myServerB:9092") + .withTopic("myTopic") + .withValueCoder(BigEndianLongCoder.of()) + .withProducerFactoryFn(new ProducerFactoryFn()); + + DisplayData displayData = DisplayData.from(write); + + assertThat(displayData, hasDisplayItem("topic", "myTopic")); + assertThat(displayData, hasDisplayItem("bootstrap.servers", "myServerA:9092,myServerB:9092")); + assertThat(displayData, hasDisplayItem("retries", 3)); + } + private static void verifyProducerRecords(String topic, int numElements, boolean keyIsAbsent) { // verify that appropriate messages are written to kafka From 4256801a6c4a0e7c15232c5fae22f7eef5fdf914 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Sun, 12 Feb 2017 06:52:45 +0100 Subject: [PATCH 04/46] [BEAM-1297] Update maven shade plugin, fix typo and remove unneeded version [BEAM-1297] Enable tiered compilation to make the JVM startup times faster This makes the build faster because every time maven forks it does not necessarily reuse the JVM instance. [BEAM-1297] Enable parallel builds in travis (1C = 1 per core) --- .jenkins/common_job_properties.groovy | 4 ++++ .travis.yml | 7 ++++--- pom.xml | 5 ++--- sdks/java/javadoc/pom.xml | 1 - 4 files changed, 10 insertions(+), 7 deletions(-) diff --git a/.jenkins/common_job_properties.groovy b/.jenkins/common_job_properties.groovy index 19cf47100a7c..bcac19e6f33b 100644 --- a/.jenkins/common_job_properties.groovy +++ b/.jenkins/common_job_properties.groovy @@ -166,6 +166,10 @@ class common_job_properties { context.mavenInstallation('Maven 3.3.3') context.mavenOpts('-Dorg.slf4j.simpleLogger.showDateTime=true') context.mavenOpts('-Dorg.slf4j.simpleLogger.dateTimeFormat=yyyy-MM-dd\\\'T\\\'HH:mm:ss.SSS') + // The -XX:+TieredCompilation -XX:TieredStopAtLevel=1 JVM options enable + // tiered compilation to make the JVM startup times faster during the tests. + context.mavenOpts('-XX:+TieredCompilation') + context.mavenOpts('-XX:TieredStopAtLevel=1') context.rootPOM('pom.xml') // Use a repository local to the workspace for better isolation of jobs. context.localRepository(LocalRepositoryLocation.LOCAL_TO_WORKSPACE) diff --git a/.travis.yml b/.travis.yml index a392f7dc7c43..c89643135303 100644 --- a/.travis.yml +++ b/.travis.yml @@ -58,9 +58,10 @@ matrix: - os: linux env: TEST_PYTHON="1" - before_install: - - echo 'MAVEN_OPTS="$MAVEN_OPTS -Xmx1024m -XX:MaxPermSize=512m -XX:+BytecodeVerificationLocal"' >> ~/.mavenrc + # The -XX:+TieredCompilation -XX:TieredStopAtLevel=1 JVM options enable + # tiered compilation to make the JVM startup times faster during the tests. + - echo 'MAVEN_OPTS="$MAVEN_OPTS -Xmx1024m -XX:MaxPermSize=512m -XX:+TieredCompilation -XX:TieredStopAtLevel=1 -XX:+BytecodeVerificationLocal"' >> ~/.mavenrc - echo $'MAVEN_OPTS="$MAVEN_OPTS -Dorg.slf4j.simpleLogger.showDateTime=true -Dorg.slf4j.simpleLogger.dateTimeFormat=yyyy-MM-dd\'T\'HH:mm:ss.SSS"' >> ~/.mavenrc - cat ~/.mavenrc - if [ "$TRAVIS_OS_NAME" == "osx" ]; then export JAVA_HOME=$(/usr/libexec/java_home); fi @@ -80,7 +81,7 @@ install: script: - if [ "$TEST_PYTHON" ]; then travis_retry $TOX_HOME/tox -e $TOX_ENV -c sdks/python/tox.ini; fi - - if [ ! "$TEST_PYTHON" ]; then travis_retry mvn --batch-mode --update-snapshots --no-snapshot-updates $MAVEN_OVERRIDE install && travis_retry bash -ex .travis/test_wordcount.sh; fi + - if [ ! "$TEST_PYTHON" ]; then travis_retry mvn --batch-mode --update-snapshots --no-snapshot-updates --threads 1C $MAVEN_OVERRIDE install && travis_retry bash -ex .travis/test_wordcount.sh; fi cache: directories: diff --git a/pom.xml b/pom.xml index 65f67236e45f..a37f1af4d4d0 100644 --- a/pom.xml +++ b/pom.xml @@ -1310,7 +1310,7 @@ org.apache.maven.plugins maven-shade-plugin - 2.4.3 + 3.0.0 @@ -1448,7 +1448,7 @@ [1.7,) - + [3.2,) @@ -1483,7 +1483,6 @@ - 3.2 diff --git a/sdks/java/javadoc/pom.xml b/sdks/java/javadoc/pom.xml index 145dcf0e4f91..243dae5d448d 100644 --- a/sdks/java/javadoc/pom.xml +++ b/sdks/java/javadoc/pom.xml @@ -232,7 +232,6 @@ org.apache.maven.plugins maven-enforcer-plugin - 1.3.1 enforce From 44624c382ac5ff062191d41df1dcd008839352e0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Sat, 25 Feb 2017 05:20:58 +0100 Subject: [PATCH 05/46] [BEAM-111] Move WritableCoder to hadoop-common --- runners/spark/pom.xml | 6 + .../spark/coders/NullWritableCoder.java | 76 ----------- .../runners/spark/coders/WritableCoder.java | 122 ------------------ .../spark/coders/WritableCoderTest.java | 45 ------- .../hadoop/HadoopFileFormatPipelineTest.java | 2 +- sdks/java/io/hadoop-common/pom.xml | 10 ++ .../beam/sdk/io/hadoop}/WritableCoder.java | 2 +- .../sdk/io/hadoop}/WritableCoderTest.java | 2 +- sdks/java/io/hdfs/pom.xml | 5 - .../beam/sdk/io/hdfs/HDFSFileSource.java | 1 + 10 files changed, 20 insertions(+), 251 deletions(-) delete mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java delete mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java delete mode 100644 runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java rename sdks/java/io/{hdfs/src/main/java/org/apache/beam/sdk/io/hdfs => hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop}/WritableCoder.java (98%) rename sdks/java/io/{hdfs/src/test/java/org/apache/beam/sdk/io/hdfs => hadoop-common/src/test/java/org/apache/beam/sdk/io/hadoop}/WritableCoderTest.java (97%) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 409fc277ffd6..8c35178c164a 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -305,6 +305,12 @@ + + org.apache.beam + beam-sdks-java-io-hadoop-common + test + + org.mockito mockito-all diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java deleted file mode 100644 index ebbab1a235b9..000000000000 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.beam.runners.spark.coders; - -import com.fasterxml.jackson.annotation.JsonCreator; -import java.io.InputStream; -import java.io.OutputStream; -import org.apache.beam.sdk.coders.Coder; -import org.apache.hadoop.io.NullWritable; - -/** - * Simple writable coder for Null. - */ -public final class NullWritableCoder extends WritableCoder { - private static final long serialVersionUID = 1L; - - @JsonCreator - public static NullWritableCoder of() { - return INSTANCE; - } - - private static final NullWritableCoder INSTANCE = new NullWritableCoder(); - - private NullWritableCoder() { - super(NullWritable.class); - } - - @Override - public void encode(NullWritable value, OutputStream outStream, Context context) { - // nothing to write - } - - @Override - public NullWritable decode(InputStream inStream, Context context) { - return NullWritable.get(); - } - - @Override - public boolean consistentWithEquals() { - return true; - } - - /** - * Returns true since registerByteSizeObserver() runs in constant time. - */ - @Override - public boolean isRegisterByteSizeObserverCheap(NullWritable value, Context context) { - return true; - } - - @Override - protected long getEncodedElementByteSize(NullWritable value, Context context) { - return 0; - } - - @Override - public void verifyDeterministic() throws Coder.NonDeterministicException { - // NullWritableCoder is deterministic - } -} diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java deleted file mode 100644 index 40c26275934a..000000000000 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java +++ /dev/null @@ -1,122 +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.coders; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.lang.reflect.InvocationTargetException; -import java.util.List; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.StandardCoder; -import org.apache.beam.sdk.util.CloudObject; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Writable; - -/** - * A {@code WritableCoder} is a {@link Coder} for a Java class that implements {@link Writable}. - * - *

To use, specify the coder type on a PCollection: - *

- * {@code
- *   PCollection records =
- *       foo.apply(...).setCoder(WritableCoder.of(MyRecord.class));
- * }
- * 
- * - * @param the type of elements handled by this coder - */ -public class WritableCoder extends StandardCoder { - private static final long serialVersionUID = 0L; - - /** - * Returns a {@code WritableCoder} instance for the provided element class. - * @param the element type - * @param clazz the element class - * @return a {@code WritableCoder} instance for the provided element class - */ - public static WritableCoder of(Class clazz) { - if (clazz.equals(NullWritable.class)) { - @SuppressWarnings("unchecked") - WritableCoder result = (WritableCoder) NullWritableCoder.of(); - return result; - } - return new WritableCoder<>(clazz); - } - - @JsonCreator - @SuppressWarnings("unchecked") - public static WritableCoder of(@JsonProperty("type") String classType) - throws ClassNotFoundException { - Class clazz = Class.forName(classType); - if (!Writable.class.isAssignableFrom(clazz)) { - throw new ClassNotFoundException( - "Class " + classType + " does not implement Writable"); - } - return of((Class) clazz); - } - - private final Class type; - - public WritableCoder(Class type) { - this.type = type; - } - - @Override - public void encode(T value, OutputStream outStream, Context context) throws IOException { - value.write(new DataOutputStream(outStream)); - } - - @Override - public T decode(InputStream inStream, Context context) throws IOException { - try { - T t = type.getConstructor().newInstance(); - t.readFields(new DataInputStream(inStream)); - return t; - } catch (NoSuchMethodException | InstantiationException | IllegalAccessException e) { - throw new CoderException("unable to deserialize record", e); - } catch (InvocationTargetException ite) { - throw new CoderException("unable to deserialize record", ite.getCause()); - } - } - - @Override - public List> getCoderArguments() { - return null; - } - - @Override - protected CloudObject initializeCloudObject() { - CloudObject result = CloudObject.forClass(getClass()); - result.put("type", type.getName()); - return result; - } - - @Override - public void verifyDeterministic() throws Coder.NonDeterministicException { - throw new NonDeterministicException(this, - "Hadoop Writable may be non-deterministic."); - } - -} diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java deleted file mode 100644 index 538fd97a8bb0..000000000000 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.beam.runners.spark.coders; - -import org.apache.beam.sdk.testing.CoderProperties; -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.NullWritable; -import org.junit.Test; - -/** - * Tests for WritableCoder. - */ -public class WritableCoderTest { - - @Test - public void testIntWritableEncoding() throws Exception { - IntWritable value = new IntWritable(42); - WritableCoder coder = WritableCoder.of(IntWritable.class); - - CoderProperties.coderDecodeEncodeEqual(coder, value); - } - - @Test - public void testNullWritableEncoding() throws Exception { - WritableCoder coder = WritableCoder.of(NullWritable.class); - - CoderProperties.coderDecodeEncodeEqual(coder, NullWritable.get()); - } -} diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java index a5072d63d2b6..48b54339f8b1 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java @@ -23,9 +23,9 @@ import java.io.File; import java.io.IOException; import org.apache.beam.runners.spark.PipelineRule; -import org.apache.beam.runners.spark.coders.WritableCoder; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.io.hadoop.WritableCoder; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.hadoop.conf.Configuration; diff --git a/sdks/java/io/hadoop-common/pom.xml b/sdks/java/io/hadoop-common/pom.xml index 13e159c6f9d0..fcd984fb4d57 100644 --- a/sdks/java/io/hadoop-common/pom.xml +++ b/sdks/java/io/hadoop-common/pom.xml @@ -31,6 +31,16 @@ Library to add shared Hadoop classes among Beam IOs. + + org.apache.beam + beam-sdks-java-core + + + + com.fasterxml.jackson.core + jackson-annotations + + org.apache.hadoop hadoop-client diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/WritableCoder.java b/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/WritableCoder.java similarity index 98% rename from sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/WritableCoder.java rename to sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/WritableCoder.java index d958cda88923..0ba367dd77da 100644 --- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/WritableCoder.java +++ b/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/WritableCoder.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.io.hdfs; +package org.apache.beam.sdk.io.hadoop; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/WritableCoderTest.java b/sdks/java/io/hadoop-common/src/test/java/org/apache/beam/sdk/io/hadoop/WritableCoderTest.java similarity index 97% rename from sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/WritableCoderTest.java rename to sdks/java/io/hadoop-common/src/test/java/org/apache/beam/sdk/io/hadoop/WritableCoderTest.java index e78f850c6635..8127773e03e6 100644 --- a/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/WritableCoderTest.java +++ b/sdks/java/io/hadoop-common/src/test/java/org/apache/beam/sdk/io/hadoop/WritableCoderTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.io.hdfs; +package org.apache.beam.sdk.io.hadoop; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.hadoop.io.IntWritable; diff --git a/sdks/java/io/hdfs/pom.xml b/sdks/java/io/hdfs/pom.xml index 1212b0ef2449..f3a1a27bb9b9 100644 --- a/sdks/java/io/hdfs/pom.xml +++ b/sdks/java/io/hdfs/pom.xml @@ -93,11 +93,6 @@ beam-sdks-java-io-hadoop-common - - com.fasterxml.jackson.core - jackson-annotations - - com.google.auto.service auto-service diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java index 2a731fb12004..0e3146fc2961 100644 --- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java @@ -48,6 +48,7 @@ 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; From 3408f6049ba3692f9edbbeead75626125954d4b6 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 23 Feb 2017 17:32:01 -0800 Subject: [PATCH 06/46] Remove PipelineRunner#apply All existing Pipeline Runners that use the Java SDK modify Pipeline graphs with the Pipeline Surgery APIs. Apply is now superflous. Add an AssertionCountingVisitor to enable TestRunners to track the number of assertions in the Pipeline. --- .../beam/runners/apex/TestApexRunner.java | 10 --- .../beam/runners/flink/TestFlinkRunner.java | 9 --- .../dataflow/testing/TestDataflowRunner.java | 17 +---- .../testing/TestDataflowRunnerTest.java | 3 +- runners/spark/pom.xml | 4 + .../beam/runners/spark/TestSparkRunner.java | 76 +++++++++++-------- .../runners/spark/ForceStreamingTest.java | 2 + .../java/org/apache/beam/sdk/Pipeline.java | 4 +- .../beam/sdk/runners/PipelineRunner.java | 14 ---- .../org/apache/beam/sdk/testing/PAssert.java | 64 ++++++++++++++++ .../apache/beam/sdk/testing/PAssertTest.java | 30 ++++++++ 11 files changed, 154 insertions(+), 79 deletions(-) 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 e447e3718bc7..a64ac549ec70 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 @@ -18,14 +18,10 @@ package org.apache.beam.runners.apex; import java.io.IOException; - import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.runners.PipelineRunner; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.values.PInput; -import org.apache.beam.sdk.values.POutput; import org.joda.time.Duration; /** @@ -48,12 +44,6 @@ public static TestApexRunner fromOptions(PipelineOptions options) { return new TestApexRunner(apexOptions); } - @Override - public - OutputT apply(PTransform transform, InputT input) { - return delegate.apply(transform, input); - } - @Override public ApexRunnerResult run(Pipeline pipeline) { ApexRunnerResult result = delegate.run(pipeline); diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java index 30a94a19b8b3..ef56b55dc9bd 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java @@ -24,10 +24,7 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.runners.PipelineRunner; -import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.values.PInput; -import org.apache.beam.sdk.values.POutput; /** * Test Flink runner. @@ -55,12 +52,6 @@ public static TestFlinkRunner create(boolean streaming) { return TestFlinkRunner.fromOptions(flinkOptions); } - @Override - public - OutputT apply(PTransform transform, InputT input) { - return delegate.apply(transform, input); - } - @Override public PipelineResult run(Pipeline pipeline) { try { 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 056444877734..53156718331d 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 @@ -46,9 +46,6 @@ import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.values.PInput; -import org.apache.beam.sdk.values.POutput; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -102,6 +99,7 @@ public DataflowPipelineJob run(Pipeline pipeline) { } DataflowPipelineJob run(Pipeline pipeline, DataflowRunner runner) { + updatePAssertCount(pipeline); TestPipelineOptions testPipelineOptions = pipeline.getOptions().as(TestPipelineOptions.class); final DataflowPipelineJob job; @@ -183,16 +181,9 @@ public Optional call() throws Exception { return job; } - @Override - public OutputT apply( - PTransform transform, InputT input) { - if (transform instanceof PAssert.OneSideInputAssert - || transform instanceof PAssert.GroupThenAssert - || transform instanceof PAssert.GroupThenAssertForSingleton) { - expectedNumberOfAssertions += 1; - } - - return runner.apply(transform, input); + @VisibleForTesting + void updatePAssertCount(Pipeline pipeline) { + expectedNumberOfAssertions = PAssert.countAsserts(pipeline); } /** 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 da5630b487b2..1e906d2f44f3 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 @@ -378,7 +378,8 @@ public void testCheckingForSuccessSkipsNonTentativeMetrics() throws Exception { PCollection pc = p.apply(Create.of(1, 2, 3)); PAssert.that(pc).containsInAnyOrder(1, 2, 3); - TestDataflowRunner runner = (TestDataflowRunner) p.getRunner(); + TestDataflowRunner runner = TestDataflowRunner.fromOptions(options); + runner.updatePAssertCount(p); doReturn(State.RUNNING).when(job).getState(); JobMetrics metrics = buildJobMetrics( generateMockMetrics(true /* success */, false /* tentative */)); diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 8c35178c164a..44f20ccd0b9c 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -224,6 +224,10 @@ + + org.apache.beam + beam-runners-core-construction-java + org.apache.beam beam-runners-core-java 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 035da0032adf..16ddc9e2d372 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 @@ -22,9 +22,14 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.is; +import com.google.common.annotations.VisibleForTesting; import java.io.File; import java.io.IOException; +import java.util.List; +import java.util.Map; import org.apache.beam.runners.core.UnboundedReadFromBoundedSource; +import org.apache.beam.runners.core.construction.PTransformMatchers; +import org.apache.beam.runners.core.construction.ReplacementOutputs; import org.apache.beam.runners.spark.aggregators.AggregatorsAccumulator; import org.apache.beam.runners.spark.metrics.SparkMetricsContainer; import org.apache.beam.runners.spark.util.GlobalWatermarkHolder; @@ -32,6 +37,7 @@ 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.PTransformOverrideFactory; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.transforms.PTransform; @@ -39,14 +45,13 @@ 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.PInput; -import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TaggedPValue; import org.apache.commons.io.FileUtils; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - /** * The SparkRunner translate operations defined on a pipeline to a representation executable * by Spark, and then submitting the job to Spark to be executed. If we wanted to run a Beam @@ -74,7 +79,6 @@ public final class TestSparkRunner extends PipelineRunner { private SparkRunner delegate; private boolean isForceStreaming; - private int expectedNumberOfAssertions = 0; private TestSparkRunner(TestSparkPipelineOptions options) { this.delegate = SparkRunner.fromOptions(options); @@ -88,37 +92,22 @@ public static TestSparkRunner fromOptions(PipelineOptions options) { return new TestSparkRunner(sparkOptions); } - /** - * Overrides for the test runner. - */ - @SuppressWarnings("unchecked") @Override - public OutputT apply( - PTransform transform, InputT input) { + 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. - if (isForceStreaming && transform instanceof BoundedReadFromUnboundedSource) { - return (OutputT) delegate.apply(new AdaptedBoundedAsUnbounded( - (BoundedReadFromUnboundedSource) transform), input); - } else { - // no actual override, simply counts asserting transforms in the pipeline. - if (transform instanceof PAssert.OneSideInputAssert - || transform instanceof PAssert.GroupThenAssert - || transform instanceof PAssert.GroupThenAssertForSingleton) { - expectedNumberOfAssertions += 1; - } - - return delegate.apply(transform, input); + if (isForceStreaming) { + adaptBoundedReads(pipeline); } - } - - @Override - public SparkPipelineResult run(Pipeline pipeline) { - TestSparkPipelineOptions testSparkPipelineOptions = - pipeline.getOptions().as(TestSparkPipelineOptions.class); SparkPipelineResult result = null; - // clear state of Aggregators, Metrics and Watermarks. + + int expectedNumberOfAssertions = PAssert.countAsserts(pipeline); + + // clear state of Aggregators, Metrics and Watermarks if exists. AggregatorsAccumulator.clear(); SparkMetricsContainer.clear(); GlobalWatermarkHolder.clear(); @@ -170,6 +159,13 @@ public SparkPipelineResult run(Pipeline pipeline) { return result; } + @VisibleForTesting + void adaptBoundedReads(Pipeline pipeline) { + pipeline.replace( + PTransformMatchers.classEqualTo(BoundedReadFromUnboundedSource.class), + new AdaptedBoundedAsUnbounded.Factory()); + } + private static class AdaptedBoundedAsUnbounded extends PTransform> { private final BoundedReadFromUnboundedSource source; @@ -185,6 +181,26 @@ public PCollection expand(PBegin input) { return (PCollection) input.apply(replacingTransform) .apply("StripIds", ParDo.of(new ValueWithRecordId.StripIdsDoFn())); } - } + static class Factory + implements PTransformOverrideFactory< + PBegin, PCollection, BoundedReadFromUnboundedSource> { + @Override + public PTransform> getReplacementTransform( + BoundedReadFromUnboundedSource transform) { + return new AdaptedBoundedAsUnbounded<>(transform); + } + + @Override + public PBegin getInput(List inputs, Pipeline p) { + return p.begin(); + } + + @Override + public Map mapOutputs( + List outputs, PCollection newOutput) { + return ReplacementOutputs.singleton(outputs, newOutput); + } + } + } } 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 9b39558e7468..b60faf2e238d 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 @@ -56,6 +56,8 @@ public void test() throws IOException { Read.from(CountingSource.unbounded()).withMaxNumRecords(-1); //noinspection unchecked pipeline.apply(boundedRead); + TestSparkRunner runner = TestSparkRunner.fromOptions(pipelineRule.getOptions()); + runner.adaptBoundedReads(pipeline); UnboundedReadDetector unboundedReadDetector = new UnboundedReadDetector(); pipeline.traverseTopologically(unboundedReadDetector); 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 109424d9fdda..fe1d526cee5e 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 @@ -417,7 +417,7 @@ private OutputT applyInternal( try { transforms.finishSpecifyingInput(); transform.validate(input); - OutputT output = runner.apply(transform, input); + OutputT output = transform.expand(input); transforms.setOutput(output); return output; @@ -444,7 +444,7 @@ void applyReplacement( LOG.debug("Replacing {} with {}", original, replacement); transforms.replaceNode(original, originalInput, replacement); try { - OutputT newOutput = runner.apply(replacement, originalInput); + OutputT newOutput = replacement.expand(originalInput); Map originalToReplacement = replacementFactory.mapOutputs(original.getOutputs(), newOutput); // Ensure the internal TransformHierarchy data structures are consistent. 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 8604dbc65c48..80bb90f8bf47 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,11 +24,8 @@ import org.apache.beam.sdk.options.GcsOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; -import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.InstanceBuilder; -import org.apache.beam.sdk.values.PInput; -import org.apache.beam.sdk.values.POutput; /** * A {@link PipelineRunner} can execute, translate, or otherwise process a @@ -64,15 +61,4 @@ public static PipelineRunner fromOptions(PipelineOptio * Processes the given Pipeline, returning the results. */ public abstract ResultT run(Pipeline pipeline); - - /** - * Applies a transform to the given input, returning the output. - * - *

The default implementation calls PTransform.apply(input), but can be overridden - * to customize behavior for a particular runner. - */ - public OutputT apply( - PTransform transform, InputT input) { - return transform.expand(input); - } } 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 b57f4a9ff573..a6fb232e6b3e 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 @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.testing; +import static com.google.common.base.Preconditions.checkState; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.not; @@ -32,6 +33,7 @@ import java.util.Map; import java.util.NoSuchElementException; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.IterableCoder; @@ -40,6 +42,7 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.runners.TransformHierarchy.Node; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; @@ -1334,4 +1337,65 @@ public void processElement(ProcessContext c, BoundedWindow window) throws Except } } } + + public static int countAsserts(Pipeline pipeline) { + AssertionCountingVisitor visitor = new AssertionCountingVisitor(); + pipeline.traverseTopologically(visitor); + return visitor.getPAssertCount(); + } + + /** + * A {@link PipelineVisitor} that counts the number of total {@link PAssert PAsserts} in a + * {@link Pipeline}. + */ + private static class AssertionCountingVisitor extends PipelineVisitor.Defaults { + private int assertCount; + private boolean pipelineVisited; + + private AssertionCountingVisitor() { + assertCount = 0; + pipelineVisited = false; + } + + @Override + public CompositeBehavior enterCompositeTransform(Node node) { + if (node.isRootNode()) { + checkState( + !pipelineVisited, + "Tried to visit a pipeline with an already used %s", + AssertionCountingVisitor.class.getSimpleName()); + } + if (!node.isRootNode() + && (node.getTransform() instanceof PAssert.OneSideInputAssert + || node.getTransform() instanceof PAssert.GroupThenAssert + || node.getTransform() instanceof PAssert.GroupThenAssertForSingleton)) { + assertCount++; + } + return CompositeBehavior.ENTER_TRANSFORM; + } + + public void leaveCompositeTransform(Node node) { + if (node.isRootNode()) { + pipelineVisited = true; + } + } + + @Override + public void visitPrimitiveTransform(Node node) { + if + (node.getTransform() instanceof PAssert.OneSideInputAssert + || node.getTransform() instanceof PAssert.GroupThenAssert + || node.getTransform() instanceof PAssert.GroupThenAssertForSingleton) { + assertCount++; + } + } + + /** + * Gets the number of {@link PAssert PAsserts} in the pipeline. + */ + int getPAssertCount() { + checkState(pipelineVisited); + return assertCount; + } + } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java index e57a254603da..777e1af8a13d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.testing; import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; @@ -29,6 +30,7 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; +import java.util.Collections; import java.util.regex.Pattern; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AtomicCoder; @@ -37,12 +39,14 @@ import org.apache.beam.sdk.io.CountingInput; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; 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.common.ElementByteSizeObserver; +import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Duration; @@ -403,4 +407,30 @@ private static Throwable runExpectingAssertionFailure(Pipeline pipeline) { fail("assertion should have failed"); throw new RuntimeException("unreachable"); } + + @Test + public void countAssertsSucceeds() { + PCollection create = pipeline.apply("FirstCreate", Create.of(1, 2, 3)); + + PAssert.that(create).containsInAnyOrder(1, 2, 3); + PAssert.thatSingleton(create.apply(Sum.integersGlobally())).isEqualTo(6); + PAssert.thatMap(pipeline.apply("CreateMap", Create.of(KV.of(1, 2)))) + .isEqualTo(Collections.singletonMap(1, 2)); + + assertThat(PAssert.countAsserts(pipeline), equalTo(3)); + } + + @Test + public void countAssertsMultipleCallsIndependent() { + PCollection create = pipeline.apply("FirstCreate", Create.of(1, 2, 3)); + + PAssert.that(create).containsInAnyOrder(1, 2, 3); + PAssert.thatSingleton(create.apply(Sum.integersGlobally())).isEqualTo(6); + assertThat(PAssert.countAsserts(pipeline), equalTo(2)); + + PAssert.thatMap(pipeline.apply("CreateMap", Create.of(KV.of(1, 2)))) + .isEqualTo(Collections.singletonMap(1, 2)); + + assertThat(PAssert.countAsserts(pipeline), equalTo(3)); + } } From 018b6ffbf85fca3fb3cd4f37bef4931dad3925a0 Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Thu, 23 Feb 2017 14:00:54 -0800 Subject: [PATCH 07/46] Upgrade dill to 0.2.6 and pin it Upgrade dill to the latest version and pin it. There were potential compatibility issues between 0.2.5 and 0.2.6, and keeping this as a range requirement is risky going forward. --- sdks/python/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 6856e00cc075..022d69d133a1 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -86,7 +86,7 @@ def get_version(): REQUIRED_PACKAGES = [ 'avro>=1.7.7,<2.0.0', 'crcmod>=1.7,<2.0', - 'dill>=0.2.5,<0.3', + 'dill==0.2.6', 'httplib2>=0.8,<0.10', 'mock>=1.0.1,<3.0.0', 'oauth2client>=2.0.1,<4.0.0', From b34e50c649adc8861670c42228cb96688abf4038 Mon Sep 17 00:00:00 2001 From: Pablo Date: Mon, 27 Feb 2017 17:28:26 -0800 Subject: [PATCH 08/46] Adding per-stage matching to metrics filters --- .../beam/runners/direct/DirectMetrics.java | 31 +++++++- .../runners/direct/DirectMetricsTest.java | 70 +++++++++++++++++++ sdks/python/apache_beam/metrics/metric.py | 53 +++++++++++--- .../python/apache_beam/metrics/metric_test.py | 43 ++++++++++++ 4 files changed, 187 insertions(+), 10 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectMetrics.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectMetrics.java index 145326f65839..fa8f9c3ed92a 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectMetrics.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectMetrics.java @@ -275,13 +275,40 @@ private boolean matches(MetricsFilter filter, MetricKey key) { && matchesScope(key.stepName(), filter.steps()); } - private boolean matchesScope(String actualScope, Set scopes) { + /** + * {@code subPathMatches(haystack, needle)} returns true if {@code needle} + * represents a path within {@code haystack}. For example, "foo/bar" is in "a/foo/bar/b", + * but not "a/fool/bar/b" or "a/foo/bart/b". + */ + public boolean subPathMatches(String haystack, String needle) { + int location = haystack.indexOf(needle); + int end = location + needle.length(); + if (location == -1) { + return false; // needle not found + } else if (location != 0 && haystack.charAt(location - 1) != '/') { + return false; // the first entry in needle wasn't exactly matched + } else if (end != haystack.length() && haystack.charAt(end) != '/') { + return false; // the last entry in needle wasn't exactly matched + } else { + return true; + } + } + + /** + * {@code matchesScope(actualScope, scopes)} returns true if the scope of a metric is matched + * by any of the filters in {@code scopes}. A metric scope is a path of type "A/B/D". A + * path is matched by a filter if the filter is equal to the path (e.g. "A/B/D", or + * if it represents a subpath within it (e.g. "A/B" or "B/D", but not "A/D"). */ + public boolean matchesScope(String actualScope, Set scopes) { if (scopes.isEmpty() || scopes.contains(actualScope)) { return true; } + // If there is no perfect match, a stage name-level match is tried. + // This is done by a substring search over the levels of the scope. + // e.g. a scope "A/B/C/D" is matched by "A/B", but not by "A/C". for (String scope : scopes) { - if (actualScope.startsWith(scope)) { + if (subPathMatches(actualScope, scope)) { return true; } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectMetricsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectMetricsTest.java index 3ad2bdc523c2..77229bf8886b 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectMetricsTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectMetricsTest.java @@ -23,9 +23,13 @@ import static org.apache.beam.sdk.metrics.MetricNameFilter.inNamespace; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; import com.google.common.collect.ImmutableList; +import java.util.HashSet; +import java.util.Set; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.sdk.metrics.DistributionData; import org.apache.beam.sdk.metrics.DistributionResult; @@ -125,6 +129,72 @@ public void testApplyAttemptedCountersQueryOneNamespace() { committedMetricsResult("ns1", "name1", "step2", 0L))); } + private boolean matchesSubPath(String actualScope, String subPath) { + return metrics.subPathMatches(actualScope, subPath); + } + + @Test + public void testMatchesSubPath() { + assertTrue("Match of the first element", + matchesSubPath("Top1/Outer1/Inner1/Bottom1", "Top1")); + assertTrue("Match of the first elements", + matchesSubPath("Top1/Outer1/Inner1/Bottom1", "Top1/Outer1")); + assertTrue("Match of the last elements", + matchesSubPath("Top1/Outer1/Inner1/Bottom1", "Inner1/Bottom1")); + assertFalse("Substring match but no subpath match", + matchesSubPath("Top1/Outer1/Inner1/Bottom1", "op1/Outer1/Inner1")); + assertFalse("Substring match from start - but no subpath match", + matchesSubPath("Top1/Outer1/Inner1/Bottom1", "Top")); + } + + private boolean matchesScopeWithSingleFilter(String actualScope, String filter) { + Set scopeFilter = new HashSet(); + scopeFilter.add(filter); + return metrics.matchesScope(actualScope, scopeFilter); + } + + @Test + public void testMatchesScope() { + assertTrue(matchesScopeWithSingleFilter("Top1/Outer1/Inner1/Bottom1", "Top1")); + assertTrue(matchesScopeWithSingleFilter( + "Top1/Outer1/Inner1/Bottom1", "Top1/Outer1/Inner1/Bottom1")); + assertTrue(matchesScopeWithSingleFilter("Top1/Outer1/Inner1/Bottom1", "Top1/Outer1")); + assertTrue(matchesScopeWithSingleFilter("Top1/Outer1/Inner1/Bottom1", "Top1/Outer1/Inner1")); + assertFalse(matchesScopeWithSingleFilter("Top1/Outer1/Inner1/Bottom1", "Top1/Inner1")); + assertFalse(matchesScopeWithSingleFilter("Top1/Outer1/Inner1/Bottom1", "Top1/Outer1/Inn")); + } + + @SuppressWarnings("unchecked") + @Test + public void testPartialScopeMatchingInMetricsQuery() { + metrics.updatePhysical(bundle1, MetricUpdates.create( + ImmutableList.of( + MetricUpdate.create(MetricKey.create("Top1/Outer1/Inner1", NAME1), 5L), + MetricUpdate.create(MetricKey.create("Top1/Outer1/Inner2", NAME1), 8L)), + ImmutableList.>of())); + metrics.updatePhysical(bundle1, MetricUpdates.create( + ImmutableList.of( + MetricUpdate.create(MetricKey.create("Top2/Outer1/Inner1", NAME1), 12L), + MetricUpdate.create(MetricKey.create("Top1/Outer2/Inner2", NAME1), 18L)), + ImmutableList.>of())); + + MetricQueryResults results = metrics.queryMetrics( + MetricsFilter.builder().addStep("Top1/Outer1").build()); + + assertThat(results.counters(), + containsInAnyOrder( + attemptedMetricsResult("ns1", "name1", "Top1/Outer1/Inner1", 5L), + attemptedMetricsResult("ns1", "name1", "Top1/Outer1/Inner2", 8L))); + + results = metrics.queryMetrics( + MetricsFilter.builder().addStep("Inner2").build()); + + assertThat(results.counters(), + containsInAnyOrder( + attemptedMetricsResult("ns1", "name1", "Top1/Outer1/Inner2", 8L), + attemptedMetricsResult("ns1", "name1", "Top1/Outer2/Inner2", 18L))); + } + @SuppressWarnings("unchecked") @Test public void testApplyAttemptedQueryCompositeScope() { diff --git a/sdks/python/apache_beam/metrics/metric.py b/sdks/python/apache_beam/metrics/metric.py index a0e3cbab2aa0..f6a0923057bf 100644 --- a/sdks/python/apache_beam/metrics/metric.py +++ b/sdks/python/apache_beam/metrics/metric.py @@ -32,8 +32,7 @@ class Metrics(object): - """Lets users create/access metric objects during pipeline execution. - """ + """Lets users create/access metric objects during pipeline execution.""" @staticmethod def get_namespace(namespace): if inspect.isclass(namespace): @@ -93,14 +92,52 @@ def update(self, value): class MetricResults(object): + + @staticmethod + def _matches_name(filter, metric_key): + if not filter.names and not filter.namespaces: + return True + + if ((filter.namespaces and + metric_key.metric.namespace in filter.namespaces) or + (filter.names and + metric_key.metric.name in filter.names)): + return True + else: + return False + + @staticmethod + def _matches_sub_path(actual_scope, filter_scope): + start_pos = actual_scope.find(filter_scope) + end_pos = start_pos + len(filter_scope) + + if start_pos == -1: + return False # No match at all + elif start_pos != 0 and actual_scope[start_pos - 1] != '/': + return False # The first entry was not exactly matched + elif end_pos != len(actual_scope) and actual_scope[end_pos] != '/': + return False # The last entry was not exactly matched + else: + return True + + @staticmethod + def _matches_scope(filter, metric_key): + if not filter.steps: + return True + + for step in filter.steps: + if MetricResults._matches_sub_path(metric_key.step, step): + return True + + return False + @staticmethod def matches(filter, metric_key): if filter is None: return True - if (metric_key.step in filter.steps and - metric_key.metric.namespace in filter.namespaces and - metric_key.metric.name in filter.names): + if (MetricResults._matches_name(filter, metric_key) and + MetricResults._matches_scope(filter, metric_key)): return True else: return False @@ -139,9 +176,9 @@ def with_name(self, name): def with_names(self, names): if isinstance(names, str): - raise ValueError('Names must be an iterable, not a string') + raise ValueError('Names must be a collection, not a string') - self._steps.update(names) + self._names.update(names) return self def with_namespace(self, namespace): @@ -158,7 +195,7 @@ def with_step(self, step): return self.with_steps([step]) def with_steps(self, steps): - if isinstance(namespaces, str): + if isinstance(steps, str): raise ValueError('Steps must be an iterable, not a string') self._steps.update(steps) diff --git a/sdks/python/apache_beam/metrics/metric_test.py b/sdks/python/apache_beam/metrics/metric_test.py index 4860edf3d013..56b74680a97d 100644 --- a/sdks/python/apache_beam/metrics/metric_test.py +++ b/sdks/python/apache_beam/metrics/metric_test.py @@ -22,6 +22,8 @@ from apache_beam.metrics.execution import MetricsContainer from apache_beam.metrics.execution import MetricsEnvironment from apache_beam.metrics.metric import Metrics +from apache_beam.metrics.metric import MetricsFilter +from apache_beam.metrics.metric import MetricResults from apache_beam.metrics.metricbase import MetricName @@ -39,6 +41,47 @@ def test_basic_metric_name(self): self.assertEqual(key, MetricKey('step1', MetricName('namespace1', 'name1'))) +class MetricResultsTest(unittest.TestCase): + + def test_metric_filter_namespace_matching(self): + filter = MetricsFilter().with_namespace('ns1') + name = MetricName('ns1', 'name1') + key = MetricKey('step1', name) + self.assertTrue(MetricResults.matches(filter, key)) + + def test_metric_filter_name_matching(self): + filter = MetricsFilter().with_name('name1').with_namespace('ns1') + name = MetricName('ns1', 'name1') + key = MetricKey('step1', name) + self.assertTrue(MetricResults.matches(filter, key)) + + filter = MetricsFilter().with_name('name1') + name = MetricName('ns1', 'name1') + key = MetricKey('step1', name) + self.assertTrue(MetricResults.matches(filter, key)) + + def test_metric_filter_step_matching(self): + filter = MetricsFilter().with_step('Top1/Outer1/Inner1') + name = MetricName('ns1', 'name1') + key = MetricKey('Top1/Outer1/Inner1', name) + self.assertTrue(MetricResults.matches(filter, key)) + + filter = MetricsFilter().with_step('step1') + name = MetricName('ns1', 'name1') + key = MetricKey('step1', name) + self.assertTrue(MetricResults.matches(filter, key)) + + filter = MetricsFilter().with_step('Top1/Outer1') + name = MetricName('ns1', 'name1') + key = MetricKey('Top1/Outer1/Inner1', name) + self.assertTrue(MetricResults.matches(filter, key)) + + filter = MetricsFilter().with_step('Top1/Inner1') + name = MetricName('ns1', 'name1') + key = MetricKey('Top1/Outer1/Inner1', name) + self.assertFalse(MetricResults.matches(filter, key)) + + class MetricsTest(unittest.TestCase): def test_get_namespace_class(self): class MyClass(object): From 71a3fd769ff4622df07633b5e7b27b5c309de29e Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Wed, 1 Mar 2017 16:04:49 -0800 Subject: [PATCH 09/46] Update javadoc ant to include runners/ and exclude modules with a wildcard --- sdks/java/javadoc/ant.xml | 33 +++++++++++++++++++-------------- 1 file changed, 19 insertions(+), 14 deletions(-) diff --git a/sdks/java/javadoc/ant.xml b/sdks/java/javadoc/ant.xml index 48b8913fa3ee..80dbdc2fe090 100644 --- a/sdks/java/javadoc/ant.xml +++ b/sdks/java/javadoc/ant.xml @@ -35,6 +35,16 @@ results in one Java source tree. --> + + + + + + + + + - - - - - - - - - - - - - - + + + + + + + + + From 05a006a14de2bc42d31321f34a49706ff3814962 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Wed, 1 Mar 2017 17:17:10 -0800 Subject: [PATCH 10/46] Fix DataflowRunner message about uploaded vs cached files --- .../java/org/apache/beam/runners/dataflow/util/PackageUtil.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java index 685d48c4672d..482ddd935565 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java @@ -330,7 +330,7 @@ public void run() { LOG.info( "Staging files complete: {} files cached, {} files newly uploaded", - numUploaded.get(), numCached.get()); + numCached.get(), numUploaded.get()); return packages; } From eaf9b9b36dec1cc421335b27f225663ce42d0cca Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 24 Feb 2017 11:29:42 -0800 Subject: [PATCH 11/46] Do not Reassign Windows when WindowFn is null Adjusting the Windowing Strategy should not change any elements of the data. This is also potentially type-unsafe, as the upstream WindowFn may only take elements of a type which is not the input element of the downstream PTransform. Introduce Window.Assign, which replaces Window.Bound as the primitive to "assign elements to windows based on the WindowFn". This converts Window.Bound into a composite in all cases. Use a Flatten to improve performance on many runners, without needing an opaque DoFn. --- .../translation/ApexPipelineTranslator.java | 2 +- ...lator.java => WindowAssignTranslator.java} | 4 +- .../direct/TransformEvaluatorRegistry.java | 2 +- .../direct/WindowEvaluatorFactory.java | 11 +- .../direct/WindowEvaluatorFactoryTest.java | 46 +----- .../flink/FlinkBatchTransformTranslators.java | 8 +- .../FlinkStreamingTransformTranslators.java | 8 +- .../functions/FlinkAssignWindows.java | 2 +- .../dataflow/DataflowPipelineTranslator.java | 9 +- .../translation/TransformTranslator.java | 8 +- .../spark/translation/TranslationUtils.java | 4 +- .../StreamingTransformTranslator.java | 8 +- .../beam/sdk/transforms/windowing/Window.java | 43 +++++- .../sdk/transforms/windowing/WindowTest.java | 136 ++++++++++++++++++ 14 files changed, 214 insertions(+), 77 deletions(-) rename runners/apex/src/main/java/org/apache/beam/runners/apex/translation/{WindowBoundTranslator.java => WindowAssignTranslator.java} (94%) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java index e9d657179353..951a286fb3af 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java @@ -71,7 +71,7 @@ public class ApexPipelineTranslator implements Pipeline.PipelineVisitor { new CreateApexPCollectionViewTranslator()); registerTransformTranslator(CreatePCollectionView.class, new CreatePCollectionViewTranslator()); - registerTransformTranslator(Window.Bound.class, new WindowBoundTranslator()); + registerTransformTranslator(Window.Assign.class, new WindowAssignTranslator()); } public ApexPipelineTranslator(ApexPipelineOptions options) { diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowAssignTranslator.java similarity index 94% rename from runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java rename to runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowAssignTranslator.java index a241cad670d1..b3aef8dba878 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowAssignTranslator.java @@ -38,11 +38,11 @@ * {@link Window.Bound} is translated to {link ApexParDoOperator} that wraps an {@link * AssignWindowsDoFn}. */ -class WindowBoundTranslator implements TransformTranslator> { +class WindowAssignTranslator implements TransformTranslator> { private static final long serialVersionUID = 1L; @Override - public void translate(Window.Bound transform, TranslationContext context) { + public void translate(Window.Assign transform, TranslationContext context) { PCollection output = (PCollection) context.getOutput(); PCollection input = (PCollection) context.getInput(); @SuppressWarnings("unchecked") 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 9fdefc385cbc..62fee5309a38 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 @@ -55,7 +55,7 @@ public static TransformEvaluatorRegistry defaultRegistry(EvaluationContext ctxt) .put(StatefulParDo.class, new StatefulParDoEvaluatorFactory<>(ctxt)) .put(PCollections.class, new FlattenEvaluatorFactory(ctxt)) .put(ViewEvaluatorFactory.WriteView.class, new ViewEvaluatorFactory(ctxt)) - .put(Window.Bound.class, new WindowEvaluatorFactory(ctxt)) + .put(Window.Assign.class, new WindowEvaluatorFactory(ctxt)) // Runner-specific primitives used in expansion of GroupByKey .put(DirectGroupByKeyOnly.class, new GroupByKeyOnlyEvaluatorFactory(ctxt)) .put(DirectGroupAlsoByWindow.class, new GroupAlsoByWindowEvaluatorFactory(ctxt)) 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 3cf178c8f4ce..8974c672d05d 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 @@ -26,7 +26,6 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.transforms.windowing.Window.Bound; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; @@ -34,7 +33,7 @@ /** * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the - * {@link Bound Window.Bound} primitive {@link PTransform}. + * {@link Window.Assign} primitive {@link PTransform}. */ class WindowEvaluatorFactory implements TransformEvaluatorFactory { private final EvaluationContext evaluationContext; @@ -53,7 +52,8 @@ public TransformEvaluator forApplication( } private TransformEvaluator createTransformEvaluator( - AppliedPTransform, PCollection, Window.Bound> transform) { + AppliedPTransform, PCollection, Window.Assign> + transform) { WindowFn fn = transform.getTransform().getWindowFn(); UncommittedBundle outputBundle = evaluationContext.createBundle( @@ -68,14 +68,15 @@ private TransformEvaluator createTransformEvaluator( public void cleanup() {} private static class WindowIntoEvaluator implements TransformEvaluator { - private final AppliedPTransform, PCollection, Window.Bound> + private final AppliedPTransform, PCollection, Window.Assign> transform; private final WindowFn windowFn; private final UncommittedBundle outputBundle; @SuppressWarnings("unchecked") public WindowIntoEvaluator( - AppliedPTransform, PCollection, Window.Bound> transform, + AppliedPTransform, PCollection, Window.Assign> + transform, WindowFn windowFn, UncommittedBundle outputBundle) { this.outputBundle = outputBundle; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java index 7e6eb2f5285d..ca52852c7146 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java @@ -33,11 +33,7 @@ import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.coders.Coder; 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.transforms.windowing.AfterPane; -import org.apache.beam.sdk.transforms.windowing.AfterWatermark; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; @@ -113,30 +109,6 @@ public void setup() { factory = new WindowEvaluatorFactory(evaluationContext); } - @Test - public void nullWindowFunSucceeds() throws Exception { - Bound transform = - Window.triggering( - AfterWatermark.pastEndOfWindow().withEarlyFirings(AfterPane.elementCountAtLeast(1))) - .accumulatingFiredPanes(); - PCollection triggering = input.apply(transform); - - CommittedBundle inputBundle = createInputBundle(); - - UncommittedBundle outputBundle = createOutputBundle(triggering, inputBundle); - - TransformResult result = runEvaluator(triggering, inputBundle, transform); - - assertThat( - Iterables.getOnlyElement(result.getOutputBundles()), - Matchers.>equalTo(outputBundle)); - CommittedBundle committed = outputBundle.commit(Instant.now()); - assertThat( - committed.getElements(), - containsInAnyOrder( - valueInIntervalWindow, valueInGlobalWindow, valueInGlobalAndTwoIntervalWindows)); - } - @Test public void singleWindowFnSucceeds() throws Exception { Duration windowDuration = Duration.standardDays(7); @@ -150,7 +122,7 @@ public void singleWindowFnSucceeds() throws Exception { BoundedWindow firstSecondWindow = new IntervalWindow(EPOCH, EPOCH.plus(windowDuration)); BoundedWindow thirdWindow = new IntervalWindow(EPOCH.minus(windowDuration), EPOCH); - TransformResult result = runEvaluator(windowed, inputBundle, transform); + TransformResult result = runEvaluator(windowed, inputBundle); assertThat( Iterables.getOnlyElement(result.getOutputBundles()), @@ -185,7 +157,7 @@ public void multipleWindowsWindowFnSucceeds() throws Exception { CommittedBundle inputBundle = createInputBundle(); UncommittedBundle outputBundle = createOutputBundle(windowed, inputBundle); - TransformResult result = runEvaluator(windowed, inputBundle, transform); + TransformResult result = runEvaluator(windowed, inputBundle); assertThat( Iterables.getOnlyElement(result.getOutputBundles()), @@ -242,7 +214,7 @@ public void referencesEarlierWindowsSucceeds() throws Exception { CommittedBundle inputBundle = createInputBundle(); UncommittedBundle outputBundle = createOutputBundle(windowed, inputBundle); - TransformResult result = runEvaluator(windowed, inputBundle, transform); + TransformResult result = runEvaluator(windowed, inputBundle); assertThat( Iterables.getOnlyElement(result.getOutputBundles()), @@ -307,17 +279,9 @@ private UncommittedBundle createOutputBundle( } private TransformResult runEvaluator( - PCollection windowed, - CommittedBundle inputBundle, - Window.Bound windowTransform /* Required while Window.Bound is a composite */) - throws Exception { + PCollection windowed, CommittedBundle inputBundle) throws Exception { TransformEvaluator evaluator = - factory.forApplication( - AppliedPTransform - ., PCollection, - PTransform, PCollection>> - of("Window", input.expand(), windowed.expand(), windowTransform, p), - inputBundle); + factory.forApplication(DirectGraphs.getProducer(windowed), inputBundle); evaluator.processElement(valueInGlobalWindow); evaluator.processElement(valueInGlobalAndTwoIntervalWindows); diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java index acc204d52a94..f043c901391f 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java @@ -110,7 +110,7 @@ class FlinkBatchTransformTranslators { TRANSLATORS.put(Flatten.PCollections.class, new FlattenPCollectionTranslatorBatch()); - TRANSLATORS.put(Window.Bound.class, new WindowBoundTranslatorBatch()); + TRANSLATORS.put(Window.Assign.class, new WindowAssignTranslatorBatch()); TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundTranslatorBatch()); TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiTranslatorBatch()); @@ -145,11 +145,11 @@ public void translateNode(Read.Bounded transform, FlinkBatchTranslationContex } } - private static class WindowBoundTranslatorBatch - implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static class WindowAssignTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { @Override - public void translateNode(Window.Bound transform, FlinkBatchTranslationContext context) { + public void translateNode(Window.Assign transform, FlinkBatchTranslationContext context) { PValue input = context.getInput(transform); TypeInformation> resultTypeInfo = diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java index 03f567d3cc14..c7df91dc1c1b 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java @@ -124,7 +124,7 @@ class FlinkStreamingTransformTranslators { TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundStreamingTranslator()); TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiStreamingTranslator()); - TRANSLATORS.put(Window.Bound.class, new WindowBoundTranslator()); + TRANSLATORS.put(Window.Assign.class, new WindowAssignTranslator()); TRANSLATORS.put(Flatten.PCollections.class, new FlattenPCollectionTranslator()); TRANSLATORS.put( FlinkStreamingViewOverrides.CreateFlinkPCollectionView.class, @@ -702,12 +702,12 @@ public void translateNode( } } - private static class WindowBoundTranslator - extends FlinkStreamingPipelineTranslator.StreamTransformTranslator> { + private static class WindowAssignTranslator + extends FlinkStreamingPipelineTranslator.StreamTransformTranslator> { @Override public void translateNode( - Window.Bound transform, + Window.Assign transform, FlinkStreamingTranslationContext context) { @SuppressWarnings("unchecked") diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java index f241ad0dc5c0..c3a5095bc1eb 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java @@ -26,7 +26,7 @@ /** * Flink {@link FlatMapFunction} for implementing - * {@link org.apache.beam.sdk.transforms.windowing.Window.Bound}. + * {@link org.apache.beam.sdk.transforms.windowing.Window.Assign}. */ public class FlinkAssignWindows implements FlatMapFunction, WindowedValue> { 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 fe5db5a6a36c..7e559e9e4885 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 @@ -85,7 +85,6 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.transforms.windowing.Window.Bound; import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.CloudObject; import org.apache.beam.sdk.util.PropertyNames; @@ -878,14 +877,14 @@ private void translateSingleHelper( }); registerTransformTranslator( - Window.Bound.class, - new TransformTranslator() { + Window.Assign.class, + new TransformTranslator() { @Override - public void translate(Window.Bound transform, TranslationContext context) { + public void translate(Window.Assign transform, TranslationContext context) { translateHelper(transform, context); } - private void translateHelper(Window.Bound transform, TranslationContext context) { + private void translateHelper(Window.Assign transform, TranslationContext context) { StepTranslationContext stepContext = context.addStep(transform, "Bucket"); PCollection input = context.getInput(transform); stepContext.addInput(PropertyNames.PARALLEL_INPUT, input); 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 7fc09ad0ba6e..8ebb49679222 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 @@ -611,10 +611,10 @@ private static void writeHadoopFile(JavaPairRDD rdd, Configuration rdd.saveAsNewAPIHadoopFile(outputDir, keyClass, valueClass, formatClass, conf); } - private static TransformEvaluator> window() { - return new TransformEvaluator>() { + private static TransformEvaluator> window() { + return new TransformEvaluator>() { @Override - public void evaluate(Window.Bound transform, EvaluationContext context) { + public void evaluate(Window.Assign transform, EvaluationContext context) { @SuppressWarnings("unchecked") JavaRDD> inRDD = ((BoundedDataset) context.borrowDataset(transform)).getRDD(); @@ -734,7 +734,7 @@ public void evaluate(StorageLevelPTransform transform, EvaluationContext context EVALUATORS.put(View.AsSingleton.class, viewAsSingleton()); EVALUATORS.put(View.AsIterable.class, viewAsIter()); EVALUATORS.put(View.CreatePCollectionView.class, createPCollView()); - EVALUATORS.put(Window.Bound.class, window()); + EVALUATORS.put(Window.Assign.class, window()); // mostly test evaluators EVALUATORS.put(StorageLevelPTransform.class, storageLevel()); } 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 6b27436dc789..158593e475e4 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 @@ -101,14 +101,14 @@ public WindowedValue> call(WindowedValue>> * with triggering or allowed lateness). *

* - * @param transform The {@link Window.Bound} transformation. + * @param transform The {@link Window.Assign} transformation. * @param context The {@link EvaluationContext}. * @param PCollection type. * @param {@link BoundedWindow} type. * @return if to apply the transformation. */ public static boolean - skipAssignWindows(Window.Bound transform, EvaluationContext context) { + skipAssignWindows(Window.Assign transform, EvaluationContext context) { @SuppressWarnings("unchecked") WindowFn windowFn = (WindowFn) transform.getWindowFn(); return windowFn == null 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 a856897e85cd..e3445bf5baab 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 @@ -210,10 +210,10 @@ public void evaluate(Flatten.PCollections transform, EvaluationContext contex }; } - private static TransformEvaluator> window() { - return new TransformEvaluator>() { + private static TransformEvaluator> window() { + return new TransformEvaluator>() { @Override - public void evaluate(final Window.Bound transform, EvaluationContext context) { + public void evaluate(final Window.Assign transform, EvaluationContext context) { @SuppressWarnings("unchecked") UnboundedDataset unboundedDataset = ((UnboundedDataset) context.borrowDataset(transform)); @@ -444,7 +444,7 @@ public JavaPairRDD, WindowedValue> call( EVALUATORS.put(ParDo.BoundMulti.class, multiDo()); EVALUATORS.put(ConsoleIO.Write.Unbound.class, print()); EVALUATORS.put(CreateStream.class, createFromQueue()); - EVALUATORS.put(Window.Bound.class, window()); + EVALUATORS.put(Window.Assign.class, window()); EVALUATORS.put(Flatten.PCollections.class, flattenPColl()); } 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 65dfaa909970..94870ff23116 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 @@ -22,6 +22,7 @@ import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; +import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; @@ -30,6 +31,7 @@ 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.joda.time.Duration; /** @@ -152,7 +154,7 @@ public enum ClosingBehavior { * *

This is the default behavior. */ - FIRE_IF_NON_EMPTY; + FIRE_IF_NON_EMPTY } @@ -469,8 +471,16 @@ private boolean canProduceMultiplePanes(WindowingStrategy strategy) { public PCollection expand(PCollection input) { WindowingStrategy outputStrategy = getOutputStrategyInternal(input.getWindowingStrategy()); - return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), outputStrategy, input.isBounded()); + if (windowFn == null) { + // A new PCollection must be created in case input is reused in a different location as the + // two PCollections will, in general, have a different windowing strategy. + return PCollectionList.of(input) + .apply(Flatten.pCollections()) + .setWindowingStrategyInternal(outputStrategy); + } else { + // This is the AssignWindows primitive + return input.apply(new Assign(outputStrategy)); + } } @Override @@ -522,6 +532,33 @@ protected String getKindString() { } } + + /** + * A Primitive {@link PTransform} that assigns windows to elements based on a {@link WindowFn}. + */ + public static class Assign extends PTransform, PCollection> { + private final WindowingStrategy updatedStrategy; + + /** + * Create a new {@link Assign} where the output is windowed with the updated {@link + * WindowingStrategy}. Windows should be assigned using the {@link WindowFn} returned by + * {@link #getWindowFn()}. + */ + private Assign(WindowingStrategy updatedStrategy) { + this.updatedStrategy = updatedStrategy; + } + + @Override + public PCollection expand(PCollection input) { + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), updatedStrategy, input.isBounded()); + } + + public WindowFn getWindowFn() { + return updatedStrategy.getWindowFn(); + } + } + /** * Creates a {@code Window} {@code PTransform} that does not change assigned * windows, but will cause windows to be merged again as part of the next 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 55c729750339..1101ebc64526 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 @@ -21,6 +21,7 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasKey; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFor; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.isOneOf; import static org.hamcrest.Matchers.not; @@ -30,15 +31,24 @@ import static org.mockito.Mockito.when; import java.io.Serializable; +import java.util.Collection; +import java.util.Collections; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.CountingInput; +import org.apache.beam.sdk.runners.TransformHierarchy; +import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; +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.transforms.display.DisplayData; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; @@ -153,6 +163,51 @@ public void testWindowIntoPropagatesLateness() { assertEquals(fixed25, strategy.getWindowFn()); } + /** + * With {@link #testWindowIntoNullWindowFnNoAssign()}, demonstrates that the expansions of the + * {@link Window.Bound} transform depends on if it actually assigns elements to windows. + */ + @Test + public void testWindowIntoWindowFnAssign() { + pipeline + .apply(Create.of(1, 2, 3)) + .apply( + Window.into( + FixedWindows.of(Duration.standardMinutes(11L).plus(Duration.millis(1L))))); + + final AtomicBoolean foundAssign = new AtomicBoolean(false); + pipeline.traverseTopologically( + new PipelineVisitor.Defaults() { + public void visitPrimitiveTransform(TransformHierarchy.Node node) { + if (node.getTransform() instanceof Window.Assign) { + foundAssign.set(true); + } + } + }); + assertThat(foundAssign.get(), is(true)); + } + + /** + * With {@link #testWindowIntoWindowFnAssign()}, demonstrates that the expansions of the + * {@link Window.Bound} transform depends on if it actually assigns elements to windows. + */ + @Test + public void testWindowIntoNullWindowFnNoAssign() { + pipeline + .apply(Create.of(1, 2, 3)) + .apply( + Window.triggering(AfterWatermark.pastEndOfWindow()) + .withAllowedLateness(Duration.ZERO) + .accumulatingFiredPanes()); + + pipeline.traverseTopologically( + new PipelineVisitor.Defaults() { + public void visitPrimitiveTransform(TransformHierarchy.Node node) { + assertThat(node.getTransform(), not(instanceOf(Window.Assign.class))); + } + }); + } + @Test public void testWindowGetName() { assertEquals("Window.Into()", @@ -220,6 +275,87 @@ public void testMissingLateness() { .apply("Trigger", Window.triggering(trigger)); } + private static class WindowOddEvenBuckets extends NonMergingWindowFn { + private static final IntervalWindow EVEN_WINDOW = + new IntervalWindow( + BoundedWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE.maxTimestamp()); + private static final IntervalWindow ODD_WINDOW = + new IntervalWindow( + BoundedWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE.maxTimestamp().minus(1)); + + @Override + public Collection assignWindows(AssignContext c) throws Exception { + if (c.element() % 2 == 0) { + return Collections.singleton(EVEN_WINDOW); + } + return Collections.singleton(ODD_WINDOW); + } + + @Override + public boolean isCompatible(WindowFn other) { + return other instanceof WindowOddEvenBuckets; + } + + @Override + public Coder windowCoder() { + return new IntervalWindow.IntervalWindowCoder(); + } + + @Override + public IntervalWindow getSideInputWindow(BoundedWindow window) { + throw new UnsupportedOperationException( + String.format("Can't use %s for side inputs", getClass().getSimpleName())); + } + } + + @Test + @Category(RunnableOnService.class) + public void testNoWindowFnDoesNotReassignWindows() { + pipeline.enableAbandonedNodeEnforcement(true); + + final PCollection initialWindows = + pipeline + .apply(CountingInput.upTo(10L)) + .apply("AssignWindows", Window.into(new WindowOddEvenBuckets())); + + // Sanity check the window assignment to demonstrate the baseline + PAssert.that(initialWindows) + .inWindow(WindowOddEvenBuckets.EVEN_WINDOW) + .containsInAnyOrder(0L, 2L, 4L, 6L, 8L); + PAssert.that(initialWindows) + .inWindow(WindowOddEvenBuckets.ODD_WINDOW) + .containsInAnyOrder(1L, 3L, 5L, 7L, 9L); + + PCollection upOne = + initialWindows.apply( + "ModifyTypes", + MapElements.via( + new SimpleFunction() { + @Override + public Boolean apply(Long input) { + return input % 2 == 0; + } + })); + PAssert.that(upOne) + .inWindow(WindowOddEvenBuckets.EVEN_WINDOW) + .containsInAnyOrder(true, true, true, true, true); + PAssert.that(upOne) + .inWindow(WindowOddEvenBuckets.ODD_WINDOW) + .containsInAnyOrder(false, false, false, false, false); + + // The elements should be in the same windows, even though they would not be assigned to the + // same windows with the updated timestamps. If we try to apply the original WindowFn, the type + // will not be appropriate and the runner should crash, as a Boolean cannot be converted into + // a long. + PCollection updatedTrigger = + upOne.apply( + "UpdateWindowingStrategy", + Window.triggering(Never.ever()) + .withAllowedLateness(Duration.ZERO) + .accumulatingFiredPanes()); + pipeline.run(); + } + /** * Tests that when two elements are combined via a GroupByKey their output timestamp agrees * with the windowing function default, the end of the window. From 1a770ef2f21b96fbdc5ff06ea8642351f136328f Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Wed, 1 Mar 2017 18:21:40 -0800 Subject: [PATCH 12/46] [maven-release-plugin] prepare branch release-0.6.0 --- pom.xml | 2 +- runners/core-construction-java/pom.xml | 4 +--- sdks/common/fn-api/pom.xml | 4 +--- sdks/common/runner-api/pom.xml | 4 +--- sdks/java/extensions/jackson/pom.xml | 4 +--- sdks/java/harness/pom.xml | 4 +--- sdks/java/javadoc/pom.xml | 10 +++++----- 7 files changed, 11 insertions(+), 21 deletions(-) diff --git a/pom.xml b/pom.xml index a37f1af4d4d0..eded6840dfd8 100644 --- a/pom.xml +++ b/pom.xml @@ -48,7 +48,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/beam.git scm:git:https://git-wip-us.apache.org/repos/asf/beam.git https://git-wip-us.apache.org/repos/asf?p=beam.git;a=summary - HEAD + release-0.6.0 diff --git a/runners/core-construction-java/pom.xml b/runners/core-construction-java/pom.xml index 868f74324de3..b602f5dca8f0 100644 --- a/runners/core-construction-java/pom.xml +++ b/runners/core-construction-java/pom.xml @@ -17,9 +17,7 @@ ~ limitations under the License. --> - + 4.0.0 diff --git a/sdks/common/fn-api/pom.xml b/sdks/common/fn-api/pom.xml index 1f6193ffe58c..5a41d9e365b9 100644 --- a/sdks/common/fn-api/pom.xml +++ b/sdks/common/fn-api/pom.xml @@ -15,9 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 jar diff --git a/sdks/common/runner-api/pom.xml b/sdks/common/runner-api/pom.xml index 8eaeb8e321af..9c6de1ebfad1 100644 --- a/sdks/common/runner-api/pom.xml +++ b/sdks/common/runner-api/pom.xml @@ -15,9 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 jar diff --git a/sdks/java/extensions/jackson/pom.xml b/sdks/java/extensions/jackson/pom.xml index be5c9537da95..1dfbd72d6fa9 100644 --- a/sdks/java/extensions/jackson/pom.xml +++ b/sdks/java/extensions/jackson/pom.xml @@ -15,9 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 diff --git a/sdks/java/harness/pom.xml b/sdks/java/harness/pom.xml index 3abe70b171b1..80b01ca88695 100644 --- a/sdks/java/harness/pom.xml +++ b/sdks/java/harness/pom.xml @@ -15,9 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 jar diff --git a/sdks/java/javadoc/pom.xml b/sdks/java/javadoc/pom.xml index 243dae5d448d..b785c986dbdd 100644 --- a/sdks/java/javadoc/pom.xml +++ b/sdks/java/javadoc/pom.xml @@ -254,12 +254,12 @@ package - - - - + + + + - + From 15e1495106ceddcbb5c9f158592a9fc785732f4a Mon Sep 17 00:00:00 2001 From: Aviem Zur Date: Thu, 2 Mar 2017 10:20:57 +0200 Subject: [PATCH 13/46] [BEAM-1565] Update Spark runner PostCommit Jenkins job. --- .../job_beam_PostCommit_Java_RunnableOnService_Spark.groovy | 2 +- runners/spark/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Spark.groovy b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Spark.groovy index 6927902febe8..a1983bdd683d 100644 --- a/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Spark.groovy +++ b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Spark.groovy @@ -40,5 +40,5 @@ mavenJob('beam_PostCommit_Java_RunnableOnService_Spark') { 'Run Spark RunnableOnService') // Maven goals for this job. - goals('-B -e clean verify -am -pl runners/spark -Prunnable-on-service-tests -Plocal-runnable-on-service-tests -Dspark.port.maxRetries=64 -Dspark.ui.enabled=false') + goals('-B -e clean verify -am -pl runners/spark -Prunnable-on-service-tests -Plocal-runnable-on-service-tests -Dspark.ui.enabled=false') } diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 44f20ccd0b9c..0d4c413e4790 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -57,7 +57,7 @@ - runnable-on-service-tests + local-runnable-on-service-tests false From a7f69bd2d2fcf5a4852739e3b3a9b705fa7b1fb5 Mon Sep 17 00:00:00 2001 From: tedyu Date: Mon, 27 Feb 2017 16:28:19 -0800 Subject: [PATCH 14/46] BEAM-1567 hashStream should be closed in PackageUtil#createPackageAttributes() --- .../beam/runners/dataflow/util/PackageUtil.java | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java index 482ddd935565..0d52c5db67c1 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java @@ -90,14 +90,12 @@ class PackageUtil { */ static PackageAttributes createPackageAttributes(File source, String stagingPath, @Nullable String overridePackageName) { - try { - boolean directory = source.isDirectory(); - - // Compute size and hash in one pass over file or directory. - Hasher hasher = Hashing.md5().newHasher(); - OutputStream hashStream = Funnels.asOutputStream(hasher); - CountingOutputStream countingOutputStream = new CountingOutputStream(hashStream); + boolean directory = source.isDirectory(); + // Compute size and hash in one pass over file or directory. + Hasher hasher = Hashing.md5().newHasher(); + OutputStream hashStream = Funnels.asOutputStream(hasher); + try (CountingOutputStream countingOutputStream = new CountingOutputStream(hashStream)) { if (!directory) { // Files are staged as-is. Files.asByteSource(source).copyTo(countingOutputStream); @@ -105,6 +103,7 @@ static PackageAttributes createPackageAttributes(File source, // Directories are recursively zipped. ZipFiles.zipDirectory(source, countingOutputStream); } + countingOutputStream.flush(); long size = countingOutputStream.getCount(); String hash = Base64Variants.MODIFIED_FOR_URL.encode(hasher.hash().asBytes()); From dd32c26622ef9d0aa9e8d0c3863ac6660ed336b7 Mon Sep 17 00:00:00 2001 From: Mark Liu Date: Tue, 21 Feb 2017 18:48:34 -0800 Subject: [PATCH 15/46] [BEAM-1188] Python Bigquery Verifier For E2E Test --- .../cookbook/bigquery_tornadoes_it_test.py | 62 ++++++++++ .../apache_beam/io/gcp/tests/__init__.py | 16 +++ .../io/gcp/tests/bigquery_matcher.py | 108 ++++++++++++++++++ .../io/gcp/tests/bigquery_matcher_test.py | 108 ++++++++++++++++++ .../apache_beam/tests/pipeline_verifiers.py | 12 +- sdks/python/apache_beam/tests/test_utils.py | 12 ++ sdks/python/setup.py | 3 + 7 files changed, 313 insertions(+), 8 deletions(-) create mode 100644 sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_it_test.py create mode 100644 sdks/python/apache_beam/io/gcp/tests/__init__.py create mode 100644 sdks/python/apache_beam/io/gcp/tests/bigquery_matcher.py create mode 100644 sdks/python/apache_beam/io/gcp/tests/bigquery_matcher_test.py diff --git a/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_it_test.py b/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_it_test.py new file mode 100644 index 000000000000..306a09e442da --- /dev/null +++ b/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_it_test.py @@ -0,0 +1,62 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +"""End-to-end test for Bigquery tornadoes example.""" + +import logging +import time +import unittest + +from hamcrest.core.core.allof import all_of +from nose.plugins.attrib import attr + +from apache_beam.examples.cookbook import bigquery_tornadoes +from apache_beam.io.gcp.tests.bigquery_matcher import BigqueryMatcher +from apache_beam.test_pipeline import TestPipeline +from apache_beam.tests.pipeline_verifiers import PipelineStateMatcher + + +class BigqueryTornadoesIT(unittest.TestCase): + + # The default checksum is a SHA-1 hash generated from sorted rows reading + # from expected Bigquery table. + DEFAULT_CHECKSUM = '83789a7c1bca7959dcf23d3bc37e9204e594330f' + + @attr('IT') + def test_bigquery_tornadoes_it(self): + test_pipeline = TestPipeline(is_integration_test=True) + + # Set extra options to the pipeline for test purpose + output_table = ('BigQueryTornadoesIT' + '.monthly_tornadoes_%s' % int(round(time.time() * 1000))) + query = 'SELECT month, tornado_count FROM [%s]' % output_table + pipeline_verifiers = [PipelineStateMatcher(), + BigqueryMatcher( + project=test_pipeline.get_option('project'), + query=query, + checksum=self.DEFAULT_CHECKSUM)] + extra_opts = {'output': output_table, + 'on_success_matcher': all_of(*pipeline_verifiers)} + + # Get pipeline options from command argument: --test-pipeline-options, + # and start pipeline job by calling pipeline main function. + bigquery_tornadoes.run( + test_pipeline.get_full_options_as_args(**extra_opts)) + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + unittest.main() diff --git a/sdks/python/apache_beam/io/gcp/tests/__init__.py b/sdks/python/apache_beam/io/gcp/tests/__init__.py new file mode 100644 index 000000000000..cce3acad34a4 --- /dev/null +++ b/sdks/python/apache_beam/io/gcp/tests/__init__.py @@ -0,0 +1,16 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# diff --git a/sdks/python/apache_beam/io/gcp/tests/bigquery_matcher.py b/sdks/python/apache_beam/io/gcp/tests/bigquery_matcher.py new file mode 100644 index 000000000000..cc26689a56c3 --- /dev/null +++ b/sdks/python/apache_beam/io/gcp/tests/bigquery_matcher.py @@ -0,0 +1,108 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +"""Bigquery data verifier for end-to-end test.""" + +import logging + +from hamcrest.core.base_matcher import BaseMatcher + +from apache_beam.tests.test_utils import compute_hash +from apache_beam.utils import retry + +# Protect against environments where bigquery library is not available. +# pylint: disable=wrong-import-order, wrong-import-position +try: + from google.cloud import bigquery + from google.cloud.exceptions import GoogleCloudError +except ImportError: + bigquery = None +# pylint: enable=wrong-import-order, wrong-import-position + +MAX_RETRIES = 4 + + +def retry_on_http_and_value_error(exception): + """Filter allowing retries on Bigquery errors and value error.""" + return isinstance(exception, GoogleCloudError) or \ + isinstance(exception, ValueError) + + +class BigqueryMatcher(BaseMatcher): + """Matcher that verifies Bigquery data with given query. + + Fetch Bigquery data with given query, compute a hash string and compare + with expected checksum. + """ + + def __init__(self, project, query, checksum): + if bigquery is None: + raise ImportError( + 'Bigquery dependencies are not installed.') + if not query or not isinstance(query, str): + raise ValueError( + 'Invalid argument: query. Please use non-empty string') + if not checksum or not isinstance(checksum, str): + raise ValueError( + 'Invalid argument: checksum. Please use non-empty string') + self.project = project + self.query = query + self.expected_checksum = checksum + + def _matches(self, _): + logging.info('Start verify Bigquery data.') + # Run query + bigquery_client = bigquery.Client(project=self.project) + response = self._query_with_retry(bigquery_client) + logging.info('Read from given query (%s), total rows %d', + self.query, len(response)) + + # Compute checksum + self.checksum = compute_hash(response) + logging.info('Generate checksum: %s', self.checksum) + + # Verify result + return self.checksum == self.expected_checksum + + @retry.with_exponential_backoff( + num_retries=MAX_RETRIES, + retry_filter=retry_on_http_and_value_error) + def _query_with_retry(self, bigquery_client): + """Run Bigquery query with retry if got error http response""" + query = bigquery_client.run_sync_query(self.query) + query.run() + + # Fetch query data one page at a time. + page_token = None + results = [] + while True: + rows, _, page_token = query.fetch_data(page_token=page_token) + results.extend(rows) + if not page_token: + break + + return results + + def describe_to(self, description): + description \ + .append_text("Expected checksum is ") \ + .append_text(self.expected_checksum) + + def describe_mismatch(self, pipeline_result, mismatch_description): + mismatch_description \ + .append_text("Actual checksum is ") \ + .append_text(self.checksum) diff --git a/sdks/python/apache_beam/io/gcp/tests/bigquery_matcher_test.py b/sdks/python/apache_beam/io/gcp/tests/bigquery_matcher_test.py new file mode 100644 index 000000000000..d8aa1489dc15 --- /dev/null +++ b/sdks/python/apache_beam/io/gcp/tests/bigquery_matcher_test.py @@ -0,0 +1,108 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +"""Unit test for Bigquery verifier""" + +import logging +import unittest + +from hamcrest import assert_that as hc_assert_that +from mock import Mock, patch + +from apache_beam.io.gcp.tests import bigquery_matcher as bq_verifier +from apache_beam.tests.test_utils import patch_retry + +# Protect against environments where bigquery library is not available. +# pylint: disable=wrong-import-order, wrong-import-position +try: + from google.cloud import bigquery + from google.cloud.exceptions import NotFound +except ImportError: + bigquery = None +# pylint: enable=wrong-import-order, wrong-import-position + + +@unittest.skipIf(bigquery is None, 'Bigquery dependencies are not installed.') +class BigqueryMatcherTest(unittest.TestCase): + + def setUp(self): + self._mock_result = Mock() + patch_retry(self, bq_verifier) + + @patch.object(bigquery, 'Client') + def test_bigquery_matcher_success(self, mock_bigquery): + mock_query = Mock() + mock_client = mock_bigquery.return_value + mock_client.run_sync_query.return_value = mock_query + mock_query.fetch_data.return_value = ([], None, None) + + matcher = bq_verifier.BigqueryMatcher( + 'mock_project', + 'mock_query', + 'da39a3ee5e6b4b0d3255bfef95601890afd80709') + hc_assert_that(self._mock_result, matcher) + + @patch.object(bigquery, 'Client') + def test_bigquery_matcher_query_run_error(self, mock_bigquery): + mock_query = Mock() + mock_client = mock_bigquery.return_value + mock_client.run_sync_query.return_value = mock_query + mock_query.run.side_effect = ValueError('job is already running') + + matcher = bq_verifier.BigqueryMatcher('mock_project', + 'mock_query', + 'mock_checksum') + with self.assertRaises(ValueError): + hc_assert_that(self._mock_result, matcher) + self.assertTrue(mock_query.run.called) + self.assertEqual(bq_verifier.MAX_RETRIES + 1, mock_query.run.call_count) + + @patch.object(bigquery, 'Client') + def test_bigquery_matcher_fetch_data_error(self, mock_bigquery): + mock_query = Mock() + mock_client = mock_bigquery.return_value + mock_client.run_sync_query.return_value = mock_query + mock_query.fetch_data.side_effect = ValueError('query job not executed') + + matcher = bq_verifier.BigqueryMatcher('mock_project', + 'mock_query', + 'mock_checksum') + with self.assertRaises(ValueError): + hc_assert_that(self._mock_result, matcher) + self.assertTrue(mock_query.fetch_data.called) + self.assertEqual(bq_verifier.MAX_RETRIES + 1, + mock_query.fetch_data.call_count) + + @patch.object(bigquery, 'Client') + def test_bigquery_matcher_query_responds_error_code(self, mock_bigquery): + mock_query = Mock() + mock_client = mock_bigquery.return_value + mock_client.run_sync_query.return_value = mock_query + mock_query.run.side_effect = NotFound('table is not found') + + matcher = bq_verifier.BigqueryMatcher('mock_project', + 'mock_query', + 'mock_checksum') + with self.assertRaises(NotFound): + hc_assert_that(self._mock_result, matcher) + self.assertTrue(mock_query.run.called) + self.assertEqual(bq_verifier.MAX_RETRIES + 1, mock_query.run.call_count) + + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + unittest.main() diff --git a/sdks/python/apache_beam/tests/pipeline_verifiers.py b/sdks/python/apache_beam/tests/pipeline_verifiers.py index 41dfc07c9aa7..379a96fa891f 100644 --- a/sdks/python/apache_beam/tests/pipeline_verifiers.py +++ b/sdks/python/apache_beam/tests/pipeline_verifiers.py @@ -22,13 +22,13 @@ `hamcrest.core.base_matcher.BaseMatcher` and override _matches. """ -import hashlib import logging from hamcrest.core.base_matcher import BaseMatcher from apache_beam.io.fileio import ChannelFactory from apache_beam.runners.runner import PipelineState +from apache_beam.tests import test_utils as utils from apache_beam.utils import retry try: @@ -76,7 +76,7 @@ class FileChecksumMatcher(BaseMatcher): """Matcher that verifies file(s) content by comparing file checksum. Use apache_beam.io.fileio to fetch file(s) from given path. File checksum - is a SHA-1 hash computed from content of file(s). + is a hash string computed from content of file(s). """ def __init__(self, file_path, expected_checksum): @@ -103,13 +103,9 @@ def _matches(self, _): read_lines = self._read_with_retry() # Compute checksum - read_lines.sort() - m = hashlib.new('sha1') - for line in read_lines: - m.update(line) - self.checksum, num_lines = (m.hexdigest(), len(read_lines)) + self.checksum = utils.compute_hash(read_lines) logging.info('Read from given path %s, %d lines, checksum: %s.', - self.file_path, num_lines, self.checksum) + self.file_path, len(read_lines), self.checksum) return self.checksum == self.expected_checksum def describe_to(self, description): diff --git a/sdks/python/apache_beam/tests/test_utils.py b/sdks/python/apache_beam/tests/test_utils.py index 3fdfe883453c..666207eb4d75 100644 --- a/sdks/python/apache_beam/tests/test_utils.py +++ b/sdks/python/apache_beam/tests/test_utils.py @@ -17,11 +17,23 @@ """Utility methods for testing""" +import hashlib import imp from mock import Mock, patch from apache_beam.utils import retry +DEFAULT_HASHING_ALG = 'sha1' + + +def compute_hash(content, hashing_alg=DEFAULT_HASHING_ALG): + """Compute a hash value from a list of string.""" + content.sort() + m = hashlib.new(hashing_alg) + for elem in content: + m.update(str(elem)) + return m.hexdigest() + def patch_retry(testcase, module): """A function to patch retry module to use mock clock and logger. diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 022d69d133a1..cf210d9392a6 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -87,6 +87,7 @@ def get_version(): 'avro>=1.7.7,<2.0.0', 'crcmod>=1.7,<2.0', 'dill==0.2.6', + 'google-cloud-bigquery>=0.22.1,<1.0.0', 'httplib2>=0.8,<0.10', 'mock>=1.0.1,<3.0.0', 'oauth2client>=2.0.1,<4.0.0', @@ -102,6 +103,8 @@ def get_version(): 'google-apitools>=0.5.6,<1.0.0', 'proto-google-cloud-datastore-v1==0.90.0', 'googledatastore==7.0.0', + # GCP packages required by tests + 'google-cloud-bigquery>=0.22.1,<0.23', ] From 97a7ae449e1ccf6b08a0ee0bc2fc0a1b49924f1f Mon Sep 17 00:00:00 2001 From: Chamikara Jayalath Date: Wed, 4 Jan 2017 19:10:09 -0800 Subject: [PATCH 16/46] Updates Python SDK source API so that sources can report limited parallelism signals. With this update Python BoundedSource/RangeTracker API can report consumed and remaining number of split points while performing a source read operations, similar to Java SDK sources. These signals can be used by runner implementations, for example, to perform scaling decisions. --- sdks/python/apache_beam/io/avroio.py | 33 +++- sdks/python/apache_beam/io/avroio_test.py | 31 ++++ sdks/python/apache_beam/io/iobase.py | 157 +++++++++++++++++- sdks/python/apache_beam/io/range_trackers.py | 41 ++++- .../apache_beam/io/range_trackers_test.py | 52 ++++++ sdks/python/apache_beam/io/textio.py | 17 +- sdks/python/apache_beam/io/textio_test.py | 13 ++ .../runners/dataflow/native_io/iobase.py | 13 +- 8 files changed, 342 insertions(+), 15 deletions(-) diff --git a/sdks/python/apache_beam/io/avroio.py b/sdks/python/apache_beam/io/avroio.py index 5dab65114ed8..ab985307ecd7 100644 --- a/sdks/python/apache_beam/io/avroio.py +++ b/sdks/python/apache_beam/io/avroio.py @@ -28,6 +28,7 @@ import apache_beam as beam from apache_beam.io import filebasedsource from apache_beam.io import fileio +from apache_beam.io import iobase from apache_beam.io.iobase import Read from apache_beam.transforms import PTransform @@ -135,6 +136,7 @@ def read_block_from_file(f, codec, schema, expected_sync_marker): ValueError: If the block cannot be read properly because the file doesn't match the specification. """ + offset = f.tell() decoder = avroio.BinaryDecoder(f) num_records = decoder.read_long() block_size = decoder.read_long() @@ -144,7 +146,8 @@ def read_block_from_file(f, codec, schema, expected_sync_marker): raise ValueError('Unexpected sync marker (actual "%s" vs expected "%s"). ' 'Maybe the underlying avro file is corrupted?', sync_marker, expected_sync_marker) - return _AvroBlock(block_bytes, num_records, codec, schema) + size = f.tell() - offset + return _AvroBlock(block_bytes, num_records, codec, schema, offset, size) @staticmethod def advance_file_past_next_sync_marker(f, sync_marker): @@ -172,13 +175,22 @@ def advance_file_past_next_sync_marker(f, sync_marker): class _AvroBlock(object): """Represents a block of an Avro file.""" - def __init__(self, block_bytes, num_records, codec, schema_string): + def __init__(self, block_bytes, num_records, codec, schema_string, + offset, size): # Decompress data early on (if needed) and thus decrease the number of # parallel copies of the data in memory at any given in time during # block iteration. self._decompressed_block_bytes = self._decompress_bytes(block_bytes, codec) self._num_records = num_records self._schema = schema.parse(schema_string) + self._offset = offset + self._size = size + + def size(self): + return self._size + + def offset(self): + return self._offset @staticmethod def _decompress_bytes(data, codec): @@ -232,12 +244,26 @@ class _AvroSource(filebasedsource.FileBasedSource): """ def read_records(self, file_name, range_tracker): + next_block_start = -1 + + def split_points_unclaimed(stop_position): + if next_block_start >= stop_position: + # Next block starts at or after the suggested stop position. Hence + # there will not be split points to be claimed for the range ending at + # suggested stop position. + return 0 + + return iobase.RangeTracker.SPLIT_POINTS_UNKNOWN + + range_tracker.set_split_points_unclaimed_callback(split_points_unclaimed) + start_offset = range_tracker.start_position() if start_offset is None: start_offset = 0 with self.open_file(file_name) as f: - codec, schema_string, sync_marker = _AvroUtils.read_meta_data_from_file(f) + codec, schema_string, sync_marker = _AvroUtils.read_meta_data_from_file( + f) # We have to start at current position if previous bundle ended at the # end of a sync marker. @@ -248,6 +274,7 @@ def read_records(self, file_name, range_tracker): while range_tracker.try_claim(f.tell()): block = _AvroUtils.read_block_from_file(f, codec, schema_string, sync_marker) + next_block_start = block.offset() + block.size() for record in block.records(): yield record diff --git a/sdks/python/apache_beam/io/avroio_test.py b/sdks/python/apache_beam/io/avroio_test.py index 233ab692f74b..8b14443926d9 100644 --- a/sdks/python/apache_beam/io/avroio_test.py +++ b/sdks/python/apache_beam/io/avroio_test.py @@ -22,6 +22,7 @@ import unittest import apache_beam as beam +from apache_beam.io import iobase from apache_beam.io import avroio from apache_beam.io import filebasedsource from apache_beam.io import source_test_utils @@ -248,6 +249,36 @@ def test_read_with_splitting_multiple_blocks(self): expected_result = self.RECORDS * 2000 self._run_avro_test(file_name, 10000, True, expected_result) + def test_split_points(self): + file_name = self._write_data(count=12000) + source = AvroSource(file_name) + + splits = [ + split + for split in source.split(desired_bundle_size=float('inf')) + ] + assert len(splits) == 1 + + range_tracker = splits[0].source.get_range_tracker( + splits[0].start_position, splits[0].stop_position) + + split_points_report = [] + + for _ in splits[0].source.read(range_tracker): + split_points_report.append(range_tracker.split_points()) + + # There are a total of three blocks. Each block has more than 10 records. + + # When reading records of the first block, range_tracker.split_points() + # should return (0, iobase.RangeTracker.SPLIT_POINTS_UNKNOWN) + self.assertEquals( + split_points_report[:10], + [(0, iobase.RangeTracker.SPLIT_POINTS_UNKNOWN)] * 10) + + # When reading records of last block, range_tracker.split_points() should + # return (2, 1) + self.assertEquals(split_points_report[-10:], [(2, 1)] * 10) + def test_read_without_splitting_compressed_deflate(self): file_name = self._write_data(codec='deflate') expected_result = self.RECORDS diff --git a/sdks/python/apache_beam/io/iobase.py b/sdks/python/apache_beam/io/iobase.py index e139a24f8654..bd40a3e78f16 100644 --- a/sdks/python/apache_beam/io/iobase.py +++ b/sdks/python/apache_beam/io/iobase.py @@ -300,6 +300,8 @@ class RangeTracker(object): the current reader and by a reader of the task starting at 43). """ + SPLIT_POINTS_UNKNOWN = object() + def start_position(self): """Returns the starting position of the current range, inclusive.""" raise NotImplementedError(type(self)) @@ -317,8 +319,8 @@ def try_claim(self, position): # pylint: disable=unused-argument ** Thread safety ** - This method along with several other methods of this class may be invoked by - multiple threads, hence must be made thread-safe, e.g. by using a single + Methods of the class ``RangeTracker`` including this method may get invoked + by different threads, hence must be made thread-safe, e.g. by using a single lock object. Args: @@ -352,8 +354,8 @@ def position_at_fraction(self, fraction): ** Thread safety ** - This method along with several other methods of this class may be invoked by - multiple threads, hence must be made thread-safe, e.g. by using a single + Methods of the class ``RangeTracker`` including this method may get invoked + by different threads, hence must be made thread-safe, e.g. by using a single lock object. Args: @@ -387,8 +389,8 @@ def try_split(self, position): ** Thread safety ** - This method along with several other methods of this class may be invoked by - multiple threads, hence must be made thread-safe, e.g. by using a single + Methods of the class ``RangeTracker`` including this method may get invoked + by different threads, hence must be made thread-safe, e.g. by using a single lock object. Args: @@ -405,8 +407,8 @@ def fraction_consumed(self): ** Thread safety ** - This method along with several other methods of this class may be invoked by - multiple threads, hence must be made thread-safe, e.g. by using a single + Methods of the class ``RangeTracker`` including this method may get invoked + by different threads, hence must be made thread-safe, e.g. by using a single lock object. Returns: @@ -416,6 +418,145 @@ def fraction_consumed(self): """ raise NotImplementedError + def split_points(self): + """Gives the number of split points consumed and remaining. + + For a ``RangeTracker`` used by a ``BoundedSource`` (within a + ``BoundedSource.read()`` invocation) this method produces a 2-tuple that + gives the number of split points consumed by the ``BoundedSource`` and the + number of split points remaining within the range of the ``RangeTracker`` + that has not been consumed by the ``BoundedSource``. + + More specifically, given that the position of the current record being read + by ``BoundedSource`` is current_position this method produces a tuple that + consists of + (1) number of split points in the range [self.start_position(), + current_position) without including the split point that is currently being + consumed. This represents the total amount of parallelism in the consumed + part of the source. + (2) number of split points within the range + [current_position, self.stop_position()) including the split point that is + currently being consumed. This represents the total amount of parallelism in + the unconsumed part of the source. + + Methods of the class ``RangeTracker`` including this method may get invoked + by different threads, hence must be made thread-safe, e.g. by using a single + lock object. + + ** General information about consumed and remaining number of split + points returned by this method. ** + + * Before a source read (``BoundedSource.read()`` invocation) claims the + first split point, number of consumed split points is 0. This condition + holds independent of whether the input is "splittable". A splittable + source is a source that has more than one split point. + * Any source read that has only claimed one split point has 0 consumed + split points since the first split point is the current split point and + is still being processed. This condition holds independent of whether + the input is splittable. + * For an empty source read which never invokes + ``RangeTracker.try_claim()``, the consumed number of split points is 0. + This condition holds independent of whether the input is splittable. + * For a source read which has invoked ``RangeTracker.try_claim()`` n + times, the consumed number of split points is n -1. + * If a ``BoundedSource`` sets a callback through function + ``set_split_points_unclaimed_callback()``, ``RangeTracker`` can use that + callback when determining remaining number of split points. + * Remaining split points should include the split point that is currently + being consumed by the source read. Hence if the above callback returns + an integer value n, remaining number of split points should be (n + 1). + * After last split point is claimed remaining split points becomes 1, + because this unfinished read itself represents an unfinished split + point. + * After all records of the source has been consumed, remaining number of + split points becomes 0 and consumed number of split points becomes equal + to the total number of split points within the range being read by the + source. This method does not address this condition and will continue to + report number of consumed split points as + ("total number of split points" - 1) and number of remaining split + points as 1. A runner that performs the reading of the source can + detect when all records have been consumed and adjust remaining and + consumed number of split points accordingly. + + ** Examples ** + + (1) A "perfectly splittable" input which can be read in parallel down to the + individual records. + + Consider a perfectly splittable input that consists of 50 split points. + + * Before a source read (``BoundedSource.read()`` invocation) claims the + first split point, number of consumed split points is 0 number of + remaining split points is 50. + * After claiming first split point, consumed number of split points is 0 + and remaining number of split is 50. + * After claiming split point #30, consumed number of split points is 29 + and remaining number of split points is 21. + * After claiming all 50 split points, consumed number of split points is + 49 and remaining number of split points is 1. + + (2) a "block-compressed" file format such as ``avroio``, in which a block of + records has to be read as a whole, but different blocks can be read in + parallel. + + Consider a block compressed input that consists of 5 blocks. + + * Before a source read (``BoundedSource.read()`` invocation) claims the + first split point (first block), number of consumed split points is 0 + number of remaining split points is 5. + * After claiming first split point, consumed number of split points is 0 + and remaining number of split is 5. + * After claiming split point #3, consumed number of split points is 2 + and remaining number of split points is 3. + * After claiming all 5 split points, consumed number of split points is + 4 and remaining number of split points is 1. + + (3) an "unsplittable" input such as a cursor in a database or a gzip + compressed file. + + Such an input is considered to have only a single split point. Number of + consumed split points is always 0 and number of remaining split points + is always 1. + + By default ``RangeTracker` returns ``RangeTracker.SPLIT_POINTS_UNKNOWN`` for + both consumed and remaining number of split points, which indicates that the + number of split points consumed and remaining is unknown. + + Returns: + A pair that gives consumed and remaining number of split points. Consumed + number of split points should be an integer larger than or equal to zero + or ``RangeTracker.SPLIT_POINTS_UNKNOWN``. Remaining number of split points + should be an integer larger than zero or + ``RangeTracker.SPLIT_POINTS_UNKNOWN``. + """ + return (RangeTracker.SPLIT_POINTS_UNKNOWN, + RangeTracker.SPLIT_POINTS_UNKNOWN) + + def set_split_points_unclaimed_callback(self, callback): + """Sets a callback for determining the unclaimed number of split points. + + By invoking this function, a ``BoundedSource`` can set a callback function + that may get invoked by the ``RangeTracker`` to determine the number of + unclaimed split points. A split point is unclaimed if + ``RangeTracker.try_claim()`` method has not been successfully invoked for + that particular split point. The callback function accepts a single + parameter, a stop position for the BoundedSource (stop_position). If the + record currently being consumed by the ``BoundedSource`` is at position + current_position, callback should return the number of split points within + the range (current_position, stop_position). Note that, this should not + include the split point that is currently being consumed by the source. + + This function must be implemented by subclasses before being used. + + Args: + callback: a function that takes a single parameter, a stop position, + and returns unclaimed number of split points for the source read + operation that is calling this function. Value returned from + callback should be either an integer larger than or equal to + zero or ``RangeTracker.SPLIT_POINTS_UNKNOWN``. + """ + raise NotImplementedError + class Sink(HasDisplayData): """A resource that can be written to using the ``df.io.Write`` transform. diff --git a/sdks/python/apache_beam/io/range_trackers.py b/sdks/python/apache_beam/io/range_trackers.py index 4c8f7eb14fda..059b6cae7395 100644 --- a/sdks/python/apache_beam/io/range_trackers.py +++ b/sdks/python/apache_beam/io/range_trackers.py @@ -55,6 +55,9 @@ def __init__(self, start, end): self._offset_of_last_split_point = -1 self._lock = threading.Lock() + self._split_points_seen = 0 + self._split_points_unclaimed_callback = None + def start_position(self): return self._start_offset @@ -106,6 +109,7 @@ def try_claim(self, record_start): return False self._offset_of_last_split_point = record_start self._last_record_start = record_start + self._split_points_seen += 1 return True def set_current_position(self, record_start): @@ -167,6 +171,24 @@ def position_at_fraction(self, fraction): return int(math.ceil(self.start_position() + fraction * ( self.stop_position() - self.start_position()))) + def split_points(self): + with self._lock: + split_points_consumed = ( + 0 if self._split_points_seen == 0 else self._split_points_seen - 1) + split_points_unclaimed = ( + self._split_points_unclaimed_callback(self.stop_position()) + if self._split_points_unclaimed_callback + else iobase.RangeTracker.SPLIT_POINTS_UNKNOWN) + split_points_remaining = ( + iobase.RangeTracker.SPLIT_POINTS_UNKNOWN if + split_points_unclaimed == iobase.RangeTracker.SPLIT_POINTS_UNKNOWN + else (split_points_unclaimed + 1)) + + return (split_points_consumed, split_points_remaining) + + def set_split_points_unclaimed_callback(self, callback): + self._split_points_unclaimed_callback = callback + class GroupedShuffleRangeTracker(iobase.RangeTracker): """A 'RangeTracker' for positions used by'GroupedShuffleReader'. @@ -184,6 +206,7 @@ def __init__(self, decoded_start_pos, decoded_stop_pos): self._decoded_stop_pos = decoded_stop_pos self._decoded_last_group_start = None self._last_group_was_at_a_split_point = False + self._split_points_seen = 0 self._lock = threading.Lock() def start_position(self): @@ -240,6 +263,7 @@ def try_claim(self, decoded_group_start): self._decoded_last_group_start = decoded_group_start self._last_group_was_at_a_split_point = True + self._split_points_seen += 1 return True def set_current_position(self, decoded_group_start): @@ -285,6 +309,14 @@ def fraction_consumed(self): ' consumed due to positions being opaque strings' ' that are interpreted by the service') + def split_points(self): + with self._lock: + splits_points_consumed = ( + 0 if self._split_points_seen <= 1 else (self._split_points_seen - 1)) + + return (splits_points_consumed, + iobase.RangeTracker.SPLIT_POINTS_UNKNOWN) + class OrderedPositionRangeTracker(iobase.RangeTracker): """ @@ -380,7 +412,7 @@ def __init__(self, range_tracker): range_tracker: a ``RangeTracker`` to which all method calls expect calls to ``try_split()`` will be delegated. """ - assert range_tracker + assert isinstance(range_tracker, iobase.RangeTracker) self._range_tracker = range_tracker def start_position(self): @@ -404,6 +436,13 @@ def set_current_position(self, position): def fraction_consumed(self): return self._range_tracker.fraction_consumed() + def split_points(self): + # An unsplittable range only contains a single split point. + return (0, 1) + + def set_split_points_unclaimed_callback(self, callback): + self._range_tracker.set_split_points_unclaimed_callback(callback) + class LexicographicKeyRangeTracker(OrderedPositionRangeTracker): """ diff --git a/sdks/python/apache_beam/io/range_trackers_test.py b/sdks/python/apache_beam/io/range_trackers_test.py index b80d1f3b07f0..edb6386379b2 100644 --- a/sdks/python/apache_beam/io/range_trackers_test.py +++ b/sdks/python/apache_beam/io/range_trackers_test.py @@ -24,6 +24,7 @@ import unittest +from apache_beam.io import iobase from apache_beam.io import range_trackers @@ -158,6 +159,35 @@ def test_try_return_record_non_monotonic(self): with self.assertRaises(Exception): tracker.try_claim(110) + def test_try_split_points(self): + tracker = range_trackers.OffsetRangeTracker(100, 400) + + def dummy_callback(stop_position): + return int(stop_position / 5) + + tracker.set_split_points_unclaimed_callback(dummy_callback) + + self.assertEqual(tracker.split_points(), + (0, 81)) + self.assertTrue(tracker.try_claim(120)) + self.assertEqual(tracker.split_points(), + (0, 81)) + self.assertTrue(tracker.try_claim(140)) + self.assertEqual(tracker.split_points(), + (1, 81)) + tracker.try_split(200) + self.assertEqual(tracker.split_points(), + (1, 41)) + self.assertTrue(tracker.try_claim(150)) + self.assertEqual(tracker.split_points(), + (2, 41)) + self.assertTrue(tracker.try_claim(180)) + self.assertEqual(tracker.split_points(), + (3, 41)) + self.assertFalse(tracker.try_claim(210)) + self.assertEqual(tracker.split_points(), + (3, 41)) + class GroupedShuffleRangeTrackerTest(unittest.TestCase): @@ -319,6 +349,28 @@ def test_try_test_split_at_position_finite_range(self): self.assertFalse(tracker.try_claim( self.bytes_to_position([3, 2, 1]))) + def test_split_points(self): + tracker = range_trackers.GroupedShuffleRangeTracker( + self.bytes_to_position([1, 0, 0]), + self.bytes_to_position([5, 0, 0])) + self.assertEqual(tracker.split_points(), + (0, iobase.RangeTracker.SPLIT_POINTS_UNKNOWN)) + self.assertTrue(tracker.try_claim(self.bytes_to_position([1, 2, 3]))) + self.assertEqual(tracker.split_points(), + (0, iobase.RangeTracker.SPLIT_POINTS_UNKNOWN)) + self.assertTrue(tracker.try_claim(self.bytes_to_position([1, 2, 5]))) + self.assertEqual(tracker.split_points(), + (1, iobase.RangeTracker.SPLIT_POINTS_UNKNOWN)) + self.assertTrue(tracker.try_claim(self.bytes_to_position([3, 6, 8]))) + self.assertEqual(tracker.split_points(), + (2, iobase.RangeTracker.SPLIT_POINTS_UNKNOWN)) + self.assertTrue(tracker.try_claim(self.bytes_to_position([4, 255, 255]))) + self.assertEqual(tracker.split_points(), + (3, iobase.RangeTracker.SPLIT_POINTS_UNKNOWN)) + self.assertFalse(tracker.try_claim(self.bytes_to_position([5, 1, 0]))) + self.assertEqual(tracker.split_points(), + (3, iobase.RangeTracker.SPLIT_POINTS_UNKNOWN)) + class OrderedPositionRangeTrackerTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/io/textio.py b/sdks/python/apache_beam/io/textio.py index 19980cb84229..5bb1a9dd4524 100644 --- a/sdks/python/apache_beam/io/textio.py +++ b/sdks/python/apache_beam/io/textio.py @@ -24,6 +24,7 @@ from apache_beam import coders from apache_beam.io import filebasedsource from apache_beam.io import fileio +from apache_beam.io import iobase from apache_beam.io.iobase import Read from apache_beam.io.iobase import Write from apache_beam.transforms import PTransform @@ -116,6 +117,14 @@ def read_records(self, file_name, range_tracker): start_offset = range_tracker.start_position() read_buffer = _TextSource.ReadBuffer('', 0) + next_record_start_position = -1 + + def split_points_unclaimed(stop_position): + return (0 if stop_position <= next_record_start_position + else iobase.RangeTracker.SPLIT_POINTS_UNKNOWN) + + range_tracker.set_split_points_unclaimed_callback(split_points_unclaimed) + with self.open_file(file_name) as file_to_read: position_after_skipping_header_lines = self._skip_lines( file_to_read, read_buffer, @@ -153,10 +162,14 @@ def read_records(self, file_name, range_tracker): if len(record) == 0 and num_bytes_to_next_record < 0: break + # Record separator must be larger than zero bytes. + assert num_bytes_to_next_record != 0 + if num_bytes_to_next_record > 0: + next_record_start_position += num_bytes_to_next_record + yield self._coder.decode(record) if num_bytes_to_next_record < 0: break - next_record_start_position += num_bytes_to_next_record def _find_separator_bounds(self, file_to_read, read_buffer): # Determines the start and end positions within 'read_buffer.data' of the @@ -220,7 +233,7 @@ def _skip_lines(self, file_to_read, read_buffer, num_lines): def _read_record(self, file_to_read, read_buffer): # Returns a tuple containing the current_record and number of bytes to the - # next record starting from 'self._next_position_in_buffer'. If EOF is + # next record starting from 'read_buffer.position'. If EOF is # reached, returns a tuple containing the current record and -1. if read_buffer.position > self._buffer_size: diff --git a/sdks/python/apache_beam/io/textio_test.py b/sdks/python/apache_beam/io/textio_test.py index f3ce8430e09f..04cf44cd0de5 100644 --- a/sdks/python/apache_beam/io/textio_test.py +++ b/sdks/python/apache_beam/io/textio_test.py @@ -27,6 +27,7 @@ import unittest import apache_beam as beam +from apache_beam.io import iobase import apache_beam.io.source_test_utils as source_test_utils # Importing following private classes for testing. @@ -265,13 +266,25 @@ def test_progress(self): splits = [split for split in source.split(desired_bundle_size=100000)] assert len(splits) == 1 fraction_consumed_report = [] + split_points_report = [] range_tracker = splits[0].source.get_range_tracker( splits[0].start_position, splits[0].stop_position) for _ in splits[0].source.read(range_tracker): fraction_consumed_report.append(range_tracker.fraction_consumed()) + split_points_report.append(range_tracker.split_points()) self.assertEqual( [float(i) / 10 for i in range(0, 10)], fraction_consumed_report) + expected_split_points_report = [ + ((i - 1), iobase.RangeTracker.SPLIT_POINTS_UNKNOWN) + for i in range(1, 10)] + + # At last split point, the remaining split points callback returns 1 since + # the expected position of next record becomes equal to the stop position. + expected_split_points_report.append((9, 1)) + + self.assertEqual( + expected_split_points_report, split_points_report) def test_read_reentrant_without_splitting(self): file_name, expected_data = write_data(10) diff --git a/sdks/python/apache_beam/runners/dataflow/native_io/iobase.py b/sdks/python/apache_beam/runners/dataflow/native_io/iobase.py index 529d4141fef8..26ebe08c435f 100644 --- a/sdks/python/apache_beam/runners/dataflow/native_io/iobase.py +++ b/sdks/python/apache_beam/runners/dataflow/native_io/iobase.py @@ -136,7 +136,8 @@ def request_dynamic_split(self, dynamic_split_request): class ReaderProgress(object): """A representation of how far a NativeSourceReader has read.""" - def __init__(self, position=None, percent_complete=None, remaining_time=None): + def __init__(self, position=None, percent_complete=None, remaining_time=None, + consumed_split_points=None, remaining_split_points=None): self._position = position @@ -149,6 +150,8 @@ def __init__(self, position=None, percent_complete=None, remaining_time=None): self._percent_complete = percent_complete self._remaining_time = remaining_time + self._consumed_split_points = consumed_split_points + self._remaining_split_points = remaining_split_points @property def position(self): @@ -172,6 +175,14 @@ def remaining_time(self): """Returns progress, represented as an estimated time remaining.""" return self._remaining_time + @property + def consumed_split_points(self): + return self._consumed_split_points + + @property + def remaining_split_points(self): + return self._remaining_split_points + class ReaderPosition(object): """A representation of position in an iteration of a 'NativeSourceReader'.""" From 131c9f916dae6345ec77a869112ae5901b568f23 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Wed, 1 Mar 2017 23:06:11 -0800 Subject: [PATCH 17/46] DataflowRunner: experimental support for issuing FnAPI based jobs Also cleanup some code around checking for existence of experiments. --- runners/google-cloud-dataflow-java/pom.xml | 3 +- .../dataflow/DataflowPipelineTranslator.java | 3 +- .../beam/runners/dataflow/DataflowRunner.java | 46 +++++++++++-------- .../runners/dataflow/DataflowRunnerInfo.java | 38 +++++++-------- .../options/DataflowPipelineDebugOptions.java | 2 + .../DataflowPipelineWorkerPoolOptions.java | 10 ++-- .../beam/runners/dataflow/dataflow.properties | 8 ++-- .../dataflow/DataflowRunnerInfoTest.java | 23 ++++++---- .../runners/dataflow/DataflowRunnerTest.java | 17 +++++++ 9 files changed, 92 insertions(+), 58 deletions(-) diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index fdd088f55ab1..fb067975d4ed 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -34,7 +34,8 @@ beam-master-20170228 - 6 + 1 + 6 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 7e559e9e4885..06e50483440d 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 @@ -327,8 +327,7 @@ public Job translate(List packages) { workerPool.setNumWorkers(options.getNumWorkers()); if (options.isStreaming() - && (options.getExperiments() == null - || !options.getExperiments().contains("enable_windmill_service"))) { + && !DataflowRunner.hasExperiment(options, "enable_windmill_service")) { // Use separate data disk for streaming. Disk disk = new Disk(); disk.setDiskType(options.getWorkerDiskType()); 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 dbf195861ad7..50b6b4fe5588 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 @@ -51,7 +51,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -307,14 +306,12 @@ public static DataflowRunner fromOptions(PipelineOptions options) { PTransformMatchers.parDoWithFnType(unsupported), UnsupportedOverrideFactory.withMessage(getUnsupportedMessage(unsupported, true))); } - if (options.getExperiments() == null - || !options.getExperiments().contains("enable_custom_pubsub_source")) { + if (!hasExperiment(options, "enable_custom_pubsub_source")) { ptoverrides.put( PTransformMatchers.classEqualTo(PubsubUnboundedSource.class), new ReflectiveRootOverrideFactory(StreamingPubsubIORead.class, this)); } - if (options.getExperiments() == null - || !options.getExperiments().contains("enable_custom_pubsub_sink")) { + if (!hasExperiment(options, "enable_custom_pubsub_sink")) { ptoverrides.put( PTransformMatchers.classEqualTo(PubsubUnboundedSink.class), new StreamingPubsubIOWriteOverrideFactory(this)); @@ -559,20 +556,7 @@ public DataflowPipelineJob run(Pipeline pipeline) { workerPool.setWorkerHarnessContainerImage(workerHarnessContainerImage); } - // Requirements about the service. - Map environmentVersion = new HashMap<>(); - environmentVersion.put( - PropertyNames.ENVIRONMENT_VERSION_MAJOR_KEY, - DataflowRunnerInfo.getDataflowRunnerInfo().getEnvironmentMajorVersion()); - newJob.getEnvironment().setVersion(environmentVersion); - // Default jobType is JAVA_BATCH_AUTOSCALING: A Java job with workers that the job can - // autoscale if specified. - String jobType = "JAVA_BATCH_AUTOSCALING"; - - if (options.isStreaming()) { - jobType = "STREAMING"; - } - environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_JOB_TYPE_KEY, jobType); + newJob.getEnvironment().setVersion(getEnvironmentVersion(options)); if (hooks != null) { hooks.modifyEnvironmentBeforeSubmission(newJob.getEnvironment()); @@ -680,6 +664,30 @@ public DataflowPipelineJob run(Pipeline pipeline) { return dataflowPipelineJob; } + /** Returns true if the specified experiment is enabled, handling null experiments. */ + public static boolean hasExperiment(DataflowPipelineDebugOptions options, String experiment) { + List experiments = + firstNonNull(options.getExperiments(), Collections.emptyList()); + return experiments.contains(experiment); + } + + /** Helper to configure the Dataflow Job Environment based on the user's job options. */ + private static Map getEnvironmentVersion(DataflowPipelineOptions options) { + DataflowRunnerInfo runnerInfo = DataflowRunnerInfo.getDataflowRunnerInfo(); + String majorVersion; + String jobType; + if (hasExperiment(options, "beam_fn_api")) { + majorVersion = runnerInfo.getFnApiEnvironmentMajorVersion(); + jobType = options.isStreaming() ? "FNAPI_STREAMING" : "FNAPI_BATCH"; + } else { + majorVersion = runnerInfo.getLegacyEnvironmentMajorVersion(); + jobType = options.isStreaming() ? "STREAMING" : "JAVA_BATCH_AUTOSCALING"; + } + return ImmutableMap.of( + PropertyNames.ENVIRONMENT_VERSION_MAJOR_KEY, majorVersion, + PropertyNames.ENVIRONMENT_VERSION_JOB_TYPE_KEY, jobType); + } + @VisibleForTesting void replaceTransforms(Pipeline pipeline) { for (Map.Entry override : overrides.entrySet()) { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerInfo.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerInfo.java index 59cb8a49de26..12b3f384cbc3 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerInfo.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerInfo.java @@ -47,32 +47,34 @@ public static DataflowRunnerInfo getDataflowRunnerInfo() { private Properties properties; - private static final String ENVIRONMENT_MAJOR_VERSION_KEY = "environment.major.version"; - private static final String BATCH_WORKER_HARNESS_CONTAINER_IMAGE_KEY = "worker.image.batch"; - private static final String STREAMING_WORKER_HARNESS_CONTAINER_IMAGE_KEY = - "worker.image.streaming"; + private static final String FNAPI_ENVIRONMENT_MAJOR_VERSION_KEY = + "fnapi.environment.major.version"; + private static final String LEGACY_ENVIRONMENT_MAJOR_VERSION_KEY = + "legacy.environment.major.version"; + private static final String CONTAINER_VERSION_KEY = "container.version"; - /** Provides the environment's major version number. */ - public String getEnvironmentMajorVersion() { + /** Provides the legacy environment's major version number. */ + public String getLegacyEnvironmentMajorVersion() { checkState( - properties.containsKey(ENVIRONMENT_MAJOR_VERSION_KEY), "Unknown environment major version"); - return properties.getProperty(ENVIRONMENT_MAJOR_VERSION_KEY); + properties.containsKey(LEGACY_ENVIRONMENT_MAJOR_VERSION_KEY), + "Unknown legacy environment major version"); + return properties.getProperty(LEGACY_ENVIRONMENT_MAJOR_VERSION_KEY); } - /** Provides the batch worker harness container image name. */ - public String getBatchWorkerHarnessContainerImage() { + /** Provides the FnAPI environment's major version number. */ + public String getFnApiEnvironmentMajorVersion() { checkState( - properties.containsKey(BATCH_WORKER_HARNESS_CONTAINER_IMAGE_KEY), - "Unknown batch worker harness container image"); - return properties.getProperty(BATCH_WORKER_HARNESS_CONTAINER_IMAGE_KEY); + properties.containsKey(FNAPI_ENVIRONMENT_MAJOR_VERSION_KEY), + "Unknown FnAPI environment major version"); + return properties.getProperty(FNAPI_ENVIRONMENT_MAJOR_VERSION_KEY); } - /** Provides the streaming worker harness container image name. */ - public String getStreamingWorkerHarnessContainerImage() { + /** Provides the container version that will be used for constructing harness image paths. */ + public String getContainerVersion() { checkState( - properties.containsKey(STREAMING_WORKER_HARNESS_CONTAINER_IMAGE_KEY), - "Unknown streaming worker harness container image"); - return properties.getProperty(STREAMING_WORKER_HARNESS_CONTAINER_IMAGE_KEY); + properties.containsKey(CONTAINER_VERSION_KEY), + "Unknown container version"); + return properties.getProperty(CONTAINER_VERSION_KEY); } private DataflowRunnerInfo(String resourcePath) { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java index cdfa3f59333a..729bca47b585 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java @@ -21,6 +21,7 @@ import com.google.api.services.dataflow.Dataflow; import java.util.List; import java.util.Map; +import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.util.DataflowTransport; import org.apache.beam.runners.dataflow.util.GcsStager; import org.apache.beam.runners.dataflow.util.Stager; @@ -53,6 +54,7 @@ public interface DataflowPipelineDebugOptions extends PipelineOptions { + "be enabled with this flag. Please sync with the Dataflow team before enabling any " + "experiments.") @Experimental + @Nullable List getExperiments(); void setExperiments(List value); diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java index 3c5d05afd1ae..e2c4bf4eece0 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java @@ -20,6 +20,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import java.util.List; import javax.annotation.Nullable; +import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.runners.dataflow.DataflowRunnerInfo; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.options.Default; @@ -129,11 +130,14 @@ class WorkerHarnessContainerImageFactory @Override public String create(PipelineOptions options) { DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); - if (dataflowOptions.isStreaming()) { - return DataflowRunnerInfo.getDataflowRunnerInfo().getStreamingWorkerHarnessContainerImage(); + String containerVersion = DataflowRunnerInfo.getDataflowRunnerInfo().getContainerVersion(); + String containerType; + if (DataflowRunner.hasExperiment(dataflowOptions, "beam_fn_api")) { + containerType = "java"; } else { - return DataflowRunnerInfo.getDataflowRunnerInfo().getBatchWorkerHarnessContainerImage(); + containerType = dataflowOptions.isStreaming() ? "beam-java-streaming" : "beam-java-batch"; } + return String.format("dataflow.gcr.io/v1beta3/%s:%s", containerType, containerVersion); } } diff --git a/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties b/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties index 47e316c26c42..ac689708012f 100644 --- a/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties +++ b/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties @@ -16,8 +16,6 @@ # # Dataflow runtime properties -environment.major.version=${dataflow.environment_major_version} - -worker.image.batch=dataflow.gcr.io/v1beta3/beam-java-batch:${dataflow.container_version} - -worker.image.streaming=dataflow.gcr.io/v1beta3/beam-java-streaming:${dataflow.container_version} +legacy.environment.major.version=${dataflow.legacy_environment_major_version} +fnapi.environment.major.version=${dataflow.fnapi_environment_major_version} +container.version=${dataflow.container_version} diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerInfoTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerInfoTest.java index 9b5b37497367..350204099ebc 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerInfoTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerInfoTest.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.dataflow; import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; @@ -32,20 +33,22 @@ public class DataflowRunnerInfoTest { public void getDataflowRunnerInfo() throws Exception { DataflowRunnerInfo info = DataflowRunnerInfo.getDataflowRunnerInfo(); - String version = info.getEnvironmentMajorVersion(); + String version = info.getLegacyEnvironmentMajorVersion(); // Validate major version is a number assertTrue( - String.format("Environment major version number %s is not a number", version), + String.format("Legacy environment major version number %s is not a number", version), version.matches("\\d+")); - // Validate container images contain gcr.io - assertThat( - "batch worker harness container image invalid", - info.getBatchWorkerHarnessContainerImage(), - containsString("gcr.io")); + version = info.getFnApiEnvironmentMajorVersion(); + // Validate major version is a number + assertTrue( + String.format("FnAPI environment major version number %s is not a number", version), + version.matches("\\d+")); + + // Validate container version does not contain a $ (indicating it was not filled in). assertThat( - "streaming worker harness container image invalid", - info.getStreamingWorkerHarnessContainerImage(), - containsString("gcr.io")); + "container version invalid", + info.getContainerVersion(), + not(containsString("$"))); } } 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 a7880777a7c4..246feb03d081 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 @@ -24,6 +24,7 @@ import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.startsWith; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; @@ -1118,4 +1119,20 @@ public void testTemplateRunnerLoggedErrorForFile() throws Exception { thrown.expect(RuntimeException.class); p.run(); } + + @Test + public void testHasExperiment() { + DataflowPipelineDebugOptions options = + PipelineOptionsFactory.as(DataflowPipelineDebugOptions.class); + + options.setExperiments(null); + assertFalse(DataflowRunner.hasExperiment(options, "foo")); + + options.setExperiments(ImmutableList.of("foo", "bar")); + assertTrue(DataflowRunner.hasExperiment(options, "foo")); + assertTrue(DataflowRunner.hasExperiment(options, "bar")); + assertFalse(DataflowRunner.hasExperiment(options, "baz")); + assertFalse(DataflowRunner.hasExperiment(options, "ba")); + assertFalse(DataflowRunner.hasExperiment(options, "BAR")); + } } From eb87ffc769df6ab410b90a599489034a83aede16 Mon Sep 17 00:00:00 2001 From: Stephen Sisk Date: Thu, 23 Feb 2017 14:01:18 -0800 Subject: [PATCH 18/46] [BEAM-1310] Add running integration tests in JdbcIO on Spark and Dataflow runners --- sdks/java/io/jdbc/pom.xml | 50 +++++++++++++++++++ .../org/apache/beam/sdk/io/jdbc/JdbcIOIT.java | 27 +++++----- sdks/java/io/pom.xml | 37 ++++++++++++++ 3 files changed, 102 insertions(+), 12 deletions(-) diff --git a/sdks/java/io/jdbc/pom.xml b/sdks/java/io/jdbc/pom.xml index afe428a0527b..fd5c52b6b339 100644 --- a/sdks/java/io/jdbc/pom.xml +++ b/sdks/java/io/jdbc/pom.xml @@ -30,6 +30,56 @@ Apache Beam :: SDKs :: Java :: IO :: JDBC IO to read and write on JDBC datasource. + + + + + + spark-runner + + + org.apache.beam + beam-runners-spark + runtime + + + org.apache.spark + spark-streaming_2.10 + ${spark.version} + runtime + + + org.apache.spark + spark-core_2.10 + ${spark.version} + runtime + + + org.slf4j + jul-to-slf4j + + + + + + + + + dataflow-runner + + + org.apache.beam + beam-runners-google-cloud-dataflow-java + runtime + + + + + org.apache.beam 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 15206c7ff827..a09db48c7b78 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 @@ -38,6 +38,7 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -50,16 +51,19 @@ *

This test requires a running instance of Postgres, and the test dataset must exist in the * database. `JdbcTestDataSet` will create the read table. * - *

You can run just this test by doing the following: + *

You can run this test by doing the following: *

- * mvn test-compile compile failsafe:integration-test -D beamTestPipelineOptions='[
- * "--postgresServerName=1.2.3.4",
- * "--postgresUsername=postgres",
- * "--postgresDatabaseName=myfancydb",
- * "--postgresPassword=yourpassword",
- * "--postgresSsl=false"
- * ]' -DskipITs=false -Dit.test=org.apache.beam.sdk.io.jdbc.JdbcIOIT -DfailIfNoTests=false
+ *  mvn -e -Pio-it verify -pl sdks/java/io/jdbc -DintegrationTestPipelineOptions='[
+ *  "--postgresServerName=1.2.3.4",
+ *  "--postgresUsername=postgres",
+ *  "--postgresDatabaseName=myfancydb",
+ *  "--postgresPassword=mypass",
+ *  "--postgresSsl=false" ]'
  * 
+ * + *

If you want to run this with a runner besides directrunner, there are profiles for dataflow + * and spark in the jdbc pom. You'll want to activate those in addition to the normal test runner + * invocation pipeline options. */ @RunWith(JUnit4.class) public class JdbcIOIT { @@ -103,6 +107,9 @@ public void setParameters(KV element, PreparedStatement stateme } } + @Rule + public TestPipeline pipeline = TestPipeline.create(); + /** * Does a test read of a few rows from a postgres database. * @@ -113,8 +120,6 @@ public void setParameters(KV element, PreparedStatement stateme public void testRead() throws SQLException { String tableName = JdbcTestDataSet.READ_TABLE_NAME; - TestPipeline pipeline = TestPipeline.create(); - PCollection> output = pipeline.apply(JdbcIO.>read() .withDataSourceConfiguration(JdbcIO.DataSourceConfiguration.create(dataSource)) .withQuery("select name,id from " + tableName) @@ -148,8 +153,6 @@ public void testRead() throws SQLException { public void testWrite() throws SQLException { writeTableName = JdbcTestDataSet.createWriteDataTable(dataSource); - TestPipeline pipeline = TestPipeline.create(); - ArrayList> data = new ArrayList<>(); for (int i = 0; i < 1000; i++) { KV kv = KV.of(i, "Test"); diff --git a/sdks/java/io/pom.xml b/sdks/java/io/pom.xml index 082f26b027e7..8d5b69b359e8 100644 --- a/sdks/java/io/pom.xml +++ b/sdks/java/io/pom.xml @@ -77,4 +77,41 @@ mqtt + + + io-it + + io-it + + + + + org.apache.maven.plugins + maven-failsafe-plugin + + + + integration-test + verify + + + false + ${skipDefaultIT} + all + 4 + + ${integrationTestPipelineOptions} + + + + + + + + + false + + + + From 7b94c99be43d82bcab9370f63c0d63646146ca97 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 3 Mar 2017 10:56:29 -0800 Subject: [PATCH 19/46] Only Override CreatePCollectionView in Streaming This permits us to use the appropriate view token for the StreamingPCollectionViewWriterFn. --- .../beam/runners/dataflow/DataflowRunner.java | 92 +++--- .../dataflow/StreamingViewOverrides.java | 287 +++--------------- 2 files changed, 76 insertions(+), 303 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 50b6b4fe5588..c609b540035f 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 @@ -64,6 +64,7 @@ import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; import org.apache.beam.runners.core.construction.UnsupportedOverrideFactory; import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.JobSpecification; +import org.apache.beam.runners.dataflow.StreamingViewOverrides.StreamingCreatePCollectionViewFactory; import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.runners.dataflow.options.DataflowPipelineWorkerPoolOptions; @@ -75,6 +76,8 @@ import org.apache.beam.sdk.PipelineResult.State; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.Coder.NonDeterministicException; +import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.FileBasedSink; @@ -94,18 +97,12 @@ import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.sdk.transforms.Combine.GloballyAsSingletonView; import org.apache.beam.sdk.transforms.Combine.GroupedValues; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.View; -import org.apache.beam.sdk.transforms.View.AsIterable; -import org.apache.beam.sdk.transforms.View.AsList; -import org.apache.beam.sdk.transforms.View.AsMap; -import org.apache.beam.sdk.transforms.View.AsMultimap; -import org.apache.beam.sdk.transforms.View.AsSingleton; import org.apache.beam.sdk.transforms.WithKeys; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -326,29 +323,8 @@ public static DataflowRunner fromOptions(PipelineOptions options) { PTransformMatchers.classEqualTo(Read.Unbounded.class), new ReflectiveRootOverrideFactory(StreamingUnboundedRead.class, this)) .put( - PTransformMatchers.classEqualTo(GloballyAsSingletonView.class), - new ReflectiveOneToOneOverrideFactory( - StreamingViewOverrides.StreamingCombineGloballyAsSingletonView.class, this)) - .put( - PTransformMatchers.classEqualTo(AsMap.class), - new ReflectiveOneToOneOverrideFactory( - StreamingViewOverrides.StreamingViewAsMap.class, this)) - .put( - PTransformMatchers.classEqualTo(AsMultimap.class), - new ReflectiveOneToOneOverrideFactory( - StreamingViewOverrides.StreamingViewAsMultimap.class, this)) - .put( - PTransformMatchers.classEqualTo(AsSingleton.class), - new ReflectiveOneToOneOverrideFactory( - StreamingViewOverrides.StreamingViewAsSingleton.class, this)) - .put( - PTransformMatchers.classEqualTo(AsList.class), - new ReflectiveOneToOneOverrideFactory( - StreamingViewOverrides.StreamingViewAsList.class, this)) - .put( - PTransformMatchers.classEqualTo(AsIterable.class), - new ReflectiveOneToOneOverrideFactory( - StreamingViewOverrides.StreamingViewAsIterable.class, this)); + PTransformMatchers.classEqualTo(View.CreatePCollectionView.class), + new StreamingCreatePCollectionViewFactory()); } else { // In batch mode must use the custom Pubsub bounded source/sink. for (Class unsupported : @@ -719,30 +695,40 @@ private void logWarningIfPCollectionViewHasNonDeterministicKeyCoder(Pipeline pip // have just recorded the full names during apply time. if (!ptransformViewsWithNonDeterministicKeyCoders.isEmpty()) { final SortedSet ptransformViewNamesWithNonDeterministicKeyCoders = new TreeSet<>(); - pipeline.traverseTopologically(new PipelineVisitor() { - @Override - public void visitValue(PValue value, TransformHierarchy.Node producer) { - } - - @Override - public void visitPrimitiveTransform(TransformHierarchy.Node node) { - if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) { - ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName()); - } - } - - @Override - public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { - if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) { - ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName()); - } - return CompositeBehavior.ENTER_TRANSFORM; - } - - @Override - public void leaveCompositeTransform(TransformHierarchy.Node node) { - } - }); + pipeline.traverseTopologically( + new PipelineVisitor() { + @Override + public void visitValue(PValue value, TransformHierarchy.Node producer) {} + + @Override + public void visitPrimitiveTransform(TransformHierarchy.Node node) { + if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) { + ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName()); + } + } + + @Override + public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { + if (node.getTransform() instanceof View.AsMap + || node.getTransform() instanceof View.AsMultimap) { + PCollection> input = + (PCollection>) Iterables.getOnlyElement(node.getInputs()).getValue(); + KvCoder inputCoder = (KvCoder) input.getCoder(); + try { + inputCoder.getKeyCoder().verifyDeterministic(); + } catch (NonDeterministicException e) { + ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName()); + } + } + if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) { + ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName()); + } + return CompositeBehavior.ENTER_TRANSFORM; + } + + @Override + public void leaveCompositeTransform(TransformHierarchy.Node node) {} + }); LOG.warn("Unable to use indexed implementation for View.AsMap and View.AsMultimap for {} " + "because the key coder is not deterministic. Falling back to singleton implementation " 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 bab115f4f6d6..8e005cfb574d 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 @@ -19,23 +19,18 @@ package org.apache.beam.runners.dataflow; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; -import java.util.Map; +import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; import org.apache.beam.runners.dataflow.DataflowRunner.StreamingPCollectionViewWriterFn; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.Coder.NonDeterministicException; import org.apache.beam.sdk.coders.CoderRegistry; -import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Combine.CombineFn; -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.View; -import org.apache.beam.sdk.util.PCollectionViews; -import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.transforms.View.CreatePCollectionView; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; @@ -44,261 +39,58 @@ * types. */ class StreamingViewOverrides { - static class StreamingCombineGloballyAsSingletonView - extends PTransform, PCollectionView> { - Combine.GloballyAsSingletonView transform; - - /** - * Builds an instance of this class from the overridden transform. - */ - @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply() - public StreamingCombineGloballyAsSingletonView( - DataflowRunner runner, - Combine.GloballyAsSingletonView transform) { - this.transform = transform; - } - + static class StreamingCreatePCollectionViewFactory + extends SingleInputOutputOverrideFactory< + PCollection, PCollectionView, CreatePCollectionView> { @Override - public PCollectionView expand(PCollection input) { - PCollection combined = - input.apply(Combine.globally(transform.getCombineFn()) - .withoutDefaults() - .withFanout(transform.getFanout())); - - PCollectionView view = PCollectionViews.singletonView( - combined.getPipeline(), - combined.getWindowingStrategy(), - transform.getInsertDefault(), - transform.getInsertDefault() - ? transform.getCombineFn().defaultValue() : null, - combined.getCoder()); - return combined - .apply(ParDo.of(new WrapAsList())) - .apply(ParDo.of(StreamingPCollectionViewWriterFn.create(view, combined.getCoder()))) - .apply(View.CreatePCollectionView.of(view)); - } - - @Override - protected String getKindString() { - return "StreamingCombineGloballyAsSingletonView"; - } - } - - private static class WrapAsList extends DoFn> { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(Arrays.asList(c.element())); + public PTransform, PCollectionView> getReplacementTransform( + final CreatePCollectionView transform) { + return new StreamingCreatePCollectionView<>(transform.getView()); } - } - /** - * Specialized implementation for - * {@link org.apache.beam.sdk.transforms.View.AsMap View.AsMap} - * for the Dataflow runner in streaming mode. - */ - static class StreamingViewAsMap - extends PTransform>, PCollectionView>> { - private final DataflowRunner runner; + private static class StreamingCreatePCollectionView + extends PTransform, PCollectionView> { + private final PCollectionView view; - @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply() - public StreamingViewAsMap(DataflowRunner runner, View.AsMap transform) { - this.runner = runner; - } - - @Override - public PCollectionView> expand(PCollection> input) { - PCollectionView> view = - PCollectionViews.mapView( - input.getPipeline(), - input.getWindowingStrategy(), - input.getCoder()); - - @SuppressWarnings({"rawtypes", "unchecked"}) - KvCoder inputCoder = (KvCoder) input.getCoder(); - try { - inputCoder.getKeyCoder().verifyDeterministic(); - } catch (NonDeterministicException e) { - runner.recordViewUsesNonDeterministicKeyCoder(this); + private StreamingCreatePCollectionView(PCollectionView view) { + this.view = view; } - return input - .apply(Combine.globally(new Concatenate>()).withoutDefaults()) - .apply(ParDo.of(StreamingPCollectionViewWriterFn.create(view, input.getCoder()))) - .apply(View.CreatePCollectionView., Map>of(view)); - } - - @Override - protected String getKindString() { - return "StreamingViewAsMap"; - } - } - - /** - * Specialized expansion for {@link - * org.apache.beam.sdk.transforms.View.AsMultimap View.AsMultimap} for the - * Dataflow runner in streaming mode. - */ - static class StreamingViewAsMultimap - extends PTransform>, PCollectionView>>> { - private final DataflowRunner runner; - - /** - * Builds an instance of this class from the overridden transform. - */ - @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply() - public StreamingViewAsMultimap(DataflowRunner runner, View.AsMultimap transform) { - this.runner = runner; - } - - @Override - public PCollectionView>> expand(PCollection> input) { - PCollectionView>> view = - PCollectionViews.multimapView( - input.getPipeline(), - input.getWindowingStrategy(), - input.getCoder()); - - @SuppressWarnings({"rawtypes", "unchecked"}) - KvCoder inputCoder = (KvCoder) input.getCoder(); - try { - inputCoder.getKeyCoder().verifyDeterministic(); - } catch (NonDeterministicException e) { - runner.recordViewUsesNonDeterministicKeyCoder(this); + @Override + public PCollectionView expand(PCollection input) { + return input + .apply(Combine.globally(new Concatenate()).withoutDefaults()) + .apply(ParDo.of(StreamingPCollectionViewWriterFn.create(view, input.getCoder()))) + .apply(View.CreatePCollectionView.of(view)); } - - return input - .apply(Combine.globally(new Concatenate>()).withoutDefaults()) - .apply(ParDo.of(StreamingPCollectionViewWriterFn.create(view, input.getCoder()))) - .apply(View.CreatePCollectionView., Map>>of(view)); - } - - @Override - protected String getKindString() { - return "StreamingViewAsMultimap"; } } - /** - * Specialized implementation for - * {@link org.apache.beam.sdk.transforms.View.AsList View.AsList} for the - * Dataflow runner in streaming mode. - */ - static class StreamingViewAsList - extends PTransform, PCollectionView>> { - /** - * Builds an instance of this class from the overridden transform. - */ - @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply() - public StreamingViewAsList(DataflowRunner runner, View.AsList transform) {} + private static class SingletonCombine extends Combine.BinaryCombineFn { + private boolean hasDefaultValue; + private T defaultValue; - @Override - public PCollectionView> expand(PCollection input) { - PCollectionView> view = - PCollectionViews.listView( - input.getPipeline(), - input.getWindowingStrategy(), - input.getCoder()); - - return input.apply(Combine.globally(new Concatenate()).withoutDefaults()) - .apply(ParDo.of(StreamingPCollectionViewWriterFn.create(view, input.getCoder()))) - .apply(View.CreatePCollectionView.>of(view)); + SingletonCombine(boolean hasDefaultValue, T defaultValue) { + this.hasDefaultValue = hasDefaultValue; + this.defaultValue = defaultValue; } @Override - protected String getKindString() { - return "StreamingViewAsList"; - } - } - - /** - * Specialized implementation for - * {@link org.apache.beam.sdk.transforms.View.AsIterable View.AsIterable} for the - * Dataflow runner in streaming mode. - */ - static class StreamingViewAsIterable - extends PTransform, PCollectionView>> { - /** - * Builds an instance of this class from the overridden transform. - */ - @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply() - public StreamingViewAsIterable(DataflowRunner runner, View.AsIterable transform) { } - - @Override - public PCollectionView> expand(PCollection input) { - PCollectionView> view = - PCollectionViews.iterableView( - input.getPipeline(), - input.getWindowingStrategy(), - input.getCoder()); - - return input.apply(Combine.globally(new Concatenate()).withoutDefaults()) - .apply(ParDo.of(StreamingPCollectionViewWriterFn.create(view, input.getCoder()))) - .apply(View.CreatePCollectionView.>of(view)); + public T apply(T left, T right) { + throw new IllegalArgumentException( + "PCollection with more than one element " + + "accessed as a singleton view. Consider using Combine.globally().asSingleton() to " + + "combine the PCollection into a single value"); } @Override - protected String getKindString() { - return "StreamingViewAsIterable"; - } - } - - /** - * Specialized expansion for - * {@link org.apache.beam.sdk.transforms.View.AsSingleton View.AsSingleton} for the - * Dataflow runner in streaming mode. - */ - static class StreamingViewAsSingleton - extends PTransform, PCollectionView> { - private View.AsSingleton transform; - - /** - * Builds an instance of this class from the overridden transform. - */ - @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply() - public StreamingViewAsSingleton(DataflowRunner runner, View.AsSingleton transform) { - this.transform = transform; - } - - @Override - public PCollectionView expand(PCollection input) { - Combine.Globally combine = Combine.globally( - new SingletonCombine<>(transform.hasDefaultValue(), transform.defaultValue())); - if (!transform.hasDefaultValue()) { - combine = combine.withoutDefaults(); - } - return input.apply(combine.asSingletonView()); - } - - @Override - protected String getKindString() { - return "StreamingViewAsSingleton"; - } - - private static class SingletonCombine extends Combine.BinaryCombineFn { - private boolean hasDefaultValue; - private T defaultValue; - - SingletonCombine(boolean hasDefaultValue, T defaultValue) { - this.hasDefaultValue = hasDefaultValue; - this.defaultValue = defaultValue; - } - - @Override - public T apply(T left, T right) { - throw new IllegalArgumentException("PCollection with more than one element " - + "accessed as a singleton view. Consider using Combine.globally().asSingleton() to " - + "combine the PCollection into a single value"); - } - - @Override - public T identity() { - if (hasDefaultValue) { - return defaultValue; - } else { - throw new IllegalArgumentException( - "Empty PCollection accessed as a singleton view. " - + "Consider setting withDefault to provide a default value"); - } + public T identity() { + if (hasDefaultValue) { + return defaultValue; + } else { + throw new IllegalArgumentException( + "Empty PCollection accessed as a singleton view. " + + "Consider setting withDefault to provide a default value"); } } } @@ -306,11 +98,6 @@ public T identity() { /** * Combiner that combines {@code T}s into a single {@code List} containing all inputs. * - *

For internal use by {@link StreamingViewAsMap}, {@link StreamingViewAsMultimap}, - * {@link StreamingViewAsList}, {@link StreamingViewAsIterable}. - * They require the input {@link PCollection} fits in memory. - * For a large {@link PCollection} this is expected to crash! - * * @param the type of elements to concatenate. */ private static class Concatenate extends CombineFn, List> { From 079966cad99442c63e0f3147a6361139bd601c8c Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 3 Mar 2017 10:57:50 -0800 Subject: [PATCH 20/46] Remove SingletonCombine It is unused with the update to Streaming View Overrides. --- .../dataflow/StreamingViewOverrides.java | 29 ------------------- 1 file changed, 29 deletions(-) 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 8e005cfb574d..5f0cb26985b1 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 @@ -66,35 +66,6 @@ public PCollectionView expand(PCollection input) { } } - private static class SingletonCombine extends Combine.BinaryCombineFn { - private boolean hasDefaultValue; - private T defaultValue; - - SingletonCombine(boolean hasDefaultValue, T defaultValue) { - this.hasDefaultValue = hasDefaultValue; - this.defaultValue = defaultValue; - } - - @Override - public T apply(T left, T right) { - throw new IllegalArgumentException( - "PCollection with more than one element " - + "accessed as a singleton view. Consider using Combine.globally().asSingleton() to " - + "combine the PCollection into a single value"); - } - - @Override - public T identity() { - if (hasDefaultValue) { - return defaultValue; - } else { - throw new IllegalArgumentException( - "Empty PCollection accessed as a singleton view. " - + "Consider setting withDefault to provide a default value"); - } - } - } - /** * Combiner that combines {@code T}s into a single {@code List} containing all inputs. * From 3de44a348e3e0934c644c718255a43b8f42a3534 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 3 Mar 2017 11:24:14 -0800 Subject: [PATCH 21/46] Add a Test for windowed CombineGloballyAsSingletonView --- .../beam/sdk/transforms/CombineTest.java | 46 +++++++++++++++++++ 1 file changed, 46 insertions(+) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java index 5b18384d6b5d..6c62d0bd19ee 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java @@ -53,6 +53,7 @@ import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; @@ -75,8 +76,10 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.TimestampedValue; import org.hamcrest.Matchers; import org.joda.time.Duration; +import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -626,6 +629,49 @@ public void processElement(ProcessContext c) { pipeline.run(); } + @Test + @Category(RunnableOnService.class) + public void testWindowedCombineGloballyAsSingletonView() { + FixedWindows windowFn = FixedWindows.of(Duration.standardMinutes(1)); + final PCollectionView view = + pipeline + .apply( + "CreateSideInput", + Create.timestamped( + TimestampedValue.of(1, new Instant(100)), + TimestampedValue.of(3, new Instant(100)))) + .apply("WindowSideInput", Window.into(windowFn)) + .apply("CombineSideInput", Sum.integersGlobally().asSingletonView()); + + TimestampedValue nonEmptyElement = TimestampedValue.of(null, new Instant(100)); + TimestampedValue emptyElement = TimestampedValue.atMinimumTimestamp(null); + PCollection output = + pipeline + .apply( + "CreateMainInput", + Create.timestamped(nonEmptyElement, emptyElement).withCoder(VoidCoder.of())) + .apply("WindowMainInput", Window.into(windowFn)) + .apply( + "OutputSideInput", + ParDo.of( + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c) { + c.output(c.sideInput(view)); + } + }) + .withSideInputs(view)); + + PAssert.that(output).containsInAnyOrder(4, 0); + PAssert.that(output) + .inWindow(windowFn.assignWindow(nonEmptyElement.getTimestamp())) + .containsInAnyOrder(4); + PAssert.that(output) + .inWindow(windowFn.assignWindow(emptyElement.getTimestamp())) + .containsInAnyOrder(0); + pipeline.run(); + } + @Test public void testCombineGetName() { assertEquals("Combine.globally(SumInts)", Combine.globally(new SumInts()).getName()); From 6253abaac62979e8496a828c18c7d1aa7214be6a Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 2 Mar 2017 14:27:29 -0800 Subject: [PATCH 22/46] Implement Single-Output ParDo as a composite This reduces the number of primitive transforms in the Java SDK. There is no functional change for any pipeline as a result of this change. --- .../translation/ApexPipelineTranslator.java | 3 +- .../translation/ParDoBoundTranslator.java | 95 --------------- ...tiTranslator.java => ParDoTranslator.java} | 4 +- .../FlattenPCollectionTranslatorTest.java | 3 +- ...atorTest.java => ParDoTranslatorTest.java} | 8 +- .../beam/runners/direct/DirectRunner.java | 18 +-- .../ParDoSingleViaMultiOverrideFactory.java | 70 ----------- ...arDoSingleViaMultiOverrideFactoryTest.java | 46 ------- .../flink/FlinkBatchTransformTranslators.java | 78 +----------- .../FlinkStreamingTransformTranslators.java | 113 +---------------- .../dataflow/DataflowPipelineTranslator.java | 29 ----- .../DataflowPipelineTranslatorTest.java | 7 +- .../translation/TransformTranslator.java | 100 ++++++++------- .../StreamingTransformTranslator.java | 115 ++++++++++-------- .../streaming/TrackStreamingSourcesTest.java | 4 +- .../org/apache/beam/sdk/transforms/ParDo.java | 8 +- 16 files changed, 145 insertions(+), 556 deletions(-) delete mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java rename runners/apex/src/main/java/org/apache/beam/runners/apex/translation/{ParDoBoundMultiTranslator.java => ParDoTranslator.java} (99%) rename runners/apex/src/test/java/org/apache/beam/runners/apex/translation/{ParDoBoundTranslatorTest.java => ParDoTranslatorTest.java} (98%) delete mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java delete mode 100644 runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactoryTest.java diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java index 951a286fb3af..7eb955123cbe 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java @@ -59,8 +59,7 @@ public class ApexPipelineTranslator implements Pipeline.PipelineVisitor { static { // register TransformTranslators - registerTransformTranslator(ParDo.Bound.class, new ParDoBoundTranslator()); - registerTransformTranslator(ParDo.BoundMulti.class, new ParDoBoundMultiTranslator<>()); + registerTransformTranslator(ParDo.BoundMulti.class, new ParDoTranslator<>()); registerTransformTranslator(Read.Unbounded.class, new ReadUnboundedTranslator()); registerTransformTranslator(Read.Bounded.class, new ReadBoundedTranslator()); registerTransformTranslator(GroupByKey.class, new GroupByKeyTranslator()); diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java deleted file mode 100644 index 5195809bdbbf..000000000000 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java +++ /dev/null @@ -1,95 +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.apex.translation; - -import java.util.List; -import org.apache.beam.runners.apex.ApexRunner; -import org.apache.beam.runners.apex.translation.operators.ApexParDoOperator; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.reflect.DoFnSignature; -import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TupleTagList; - -/** {@link ParDo.Bound} is translated to {link ApexParDoOperator} that wraps the {@link DoFn}. */ -class ParDoBoundTranslator - implements TransformTranslator> { - private static final long serialVersionUID = 1L; - - @Override - public void translate(ParDo.Bound transform, TranslationContext context) { - DoFn doFn = transform.getFn(); - DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass()); - - if (signature.processElement().isSplittable()) { - throw new UnsupportedOperationException( - String.format( - "%s does not support splittable DoFn: %s", ApexRunner.class.getSimpleName(), doFn)); - } - if (signature.stateDeclarations().size() > 0) { - throw new UnsupportedOperationException( - String.format( - "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", - DoFn.StateId.class.getSimpleName(), - doFn.getClass().getName(), - DoFn.class.getSimpleName(), - ApexRunner.class.getSimpleName())); - } - - if (signature.timerDeclarations().size() > 0) { - throw new UnsupportedOperationException( - String.format( - "Found %s annotations on %s, but %s cannot yet be used with timers in the %s.", - DoFn.TimerId.class.getSimpleName(), - doFn.getClass().getName(), - DoFn.class.getSimpleName(), - ApexRunner.class.getSimpleName())); - } - - PCollection output = (PCollection) context.getOutput(); - PCollection input = (PCollection) context.getInput(); - List> sideInputs = transform.getSideInputs(); - Coder inputCoder = input.getCoder(); - WindowedValueCoder wvInputCoder = - FullWindowedValueCoder.of( - inputCoder, input.getWindowingStrategy().getWindowFn().windowCoder()); - - ApexParDoOperator operator = - new ApexParDoOperator<>( - context.getPipelineOptions(), - doFn, - new TupleTag(), - TupleTagList.empty().getAll() /*sideOutputTags*/, - output.getWindowingStrategy(), - sideInputs, - wvInputCoder, - context.stateInternalsFactory()); - context.addOperator(operator, operator.output); - context.addStream(context.getInput(), operator.input); - if (!sideInputs.isEmpty()) { - ParDoBoundMultiTranslator.addSideInputs(operator, sideInputs, context); - } - } -} diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoTranslator.java similarity index 99% rename from runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java rename to runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoTranslator.java index f55b48cd7253..5ffc3c389a68 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoTranslator.java @@ -46,10 +46,10 @@ /** * {@link ParDo.BoundMulti} is translated to {@link ApexParDoOperator} that wraps the {@link DoFn}. */ -class ParDoBoundMultiTranslator +class ParDoTranslator implements TransformTranslator> { private static final long serialVersionUID = 1L; - private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundMultiTranslator.class); + private static final Logger LOG = LoggerFactory.getLogger(ParDoTranslator.class); @Override public void translate(ParDo.BoundMulti transform, TranslationContext context) { diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/FlattenPCollectionTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/FlattenPCollectionTranslatorTest.java index b2e29b6de085..64ca0ee4fd07 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/FlattenPCollectionTranslatorTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/FlattenPCollectionTranslatorTest.java @@ -110,7 +110,8 @@ public void testFlattenSingleCollection() { PCollectionList.of(single).apply(Flatten.pCollections()) .apply(ParDo.of(new EmbeddedCollector())); translator.translate(p, dag); - Assert.assertNotNull(dag.getOperatorMeta("ParDo(EmbeddedCollector)")); + Assert.assertNotNull( + dag.getOperatorMeta("ParDo(EmbeddedCollector)/ParMultiDo(EmbeddedCollector)")); } } diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoTranslatorTest.java similarity index 98% rename from runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslatorTest.java rename to runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoTranslatorTest.java index 2aa07208cb09..83e68f7822d4 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslatorTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoTranslatorTest.java @@ -68,11 +68,11 @@ import org.slf4j.LoggerFactory; /** - * integration test for {@link ParDoBoundTranslator}. + * integration test for {@link ParDoTranslator}. */ @RunWith(JUnit4.class) -public class ParDoBoundTranslatorTest { - private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundTranslatorTest.class); +public class ParDoTranslatorTest { + private static final Logger LOG = LoggerFactory.getLogger(ParDoTranslatorTest.class); private static final long SLEEP_MILLIS = 500; private static final long TIMEOUT_MILLIS = 30000; @@ -98,7 +98,7 @@ public void test() throws Exception { Assert.assertNotNull(om); Assert.assertEquals(om.getOperator().getClass(), ApexReadUnboundedInputOperator.class); - om = dag.getOperatorMeta("ParDo(Add)"); + om = dag.getOperatorMeta("ParDo(Add)/ParMultiDo(Add)"); Assert.assertNotNull(om); Assert.assertEquals(om.getOperator().getClass(), ApexParDoOperator.class); 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 f56d225f1f1b..4601262ef261 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 @@ -89,24 +89,10 @@ public class DirectRunner extends PipelineRunner { .put( PTransformMatchers.classEqualTo(TestStream.class), new DirectTestStreamFactory()) /* primitive */ - /* Single-output ParDos are implemented in terms of Multi-output ParDos. Any override - that is applied to a multi-output ParDo must first have all matching Single-output ParDos - converted to match. - */ - .put(PTransformMatchers.splittableParDoSingle(), new ParDoSingleViaMultiOverrideFactory()) - .put( - PTransformMatchers.stateOrTimerParDoSingle(), - new ParDoSingleViaMultiOverrideFactory()) - // SplittableParMultiDo is implemented in terms of nonsplittable single ParDos - .put(PTransformMatchers.splittableParDoMulti(), new ParDoMultiOverrideFactory()) - // state and timer pardos are implemented in terms of nonsplittable single ParDos - .put(PTransformMatchers.stateOrTimerParDoMulti(), new ParDoMultiOverrideFactory()) - .put( - PTransformMatchers.classEqualTo(ParDo.Bound.class), - new ParDoSingleViaMultiOverrideFactory()) /* returns a BoundMulti */ .put( PTransformMatchers.classEqualTo(BoundMulti.class), - /* returns one of two primitives; SplittableParDos are replaced above. */ + /* returns one of two primitives; SplittableParDos and ParDos with state and timers + are replaced appropriately by the override factory. */ new ParDoMultiOverrideFactory()) .put( PTransformMatchers.classEqualTo(GBKIntoKeyedWorkItems.class), diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java deleted file mode 100644 index f8597299217b..000000000000 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.direct; - -import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; -import org.apache.beam.sdk.runners.PTransformOverrideFactory; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.ParDo.Bound; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionTuple; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TupleTagList; - -/** - * A {@link PTransformOverrideFactory} that overrides single-output {@link ParDo} to implement - * it in terms of multi-output {@link ParDo}. - */ -class ParDoSingleViaMultiOverrideFactory - extends SingleInputOutputOverrideFactory< - PCollection, PCollection, Bound> { - @Override - public PTransform, PCollection> getReplacementTransform( - Bound transform) { - return new ParDoSingleViaMulti<>(transform); - } - - static class ParDoSingleViaMulti - extends PTransform, PCollection> { - private static final String MAIN_OUTPUT_TAG = "main"; - - private final ParDo.Bound underlyingParDo; - - public ParDoSingleViaMulti(ParDo.Bound underlyingParDo) { - this.underlyingParDo = underlyingParDo; - } - - @Override - public PCollection expand(PCollection input) { - - // Output tags for ParDo need only be unique up to applied transform - TupleTag mainOutputTag = new TupleTag(MAIN_OUTPUT_TAG); - - PCollectionTuple outputs = - input.apply( - ParDo.of(underlyingParDo.getFn()) - .withSideInputs(underlyingParDo.getSideInputs()) - .withOutputTags(mainOutputTag, TupleTagList.empty())); - PCollection output = outputs.get(mainOutputTag); - - output.setTypeDescriptor(underlyingParDo.getFn().getOutputTypeDescriptor()); - return output; - } - } -} diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactoryTest.java deleted file mode 100644 index 59577a82b3b9..000000000000 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactoryTest.java +++ /dev/null @@ -1,46 +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.direct; - -import static org.junit.Assert.assertThat; - -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.values.PCollection; -import org.hamcrest.Matchers; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** - * Tests for {@link ParDoSingleViaMultiOverrideFactory}. - */ -@RunWith(JUnit4.class) -public class ParDoSingleViaMultiOverrideFactoryTest { - private ParDoSingleViaMultiOverrideFactory factory = - new ParDoSingleViaMultiOverrideFactory<>(); - - @Test - public void getInputSucceeds() { - TestPipeline p = TestPipeline.create(); - PCollection input = p.apply(Create.of(1, 2, 3)); - PCollection reconstructed = factory.getInput(input.expand(), p); - assertThat(reconstructed, Matchers.>equalTo(input)); - } -} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java index f043c901391f..31a6bdace118 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java @@ -112,8 +112,7 @@ class FlinkBatchTransformTranslators { TRANSLATORS.put(Window.Assign.class, new WindowAssignTranslatorBatch()); - TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundTranslatorBatch()); - TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiTranslatorBatch()); + TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoTranslatorBatch()); TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslatorBatch()); } @@ -498,80 +497,7 @@ private static void rejectSplittable(DoFn doFn) { } } - private static class ParDoBoundTranslatorBatch - implements FlinkBatchPipelineTranslator.BatchTransformTranslator< - ParDo.Bound> { - - @Override - @SuppressWarnings("unchecked") - public void translateNode( - ParDo.Bound transform, - - FlinkBatchTranslationContext context) { - DoFn doFn = transform.getFn(); - rejectSplittable(doFn); - - DataSet> inputDataSet = - context.getInputDataSet(context.getInput(transform)); - - TypeInformation> typeInformation = - context.getTypeInfo(context.getOutput(transform)); - - List> sideInputs = transform.getSideInputs(); - - // construct a map from side input to WindowingStrategy so that - // the DoFn runner can map main-input windows to side input windows - Map, WindowingStrategy> sideInputStrategies = new HashMap<>(); - for (PCollectionView sideInput: sideInputs) { - sideInputStrategies.put(sideInput, sideInput.getWindowingStrategyInternal()); - } - - WindowingStrategy windowingStrategy = - context.getOutput(transform).getWindowingStrategy(); - - SingleInputUdfOperator, WindowedValue, ?> outputDataSet; - DoFnSignature signature = DoFnSignatures.getSignature(transform.getFn().getClass()); - if (signature.stateDeclarations().size() > 0 - || signature.timerDeclarations().size() > 0) { - - // Based on the fact that the signature is stateful, DoFnSignatures ensures - // that it is also keyed - KvCoder inputCoder = - (KvCoder) context.getInput(transform).getCoder(); - - FlinkStatefulDoFnFunction doFnWrapper = new FlinkStatefulDoFnFunction<>( - (DoFn) doFn, windowingStrategy, sideInputStrategies, context.getPipelineOptions(), - null, new TupleTag() - ); - - Grouping> grouping = - inputDataSet.groupBy(new KvKeySelector(inputCoder.getKeyCoder())); - - outputDataSet = new GroupReduceOperator( - grouping, typeInformation, doFnWrapper, transform.getName()); - - } else { - FlinkDoFnFunction doFnWrapper = - new FlinkDoFnFunction<>( - doFn, - windowingStrategy, - sideInputStrategies, - context.getPipelineOptions(), - null, new TupleTag()); - - outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, - transform.getName()); - - } - - transformSideInputs(sideInputs, outputDataSet, context); - - context.setOutputDataSet(context.getOutput(transform), outputDataSet); - - } - } - - private static class ParDoBoundMultiTranslatorBatch + private static class ParDoTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator< ParDo.BoundMulti> { diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java index c7df91dc1c1b..7227dceddbc1 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java @@ -121,8 +121,7 @@ class FlinkStreamingTransformTranslators { TRANSLATORS.put(Write.class, new WriteSinkStreamingTranslator()); TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteBoundStreamingTranslator()); - TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundStreamingTranslator()); - TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiStreamingTranslator()); + TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoStreamingTranslator()); TRANSLATORS.put(Window.Assign.class, new WindowAssignTranslator()); TRANSLATORS.put(Flatten.PCollections.class, new FlattenPCollectionTranslator()); @@ -320,114 +319,6 @@ private static void rejectSplittable(DoFn doFn) { } } - private static class ParDoBoundStreamingTranslator - extends FlinkStreamingPipelineTranslator.StreamTransformTranslator< - ParDo.Bound> { - - @Override - public void translateNode( - ParDo.Bound transform, - FlinkStreamingTranslationContext context) { - - DoFn doFn = transform.getFn(); - rejectSplittable(doFn); - - WindowingStrategy windowingStrategy = - context.getOutput(transform).getWindowingStrategy(); - - TypeInformation> typeInfo = - context.getTypeInfo(context.getOutput(transform)); - - List> sideInputs = transform.getSideInputs(); - - @SuppressWarnings("unchecked") - PCollection inputPCollection = (PCollection) context.getInput(transform); - - Coder> inputCoder = context.getCoder(inputPCollection); - - DataStream> inputDataStream = - context.getInputDataStream(context.getInput(transform)); - Coder keyCoder = null; - boolean stateful = false; - DoFnSignature signature = DoFnSignatures.getSignature(transform.getFn().getClass()); - if (signature.stateDeclarations().size() > 0 - || signature.timerDeclarations().size() > 0) { - // Based on the fact that the signature is stateful, DoFnSignatures ensures - // that it is also keyed - keyCoder = ((KvCoder) inputPCollection.getCoder()).getKeyCoder(); - inputDataStream = inputDataStream.keyBy(new KvToByteBufferKeySelector(keyCoder)); - stateful = true; - } - - if (sideInputs.isEmpty()) { - DoFnOperator> doFnOperator = - new DoFnOperator<>( - transform.getFn(), - inputCoder, - new TupleTag("main output"), - Collections.>emptyList(), - new DoFnOperator.DefaultOutputManagerFactory>(), - windowingStrategy, - new HashMap>(), /* side-input mapping */ - Collections.>emptyList(), /* side inputs */ - context.getPipelineOptions(), - keyCoder); - - SingleOutputStreamOperator> outDataStream = inputDataStream - .transform(transform.getName(), typeInfo, doFnOperator); - - context.setOutputDataStream(context.getOutput(transform), outDataStream); - } else { - Tuple2>, DataStream> transformedSideInputs = - transformSideInputs(sideInputs, context); - - DoFnOperator> doFnOperator = - new DoFnOperator<>( - transform.getFn(), - inputCoder, - new TupleTag("main output"), - Collections.>emptyList(), - new DoFnOperator.DefaultOutputManagerFactory>(), - windowingStrategy, - transformedSideInputs.f0, - sideInputs, - context.getPipelineOptions(), - keyCoder); - - SingleOutputStreamOperator> outDataStream; - if (stateful) { - // we have to manually contruct the two-input transform because we're not - // allowed to have only one input keyed, normally. - KeyedStream keyedStream = (KeyedStream) inputDataStream; - TwoInputTransformation< - WindowedValue>, - RawUnionValue, - WindowedValue> rawFlinkTransform = new TwoInputTransformation<>( - keyedStream.getTransformation(), - transformedSideInputs.f1.broadcast().getTransformation(), - transform.getName(), - (TwoInputStreamOperator) doFnOperator, - typeInfo, - keyedStream.getParallelism()); - - rawFlinkTransform.setStateKeyType(keyedStream.getKeyType()); - rawFlinkTransform.setStateKeySelectors(keyedStream.getKeySelector(), null); - - outDataStream = new SingleOutputStreamOperator( - keyedStream.getExecutionEnvironment(), - rawFlinkTransform) {}; // we have to cheat around the ctor being protected - - keyedStream.getExecutionEnvironment().addOperator(rawFlinkTransform); - } else { - outDataStream = inputDataStream - .connect(transformedSideInputs.f1.broadcast()) - .transform(transform.getName(), typeInfo, doFnOperator); - } - context.setOutputDataStream(context.getOutput(transform), outDataStream); - } - } - } - /** * Wraps each element in a {@link RawUnionValue} with the given tag id. */ @@ -505,7 +396,7 @@ public RawUnionValue map(T o) throws Exception { } - private static class ParDoBoundMultiStreamingTranslator + private static class ParDoStreamingTranslator extends FlinkStreamingPipelineTranslator.StreamTransformTranslator< ParDo.BoundMulti> { 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 06e50483440d..ab4cb9c67b04 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 @@ -45,7 +45,6 @@ import com.google.common.base.Supplier; import com.google.common.collect.BiMap; import com.google.common.collect.ImmutableBiMap; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import java.io.IOException; import java.util.ArrayList; @@ -847,34 +846,6 @@ private void translateMultiHelper( } }); - registerTransformTranslator( - ParDo.Bound.class, - new TransformTranslator() { - @Override - public void translate(ParDo.Bound transform, TranslationContext context) { - translateSingleHelper(transform, context); - } - - private void translateSingleHelper( - ParDo.Bound transform, TranslationContext context) { - - StepTranslationContext stepContext = context.addStep(transform, "ParallelDo"); - translateInputs( - stepContext, context.getInput(transform), transform.getSideInputs(), context); - long mainOutput = stepContext.addOutput(context.getOutput(transform)); - translateFn( - stepContext, - transform.getFn(), - context.getInput(transform).getWindowingStrategy(), - transform.getSideInputs(), - context.getInput(transform).getCoder(), - context, - mainOutput, - ImmutableMap.>of( - mainOutput, new TupleTag<>(PropertyNames.OUTPUT))); - } - }); - registerTransformTranslator( Window.Assign.class, new TransformTranslator() { 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 d4271e52f7c7..ccb185cbea93 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 @@ -525,7 +525,8 @@ private static Step createPredefinedStep() throws Exception { assertEquals(13, job.getSteps().size()); Step step = job.getSteps().get(1); - assertEquals(stepName, getString(step.getProperties(), PropertyNames.USER_NAME)); + assertEquals( + stepName + "/ParMultiDo(NoOp)", getString(step.getProperties(), PropertyNames.USER_NAME)); assertAllStepOutputsHaveUniqueIds(job); return step; } @@ -971,7 +972,7 @@ public void populateDisplayData(DisplayData.Builder builder) { .put("type", "JAVA_CLASS") .put("value", fn1.getClass().getName()) .put("shortValue", fn1.getClass().getSimpleName()) - .put("namespace", parDo1.getClass().getName()) + .put("namespace", ParDo.BoundMulti.class.getName()) .build(), ImmutableMap.builder() .put("key", "foo2") @@ -991,7 +992,7 @@ public void populateDisplayData(DisplayData.Builder builder) { .put("type", "JAVA_CLASS") .put("value", fn2.getClass().getName()) .put("shortValue", fn2.getClass().getSimpleName()) - .put("namespace", parDo2.getClass().getName()) + .put("namespace", ParDo.BoundMulti.class.getName()) .build(), ImmutableMap.builder() .put("key", "foo3") 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 8ebb49679222..a4939b976448 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 @@ -27,6 +27,7 @@ import static org.apache.beam.runners.spark.translation.TranslationUtils.rejectSplittable; import static org.apache.beam.runners.spark.translation.TranslationUtils.rejectStateAndTimers; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import java.io.IOException; @@ -73,6 +74,7 @@ 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.TaggedPValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.hadoop.conf.Configuration; @@ -331,38 +333,19 @@ public WindowedValue> call( }; } - private static TransformEvaluator> parDo() { - return new TransformEvaluator>() { - @Override - public void evaluate(ParDo.Bound transform, EvaluationContext context) { - String stepName = context.getCurrentTransform().getFullName(); - DoFn doFn = transform.getFn(); - rejectSplittable(doFn); - rejectStateAndTimers(doFn); - @SuppressWarnings("unchecked") - JavaRDD> inRDD = - ((BoundedDataset) context.borrowDataset(transform)).getRDD(); - WindowingStrategy windowingStrategy = - context.getInput(transform).getWindowingStrategy(); - JavaSparkContext jsc = context.getSparkContext(); - Accumulator aggAccum = - SparkAggregators.getNamedAggregators(jsc); - Accumulator metricsAccum = - MetricsAccumulator.getInstance(); - Map, KV, SideInputBroadcast>> sideInputs = - TranslationUtils.getSideInputs(transform.getSideInputs(), context); - context.putDataset(transform, - new BoundedDataset<>(inRDD.mapPartitions(new DoFnFunction<>(aggAccum, metricsAccum, - stepName, doFn, context.getRuntimeContext(), sideInputs, windowingStrategy)))); - } - }; - } - - private static TransformEvaluator> - multiDo() { + private static TransformEvaluator> parDo() { return new TransformEvaluator>() { @Override public void evaluate(ParDo.BoundMulti transform, EvaluationContext context) { + if (transform.getSideOutputTags().size() == 0) { + evaluateSingle(transform, context); + } else { + evaluateMulti(transform, context); + } + } + + private void evaluateMulti( + ParDo.BoundMulti transform, EvaluationContext context) { String stepName = context.getCurrentTransform().getFullName(); DoFn doFn = transform.getFn(); rejectSplittable(doFn); @@ -373,16 +356,21 @@ public void evaluate(ParDo.BoundMulti transform, EvaluationCont WindowingStrategy windowingStrategy = context.getInput(transform).getWindowingStrategy(); JavaSparkContext jsc = context.getSparkContext(); - Accumulator aggAccum = - SparkAggregators.getNamedAggregators(jsc); - Accumulator metricsAccum = - MetricsAccumulator.getInstance(); - JavaPairRDD, WindowedValue> all = inRDD - .mapPartitionsToPair( - new MultiDoFnFunction<>(aggAccum, metricsAccum, stepName, doFn, - context.getRuntimeContext(), transform.getMainOutputTag(), - TranslationUtils.getSideInputs(transform.getSideInputs(), context), - windowingStrategy)).cache(); + Accumulator aggAccum = SparkAggregators.getNamedAggregators(jsc); + Accumulator metricsAccum = MetricsAccumulator.getInstance(); + JavaPairRDD, WindowedValue> all = + inRDD + .mapPartitionsToPair( + new MultiDoFnFunction<>( + aggAccum, + metricsAccum, + stepName, + doFn, + context.getRuntimeContext(), + transform.getMainOutputTag(), + TranslationUtils.getSideInputs(transform.getSideInputs(), context), + windowingStrategy)) + .cache(); List pct = context.getOutputs(transform); for (TaggedPValue e : pct) { @SuppressWarnings("unchecked") @@ -395,6 +383,37 @@ public void evaluate(ParDo.BoundMulti transform, EvaluationCont context.putDataset(e.getValue(), new BoundedDataset<>(values)); } } + + private void evaluateSingle( + ParDo.BoundMulti transform, EvaluationContext context) { + String stepName = context.getCurrentTransform().getFullName(); + DoFn doFn = transform.getFn(); + rejectSplittable(doFn); + rejectStateAndTimers(doFn); + @SuppressWarnings("unchecked") + JavaRDD> inRDD = + ((BoundedDataset) context.borrowDataset(transform)).getRDD(); + WindowingStrategy windowingStrategy = + context.getInput(transform).getWindowingStrategy(); + JavaSparkContext jsc = context.getSparkContext(); + Accumulator aggAccum = SparkAggregators.getNamedAggregators(jsc); + Accumulator metricsAccum = MetricsAccumulator.getInstance(); + Map, KV, SideInputBroadcast>> sideInputs = + TranslationUtils.getSideInputs(transform.getSideInputs(), context); + PValue onlyOutput = Iterables.getOnlyElement(context.getOutputs(transform)).getValue(); + context.putDataset( + onlyOutput, + new BoundedDataset<>( + inRDD.mapPartitions( + new DoFnFunction<>( + aggAccum, + metricsAccum, + stepName, + doFn, + context.getRuntimeContext(), + sideInputs, + windowingStrategy)))); + } }; } @@ -723,8 +742,7 @@ public void evaluate(StorageLevelPTransform transform, EvaluationContext context EVALUATORS.put(Read.Bounded.class, readBounded()); EVALUATORS.put(HadoopIO.Read.Bound.class, readHadoop()); EVALUATORS.put(HadoopIO.Write.Bound.class, writeHadoop()); - EVALUATORS.put(ParDo.Bound.class, parDo()); - EVALUATORS.put(ParDo.BoundMulti.class, multiDo()); + EVALUATORS.put(ParDo.BoundMulti.class, parDo()); EVALUATORS.put(GroupByKey.class, groupByKey()); EVALUATORS.put(Combine.GroupedValues.class, combineGrouped()); EVALUATORS.put(Combine.Globally.class, combineGlobally()); 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 e3445bf5baab..628b713913d8 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 @@ -331,58 +331,20 @@ public void evaluate(final Combine.GroupedValues transform, }; } - private static TransformEvaluator> parDo() { - return new TransformEvaluator>() { - @Override - public void evaluate(final ParDo.Bound transform, - final EvaluationContext context) { - final DoFn doFn = transform.getFn(); - rejectSplittable(doFn); - rejectStateAndTimers(doFn); - final SparkRuntimeContext runtimeContext = context.getRuntimeContext(); - final WindowingStrategy windowingStrategy = - context.getInput(transform).getWindowingStrategy(); - final SparkPCollectionView pviews = context.getPViews(); - - @SuppressWarnings("unchecked") - UnboundedDataset unboundedDataset = - ((UnboundedDataset) context.borrowDataset(transform)); - JavaDStream> dStream = unboundedDataset.getDStream(); - - final String stepName = context.getCurrentTransform().getFullName(); - - JavaDStream> outStream = - dStream.transform(new Function>, - JavaRDD>>() { - @Override - public JavaRDD> call(JavaRDD> rdd) throws - Exception { - final JavaSparkContext jsc = new JavaSparkContext(rdd.context()); - final Accumulator aggAccum = - SparkAggregators.getNamedAggregators(jsc); - final Accumulator metricsAccum = - MetricsAccumulator.getInstance(); - final Map, KV, SideInputBroadcast>> sideInputs = - TranslationUtils.getSideInputs(transform.getSideInputs(), - jsc, pviews); - return rdd.mapPartitions( - new DoFnFunction<>(aggAccum, metricsAccum, stepName, doFn, runtimeContext, - sideInputs, windowingStrategy)); - } - }); - - context.putDataset(transform, - new UnboundedDataset<>(outStream, unboundedDataset.getStreamSources())); - } - }; - } - private static TransformEvaluator> multiDo() { return new TransformEvaluator>() { - @Override - public void evaluate(final ParDo.BoundMulti transform, - final EvaluationContext context) { + public void evaluate( + final ParDo.BoundMulti transform, final EvaluationContext context) { + if (transform.getSideOutputTags().size() == 0) { + evaluateSingle(transform, context); + } else { + evaluateMulti(transform, context); + } + } + + private void evaluateMulti( + final ParDo.BoundMulti transform, final EvaluationContext context) { final DoFn doFn = transform.getFn(); rejectSplittable(doFn); rejectStateAndTimers(doFn); @@ -426,10 +388,60 @@ public JavaPairRDD, WindowedValue> call( JavaDStream> values = (JavaDStream>) (JavaDStream) TranslationUtils.dStreamValues(filtered); - context.putDataset(e.getValue(), - new UnboundedDataset<>(values, unboundedDataset.getStreamSources())); + context.putDataset( + e.getValue(), new UnboundedDataset<>(values, unboundedDataset.getStreamSources())); } } + + private void evaluateSingle( + final ParDo.BoundMulti transform, final EvaluationContext context) { + final DoFn doFn = transform.getFn(); + rejectSplittable(doFn); + rejectStateAndTimers(doFn); + final SparkRuntimeContext runtimeContext = context.getRuntimeContext(); + final WindowingStrategy windowingStrategy = + context.getInput(transform).getWindowingStrategy(); + final SparkPCollectionView pviews = context.getPViews(); + + @SuppressWarnings("unchecked") + UnboundedDataset unboundedDataset = + ((UnboundedDataset) context.borrowDataset(transform)); + JavaDStream> dStream = unboundedDataset.getDStream(); + + final String stepName = context.getCurrentTransform().getFullName(); + + JavaDStream> outStream = + dStream.transform( + new Function>, JavaRDD>>() { + @Override + public JavaRDD> call(JavaRDD> rdd) + throws Exception { + final JavaSparkContext jsc = new JavaSparkContext(rdd.context()); + final Accumulator aggAccum = + SparkAggregators.getNamedAggregators(jsc); + final Accumulator metricsAccum = + MetricsAccumulator.getInstance(); + final Map, KV, SideInputBroadcast>> + sideInputs = + TranslationUtils.getSideInputs(transform.getSideInputs(), jsc, pviews); + return rdd.mapPartitions( + new DoFnFunction<>( + aggAccum, + metricsAccum, + stepName, + doFn, + runtimeContext, + sideInputs, + windowingStrategy)); + } + }); + + PCollection output = + (PCollection) + Iterables.getOnlyElement(context.getOutputs(transform)).getValue(); + context.putDataset( + output, new UnboundedDataset<>(outStream, unboundedDataset.getStreamSources())); + } }; } @@ -440,7 +452,6 @@ public JavaPairRDD, WindowedValue> call( EVALUATORS.put(Read.Unbounded.class, readUnbounded()); EVALUATORS.put(GroupByKey.class, groupByKey()); EVALUATORS.put(Combine.GroupedValues.class, combineGrouped()); - EVALUATORS.put(ParDo.Bound.class, parDo()); EVALUATORS.put(ParDo.BoundMulti.class, multiDo()); EVALUATORS.put(ConsoleIO.Write.Unbound.class, print()); EVALUATORS.put(CreateStream.class, createFromQueue()); 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 b181a042820c..d66633b4c49b 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 @@ -83,7 +83,7 @@ public void testTrackSingle() { p.apply(emptyStream).apply(ParDo.of(new PassthroughFn<>())); - p.traverseTopologically(new StreamingSourceTracker(jssc, p, ParDo.Bound.class, 0)); + p.traverseTopologically(new StreamingSourceTracker(jssc, p, ParDo.BoundMulti.class, 0)); assertThat(StreamingSourceTracker.numAssertions, equalTo(1)); } @@ -111,7 +111,7 @@ public void testTrackFlattened() { PCollectionList.of(pcol1).and(pcol2).apply(Flatten.pCollections()); flattened.apply(ParDo.of(new PassthroughFn<>())); - p.traverseTopologically(new StreamingSourceTracker(jssc, p, ParDo.Bound.class, 0, 1)); + p.traverseTopologically(new StreamingSourceTracker(jssc, p, ParDo.BoundMulti.class, 0, 1)); assertThat(StreamingSourceTracker.numAssertions, equalTo(1)); } 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 19c5a2d5b511..92252310f961 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 @@ -738,12 +738,8 @@ public BoundMulti withOutputTags( @Override public PCollection expand(PCollection input) { - validateWindowType(input, fn); - return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), - input.getWindowingStrategy(), - input.isBounded()) - .setTypeDescriptor(getFn().getOutputTypeDescriptor()); + TupleTag mainOutput = new TupleTag<>(); + return input.apply(withOutputTags(mainOutput, TupleTagList.empty())).get(mainOutput); } @Override From c5783314de1851cdf707888fe020fbf844ecdaa2 Mon Sep 17 00:00:00 2001 From: Pablo Date: Thu, 2 Mar 2017 09:52:07 -0800 Subject: [PATCH 23/46] Updating Dataflow API client protobufs --- .../clients/dataflow/dataflow_v1b3_client.py | 530 +++++++++--------- .../dataflow/dataflow_v1b3_messages.py | 345 +++++++++--- 2 files changed, 552 insertions(+), 323 deletions(-) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_client.py b/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_client.py index 725d496caba6..f280217a6bd6 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_client.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_client.py @@ -70,6 +70,33 @@ class ProjectsJobsDebugService(base_api.BaseApiService): def __init__(self, client): super(DataflowV1b3.ProjectsJobsDebugService, self).__init__(client) + self._method_configs = { + 'GetConfig': base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'dataflow.projects.jobs.debug.getConfig', + ordered_params=[u'projectId', u'jobId'], + path_params=[u'jobId', u'projectId'], + query_params=[], + relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/debug/getConfig', + request_field=u'getDebugConfigRequest', + request_type_name=u'DataflowProjectsJobsDebugGetConfigRequest', + response_type_name=u'GetDebugConfigResponse', + supports_download=False, + ), + 'SendCapture': base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'dataflow.projects.jobs.debug.sendCapture', + ordered_params=[u'projectId', u'jobId'], + path_params=[u'jobId', u'projectId'], + query_params=[], + relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/debug/sendCapture', + request_field=u'sendDebugCaptureRequest', + request_type_name=u'DataflowProjectsJobsDebugSendCaptureRequest', + response_type_name=u'SendDebugCaptureResponse', + supports_download=False, + ), + } + self._upload_configs = { } @@ -86,19 +113,6 @@ def GetConfig(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) - GetConfig.method_config = lambda: base_api.ApiMethodInfo( - http_method=u'POST', - method_id=u'dataflow.projects.jobs.debug.getConfig', - ordered_params=[u'projectId', u'jobId'], - path_params=[u'jobId', u'projectId'], - query_params=[], - relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/debug/getConfig', - request_field=u'getDebugConfigRequest', - request_type_name=u'DataflowProjectsJobsDebugGetConfigRequest', - response_type_name=u'GetDebugConfigResponse', - supports_download=False, - ) - def SendCapture(self, request, global_params=None): """Send encoded debug capture data for component. @@ -112,19 +126,6 @@ def SendCapture(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) - SendCapture.method_config = lambda: base_api.ApiMethodInfo( - http_method=u'POST', - method_id=u'dataflow.projects.jobs.debug.sendCapture', - ordered_params=[u'projectId', u'jobId'], - path_params=[u'jobId', u'projectId'], - query_params=[], - relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/debug/sendCapture', - request_field=u'sendDebugCaptureRequest', - request_type_name=u'DataflowProjectsJobsDebugSendCaptureRequest', - response_type_name=u'SendDebugCaptureResponse', - supports_download=False, - ) - class ProjectsJobsMessagesService(base_api.BaseApiService): """Service class for the projects_jobs_messages resource.""" @@ -132,6 +133,21 @@ class ProjectsJobsMessagesService(base_api.BaseApiService): def __init__(self, client): super(DataflowV1b3.ProjectsJobsMessagesService, self).__init__(client) + self._method_configs = { + 'List': base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'dataflow.projects.jobs.messages.list', + ordered_params=[u'projectId', u'jobId'], + path_params=[u'jobId', u'projectId'], + query_params=[u'endTime', u'location', u'minimumImportance', u'pageSize', u'pageToken', u'startTime'], + relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/messages', + request_field='', + request_type_name=u'DataflowProjectsJobsMessagesListRequest', + response_type_name=u'ListJobMessagesResponse', + supports_download=False, + ), + } + self._upload_configs = { } @@ -148,19 +164,6 @@ def List(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) - List.method_config = lambda: base_api.ApiMethodInfo( - http_method=u'GET', - method_id=u'dataflow.projects.jobs.messages.list', - ordered_params=[u'projectId', u'jobId'], - path_params=[u'jobId', u'projectId'], - query_params=[u'endTime', u'location', u'minimumImportance', u'pageSize', u'pageToken', u'startTime'], - relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/messages', - request_field='', - request_type_name=u'DataflowProjectsJobsMessagesListRequest', - response_type_name=u'ListJobMessagesResponse', - supports_download=False, - ) - class ProjectsJobsWorkItemsService(base_api.BaseApiService): """Service class for the projects_jobs_workItems resource.""" @@ -168,6 +171,33 @@ class ProjectsJobsWorkItemsService(base_api.BaseApiService): def __init__(self, client): super(DataflowV1b3.ProjectsJobsWorkItemsService, self).__init__(client) + self._method_configs = { + 'Lease': base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'dataflow.projects.jobs.workItems.lease', + ordered_params=[u'projectId', u'jobId'], + path_params=[u'jobId', u'projectId'], + query_params=[], + relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/workItems:lease', + request_field=u'leaseWorkItemRequest', + request_type_name=u'DataflowProjectsJobsWorkItemsLeaseRequest', + response_type_name=u'LeaseWorkItemResponse', + supports_download=False, + ), + 'ReportStatus': base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'dataflow.projects.jobs.workItems.reportStatus', + ordered_params=[u'projectId', u'jobId'], + path_params=[u'jobId', u'projectId'], + query_params=[], + relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/workItems:reportStatus', + request_field=u'reportWorkItemStatusRequest', + request_type_name=u'DataflowProjectsJobsWorkItemsReportStatusRequest', + response_type_name=u'ReportWorkItemStatusResponse', + supports_download=False, + ), + } + self._upload_configs = { } @@ -184,19 +214,6 @@ def Lease(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) - Lease.method_config = lambda: base_api.ApiMethodInfo( - http_method=u'POST', - method_id=u'dataflow.projects.jobs.workItems.lease', - ordered_params=[u'projectId', u'jobId'], - path_params=[u'jobId', u'projectId'], - query_params=[], - relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/workItems:lease', - request_field=u'leaseWorkItemRequest', - request_type_name=u'DataflowProjectsJobsWorkItemsLeaseRequest', - response_type_name=u'LeaseWorkItemResponse', - supports_download=False, - ) - def ReportStatus(self, request, global_params=None): """Reports the status of dataflow WorkItems leased by a worker. @@ -210,19 +227,6 @@ def ReportStatus(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) - ReportStatus.method_config = lambda: base_api.ApiMethodInfo( - http_method=u'POST', - method_id=u'dataflow.projects.jobs.workItems.reportStatus', - ordered_params=[u'projectId', u'jobId'], - path_params=[u'jobId', u'projectId'], - query_params=[], - relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/workItems:reportStatus', - request_field=u'reportWorkItemStatusRequest', - request_type_name=u'DataflowProjectsJobsWorkItemsReportStatusRequest', - response_type_name=u'ReportWorkItemStatusResponse', - supports_download=False, - ) - class ProjectsJobsService(base_api.BaseApiService): """Service class for the projects_jobs resource.""" @@ -230,6 +234,69 @@ class ProjectsJobsService(base_api.BaseApiService): def __init__(self, client): super(DataflowV1b3.ProjectsJobsService, self).__init__(client) + self._method_configs = { + 'Create': base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'dataflow.projects.jobs.create', + ordered_params=[u'projectId'], + path_params=[u'projectId'], + query_params=[u'location', u'replaceJobId', u'view'], + relative_path=u'v1b3/projects/{projectId}/jobs', + request_field=u'job', + request_type_name=u'DataflowProjectsJobsCreateRequest', + response_type_name=u'Job', + supports_download=False, + ), + 'Get': base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'dataflow.projects.jobs.get', + ordered_params=[u'projectId', u'jobId'], + path_params=[u'jobId', u'projectId'], + query_params=[u'location', u'view'], + relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}', + request_field='', + request_type_name=u'DataflowProjectsJobsGetRequest', + response_type_name=u'Job', + supports_download=False, + ), + 'GetMetrics': base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'dataflow.projects.jobs.getMetrics', + ordered_params=[u'projectId', u'jobId'], + path_params=[u'jobId', u'projectId'], + query_params=[u'location', u'startTime'], + relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/metrics', + request_field='', + request_type_name=u'DataflowProjectsJobsGetMetricsRequest', + response_type_name=u'JobMetrics', + supports_download=False, + ), + 'List': base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'dataflow.projects.jobs.list', + ordered_params=[u'projectId'], + path_params=[u'projectId'], + query_params=[u'filter', u'location', u'pageSize', u'pageToken', u'view'], + relative_path=u'v1b3/projects/{projectId}/jobs', + request_field='', + request_type_name=u'DataflowProjectsJobsListRequest', + response_type_name=u'ListJobsResponse', + supports_download=False, + ), + 'Update': base_api.ApiMethodInfo( + http_method=u'PUT', + method_id=u'dataflow.projects.jobs.update', + ordered_params=[u'projectId', u'jobId'], + path_params=[u'jobId', u'projectId'], + query_params=[u'location'], + relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}', + request_field=u'job', + request_type_name=u'DataflowProjectsJobsUpdateRequest', + response_type_name=u'Job', + supports_download=False, + ), + } + self._upload_configs = { } @@ -246,19 +313,6 @@ def Create(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) - Create.method_config = lambda: base_api.ApiMethodInfo( - http_method=u'POST', - method_id=u'dataflow.projects.jobs.create', - ordered_params=[u'projectId'], - path_params=[u'projectId'], - query_params=[u'location', u'replaceJobId', u'view'], - relative_path=u'v1b3/projects/{projectId}/jobs', - request_field=u'job', - request_type_name=u'DataflowProjectsJobsCreateRequest', - response_type_name=u'Job', - supports_download=False, - ) - def Get(self, request, global_params=None): """Gets the state of the specified Cloud Dataflow job. @@ -272,19 +326,6 @@ def Get(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) - Get.method_config = lambda: base_api.ApiMethodInfo( - http_method=u'GET', - method_id=u'dataflow.projects.jobs.get', - ordered_params=[u'projectId', u'jobId'], - path_params=[u'jobId', u'projectId'], - query_params=[u'location', u'view'], - relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}', - request_field='', - request_type_name=u'DataflowProjectsJobsGetRequest', - response_type_name=u'Job', - supports_download=False, - ) - def GetMetrics(self, request, global_params=None): """Request the job status. @@ -298,19 +339,6 @@ def GetMetrics(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) - GetMetrics.method_config = lambda: base_api.ApiMethodInfo( - http_method=u'GET', - method_id=u'dataflow.projects.jobs.getMetrics', - ordered_params=[u'projectId', u'jobId'], - path_params=[u'jobId', u'projectId'], - query_params=[u'location', u'startTime'], - relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/metrics', - request_field='', - request_type_name=u'DataflowProjectsJobsGetMetricsRequest', - response_type_name=u'JobMetrics', - supports_download=False, - ) - def List(self, request, global_params=None): """List the jobs of a project. @@ -324,19 +352,6 @@ def List(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) - List.method_config = lambda: base_api.ApiMethodInfo( - http_method=u'GET', - method_id=u'dataflow.projects.jobs.list', - ordered_params=[u'projectId'], - path_params=[u'projectId'], - query_params=[u'filter', u'location', u'pageSize', u'pageToken', u'view'], - relative_path=u'v1b3/projects/{projectId}/jobs', - request_field='', - request_type_name=u'DataflowProjectsJobsListRequest', - response_type_name=u'ListJobsResponse', - supports_download=False, - ) - def Update(self, request, global_params=None): """Updates the state of an existing Cloud Dataflow job. @@ -350,19 +365,6 @@ def Update(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) - Update.method_config = lambda: base_api.ApiMethodInfo( - http_method=u'PUT', - method_id=u'dataflow.projects.jobs.update', - ordered_params=[u'projectId', u'jobId'], - path_params=[u'jobId', u'projectId'], - query_params=[u'location'], - relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}', - request_field=u'job', - request_type_name=u'DataflowProjectsJobsUpdateRequest', - response_type_name=u'Job', - supports_download=False, - ) - class ProjectsLocationsJobsMessagesService(base_api.BaseApiService): """Service class for the projects_locations_jobs_messages resource.""" @@ -370,6 +372,21 @@ class ProjectsLocationsJobsMessagesService(base_api.BaseApiService): def __init__(self, client): super(DataflowV1b3.ProjectsLocationsJobsMessagesService, self).__init__(client) + self._method_configs = { + 'List': base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'dataflow.projects.locations.jobs.messages.list', + ordered_params=[u'projectId', u'location', u'jobId'], + path_params=[u'jobId', u'location', u'projectId'], + query_params=[u'endTime', u'minimumImportance', u'pageSize', u'pageToken', u'startTime'], + relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs/{jobId}/messages', + request_field='', + request_type_name=u'DataflowProjectsLocationsJobsMessagesListRequest', + response_type_name=u'ListJobMessagesResponse', + supports_download=False, + ), + } + self._upload_configs = { } @@ -386,19 +403,6 @@ def List(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) - List.method_config = lambda: base_api.ApiMethodInfo( - http_method=u'GET', - method_id=u'dataflow.projects.locations.jobs.messages.list', - ordered_params=[u'projectId', u'location', u'jobId'], - path_params=[u'jobId', u'location', u'projectId'], - query_params=[u'endTime', u'minimumImportance', u'pageSize', u'pageToken', u'startTime'], - relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs/{jobId}/messages', - request_field='', - request_type_name=u'DataflowProjectsLocationsJobsMessagesListRequest', - response_type_name=u'ListJobMessagesResponse', - supports_download=False, - ) - class ProjectsLocationsJobsWorkItemsService(base_api.BaseApiService): """Service class for the projects_locations_jobs_workItems resource.""" @@ -406,6 +410,33 @@ class ProjectsLocationsJobsWorkItemsService(base_api.BaseApiService): def __init__(self, client): super(DataflowV1b3.ProjectsLocationsJobsWorkItemsService, self).__init__(client) + self._method_configs = { + 'Lease': base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'dataflow.projects.locations.jobs.workItems.lease', + ordered_params=[u'projectId', u'location', u'jobId'], + path_params=[u'jobId', u'location', u'projectId'], + query_params=[], + relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs/{jobId}/workItems:lease', + request_field=u'leaseWorkItemRequest', + request_type_name=u'DataflowProjectsLocationsJobsWorkItemsLeaseRequest', + response_type_name=u'LeaseWorkItemResponse', + supports_download=False, + ), + 'ReportStatus': base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'dataflow.projects.locations.jobs.workItems.reportStatus', + ordered_params=[u'projectId', u'location', u'jobId'], + path_params=[u'jobId', u'location', u'projectId'], + query_params=[], + relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs/{jobId}/workItems:reportStatus', + request_field=u'reportWorkItemStatusRequest', + request_type_name=u'DataflowProjectsLocationsJobsWorkItemsReportStatusRequest', + response_type_name=u'ReportWorkItemStatusResponse', + supports_download=False, + ), + } + self._upload_configs = { } @@ -422,19 +453,6 @@ def Lease(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) - Lease.method_config = lambda: base_api.ApiMethodInfo( - http_method=u'POST', - method_id=u'dataflow.projects.locations.jobs.workItems.lease', - ordered_params=[u'projectId', u'location', u'jobId'], - path_params=[u'jobId', u'location', u'projectId'], - query_params=[], - relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs/{jobId}/workItems:lease', - request_field=u'leaseWorkItemRequest', - request_type_name=u'DataflowProjectsLocationsJobsWorkItemsLeaseRequest', - response_type_name=u'LeaseWorkItemResponse', - supports_download=False, - ) - def ReportStatus(self, request, global_params=None): """Reports the status of dataflow WorkItems leased by a worker. @@ -448,19 +466,6 @@ def ReportStatus(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) - ReportStatus.method_config = lambda: base_api.ApiMethodInfo( - http_method=u'POST', - method_id=u'dataflow.projects.locations.jobs.workItems.reportStatus', - ordered_params=[u'projectId', u'location', u'jobId'], - path_params=[u'jobId', u'location', u'projectId'], - query_params=[], - relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs/{jobId}/workItems:reportStatus', - request_field=u'reportWorkItemStatusRequest', - request_type_name=u'DataflowProjectsLocationsJobsWorkItemsReportStatusRequest', - response_type_name=u'ReportWorkItemStatusResponse', - supports_download=False, - ) - class ProjectsLocationsJobsService(base_api.BaseApiService): """Service class for the projects_locations_jobs resource.""" @@ -468,6 +473,69 @@ class ProjectsLocationsJobsService(base_api.BaseApiService): def __init__(self, client): super(DataflowV1b3.ProjectsLocationsJobsService, self).__init__(client) + self._method_configs = { + 'Create': base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'dataflow.projects.locations.jobs.create', + ordered_params=[u'projectId', u'location'], + path_params=[u'location', u'projectId'], + query_params=[u'replaceJobId', u'view'], + relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs', + request_field=u'job', + request_type_name=u'DataflowProjectsLocationsJobsCreateRequest', + response_type_name=u'Job', + supports_download=False, + ), + 'Get': base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'dataflow.projects.locations.jobs.get', + ordered_params=[u'projectId', u'location', u'jobId'], + path_params=[u'jobId', u'location', u'projectId'], + query_params=[u'view'], + relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs/{jobId}', + request_field='', + request_type_name=u'DataflowProjectsLocationsJobsGetRequest', + response_type_name=u'Job', + supports_download=False, + ), + 'GetMetrics': base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'dataflow.projects.locations.jobs.getMetrics', + ordered_params=[u'projectId', u'location', u'jobId'], + path_params=[u'jobId', u'location', u'projectId'], + query_params=[u'startTime'], + relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs/{jobId}/metrics', + request_field='', + request_type_name=u'DataflowProjectsLocationsJobsGetMetricsRequest', + response_type_name=u'JobMetrics', + supports_download=False, + ), + 'List': base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'dataflow.projects.locations.jobs.list', + ordered_params=[u'projectId', u'location'], + path_params=[u'location', u'projectId'], + query_params=[u'filter', u'pageSize', u'pageToken', u'view'], + relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs', + request_field='', + request_type_name=u'DataflowProjectsLocationsJobsListRequest', + response_type_name=u'ListJobsResponse', + supports_download=False, + ), + 'Update': base_api.ApiMethodInfo( + http_method=u'PUT', + method_id=u'dataflow.projects.locations.jobs.update', + ordered_params=[u'projectId', u'location', u'jobId'], + path_params=[u'jobId', u'location', u'projectId'], + query_params=[], + relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs/{jobId}', + request_field=u'job', + request_type_name=u'DataflowProjectsLocationsJobsUpdateRequest', + response_type_name=u'Job', + supports_download=False, + ), + } + self._upload_configs = { } @@ -484,19 +552,6 @@ def Create(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) - Create.method_config = lambda: base_api.ApiMethodInfo( - http_method=u'POST', - method_id=u'dataflow.projects.locations.jobs.create', - ordered_params=[u'projectId', u'location'], - path_params=[u'location', u'projectId'], - query_params=[u'replaceJobId', u'view'], - relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs', - request_field=u'job', - request_type_name=u'DataflowProjectsLocationsJobsCreateRequest', - response_type_name=u'Job', - supports_download=False, - ) - def Get(self, request, global_params=None): """Gets the state of the specified Cloud Dataflow job. @@ -510,19 +565,6 @@ def Get(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) - Get.method_config = lambda: base_api.ApiMethodInfo( - http_method=u'GET', - method_id=u'dataflow.projects.locations.jobs.get', - ordered_params=[u'projectId', u'location', u'jobId'], - path_params=[u'jobId', u'location', u'projectId'], - query_params=[u'view'], - relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs/{jobId}', - request_field='', - request_type_name=u'DataflowProjectsLocationsJobsGetRequest', - response_type_name=u'Job', - supports_download=False, - ) - def GetMetrics(self, request, global_params=None): """Request the job status. @@ -536,19 +578,6 @@ def GetMetrics(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) - GetMetrics.method_config = lambda: base_api.ApiMethodInfo( - http_method=u'GET', - method_id=u'dataflow.projects.locations.jobs.getMetrics', - ordered_params=[u'projectId', u'location', u'jobId'], - path_params=[u'jobId', u'location', u'projectId'], - query_params=[u'startTime'], - relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs/{jobId}/metrics', - request_field='', - request_type_name=u'DataflowProjectsLocationsJobsGetMetricsRequest', - response_type_name=u'JobMetrics', - supports_download=False, - ) - def List(self, request, global_params=None): """List the jobs of a project. @@ -562,19 +591,6 @@ def List(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) - List.method_config = lambda: base_api.ApiMethodInfo( - http_method=u'GET', - method_id=u'dataflow.projects.locations.jobs.list', - ordered_params=[u'projectId', u'location'], - path_params=[u'location', u'projectId'], - query_params=[u'filter', u'pageSize', u'pageToken', u'view'], - relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs', - request_field='', - request_type_name=u'DataflowProjectsLocationsJobsListRequest', - response_type_name=u'ListJobsResponse', - supports_download=False, - ) - def Update(self, request, global_params=None): """Updates the state of an existing Cloud Dataflow job. @@ -588,19 +604,6 @@ def Update(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) - Update.method_config = lambda: base_api.ApiMethodInfo( - http_method=u'PUT', - method_id=u'dataflow.projects.locations.jobs.update', - ordered_params=[u'projectId', u'location', u'jobId'], - path_params=[u'jobId', u'location', u'projectId'], - query_params=[], - relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs/{jobId}', - request_field=u'job', - request_type_name=u'DataflowProjectsLocationsJobsUpdateRequest', - response_type_name=u'Job', - supports_download=False, - ) - class ProjectsLocationsService(base_api.BaseApiService): """Service class for the projects_locations resource.""" @@ -608,6 +611,9 @@ class ProjectsLocationsService(base_api.BaseApiService): def __init__(self, client): super(DataflowV1b3.ProjectsLocationsService, self).__init__(client) + self._method_configs = { + } + self._upload_configs = { } @@ -618,6 +624,21 @@ class ProjectsTemplatesService(base_api.BaseApiService): def __init__(self, client): super(DataflowV1b3.ProjectsTemplatesService, self).__init__(client) + self._method_configs = { + 'Create': base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'dataflow.projects.templates.create', + ordered_params=[u'projectId'], + path_params=[u'projectId'], + query_params=[], + relative_path=u'v1b3/projects/{projectId}/templates', + request_field=u'createJobFromTemplateRequest', + request_type_name=u'DataflowProjectsTemplatesCreateRequest', + response_type_name=u'Job', + supports_download=False, + ), + } + self._upload_configs = { } @@ -634,19 +655,6 @@ def Create(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) - Create.method_config = lambda: base_api.ApiMethodInfo( - http_method=u'POST', - method_id=u'dataflow.projects.templates.create', - ordered_params=[u'projectId'], - path_params=[u'projectId'], - query_params=[], - relative_path=u'v1b3/projects/{projectId}/templates', - request_field=u'createJobFromTemplateRequest', - request_type_name=u'DataflowProjectsTemplatesCreateRequest', - response_type_name=u'Job', - supports_download=False, - ) - class ProjectsService(base_api.BaseApiService): """Service class for the projects resource.""" @@ -654,6 +662,21 @@ class ProjectsService(base_api.BaseApiService): def __init__(self, client): super(DataflowV1b3.ProjectsService, self).__init__(client) + self._method_configs = { + 'WorkerMessages': base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'dataflow.projects.workerMessages', + ordered_params=[u'projectId'], + path_params=[u'projectId'], + query_params=[], + relative_path=u'v1b3/projects/{projectId}/WorkerMessages', + request_field=u'sendWorkerMessagesRequest', + request_type_name=u'DataflowProjectsWorkerMessagesRequest', + response_type_name=u'SendWorkerMessagesResponse', + supports_download=False, + ), + } + self._upload_configs = { } @@ -669,16 +692,3 @@ def WorkerMessages(self, request, global_params=None): config = self.GetMethodConfig('WorkerMessages') return self._RunMethod( config, request, global_params=global_params) - - WorkerMessages.method_config = lambda: base_api.ApiMethodInfo( - http_method=u'POST', - method_id=u'dataflow.projects.workerMessages', - ordered_params=[u'projectId'], - path_params=[u'projectId'], - query_params=[], - relative_path=u'v1b3/projects/{projectId}/WorkerMessages', - request_field=u'sendWorkerMessagesRequest', - request_type_name=u'DataflowProjectsWorkerMessagesRequest', - response_type_name=u'SendWorkerMessagesResponse', - supports_download=False, - ) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_messages.py b/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_messages.py index a42154eb7aed..4f1ccdb68d44 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_messages.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_messages.py @@ -129,6 +129,55 @@ class AlgorithmValueValuesEnum(_messages.Enum): maxNumWorkers = _messages.IntegerField(2, variant=_messages.Variant.INT32) +class CPUTime(_messages.Message): + """Modeled after information exposed by /proc/stat. + + Fields: + rate: Average CPU utilization rate (% non-idle cpu / second) since + previous sample. + timestamp: Timestamp of the measurement. + totalMs: Total active CPU time across all cores (ie., non-idle) in + milliseconds since start-up. + """ + + rate = _messages.FloatField(1) + timestamp = _messages.StringField(2) + totalMs = _messages.IntegerField(3, variant=_messages.Variant.UINT64) + + +class ComponentSource(_messages.Message): + """Description of an interstitial value between transforms in an execution + stage. + + Fields: + name: Dataflow service generated name for this source. + originalTransformOrCollection: User name for the original user transform + or collection with which this source is most closely associated. + userName: Human-readable name for this transform; may be user or system + generated. + """ + + name = _messages.StringField(1) + originalTransformOrCollection = _messages.StringField(2) + userName = _messages.StringField(3) + + +class ComponentTransform(_messages.Message): + """Description of a transform executed as part of an execution stage. + + Fields: + name: Dataflow service generated name for this source. + originalTransform: User name for the original user transform with which + this transform is most closely associated. + userName: Human-readable name for this transform; may be user or system + generated. + """ + + name = _messages.StringField(1) + originalTransform = _messages.StringField(2) + userName = _messages.StringField(3) + + class ComputationTopology(_messages.Message): """All configuration data for a particular Computation. @@ -239,8 +288,8 @@ class CounterStructuredName(_messages.Message): structured names are the same get merged into a single value for the job. Enums: + OriginValueValuesEnum: One of the standard Origins defined above. PortionValueValuesEnum: Portion of this counter, either key or value. - StandardOriginValueValuesEnum: One of the standard Origins defined above. Fields: componentStepName: Name of the optimized step being executed by the @@ -249,14 +298,25 @@ class CounterStructuredName(_messages.Message): component steps. name: Counter name. Not necessarily globally-unique, but unique within the context of the other fields. Required. + origin: One of the standard Origins defined above. + originNamespace: A string containing a more specific namespace of the + counter's origin. originalStepName: System generated name of the original step in the user's graph, before optimization. - otherOrigin: A string containing the origin of the counter. portion: Portion of this counter, either key or value. - standardOrigin: One of the standard Origins defined above. workerId: ID of a particular worker. """ + class OriginValueValuesEnum(_messages.Enum): + """One of the standard Origins defined above. + + Values: + SYSTEM: Counter was created by the Dataflow system. + USER: Counter was created by the user. + """ + SYSTEM = 0 + USER = 1 + class PortionValueValuesEnum(_messages.Enum): """Portion of this counter, either key or value. @@ -269,23 +329,13 @@ class PortionValueValuesEnum(_messages.Enum): KEY = 1 VALUE = 2 - class StandardOriginValueValuesEnum(_messages.Enum): - """One of the standard Origins defined above. - - Values: - DATAFLOW: Counter was created by the Dataflow system. - USER: Counter was created by the user. - """ - DATAFLOW = 0 - USER = 1 - componentStepName = _messages.StringField(1) executionStepName = _messages.StringField(2) name = _messages.StringField(3) - originalStepName = _messages.StringField(4) - otherOrigin = _messages.StringField(5) - portion = _messages.EnumField('PortionValueValuesEnum', 6) - standardOrigin = _messages.EnumField('StandardOriginValueValuesEnum', 7) + origin = _messages.EnumField('OriginValueValuesEnum', 4) + originNamespace = _messages.StringField(5) + originalStepName = _messages.StringField(6) + portion = _messages.EnumField('PortionValueValuesEnum', 7) workerId = _messages.StringField(8) @@ -437,10 +487,12 @@ class ViewValueValuesEnum(_messages.Enum): JOB_VIEW_UNKNOWN: JOB_VIEW_SUMMARY: JOB_VIEW_ALL: + JOB_VIEW_DESCRIPTION: """ JOB_VIEW_UNKNOWN = 0 JOB_VIEW_SUMMARY = 1 JOB_VIEW_ALL = 2 + JOB_VIEW_DESCRIPTION = 3 job = _messages.MessageField('Job', 1) location = _messages.StringField(2) @@ -516,10 +568,12 @@ class ViewValueValuesEnum(_messages.Enum): JOB_VIEW_UNKNOWN: JOB_VIEW_SUMMARY: JOB_VIEW_ALL: + JOB_VIEW_DESCRIPTION: """ JOB_VIEW_UNKNOWN = 0 JOB_VIEW_SUMMARY = 1 JOB_VIEW_ALL = 2 + JOB_VIEW_DESCRIPTION = 3 jobId = _messages.StringField(1, required=True) location = _messages.StringField(2) @@ -570,10 +624,12 @@ class ViewValueValuesEnum(_messages.Enum): JOB_VIEW_UNKNOWN: JOB_VIEW_SUMMARY: JOB_VIEW_ALL: + JOB_VIEW_DESCRIPTION: """ JOB_VIEW_UNKNOWN = 0 JOB_VIEW_SUMMARY = 1 JOB_VIEW_ALL = 2 + JOB_VIEW_DESCRIPTION = 3 filter = _messages.EnumField('FilterValueValuesEnum', 1) location = _messages.StringField(2) @@ -703,10 +759,12 @@ class ViewValueValuesEnum(_messages.Enum): JOB_VIEW_UNKNOWN: JOB_VIEW_SUMMARY: JOB_VIEW_ALL: + JOB_VIEW_DESCRIPTION: """ JOB_VIEW_UNKNOWN = 0 JOB_VIEW_SUMMARY = 1 JOB_VIEW_ALL = 2 + JOB_VIEW_DESCRIPTION = 3 job = _messages.MessageField('Job', 1) location = _messages.StringField(2, required=True) @@ -752,10 +810,12 @@ class ViewValueValuesEnum(_messages.Enum): JOB_VIEW_UNKNOWN: JOB_VIEW_SUMMARY: JOB_VIEW_ALL: + JOB_VIEW_DESCRIPTION: """ JOB_VIEW_UNKNOWN = 0 JOB_VIEW_SUMMARY = 1 JOB_VIEW_ALL = 2 + JOB_VIEW_DESCRIPTION = 3 jobId = _messages.StringField(1, required=True) location = _messages.StringField(2, required=True) @@ -806,10 +866,12 @@ class ViewValueValuesEnum(_messages.Enum): JOB_VIEW_UNKNOWN: JOB_VIEW_SUMMARY: JOB_VIEW_ALL: + JOB_VIEW_DESCRIPTION: """ JOB_VIEW_UNKNOWN = 0 JOB_VIEW_SUMMARY = 1 JOB_VIEW_ALL = 2 + JOB_VIEW_DESCRIPTION = 3 filter = _messages.EnumField('FilterValueValuesEnum', 1) location = _messages.StringField(2, required=True) @@ -1014,6 +1076,46 @@ class Disk(_messages.Message): sizeGb = _messages.IntegerField(3, variant=_messages.Variant.INT32) +class DisplayData(_messages.Message): + """Data provided with a pipeline or transform to provide descriptive info. + + Fields: + boolValue: Contains value if the data is of a boolean type. + durationValue: Contains value if the data is of duration type. + floatValue: Contains value if the data is of float type. + int64Value: Contains value if the data is of int64 type. + javaClassValue: Contains value if the data is of java class type. + key: The key identifying the display data. This is intended to be used as + a label for the display data when viewed in a dax monitoring system. + label: An optional label to display in a dax UI for the element. + namespace: The namespace for the key. This is usually a class name or + programming language namespace (i.e. python module) which defines the + display data. This allows a dax monitoring system to specially handle + the data and perform custom rendering. + shortStrValue: A possible additional shorter value to display. For example + a java_class_name_value of com.mypackage.MyDoFn will be stored with + MyDoFn as the short_str_value and com.mypackage.MyDoFn as the + java_class_name value. short_str_value can be displayed and + java_class_name_value will be displayed as a tooltip. + strValue: Contains value if the data is of string type. + timestampValue: Contains value if the data is of timestamp type. + url: An optional full URL. + """ + + boolValue = _messages.BooleanField(1) + durationValue = _messages.StringField(2) + floatValue = _messages.FloatField(3, variant=_messages.Variant.FLOAT) + int64Value = _messages.IntegerField(4) + javaClassValue = _messages.StringField(5) + key = _messages.StringField(6) + label = _messages.StringField(7) + namespace = _messages.StringField(8) + shortStrValue = _messages.StringField(9) + strValue = _messages.StringField(10) + timestampValue = _messages.StringField(11) + url = _messages.StringField(12) + + class DistributionUpdate(_messages.Message): """A metric value representing a distribution. @@ -1210,6 +1312,59 @@ class AdditionalProperty(_messages.Message): workerPools = _messages.MessageField('WorkerPool', 10, repeated=True) +class ExecutionStageSummary(_messages.Message): + """Description of the composing transforms, names/ids, and input/outputs of + a stage of execution. Some composing transforms and sources may have been + generated by the Dataflow service during execution planning. + + Enums: + KindValueValuesEnum: Type of tranform this stage is executing. + + Fields: + componentSource: Collections produced and consumed by component transforms + of this stage. + componentTransform: Transforms that comprise this execution stage. + id: Dataflow service generated id for this stage. + inputSource: Input sources for this stage. + kind: Type of tranform this stage is executing. + name: Dataflow service generated name for this stage. + outputSource: Output sources for this stage. + """ + + class KindValueValuesEnum(_messages.Enum): + """Type of tranform this stage is executing. + + Values: + UNKNOWN_KIND: Unrecognized transform type. + PAR_DO_KIND: ParDo transform. + GROUP_BY_KEY_KIND: Group By Key transform. + FLATTEN_KIND: Flatten transform. + READ_KIND: Read transform. + WRITE_KIND: Write transform. + CONSTANT_KIND: Constructs from a constant value, such as with Create.of. + SINGLETON_KIND: Creates a Singleton view of a collection. + SHUFFLE_KIND: Opening or closing a shuffle session, often as part of a + GroupByKey. + """ + UNKNOWN_KIND = 0 + PAR_DO_KIND = 1 + GROUP_BY_KEY_KIND = 2 + FLATTEN_KIND = 3 + READ_KIND = 4 + WRITE_KIND = 5 + CONSTANT_KIND = 6 + SINGLETON_KIND = 7 + SHUFFLE_KIND = 8 + + componentSource = _messages.MessageField('ComponentSource', 1, repeated=True) + componentTransform = _messages.MessageField('ComponentTransform', 2, repeated=True) + id = _messages.IntegerField(3) + inputSource = _messages.MessageField('StageSource', 4, repeated=True) + kind = _messages.EnumField('KindValueValuesEnum', 5) + name = _messages.StringField(6) + outputSource = _messages.MessageField('StageSource', 7, repeated=True) + + class FailedLocation(_messages.Message): """Indicates which location failed to respond to a request for data. @@ -1412,8 +1567,7 @@ class Job(_messages.Message): callers cannot mutate it. currentStateTime: The timestamp associated with the current state. environment: The environment for the job. - executionInfo: Information about how the Cloud Dataflow service will run - the job. + executionInfo: Deprecated. id: The unique ID of this job. This field is set by the Cloud Dataflow service when the Job is created, and is immutable for the life of the job. @@ -1429,6 +1583,10 @@ class Job(_messages.Message): attempts to create a Job with the same name as an already-existing Job, the attempt returns the existing Job. The name must match the regular expression `[a-z]([-a-z0-9]{0,38}[a-z0-9])?` + pipelineDescription: Preliminary field: The format of this data may change + at any time. A description of the user pipeline and stages through which + it is executed. Created by Cloud Dataflow service. Only retrieved with + JOB_VIEW_DESCRIPTION or JOB_VIEW_ALL. projectId: The ID of the Cloud Platform project that the job belongs to. replaceJobId: If this job is an update of an existing job, this field is the job ID of the job it replaced. When sending a `CreateJobRequest`, @@ -1643,14 +1801,15 @@ class AdditionalProperty(_messages.Message): labels = _messages.MessageField('LabelsValue', 8) location = _messages.StringField(9) name = _messages.StringField(10) - projectId = _messages.StringField(11) - replaceJobId = _messages.StringField(12) - replacedByJobId = _messages.StringField(13) - requestedState = _messages.EnumField('RequestedStateValueValuesEnum', 14) - steps = _messages.MessageField('Step', 15, repeated=True) - tempFiles = _messages.StringField(16, repeated=True) - transformNameMapping = _messages.MessageField('TransformNameMappingValue', 17) - type = _messages.EnumField('TypeValueValuesEnum', 18) + pipelineDescription = _messages.MessageField('PipelineDescription', 11) + projectId = _messages.StringField(12) + replaceJobId = _messages.StringField(13) + replacedByJobId = _messages.StringField(14) + requestedState = _messages.EnumField('RequestedStateValueValuesEnum', 15) + steps = _messages.MessageField('Step', 16, repeated=True) + tempFiles = _messages.StringField(17, repeated=True) + transformNameMapping = _messages.MessageField('TransformNameMappingValue', 18) + type = _messages.EnumField('TypeValueValuesEnum', 19) class JobExecutionInfo(_messages.Message): @@ -2259,6 +2418,24 @@ class AdditionalProperty(_messages.Message): valueCombiningFn = _messages.MessageField('ValueCombiningFnValue', 6) +class PipelineDescription(_messages.Message): + """A descriptive representation of submitted pipeline as well as the + executed form. This data is provided by the Dataflow service for ease of + visualizing the pipeline and interpretting Dataflow provided metrics. + + Fields: + displayData: Pipeline level display data. + executionPipelineStage: Description of each stage of execution of the + pipeline. + originalPipelineTransform: Description of each transform in the pipeline + and collections between them. + """ + + displayData = _messages.MessageField('DisplayData', 1, repeated=True) + executionPipelineStage = _messages.MessageField('ExecutionStageSummary', 2, repeated=True) + originalPipelineTransform = _messages.MessageField('TransformSummary', 3, repeated=True) + + class Position(_messages.Message): """Position defines a position within a collection of data. The value can be either the end position, a key (used with ordered collections), a byte @@ -2377,43 +2554,13 @@ class ReportedParallelism(_messages.Message): class ResourceUtilizationReport(_messages.Message): """Worker metrics exported from workers. This contains resource utilization metrics accumulated from a variety of sources. For more information, see go - /df-resource-signals. Note that this proto closely follows the structure of - its DFE siblings in its contents. - - Messages: - MetricsValueListEntry: A MetricsValueListEntry object. + /df-resource-signals. Fields: - metrics: Each Struct must parallel DFE worker metrics protos (eg., - cpu_time metric will have nested values \u201ctimestamp_ms, total_ms, rate\u201d). + cpuTime: CPU utilization samples. """ - @encoding.MapUnrecognizedFields('additionalProperties') - class MetricsValueListEntry(_messages.Message): - """A MetricsValueListEntry object. - - Messages: - AdditionalProperty: An additional property for a MetricsValueListEntry - object. - - Fields: - additionalProperties: Properties of the object. - """ - - class AdditionalProperty(_messages.Message): - """An additional property for a MetricsValueListEntry object. - - Fields: - key: Name of the additional property. - value: A extra_types.JsonValue attribute. - """ - - key = _messages.StringField(1) - value = _messages.MessageField('extra_types.JsonValue', 2) - - additionalProperties = _messages.MessageField('AdditionalProperty', 1, repeated=True) - - metrics = _messages.MessageField('MetricsValueListEntry', 1, repeated=True) + cpuTime = _messages.MessageField('CPUTime', 1, repeated=True) class ResourceUtilizationReportResponse(_messages.Message): @@ -2996,6 +3143,25 @@ class SplitInt64(_messages.Message): lowBits = _messages.IntegerField(2, variant=_messages.Variant.UINT32) +class StageSource(_messages.Message): + """Description of an input or output of an execution stage. + + Fields: + name: Dataflow service generated name for this source. + originalUserTransformOrCollection: User name for the original user + transform or collection with which this source is most closely + associated. + sizeBytes: Size of the source, if measurable. + userName: Human-readable name for this source; may be user or system + generated. + """ + + name = _messages.StringField(1) + originalUserTransformOrCollection = _messages.StringField(2) + sizeBytes = _messages.IntegerField(3) + userName = _messages.StringField(4) + + class StandardQueryParameters(_messages.Message): """Query parameters accepted by all methods. @@ -3169,20 +3335,23 @@ class Step(_messages.Message): Messages: PropertiesValue: Named properties associated with the step. Each kind of - predefined step has its own required set of properties. + predefined step has its own required set of properties. Must be provided + on Create. Only retrieved with JOB_VIEW_ALL. Fields: kind: The kind of step in the Cloud Dataflow job. name: The name that identifies the step. This must be unique for each step with respect to all other steps in the Cloud Dataflow job. properties: Named properties associated with the step. Each kind of - predefined step has its own required set of properties. + predefined step has its own required set of properties. Must be provided + on Create. Only retrieved with JOB_VIEW_ALL. """ @encoding.MapUnrecognizedFields('additionalProperties') class PropertiesValue(_messages.Message): """Named properties associated with the step. Each kind of predefined step - has its own required set of properties. + has its own required set of properties. Must be provided on Create. Only + retrieved with JOB_VIEW_ALL. Messages: AdditionalProperty: An additional property for a PropertiesValue object. @@ -3502,6 +3671,56 @@ class AdditionalProperty(_messages.Message): userStageToComputationNameMap = _messages.MessageField('UserStageToComputationNameMapValue', 5) +class TransformSummary(_messages.Message): + """Description of the type, names/ids, and input/outputs for a transform. + + Enums: + KindValueValuesEnum: Type of transform. + + Fields: + displayData: Transform-specific display data. + id: SDK generated id of this transform instance. + inputCollectionName: User names for all collection inputs to this + transform. + kind: Type of transform. + name: User provided name for this transform instance. + outputCollectionName: User names for all collection outputs to this + transform. + """ + + class KindValueValuesEnum(_messages.Enum): + """Type of transform. + + Values: + UNKNOWN_KIND: Unrecognized transform type. + PAR_DO_KIND: ParDo transform. + GROUP_BY_KEY_KIND: Group By Key transform. + FLATTEN_KIND: Flatten transform. + READ_KIND: Read transform. + WRITE_KIND: Write transform. + CONSTANT_KIND: Constructs from a constant value, such as with Create.of. + SINGLETON_KIND: Creates a Singleton view of a collection. + SHUFFLE_KIND: Opening or closing a shuffle session, often as part of a + GroupByKey. + """ + UNKNOWN_KIND = 0 + PAR_DO_KIND = 1 + GROUP_BY_KEY_KIND = 2 + FLATTEN_KIND = 3 + READ_KIND = 4 + WRITE_KIND = 5 + CONSTANT_KIND = 6 + SINGLETON_KIND = 7 + SHUFFLE_KIND = 8 + + displayData = _messages.MessageField('DisplayData', 1, repeated=True) + id = _messages.StringField(2) + inputCollectionName = _messages.StringField(3, repeated=True) + kind = _messages.EnumField('KindValueValuesEnum', 4) + name = _messages.StringField(5) + outputCollectionName = _messages.StringField(6, repeated=True) + + class WorkItem(_messages.Message): """WorkItem represents basic information about a WorkItem to be executed in the cloud. From 11a381b23541a305ed115a62df4794e0a1d79cbc Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Fri, 3 Mar 2017 17:22:27 -0800 Subject: [PATCH 24/46] Revert "[maven-release-plugin] prepare branch release-0.6.0" This reverts commit 1a770ef2f21b96fbdc5ff06ea8642351f136328f. --- pom.xml | 2 +- runners/core-construction-java/pom.xml | 4 +++- sdks/common/fn-api/pom.xml | 4 +++- sdks/common/runner-api/pom.xml | 4 +++- sdks/java/extensions/jackson/pom.xml | 4 +++- sdks/java/harness/pom.xml | 4 +++- sdks/java/javadoc/pom.xml | 10 +++++----- 7 files changed, 21 insertions(+), 11 deletions(-) diff --git a/pom.xml b/pom.xml index eded6840dfd8..a37f1af4d4d0 100644 --- a/pom.xml +++ b/pom.xml @@ -48,7 +48,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/beam.git scm:git:https://git-wip-us.apache.org/repos/asf/beam.git https://git-wip-us.apache.org/repos/asf?p=beam.git;a=summary - release-0.6.0 + HEAD diff --git a/runners/core-construction-java/pom.xml b/runners/core-construction-java/pom.xml index b602f5dca8f0..868f74324de3 100644 --- a/runners/core-construction-java/pom.xml +++ b/runners/core-construction-java/pom.xml @@ -17,7 +17,9 @@ ~ limitations under the License. --> - + 4.0.0 diff --git a/sdks/common/fn-api/pom.xml b/sdks/common/fn-api/pom.xml index 5a41d9e365b9..1f6193ffe58c 100644 --- a/sdks/common/fn-api/pom.xml +++ b/sdks/common/fn-api/pom.xml @@ -15,7 +15,9 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 jar diff --git a/sdks/common/runner-api/pom.xml b/sdks/common/runner-api/pom.xml index 9c6de1ebfad1..8eaeb8e321af 100644 --- a/sdks/common/runner-api/pom.xml +++ b/sdks/common/runner-api/pom.xml @@ -15,7 +15,9 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 jar diff --git a/sdks/java/extensions/jackson/pom.xml b/sdks/java/extensions/jackson/pom.xml index 1dfbd72d6fa9..be5c9537da95 100644 --- a/sdks/java/extensions/jackson/pom.xml +++ b/sdks/java/extensions/jackson/pom.xml @@ -15,7 +15,9 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 diff --git a/sdks/java/harness/pom.xml b/sdks/java/harness/pom.xml index 80b01ca88695..3abe70b171b1 100644 --- a/sdks/java/harness/pom.xml +++ b/sdks/java/harness/pom.xml @@ -15,7 +15,9 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 jar diff --git a/sdks/java/javadoc/pom.xml b/sdks/java/javadoc/pom.xml index b785c986dbdd..243dae5d448d 100644 --- a/sdks/java/javadoc/pom.xml +++ b/sdks/java/javadoc/pom.xml @@ -254,12 +254,12 @@ package - - - - + + + + - + From 4febd954af00458032efbee45b7f9724fe0ea9ed Mon Sep 17 00:00:00 2001 From: Aviem Zur Date: Sun, 5 Mar 2017 16:17:35 +0200 Subject: [PATCH 25/46] [BEAM-1625] BoundedDataset action() does not materialize RDD --- .../beam/runners/spark/translation/BoundedDataset.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java index 5e198467abb2..7db04a8a2c85 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java @@ -20,7 +20,6 @@ import com.google.common.base.Function; import com.google.common.collect.Iterables; -import java.util.Iterator; import java.util.List; import javax.annotation.Nullable; import org.apache.beam.runners.spark.coders.CoderHelpers; @@ -107,9 +106,8 @@ public void cache(String storageLevel) { @Override public void action() { // Empty function to force computation of RDD. - rdd.foreachPartition(new VoidFunction>>() { - @Override - public void call(Iterator> windowedValueIterator) throws Exception { + rdd.foreach(new VoidFunction>() { + @Override public void call(WindowedValue tWindowedValue) throws Exception { // Empty implementation. } }); From e11d6d27174f866371a253449908c9a6033114d5 Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Sun, 5 Mar 2017 15:10:49 -0800 Subject: [PATCH 26/46] Update Guava version from 19.0 to 20.0 in example projects The Guava version was updated in the main project on January 19. --- .../src/main/resources/archetype-resources/pom.xml | 2 +- .../examples/src/main/resources/archetype-resources/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml index 55211ed5576e..e025f9ec509e 100644 --- a/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml @@ -270,7 +270,7 @@ com.google.guava guava - 19.0 + 20.0 diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml index 654973ca2d1f..6a1232475bb6 100644 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml @@ -316,7 +316,7 @@ com.google.guava guava - 19.0 + 20.0 From 630df6f0e328d52d0892a56ae8493946b78dd6b9 Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Sun, 5 Mar 2017 15:21:22 -0800 Subject: [PATCH 27/46] Java 8 examples: add SNAPSHOT repository to the pom.xml file This enables users of the archetype to reference nightly builds if they so choose. --- .../src/main/resources/archetype-resources/pom.xml | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml index e025f9ec509e..f30a35f96f86 100644 --- a/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml @@ -30,6 +30,20 @@ 0.6.0-SNAPSHOT + + + apache.snapshots + Apache Development Snapshot Repository + https://repository.apache.org/content/repositories/snapshots/ + + false + + + true + + + + From 90c8094d883983f0006065169fd3ef230963e083 Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Sun, 5 Mar 2017 15:23:07 -0800 Subject: [PATCH 28/46] Java 8 examples: add shade configuration for producing bundled/fat jar This helps users who may need the fat jar to submit it to the cluster. --- .../resources/archetype-resources/pom.xml | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml index f30a35f96f86..3907fd5ba343 100644 --- a/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml @@ -73,6 +73,38 @@ + + + + org.apache.maven.plugins + maven-shade-plugin + 2.4.1 + + + package + + shade + + + ${project.artifactId}-bundled-${project.version} + + + *:* + + META-INF/LICENSE + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + From 0360b1b3be6190d36e1af1d2d077251ef60106d8 Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Sun, 5 Mar 2017 16:43:48 -0800 Subject: [PATCH 29/46] Java examples: move shade plugin into default lifecycle This helps users running the examples on runner that require the fat/bundled jar. --- .../resources/archetype-resources/pom.xml | 64 +++++++++---------- 1 file changed, 32 insertions(+), 32 deletions(-) diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml index 6a1232475bb6..0ed23fdda579 100644 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml @@ -73,6 +73,38 @@ + + + + org.apache.maven.plugins + maven-shade-plugin + 2.4.1 + + + package + + shade + + + ${project.artifactId}-bundled-${project.version} + + + *:* + + META-INF/LICENSE + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + @@ -85,38 +117,6 @@ false - - - - org.apache.maven.plugins - maven-shade-plugin - 2.4.1 - - - package - - shade - - - ${project.artifactId}-bundled-${project.version} - - - *:* - - META-INF/LICENSE - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - From b31be1caef6a691006082e3e440d0b42ff1d4165 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Sun, 5 Mar 2017 20:03:18 -0800 Subject: [PATCH 30/46] Revert "DataflowRunner: experimental support for issuing FnAPI based jobs" This reverts commit 131c9f916dae6345ec77a869112ae5901b568f23. --- runners/google-cloud-dataflow-java/pom.xml | 3 +- .../dataflow/DataflowPipelineTranslator.java | 3 +- .../beam/runners/dataflow/DataflowRunner.java | 46 ++++++++----------- .../runners/dataflow/DataflowRunnerInfo.java | 38 ++++++++------- .../options/DataflowPipelineDebugOptions.java | 2 - .../DataflowPipelineWorkerPoolOptions.java | 10 ++-- .../beam/runners/dataflow/dataflow.properties | 8 ++-- .../dataflow/DataflowRunnerInfoTest.java | 23 ++++------ .../runners/dataflow/DataflowRunnerTest.java | 17 ------- 9 files changed, 58 insertions(+), 92 deletions(-) diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index fb067975d4ed..fdd088f55ab1 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -34,8 +34,7 @@ beam-master-20170228 - 1 - 6 + 6 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 ab4cb9c67b04..7a78a4cd2a19 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 @@ -326,7 +326,8 @@ public Job translate(List packages) { workerPool.setNumWorkers(options.getNumWorkers()); if (options.isStreaming() - && !DataflowRunner.hasExperiment(options, "enable_windmill_service")) { + && (options.getExperiments() == null + || !options.getExperiments().contains("enable_windmill_service"))) { // Use separate data disk for streaming. Disk disk = new Disk(); disk.setDiskType(options.getWorkerDiskType()); 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 c609b540035f..15147f1bfb57 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 @@ -51,6 +51,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -303,12 +304,14 @@ public static DataflowRunner fromOptions(PipelineOptions options) { PTransformMatchers.parDoWithFnType(unsupported), UnsupportedOverrideFactory.withMessage(getUnsupportedMessage(unsupported, true))); } - if (!hasExperiment(options, "enable_custom_pubsub_source")) { + if (options.getExperiments() == null + || !options.getExperiments().contains("enable_custom_pubsub_source")) { ptoverrides.put( PTransformMatchers.classEqualTo(PubsubUnboundedSource.class), new ReflectiveRootOverrideFactory(StreamingPubsubIORead.class, this)); } - if (!hasExperiment(options, "enable_custom_pubsub_sink")) { + if (options.getExperiments() == null + || !options.getExperiments().contains("enable_custom_pubsub_sink")) { ptoverrides.put( PTransformMatchers.classEqualTo(PubsubUnboundedSink.class), new StreamingPubsubIOWriteOverrideFactory(this)); @@ -532,7 +535,20 @@ public DataflowPipelineJob run(Pipeline pipeline) { workerPool.setWorkerHarnessContainerImage(workerHarnessContainerImage); } - newJob.getEnvironment().setVersion(getEnvironmentVersion(options)); + // Requirements about the service. + Map environmentVersion = new HashMap<>(); + environmentVersion.put( + PropertyNames.ENVIRONMENT_VERSION_MAJOR_KEY, + DataflowRunnerInfo.getDataflowRunnerInfo().getEnvironmentMajorVersion()); + newJob.getEnvironment().setVersion(environmentVersion); + // Default jobType is JAVA_BATCH_AUTOSCALING: A Java job with workers that the job can + // autoscale if specified. + String jobType = "JAVA_BATCH_AUTOSCALING"; + + if (options.isStreaming()) { + jobType = "STREAMING"; + } + environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_JOB_TYPE_KEY, jobType); if (hooks != null) { hooks.modifyEnvironmentBeforeSubmission(newJob.getEnvironment()); @@ -640,30 +656,6 @@ public DataflowPipelineJob run(Pipeline pipeline) { return dataflowPipelineJob; } - /** Returns true if the specified experiment is enabled, handling null experiments. */ - public static boolean hasExperiment(DataflowPipelineDebugOptions options, String experiment) { - List experiments = - firstNonNull(options.getExperiments(), Collections.emptyList()); - return experiments.contains(experiment); - } - - /** Helper to configure the Dataflow Job Environment based on the user's job options. */ - private static Map getEnvironmentVersion(DataflowPipelineOptions options) { - DataflowRunnerInfo runnerInfo = DataflowRunnerInfo.getDataflowRunnerInfo(); - String majorVersion; - String jobType; - if (hasExperiment(options, "beam_fn_api")) { - majorVersion = runnerInfo.getFnApiEnvironmentMajorVersion(); - jobType = options.isStreaming() ? "FNAPI_STREAMING" : "FNAPI_BATCH"; - } else { - majorVersion = runnerInfo.getLegacyEnvironmentMajorVersion(); - jobType = options.isStreaming() ? "STREAMING" : "JAVA_BATCH_AUTOSCALING"; - } - return ImmutableMap.of( - PropertyNames.ENVIRONMENT_VERSION_MAJOR_KEY, majorVersion, - PropertyNames.ENVIRONMENT_VERSION_JOB_TYPE_KEY, jobType); - } - @VisibleForTesting void replaceTransforms(Pipeline pipeline) { for (Map.Entry override : overrides.entrySet()) { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerInfo.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerInfo.java index 12b3f384cbc3..59cb8a49de26 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerInfo.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerInfo.java @@ -47,34 +47,32 @@ public static DataflowRunnerInfo getDataflowRunnerInfo() { private Properties properties; - private static final String FNAPI_ENVIRONMENT_MAJOR_VERSION_KEY = - "fnapi.environment.major.version"; - private static final String LEGACY_ENVIRONMENT_MAJOR_VERSION_KEY = - "legacy.environment.major.version"; - private static final String CONTAINER_VERSION_KEY = "container.version"; + private static final String ENVIRONMENT_MAJOR_VERSION_KEY = "environment.major.version"; + private static final String BATCH_WORKER_HARNESS_CONTAINER_IMAGE_KEY = "worker.image.batch"; + private static final String STREAMING_WORKER_HARNESS_CONTAINER_IMAGE_KEY = + "worker.image.streaming"; - /** Provides the legacy environment's major version number. */ - public String getLegacyEnvironmentMajorVersion() { + /** Provides the environment's major version number. */ + public String getEnvironmentMajorVersion() { checkState( - properties.containsKey(LEGACY_ENVIRONMENT_MAJOR_VERSION_KEY), - "Unknown legacy environment major version"); - return properties.getProperty(LEGACY_ENVIRONMENT_MAJOR_VERSION_KEY); + properties.containsKey(ENVIRONMENT_MAJOR_VERSION_KEY), "Unknown environment major version"); + return properties.getProperty(ENVIRONMENT_MAJOR_VERSION_KEY); } - /** Provides the FnAPI environment's major version number. */ - public String getFnApiEnvironmentMajorVersion() { + /** Provides the batch worker harness container image name. */ + public String getBatchWorkerHarnessContainerImage() { checkState( - properties.containsKey(FNAPI_ENVIRONMENT_MAJOR_VERSION_KEY), - "Unknown FnAPI environment major version"); - return properties.getProperty(FNAPI_ENVIRONMENT_MAJOR_VERSION_KEY); + properties.containsKey(BATCH_WORKER_HARNESS_CONTAINER_IMAGE_KEY), + "Unknown batch worker harness container image"); + return properties.getProperty(BATCH_WORKER_HARNESS_CONTAINER_IMAGE_KEY); } - /** Provides the container version that will be used for constructing harness image paths. */ - public String getContainerVersion() { + /** Provides the streaming worker harness container image name. */ + public String getStreamingWorkerHarnessContainerImage() { checkState( - properties.containsKey(CONTAINER_VERSION_KEY), - "Unknown container version"); - return properties.getProperty(CONTAINER_VERSION_KEY); + properties.containsKey(STREAMING_WORKER_HARNESS_CONTAINER_IMAGE_KEY), + "Unknown streaming worker harness container image"); + return properties.getProperty(STREAMING_WORKER_HARNESS_CONTAINER_IMAGE_KEY); } private DataflowRunnerInfo(String resourcePath) { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java index 729bca47b585..cdfa3f59333a 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java @@ -21,7 +21,6 @@ import com.google.api.services.dataflow.Dataflow; import java.util.List; import java.util.Map; -import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.util.DataflowTransport; import org.apache.beam.runners.dataflow.util.GcsStager; import org.apache.beam.runners.dataflow.util.Stager; @@ -54,7 +53,6 @@ public interface DataflowPipelineDebugOptions extends PipelineOptions { + "be enabled with this flag. Please sync with the Dataflow team before enabling any " + "experiments.") @Experimental - @Nullable List getExperiments(); void setExperiments(List value); diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java index e2c4bf4eece0..3c5d05afd1ae 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java @@ -20,7 +20,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import java.util.List; import javax.annotation.Nullable; -import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.runners.dataflow.DataflowRunnerInfo; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.options.Default; @@ -130,14 +129,11 @@ class WorkerHarnessContainerImageFactory @Override public String create(PipelineOptions options) { DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); - String containerVersion = DataflowRunnerInfo.getDataflowRunnerInfo().getContainerVersion(); - String containerType; - if (DataflowRunner.hasExperiment(dataflowOptions, "beam_fn_api")) { - containerType = "java"; + if (dataflowOptions.isStreaming()) { + return DataflowRunnerInfo.getDataflowRunnerInfo().getStreamingWorkerHarnessContainerImage(); } else { - containerType = dataflowOptions.isStreaming() ? "beam-java-streaming" : "beam-java-batch"; + return DataflowRunnerInfo.getDataflowRunnerInfo().getBatchWorkerHarnessContainerImage(); } - return String.format("dataflow.gcr.io/v1beta3/%s:%s", containerType, containerVersion); } } diff --git a/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties b/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties index ac689708012f..47e316c26c42 100644 --- a/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties +++ b/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties @@ -16,6 +16,8 @@ # # Dataflow runtime properties -legacy.environment.major.version=${dataflow.legacy_environment_major_version} -fnapi.environment.major.version=${dataflow.fnapi_environment_major_version} -container.version=${dataflow.container_version} +environment.major.version=${dataflow.environment_major_version} + +worker.image.batch=dataflow.gcr.io/v1beta3/beam-java-batch:${dataflow.container_version} + +worker.image.streaming=dataflow.gcr.io/v1beta3/beam-java-streaming:${dataflow.container_version} diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerInfoTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerInfoTest.java index 350204099ebc..9b5b37497367 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerInfoTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerInfoTest.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.dataflow; import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; @@ -33,22 +32,20 @@ public class DataflowRunnerInfoTest { public void getDataflowRunnerInfo() throws Exception { DataflowRunnerInfo info = DataflowRunnerInfo.getDataflowRunnerInfo(); - String version = info.getLegacyEnvironmentMajorVersion(); + String version = info.getEnvironmentMajorVersion(); // Validate major version is a number assertTrue( - String.format("Legacy environment major version number %s is not a number", version), + String.format("Environment major version number %s is not a number", version), version.matches("\\d+")); - version = info.getFnApiEnvironmentMajorVersion(); - // Validate major version is a number - assertTrue( - String.format("FnAPI environment major version number %s is not a number", version), - version.matches("\\d+")); - - // Validate container version does not contain a $ (indicating it was not filled in). + // Validate container images contain gcr.io + assertThat( + "batch worker harness container image invalid", + info.getBatchWorkerHarnessContainerImage(), + containsString("gcr.io")); assertThat( - "container version invalid", - info.getContainerVersion(), - not(containsString("$"))); + "streaming worker harness container image invalid", + info.getStreamingWorkerHarnessContainerImage(), + containsString("gcr.io")); } } 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 246feb03d081..a7880777a7c4 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 @@ -24,7 +24,6 @@ import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.startsWith; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; @@ -1119,20 +1118,4 @@ public void testTemplateRunnerLoggedErrorForFile() throws Exception { thrown.expect(RuntimeException.class); p.run(); } - - @Test - public void testHasExperiment() { - DataflowPipelineDebugOptions options = - PipelineOptionsFactory.as(DataflowPipelineDebugOptions.class); - - options.setExperiments(null); - assertFalse(DataflowRunner.hasExperiment(options, "foo")); - - options.setExperiments(ImmutableList.of("foo", "bar")); - assertTrue(DataflowRunner.hasExperiment(options, "foo")); - assertTrue(DataflowRunner.hasExperiment(options, "bar")); - assertFalse(DataflowRunner.hasExperiment(options, "baz")); - assertFalse(DataflowRunner.hasExperiment(options, "ba")); - assertFalse(DataflowRunner.hasExperiment(options, "BAR")); - } } From 47c80926564f92ac0c0b4647f005ac86b0eaf649 Mon Sep 17 00:00:00 2001 From: Sela Date: Sun, 5 Mar 2017 23:37:08 +0200 Subject: [PATCH 31/46] [BEAM-1626] Remove cache of MapWithStateDStream on read. --- .../org/apache/beam/runners/spark/io/SparkUnboundedSource.java | 2 -- 1 file changed, 2 deletions(-) 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 354461fac263..e5bbaf185e05 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 @@ -94,8 +94,6 @@ public static UnboundedDataset re // set checkpoint duration for read stream, if set. checkpointStream(mapWithStateDStream, options); - // cache since checkpointing is less frequent. - mapWithStateDStream.cache(); // report the number of input elements for this InputDStream to the InputInfoTracker. int id = inputDStream.inputDStream().id(); From d8bc618edafd07ae8e0ec692fc7f3df7395b876e Mon Sep 17 00:00:00 2001 From: Aviem Zur Date: Sun, 5 Mar 2017 07:15:32 +0200 Subject: [PATCH 32/46] [BEAM-1623] Transform Reshuffle directly in Spark runner --- .../translation/GroupCombineFunctions.java | 22 +++++++++++ .../translation/TransformTranslator.java | 38 ++++++++++++++----- .../spark/translation/TranslationUtils.java | 10 +++++ .../StreamingTransformTranslator.java | 36 ++++++++++++++++++ 4 files changed, 97 insertions(+), 9 deletions(-) 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 1e879ce54573..b2a589da2e2d 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 @@ -203,4 +203,26 @@ public static Iterable> combineGlobally( return accumulatedBytes.mapToPair(CoderHelpers.fromByteFunction(keyCoder, iterAccumCoder)); } + + /** + * An implementation of + * {@link org.apache.beam.sdk.util.Reshuffle} for the Spark runner. + */ + public static JavaRDD>> reshuffle( + JavaRDD>> rdd, + Coder keyCoder, + WindowedValueCoder wvCoder) { + + // Use coders to convert objects in the PCollection to byte arrays, so they + // can be transferred over the network for the shuffle. + return rdd + .map(new ReifyTimestampsAndWindowsFunction()) + .map(WindowingHelpers.>>unwindowFunction()) + .mapToPair(TranslationUtils.>toPairFunction()) + .mapToPair(CoderHelpers.toByteFunction(keyCoder, wvCoder)) + .repartition(rdd.getNumPartitions()) + .mapToPair(CoderHelpers.fromByteFunction(keyCoder, wvCoder)) + .map(TranslationUtils.>fromPairFunction()) + .map(TranslationUtils.toKVByWindowInValue()); + } } 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 a4939b976448..0ae731328913 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 @@ -69,6 +69,7 @@ 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.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; @@ -318,15 +319,7 @@ public Iterable> call( return sparkCombineFn.extractOutput(iter); } }).map(TranslationUtils.>fromPairFunction()) - .map(new Function>, - WindowedValue>>() { - @Override - public WindowedValue> call( - KV> kv) throws Exception { - WindowedValue wv = kv.getValue(); - return wv.withValue(KV.of(kv.getKey(), wv.getValue())); - } - }); + .map(TranslationUtils.toKVByWindowInValue()); context.putDataset(transform, new BoundedDataset<>(outRdd)); } @@ -735,6 +728,32 @@ public void evaluate(StorageLevelPTransform transform, EvaluationContext context }; } + private static TransformEvaluator> reshuffle() { + return new TransformEvaluator>() { + @Override public void evaluate(Reshuffle transform, EvaluationContext context) { + @SuppressWarnings("unchecked") + JavaRDD>> inRDD = + ((BoundedDataset>) context.borrowDataset(transform)).getRDD(); + @SuppressWarnings("unchecked") + final WindowingStrategy windowingStrategy = + (WindowingStrategy) context.getInput(transform).getWindowingStrategy(); + @SuppressWarnings("unchecked") + final KvCoder coder = (KvCoder) context.getInput(transform).getCoder(); + @SuppressWarnings("unchecked") + final WindowFn windowFn = (WindowFn) windowingStrategy.getWindowFn(); + + final Coder keyCoder = coder.getKeyCoder(); + final WindowedValue.WindowedValueCoder wvCoder = + WindowedValue.FullWindowedValueCoder.of(coder.getValueCoder(), windowFn.windowCoder()); + + JavaRDD>> reshuffled = + GroupCombineFunctions.reshuffle(inRDD, keyCoder, wvCoder); + + context.putDataset(transform, new BoundedDataset<>(reshuffled)); + } + }; + } + private static final Map, TransformEvaluator> EVALUATORS = Maps .newHashMap(); @@ -753,6 +772,7 @@ public void evaluate(StorageLevelPTransform transform, EvaluationContext context EVALUATORS.put(View.AsIterable.class, viewAsIter()); EVALUATORS.put(View.CreatePCollectionView.class, createPCollView()); EVALUATORS.put(Window.Assign.class, window()); + EVALUATORS.put(Reshuffle.class, reshuffle()); // mostly test evaluators EVALUATORS.put(StorageLevelPTransform.class, storageLevel()); } 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 158593e475e4..f2b34183e8b2 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 @@ -159,6 +159,16 @@ public Tuple2>> call( }; } + /** Extract window from a {@link KV} with {@link WindowedValue} value. */ + static Function>, WindowedValue>> toKVByWindowInValue() { + return new Function>, WindowedValue>>() { + @Override public WindowedValue> call(KV> kv) throws Exception { + WindowedValue wv = kv.getValue(); + return wv.withValue(KV.of(kv.getKey(), wv.getValue())); + } + }; + } + /** * A utility class to filter {@link TupleTag}s. * 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 628b713913d8..31307cc3781d 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 @@ -73,6 +73,7 @@ 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.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; @@ -445,6 +446,40 @@ public JavaRDD> call(JavaRDD> rdd) }; } + private static TransformEvaluator> reshuffle() { + return new TransformEvaluator>() { + @Override + public void evaluate(Reshuffle transform, EvaluationContext context) { + @SuppressWarnings("unchecked") UnboundedDataset> inputDataset = + (UnboundedDataset>) context.borrowDataset(transform); + List streamSources = inputDataset.getStreamSources(); + JavaDStream>> dStream = inputDataset.getDStream(); + @SuppressWarnings("unchecked") + final KvCoder coder = (KvCoder) context.getInput(transform).getCoder(); + @SuppressWarnings("unchecked") + final WindowingStrategy windowingStrategy = + (WindowingStrategy) context.getInput(transform).getWindowingStrategy(); + @SuppressWarnings("unchecked") + final WindowFn windowFn = (WindowFn) windowingStrategy.getWindowFn(); + + final WindowedValue.WindowedValueCoder wvCoder = + WindowedValue.FullWindowedValueCoder.of(coder.getValueCoder(), windowFn.windowCoder()); + + JavaDStream>> reshuffledStream = + dStream.transform(new Function>>, + JavaRDD>>>() { + @Override + public JavaRDD>> call( + JavaRDD>> rdd) throws Exception { + return GroupCombineFunctions.reshuffle(rdd, coder.getKeyCoder(), wvCoder); + } + }); + + context.putDataset(transform, new UnboundedDataset<>(reshuffledStream, streamSources)); + } + }; + } + private static final Map, TransformEvaluator> EVALUATORS = Maps.newHashMap(); @@ -457,6 +492,7 @@ public JavaRDD> call(JavaRDD> rdd) EVALUATORS.put(CreateStream.class, createFromQueue()); EVALUATORS.put(Window.Assign.class, window()); EVALUATORS.put(Flatten.PCollections.class, flattenPColl()); + EVALUATORS.put(Reshuffle.class, reshuffle()); } /** From 8766b03eb31b4f16de8fbf5a6902378a2c1151e0 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 6 Mar 2017 08:55:13 -0800 Subject: [PATCH 33/46] Revert "Implement Single-Output ParDo as a composite" This reverts commit 6253abaac62979e8496a828c18c7d1aa7214be6a. The reverted commit breaks Dataflow DisplayData. The actual fix will include a Dataflow override for single-output ParDos. --- .../translation/ApexPipelineTranslator.java | 3 +- ...or.java => ParDoBoundMultiTranslator.java} | 4 +- .../translation/ParDoBoundTranslator.java | 95 +++++++++++++++ .../FlattenPCollectionTranslatorTest.java | 3 +- ...est.java => ParDoBoundTranslatorTest.java} | 8 +- .../beam/runners/direct/DirectRunner.java | 18 ++- .../ParDoSingleViaMultiOverrideFactory.java | 70 +++++++++++ ...arDoSingleViaMultiOverrideFactoryTest.java | 46 +++++++ .../flink/FlinkBatchTransformTranslators.java | 78 +++++++++++- .../FlinkStreamingTransformTranslators.java | 113 ++++++++++++++++- .../dataflow/DataflowPipelineTranslator.java | 29 +++++ .../DataflowPipelineTranslatorTest.java | 7 +- .../translation/TransformTranslator.java | 100 +++++++-------- .../StreamingTransformTranslator.java | 115 ++++++++---------- .../streaming/TrackStreamingSourcesTest.java | 4 +- .../org/apache/beam/sdk/transforms/ParDo.java | 8 +- 16 files changed, 556 insertions(+), 145 deletions(-) rename runners/apex/src/main/java/org/apache/beam/runners/apex/translation/{ParDoTranslator.java => ParDoBoundMultiTranslator.java} (99%) create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java rename runners/apex/src/test/java/org/apache/beam/runners/apex/translation/{ParDoTranslatorTest.java => ParDoBoundTranslatorTest.java} (98%) create mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java create mode 100644 runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactoryTest.java diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java index 7eb955123cbe..951a286fb3af 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java @@ -59,7 +59,8 @@ public class ApexPipelineTranslator implements Pipeline.PipelineVisitor { static { // register TransformTranslators - registerTransformTranslator(ParDo.BoundMulti.class, new ParDoTranslator<>()); + registerTransformTranslator(ParDo.Bound.class, new ParDoBoundTranslator()); + registerTransformTranslator(ParDo.BoundMulti.class, new ParDoBoundMultiTranslator<>()); registerTransformTranslator(Read.Unbounded.class, new ReadUnboundedTranslator()); registerTransformTranslator(Read.Bounded.class, new ReadBoundedTranslator()); registerTransformTranslator(GroupByKey.class, new GroupByKeyTranslator()); diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java similarity index 99% rename from runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoTranslator.java rename to runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java index 5ffc3c389a68..f55b48cd7253 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java @@ -46,10 +46,10 @@ /** * {@link ParDo.BoundMulti} is translated to {@link ApexParDoOperator} that wraps the {@link DoFn}. */ -class ParDoTranslator +class ParDoBoundMultiTranslator implements TransformTranslator> { private static final long serialVersionUID = 1L; - private static final Logger LOG = LoggerFactory.getLogger(ParDoTranslator.class); + private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundMultiTranslator.class); @Override public void translate(ParDo.BoundMulti transform, TranslationContext context) { diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java new file mode 100644 index 000000000000..5195809bdbbf --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.apex.translation; + +import java.util.List; +import org.apache.beam.runners.apex.ApexRunner; +import org.apache.beam.runners.apex.translation.operators.ApexParDoOperator; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature; +import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; +import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; + +/** {@link ParDo.Bound} is translated to {link ApexParDoOperator} that wraps the {@link DoFn}. */ +class ParDoBoundTranslator + implements TransformTranslator> { + private static final long serialVersionUID = 1L; + + @Override + public void translate(ParDo.Bound transform, TranslationContext context) { + DoFn doFn = transform.getFn(); + DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass()); + + if (signature.processElement().isSplittable()) { + throw new UnsupportedOperationException( + String.format( + "%s does not support splittable DoFn: %s", ApexRunner.class.getSimpleName(), doFn)); + } + if (signature.stateDeclarations().size() > 0) { + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", + DoFn.StateId.class.getSimpleName(), + doFn.getClass().getName(), + DoFn.class.getSimpleName(), + ApexRunner.class.getSimpleName())); + } + + if (signature.timerDeclarations().size() > 0) { + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with timers in the %s.", + DoFn.TimerId.class.getSimpleName(), + doFn.getClass().getName(), + DoFn.class.getSimpleName(), + ApexRunner.class.getSimpleName())); + } + + PCollection output = (PCollection) context.getOutput(); + PCollection input = (PCollection) context.getInput(); + List> sideInputs = transform.getSideInputs(); + Coder inputCoder = input.getCoder(); + WindowedValueCoder wvInputCoder = + FullWindowedValueCoder.of( + inputCoder, input.getWindowingStrategy().getWindowFn().windowCoder()); + + ApexParDoOperator operator = + new ApexParDoOperator<>( + context.getPipelineOptions(), + doFn, + new TupleTag(), + TupleTagList.empty().getAll() /*sideOutputTags*/, + output.getWindowingStrategy(), + sideInputs, + wvInputCoder, + context.stateInternalsFactory()); + context.addOperator(operator, operator.output); + context.addStream(context.getInput(), operator.input); + if (!sideInputs.isEmpty()) { + ParDoBoundMultiTranslator.addSideInputs(operator, sideInputs, context); + } + } +} diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/FlattenPCollectionTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/FlattenPCollectionTranslatorTest.java index 64ca0ee4fd07..b2e29b6de085 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/FlattenPCollectionTranslatorTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/FlattenPCollectionTranslatorTest.java @@ -110,8 +110,7 @@ public void testFlattenSingleCollection() { PCollectionList.of(single).apply(Flatten.pCollections()) .apply(ParDo.of(new EmbeddedCollector())); translator.translate(p, dag); - Assert.assertNotNull( - dag.getOperatorMeta("ParDo(EmbeddedCollector)/ParMultiDo(EmbeddedCollector)")); + Assert.assertNotNull(dag.getOperatorMeta("ParDo(EmbeddedCollector)")); } } 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/ParDoBoundTranslatorTest.java similarity index 98% rename from runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoTranslatorTest.java rename to runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslatorTest.java index 83e68f7822d4..2aa07208cb09 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/ParDoBoundTranslatorTest.java @@ -68,11 +68,11 @@ import org.slf4j.LoggerFactory; /** - * integration test for {@link ParDoTranslator}. + * integration test for {@link ParDoBoundTranslator}. */ @RunWith(JUnit4.class) -public class ParDoTranslatorTest { - private static final Logger LOG = LoggerFactory.getLogger(ParDoTranslatorTest.class); +public class ParDoBoundTranslatorTest { + private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundTranslatorTest.class); private static final long SLEEP_MILLIS = 500; private static final long TIMEOUT_MILLIS = 30000; @@ -98,7 +98,7 @@ public void test() throws Exception { Assert.assertNotNull(om); Assert.assertEquals(om.getOperator().getClass(), ApexReadUnboundedInputOperator.class); - om = dag.getOperatorMeta("ParDo(Add)/ParMultiDo(Add)"); + om = dag.getOperatorMeta("ParDo(Add)"); Assert.assertNotNull(om); Assert.assertEquals(om.getOperator().getClass(), ApexParDoOperator.class); 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 4601262ef261..f56d225f1f1b 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 @@ -89,10 +89,24 @@ public class DirectRunner extends PipelineRunner { .put( PTransformMatchers.classEqualTo(TestStream.class), new DirectTestStreamFactory()) /* primitive */ + /* Single-output ParDos are implemented in terms of Multi-output ParDos. Any override + that is applied to a multi-output ParDo must first have all matching Single-output ParDos + converted to match. + */ + .put(PTransformMatchers.splittableParDoSingle(), new ParDoSingleViaMultiOverrideFactory()) + .put( + PTransformMatchers.stateOrTimerParDoSingle(), + new ParDoSingleViaMultiOverrideFactory()) + // SplittableParMultiDo is implemented in terms of nonsplittable single ParDos + .put(PTransformMatchers.splittableParDoMulti(), new ParDoMultiOverrideFactory()) + // state and timer pardos are implemented in terms of nonsplittable single ParDos + .put(PTransformMatchers.stateOrTimerParDoMulti(), new ParDoMultiOverrideFactory()) + .put( + PTransformMatchers.classEqualTo(ParDo.Bound.class), + new ParDoSingleViaMultiOverrideFactory()) /* returns a BoundMulti */ .put( PTransformMatchers.classEqualTo(BoundMulti.class), - /* returns one of two primitives; SplittableParDos and ParDos with state and timers - are replaced appropriately by the override factory. */ + /* returns one of two primitives; SplittableParDos are replaced above. */ new ParDoMultiOverrideFactory()) .put( PTransformMatchers.classEqualTo(GBKIntoKeyedWorkItems.class), diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java new file mode 100644 index 000000000000..f8597299217b --- /dev/null +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.direct; + +import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; +import org.apache.beam.sdk.runners.PTransformOverrideFactory; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.ParDo.Bound; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; + +/** + * A {@link PTransformOverrideFactory} that overrides single-output {@link ParDo} to implement + * it in terms of multi-output {@link ParDo}. + */ +class ParDoSingleViaMultiOverrideFactory + extends SingleInputOutputOverrideFactory< + PCollection, PCollection, Bound> { + @Override + public PTransform, PCollection> getReplacementTransform( + Bound transform) { + return new ParDoSingleViaMulti<>(transform); + } + + static class ParDoSingleViaMulti + extends PTransform, PCollection> { + private static final String MAIN_OUTPUT_TAG = "main"; + + private final ParDo.Bound underlyingParDo; + + public ParDoSingleViaMulti(ParDo.Bound underlyingParDo) { + this.underlyingParDo = underlyingParDo; + } + + @Override + public PCollection expand(PCollection input) { + + // Output tags for ParDo need only be unique up to applied transform + TupleTag mainOutputTag = new TupleTag(MAIN_OUTPUT_TAG); + + PCollectionTuple outputs = + input.apply( + ParDo.of(underlyingParDo.getFn()) + .withSideInputs(underlyingParDo.getSideInputs()) + .withOutputTags(mainOutputTag, TupleTagList.empty())); + PCollection output = outputs.get(mainOutputTag); + + output.setTypeDescriptor(underlyingParDo.getFn().getOutputTypeDescriptor()); + return output; + } + } +} diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactoryTest.java new file mode 100644 index 000000000000..59577a82b3b9 --- /dev/null +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactoryTest.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.direct; + +import static org.junit.Assert.assertThat; + +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.PCollection; +import org.hamcrest.Matchers; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for {@link ParDoSingleViaMultiOverrideFactory}. + */ +@RunWith(JUnit4.class) +public class ParDoSingleViaMultiOverrideFactoryTest { + private ParDoSingleViaMultiOverrideFactory factory = + new ParDoSingleViaMultiOverrideFactory<>(); + + @Test + public void getInputSucceeds() { + TestPipeline p = TestPipeline.create(); + PCollection input = p.apply(Create.of(1, 2, 3)); + PCollection reconstructed = factory.getInput(input.expand(), p); + assertThat(reconstructed, Matchers.>equalTo(input)); + } +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java index 31a6bdace118..f043c901391f 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java @@ -112,7 +112,8 @@ class FlinkBatchTransformTranslators { TRANSLATORS.put(Window.Assign.class, new WindowAssignTranslatorBatch()); - TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoTranslatorBatch()); + TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundTranslatorBatch()); + TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiTranslatorBatch()); TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslatorBatch()); } @@ -497,7 +498,80 @@ private static void rejectSplittable(DoFn doFn) { } } - private static class ParDoTranslatorBatch + private static class ParDoBoundTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator< + ParDo.Bound> { + + @Override + @SuppressWarnings("unchecked") + public void translateNode( + ParDo.Bound transform, + + FlinkBatchTranslationContext context) { + DoFn doFn = transform.getFn(); + rejectSplittable(doFn); + + DataSet> inputDataSet = + context.getInputDataSet(context.getInput(transform)); + + TypeInformation> typeInformation = + context.getTypeInfo(context.getOutput(transform)); + + List> sideInputs = transform.getSideInputs(); + + // construct a map from side input to WindowingStrategy so that + // the DoFn runner can map main-input windows to side input windows + Map, WindowingStrategy> sideInputStrategies = new HashMap<>(); + for (PCollectionView sideInput: sideInputs) { + sideInputStrategies.put(sideInput, sideInput.getWindowingStrategyInternal()); + } + + WindowingStrategy windowingStrategy = + context.getOutput(transform).getWindowingStrategy(); + + SingleInputUdfOperator, WindowedValue, ?> outputDataSet; + DoFnSignature signature = DoFnSignatures.getSignature(transform.getFn().getClass()); + if (signature.stateDeclarations().size() > 0 + || signature.timerDeclarations().size() > 0) { + + // Based on the fact that the signature is stateful, DoFnSignatures ensures + // that it is also keyed + KvCoder inputCoder = + (KvCoder) context.getInput(transform).getCoder(); + + FlinkStatefulDoFnFunction doFnWrapper = new FlinkStatefulDoFnFunction<>( + (DoFn) doFn, windowingStrategy, sideInputStrategies, context.getPipelineOptions(), + null, new TupleTag() + ); + + Grouping> grouping = + inputDataSet.groupBy(new KvKeySelector(inputCoder.getKeyCoder())); + + outputDataSet = new GroupReduceOperator( + grouping, typeInformation, doFnWrapper, transform.getName()); + + } else { + FlinkDoFnFunction doFnWrapper = + new FlinkDoFnFunction<>( + doFn, + windowingStrategy, + sideInputStrategies, + context.getPipelineOptions(), + null, new TupleTag()); + + outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, + transform.getName()); + + } + + transformSideInputs(sideInputs, outputDataSet, context); + + context.setOutputDataSet(context.getOutput(transform), outputDataSet); + + } + } + + private static class ParDoBoundMultiTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator< ParDo.BoundMulti> { diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java index 7227dceddbc1..c7df91dc1c1b 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java @@ -121,7 +121,8 @@ class FlinkStreamingTransformTranslators { TRANSLATORS.put(Write.class, new WriteSinkStreamingTranslator()); TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteBoundStreamingTranslator()); - TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoStreamingTranslator()); + TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundStreamingTranslator()); + TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiStreamingTranslator()); TRANSLATORS.put(Window.Assign.class, new WindowAssignTranslator()); TRANSLATORS.put(Flatten.PCollections.class, new FlattenPCollectionTranslator()); @@ -319,6 +320,114 @@ private static void rejectSplittable(DoFn doFn) { } } + private static class ParDoBoundStreamingTranslator + extends FlinkStreamingPipelineTranslator.StreamTransformTranslator< + ParDo.Bound> { + + @Override + public void translateNode( + ParDo.Bound transform, + FlinkStreamingTranslationContext context) { + + DoFn doFn = transform.getFn(); + rejectSplittable(doFn); + + WindowingStrategy windowingStrategy = + context.getOutput(transform).getWindowingStrategy(); + + TypeInformation> typeInfo = + context.getTypeInfo(context.getOutput(transform)); + + List> sideInputs = transform.getSideInputs(); + + @SuppressWarnings("unchecked") + PCollection inputPCollection = (PCollection) context.getInput(transform); + + Coder> inputCoder = context.getCoder(inputPCollection); + + DataStream> inputDataStream = + context.getInputDataStream(context.getInput(transform)); + Coder keyCoder = null; + boolean stateful = false; + DoFnSignature signature = DoFnSignatures.getSignature(transform.getFn().getClass()); + if (signature.stateDeclarations().size() > 0 + || signature.timerDeclarations().size() > 0) { + // Based on the fact that the signature is stateful, DoFnSignatures ensures + // that it is also keyed + keyCoder = ((KvCoder) inputPCollection.getCoder()).getKeyCoder(); + inputDataStream = inputDataStream.keyBy(new KvToByteBufferKeySelector(keyCoder)); + stateful = true; + } + + if (sideInputs.isEmpty()) { + DoFnOperator> doFnOperator = + new DoFnOperator<>( + transform.getFn(), + inputCoder, + new TupleTag("main output"), + Collections.>emptyList(), + new DoFnOperator.DefaultOutputManagerFactory>(), + windowingStrategy, + new HashMap>(), /* side-input mapping */ + Collections.>emptyList(), /* side inputs */ + context.getPipelineOptions(), + keyCoder); + + SingleOutputStreamOperator> outDataStream = inputDataStream + .transform(transform.getName(), typeInfo, doFnOperator); + + context.setOutputDataStream(context.getOutput(transform), outDataStream); + } else { + Tuple2>, DataStream> transformedSideInputs = + transformSideInputs(sideInputs, context); + + DoFnOperator> doFnOperator = + new DoFnOperator<>( + transform.getFn(), + inputCoder, + new TupleTag("main output"), + Collections.>emptyList(), + new DoFnOperator.DefaultOutputManagerFactory>(), + windowingStrategy, + transformedSideInputs.f0, + sideInputs, + context.getPipelineOptions(), + keyCoder); + + SingleOutputStreamOperator> outDataStream; + if (stateful) { + // we have to manually contruct the two-input transform because we're not + // allowed to have only one input keyed, normally. + KeyedStream keyedStream = (KeyedStream) inputDataStream; + TwoInputTransformation< + WindowedValue>, + RawUnionValue, + WindowedValue> rawFlinkTransform = new TwoInputTransformation<>( + keyedStream.getTransformation(), + transformedSideInputs.f1.broadcast().getTransformation(), + transform.getName(), + (TwoInputStreamOperator) doFnOperator, + typeInfo, + keyedStream.getParallelism()); + + rawFlinkTransform.setStateKeyType(keyedStream.getKeyType()); + rawFlinkTransform.setStateKeySelectors(keyedStream.getKeySelector(), null); + + outDataStream = new SingleOutputStreamOperator( + keyedStream.getExecutionEnvironment(), + rawFlinkTransform) {}; // we have to cheat around the ctor being protected + + keyedStream.getExecutionEnvironment().addOperator(rawFlinkTransform); + } else { + outDataStream = inputDataStream + .connect(transformedSideInputs.f1.broadcast()) + .transform(transform.getName(), typeInfo, doFnOperator); + } + context.setOutputDataStream(context.getOutput(transform), outDataStream); + } + } + } + /** * Wraps each element in a {@link RawUnionValue} with the given tag id. */ @@ -396,7 +505,7 @@ public RawUnionValue map(T o) throws Exception { } - private static class ParDoStreamingTranslator + private static class ParDoBoundMultiStreamingTranslator extends FlinkStreamingPipelineTranslator.StreamTransformTranslator< ParDo.BoundMulti> { 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 ab4cb9c67b04..06e50483440d 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 @@ -45,6 +45,7 @@ import com.google.common.base.Supplier; import com.google.common.collect.BiMap; import com.google.common.collect.ImmutableBiMap; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import java.io.IOException; import java.util.ArrayList; @@ -846,6 +847,34 @@ private void translateMultiHelper( } }); + registerTransformTranslator( + ParDo.Bound.class, + new TransformTranslator() { + @Override + public void translate(ParDo.Bound transform, TranslationContext context) { + translateSingleHelper(transform, context); + } + + private void translateSingleHelper( + ParDo.Bound transform, TranslationContext context) { + + StepTranslationContext stepContext = context.addStep(transform, "ParallelDo"); + translateInputs( + stepContext, context.getInput(transform), transform.getSideInputs(), context); + long mainOutput = stepContext.addOutput(context.getOutput(transform)); + translateFn( + stepContext, + transform.getFn(), + context.getInput(transform).getWindowingStrategy(), + transform.getSideInputs(), + context.getInput(transform).getCoder(), + context, + mainOutput, + ImmutableMap.>of( + mainOutput, new TupleTag<>(PropertyNames.OUTPUT))); + } + }); + registerTransformTranslator( Window.Assign.class, new TransformTranslator() { 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 ccb185cbea93..d4271e52f7c7 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 @@ -525,8 +525,7 @@ private static Step createPredefinedStep() throws Exception { assertEquals(13, job.getSteps().size()); Step step = job.getSteps().get(1); - assertEquals( - stepName + "/ParMultiDo(NoOp)", getString(step.getProperties(), PropertyNames.USER_NAME)); + assertEquals(stepName, getString(step.getProperties(), PropertyNames.USER_NAME)); assertAllStepOutputsHaveUniqueIds(job); return step; } @@ -972,7 +971,7 @@ public void populateDisplayData(DisplayData.Builder builder) { .put("type", "JAVA_CLASS") .put("value", fn1.getClass().getName()) .put("shortValue", fn1.getClass().getSimpleName()) - .put("namespace", ParDo.BoundMulti.class.getName()) + .put("namespace", parDo1.getClass().getName()) .build(), ImmutableMap.builder() .put("key", "foo2") @@ -992,7 +991,7 @@ public void populateDisplayData(DisplayData.Builder builder) { .put("type", "JAVA_CLASS") .put("value", fn2.getClass().getName()) .put("shortValue", fn2.getClass().getSimpleName()) - .put("namespace", ParDo.BoundMulti.class.getName()) + .put("namespace", parDo2.getClass().getName()) .build(), ImmutableMap.builder() .put("key", "foo3") 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 0ae731328913..725d157b4659 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 @@ -27,7 +27,6 @@ import static org.apache.beam.runners.spark.translation.TranslationUtils.rejectSplittable; import static org.apache.beam.runners.spark.translation.TranslationUtils.rejectStateAndTimers; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import java.io.IOException; @@ -75,7 +74,6 @@ 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.TaggedPValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.hadoop.conf.Configuration; @@ -326,19 +324,38 @@ public Iterable> call( }; } - private static TransformEvaluator> parDo() { - return new TransformEvaluator>() { + private static TransformEvaluator> parDo() { + return new TransformEvaluator>() { @Override - public void evaluate(ParDo.BoundMulti transform, EvaluationContext context) { - if (transform.getSideOutputTags().size() == 0) { - evaluateSingle(transform, context); - } else { - evaluateMulti(transform, context); - } + public void evaluate(ParDo.Bound transform, EvaluationContext context) { + String stepName = context.getCurrentTransform().getFullName(); + DoFn doFn = transform.getFn(); + rejectSplittable(doFn); + rejectStateAndTimers(doFn); + @SuppressWarnings("unchecked") + JavaRDD> inRDD = + ((BoundedDataset) context.borrowDataset(transform)).getRDD(); + WindowingStrategy windowingStrategy = + context.getInput(transform).getWindowingStrategy(); + JavaSparkContext jsc = context.getSparkContext(); + Accumulator aggAccum = + SparkAggregators.getNamedAggregators(jsc); + Accumulator metricsAccum = + MetricsAccumulator.getInstance(); + Map, KV, SideInputBroadcast>> sideInputs = + TranslationUtils.getSideInputs(transform.getSideInputs(), context); + context.putDataset(transform, + new BoundedDataset<>(inRDD.mapPartitions(new DoFnFunction<>(aggAccum, metricsAccum, + stepName, doFn, context.getRuntimeContext(), sideInputs, windowingStrategy)))); } + }; + } - private void evaluateMulti( - ParDo.BoundMulti transform, EvaluationContext context) { + private static TransformEvaluator> + multiDo() { + return new TransformEvaluator>() { + @Override + public void evaluate(ParDo.BoundMulti transform, EvaluationContext context) { String stepName = context.getCurrentTransform().getFullName(); DoFn doFn = transform.getFn(); rejectSplittable(doFn); @@ -349,21 +366,16 @@ private void evaluateMulti( WindowingStrategy windowingStrategy = context.getInput(transform).getWindowingStrategy(); JavaSparkContext jsc = context.getSparkContext(); - Accumulator aggAccum = SparkAggregators.getNamedAggregators(jsc); - Accumulator metricsAccum = MetricsAccumulator.getInstance(); - JavaPairRDD, WindowedValue> all = - inRDD - .mapPartitionsToPair( - new MultiDoFnFunction<>( - aggAccum, - metricsAccum, - stepName, - doFn, - context.getRuntimeContext(), - transform.getMainOutputTag(), - TranslationUtils.getSideInputs(transform.getSideInputs(), context), - windowingStrategy)) - .cache(); + Accumulator aggAccum = + SparkAggregators.getNamedAggregators(jsc); + Accumulator metricsAccum = + MetricsAccumulator.getInstance(); + JavaPairRDD, WindowedValue> all = inRDD + .mapPartitionsToPair( + new MultiDoFnFunction<>(aggAccum, metricsAccum, stepName, doFn, + context.getRuntimeContext(), transform.getMainOutputTag(), + TranslationUtils.getSideInputs(transform.getSideInputs(), context), + windowingStrategy)).cache(); List pct = context.getOutputs(transform); for (TaggedPValue e : pct) { @SuppressWarnings("unchecked") @@ -376,37 +388,6 @@ private void evaluateMulti( context.putDataset(e.getValue(), new BoundedDataset<>(values)); } } - - private void evaluateSingle( - ParDo.BoundMulti transform, EvaluationContext context) { - String stepName = context.getCurrentTransform().getFullName(); - DoFn doFn = transform.getFn(); - rejectSplittable(doFn); - rejectStateAndTimers(doFn); - @SuppressWarnings("unchecked") - JavaRDD> inRDD = - ((BoundedDataset) context.borrowDataset(transform)).getRDD(); - WindowingStrategy windowingStrategy = - context.getInput(transform).getWindowingStrategy(); - JavaSparkContext jsc = context.getSparkContext(); - Accumulator aggAccum = SparkAggregators.getNamedAggregators(jsc); - Accumulator metricsAccum = MetricsAccumulator.getInstance(); - Map, KV, SideInputBroadcast>> sideInputs = - TranslationUtils.getSideInputs(transform.getSideInputs(), context); - PValue onlyOutput = Iterables.getOnlyElement(context.getOutputs(transform)).getValue(); - context.putDataset( - onlyOutput, - new BoundedDataset<>( - inRDD.mapPartitions( - new DoFnFunction<>( - aggAccum, - metricsAccum, - stepName, - doFn, - context.getRuntimeContext(), - sideInputs, - windowingStrategy)))); - } }; } @@ -761,7 +742,8 @@ private static TransformEvaluator transform, }; } + private static TransformEvaluator> parDo() { + return new TransformEvaluator>() { + @Override + public void evaluate(final ParDo.Bound transform, + final EvaluationContext context) { + final DoFn doFn = transform.getFn(); + rejectSplittable(doFn); + rejectStateAndTimers(doFn); + final SparkRuntimeContext runtimeContext = context.getRuntimeContext(); + final WindowingStrategy windowingStrategy = + context.getInput(transform).getWindowingStrategy(); + final SparkPCollectionView pviews = context.getPViews(); + + @SuppressWarnings("unchecked") + UnboundedDataset unboundedDataset = + ((UnboundedDataset) context.borrowDataset(transform)); + JavaDStream> dStream = unboundedDataset.getDStream(); + + final String stepName = context.getCurrentTransform().getFullName(); + + JavaDStream> outStream = + dStream.transform(new Function>, + JavaRDD>>() { + @Override + public JavaRDD> call(JavaRDD> rdd) throws + Exception { + final JavaSparkContext jsc = new JavaSparkContext(rdd.context()); + final Accumulator aggAccum = + SparkAggregators.getNamedAggregators(jsc); + final Accumulator metricsAccum = + MetricsAccumulator.getInstance(); + final Map, KV, SideInputBroadcast>> sideInputs = + TranslationUtils.getSideInputs(transform.getSideInputs(), + jsc, pviews); + return rdd.mapPartitions( + new DoFnFunction<>(aggAccum, metricsAccum, stepName, doFn, runtimeContext, + sideInputs, windowingStrategy)); + } + }); + + context.putDataset(transform, + new UnboundedDataset<>(outStream, unboundedDataset.getStreamSources())); + } + }; + } + private static TransformEvaluator> multiDo() { return new TransformEvaluator>() { - public void evaluate( - final ParDo.BoundMulti transform, final EvaluationContext context) { - if (transform.getSideOutputTags().size() == 0) { - evaluateSingle(transform, context); - } else { - evaluateMulti(transform, context); - } - } - - private void evaluateMulti( - final ParDo.BoundMulti transform, final EvaluationContext context) { + @Override + public void evaluate(final ParDo.BoundMulti transform, + final EvaluationContext context) { final DoFn doFn = transform.getFn(); rejectSplittable(doFn); rejectStateAndTimers(doFn); @@ -389,60 +427,10 @@ public JavaPairRDD, WindowedValue> call( JavaDStream> values = (JavaDStream>) (JavaDStream) TranslationUtils.dStreamValues(filtered); - context.putDataset( - e.getValue(), new UnboundedDataset<>(values, unboundedDataset.getStreamSources())); + context.putDataset(e.getValue(), + new UnboundedDataset<>(values, unboundedDataset.getStreamSources())); } } - - private void evaluateSingle( - final ParDo.BoundMulti transform, final EvaluationContext context) { - final DoFn doFn = transform.getFn(); - rejectSplittable(doFn); - rejectStateAndTimers(doFn); - final SparkRuntimeContext runtimeContext = context.getRuntimeContext(); - final WindowingStrategy windowingStrategy = - context.getInput(transform).getWindowingStrategy(); - final SparkPCollectionView pviews = context.getPViews(); - - @SuppressWarnings("unchecked") - UnboundedDataset unboundedDataset = - ((UnboundedDataset) context.borrowDataset(transform)); - JavaDStream> dStream = unboundedDataset.getDStream(); - - final String stepName = context.getCurrentTransform().getFullName(); - - JavaDStream> outStream = - dStream.transform( - new Function>, JavaRDD>>() { - @Override - public JavaRDD> call(JavaRDD> rdd) - throws Exception { - final JavaSparkContext jsc = new JavaSparkContext(rdd.context()); - final Accumulator aggAccum = - SparkAggregators.getNamedAggregators(jsc); - final Accumulator metricsAccum = - MetricsAccumulator.getInstance(); - final Map, KV, SideInputBroadcast>> - sideInputs = - TranslationUtils.getSideInputs(transform.getSideInputs(), jsc, pviews); - return rdd.mapPartitions( - new DoFnFunction<>( - aggAccum, - metricsAccum, - stepName, - doFn, - runtimeContext, - sideInputs, - windowingStrategy)); - } - }); - - PCollection output = - (PCollection) - Iterables.getOnlyElement(context.getOutputs(transform)).getValue(); - context.putDataset( - output, new UnboundedDataset<>(outStream, unboundedDataset.getStreamSources())); - } }; } @@ -487,6 +475,7 @@ public JavaRDD>> call( EVALUATORS.put(Read.Unbounded.class, readUnbounded()); EVALUATORS.put(GroupByKey.class, groupByKey()); EVALUATORS.put(Combine.GroupedValues.class, combineGrouped()); + EVALUATORS.put(ParDo.Bound.class, parDo()); EVALUATORS.put(ParDo.BoundMulti.class, multiDo()); EVALUATORS.put(ConsoleIO.Write.Unbound.class, print()); EVALUATORS.put(CreateStream.class, createFromQueue()); 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 d66633b4c49b..b181a042820c 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 @@ -83,7 +83,7 @@ public void testTrackSingle() { p.apply(emptyStream).apply(ParDo.of(new PassthroughFn<>())); - p.traverseTopologically(new StreamingSourceTracker(jssc, p, ParDo.BoundMulti.class, 0)); + p.traverseTopologically(new StreamingSourceTracker(jssc, p, ParDo.Bound.class, 0)); assertThat(StreamingSourceTracker.numAssertions, equalTo(1)); } @@ -111,7 +111,7 @@ public void testTrackFlattened() { PCollectionList.of(pcol1).and(pcol2).apply(Flatten.pCollections()); flattened.apply(ParDo.of(new PassthroughFn<>())); - p.traverseTopologically(new StreamingSourceTracker(jssc, p, ParDo.BoundMulti.class, 0, 1)); + p.traverseTopologically(new StreamingSourceTracker(jssc, p, ParDo.Bound.class, 0, 1)); assertThat(StreamingSourceTracker.numAssertions, equalTo(1)); } 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 92252310f961..19c5a2d5b511 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 @@ -738,8 +738,12 @@ public BoundMulti withOutputTags( @Override public PCollection expand(PCollection input) { - TupleTag mainOutput = new TupleTag<>(); - return input.apply(withOutputTags(mainOutput, TupleTagList.empty())).get(mainOutput); + validateWindowType(input, fn); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), + input.getWindowingStrategy(), + input.isBounded()) + .setTypeDescriptor(getFn().getOutputTypeDescriptor()); } @Override From a4e8bef4fee90ef4107282f1fcb14110c8fc38be Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Thu, 2 Mar 2017 16:15:38 -0800 Subject: [PATCH 34/46] Fix tox warning for non-whitelisted find command --- sdks/python/tox.ini | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 927c211d3d66..3e4f2f27c498 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -44,6 +44,7 @@ platform = linux2 deps = nose cython +whitelist_externals=find commands = python --version pip install -e .[test] From 57886246a08e82a2cff0d6b95816b76a5b912595 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 6 Mar 2017 16:02:54 -0800 Subject: [PATCH 35/46] Add tests for serialization of BigQueryIO.TableRowInfoCoder --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 3 ++- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 17 +++++++++++++++++ 2 files changed, 19 insertions(+), 1 deletion(-) 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 be9a786fbe70..0e1c6fc1ae7b 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 @@ -2887,7 +2887,8 @@ public void verifyDeterministic() throws NonDeterministicException { VarIntCoder shardNumberCoder; } - private static class TableRowInfoCoder extends AtomicCoder { + @VisibleForTesting + static class TableRowInfoCoder extends AtomicCoder { private static final TableRowInfoCoder INSTANCE = new TableRowInfoCoder(); @JsonCreator 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 fe41703e6748..c9061a3e38ae 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 @@ -145,6 +145,7 @@ import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; @@ -154,6 +155,7 @@ import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.util.PCollectionViews; import org.apache.beam.sdk.util.Transport; +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; @@ -2474,6 +2476,21 @@ public void testShardedKeyCoderIsSerializableWithWellKnownCoderType() { CoderProperties.coderSerializable(BigQueryIO.ShardedKeyCoder.of(GlobalWindow.Coder.INSTANCE)); } + @Test + public void testTableRowInfoCoderSerializable() { + CoderProperties.coderSerializable(BigQueryIO.TableRowInfoCoder.of()); + } + + @Test + public void testComplexCoderSerializable() { + CoderProperties.coderSerializable( + WindowedValue.getFullCoder( + KvCoder.of( + BigQueryIO.ShardedKeyCoder.of(StringUtf8Coder.of()), + BigQueryIO.TableRowInfoCoder.of()), + IntervalWindow.getCoder())); + } + @Test public void testUniqueStepIdRead() { RuntimeTestOptions options = PipelineOptionsFactory.as(RuntimeTestOptions.class); From 4dda585cda61a775e2d616fa5c25698f490b9cd3 Mon Sep 17 00:00:00 2001 From: Sela Date: Mon, 6 Mar 2017 11:17:00 +0200 Subject: [PATCH 36/46] [BEAM-1556] Make PipelineOptions a lazy-singleton and init IOs as part of it. --- .../translation/SparkRuntimeContext.java | 29 +++++++++++++++---- 1 file changed, 23 insertions(+), 6 deletions(-) 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 9c3d79f0c8d6..4ccfeadcd9e3 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 @@ -32,6 +32,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.spark.Accumulator; /** @@ -40,12 +41,10 @@ */ public class SparkRuntimeContext implements Serializable { private final String serializedPipelineOptions; + private transient CoderRegistry coderRegistry; - /** - * Map fo names to Beam aggregators. - */ + // map for names to Beam aggregators. private final Map> aggregators = new HashMap<>(); - private transient CoderRegistry coderRegistry; SparkRuntimeContext(Pipeline pipeline) { this.serializedPipelineOptions = serializePipelineOptions(pipeline.getOptions()); @@ -67,8 +66,8 @@ private static PipelineOptions deserializePipelineOptions(String serializedPipel } } - public synchronized PipelineOptions getPipelineOptions() { - return deserializePipelineOptions(serializedPipelineOptions); + public PipelineOptions getPipelineOptions() { + return PipelineOptionsHolder.getOrInit(serializedPipelineOptions); } /** @@ -118,6 +117,24 @@ public CoderRegistry getCoderRegistry() { return coderRegistry; } + private static class PipelineOptionsHolder { + // on executors, this should deserialize once. + private static transient volatile PipelineOptions pipelineOptions = null; + + static PipelineOptions getOrInit(String serializedPipelineOptions) { + if (pipelineOptions == null) { + synchronized (PipelineOptionsHolder.class) { + if (pipelineOptions == null) { + pipelineOptions = deserializePipelineOptions(serializedPipelineOptions); + } + } + // register IO factories. + IOChannelUtils.registerIOFactoriesAllowOverride(pipelineOptions); + } + return pipelineOptions; + } + } + /** * Initialize spark aggregators exactly once. * From a889597e748eb752141af8dc568c56449c4eba5c Mon Sep 17 00:00:00 2001 From: Aviem Zur Date: Tue, 7 Mar 2017 15:07:03 +0200 Subject: [PATCH 37/46] [BEAM-1636] UnboundedDataset action() does not materialize RDD --- .../runners/spark/translation/BoundedDataset.java | 7 +------ .../runners/spark/translation/TranslationUtils.java | 8 ++++++++ .../spark/translation/streaming/UnboundedDataset.java | 11 ++++++++++- 3 files changed, 19 insertions(+), 7 deletions(-) diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java index 7db04a8a2c85..6e4ffc7b0334 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java @@ -32,7 +32,6 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDDLike; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.VoidFunction; import org.apache.spark.storage.StorageLevel; /** @@ -106,11 +105,7 @@ public void cache(String storageLevel) { @Override public void action() { // Empty function to force computation of RDD. - rdd.foreach(new VoidFunction>() { - @Override public void call(WindowedValue tWindowedValue) throws Exception { - // Empty implementation. - } - }); + rdd.foreach(TranslationUtils.>emptyVoidFunction()); } @Override 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 f2b34183e8b2..8545b360b31b 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 @@ -43,6 +43,7 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.api.java.function.VoidFunction; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; @@ -267,4 +268,11 @@ public static void rejectStateAndTimers(DoFn doFn) { } } + public static VoidFunction emptyVoidFunction() { + return new VoidFunction() { + @Override public void call(T t) throws Exception { + // Empty implementation. + } + }; + } } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/UnboundedDataset.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/UnboundedDataset.java index e9abe93cba1a..ccdaf113b4b1 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/UnboundedDataset.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/UnboundedDataset.java @@ -21,7 +21,10 @@ import java.util.ArrayList; import java.util.List; import org.apache.beam.runners.spark.translation.Dataset; +import org.apache.beam.runners.spark.translation.TranslationUtils; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.VoidFunction; import org.apache.spark.streaming.api.java.JavaDStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,11 +71,17 @@ public void cache(String storageLevel) { @Override public void action() { // Force computation of DStream. - dStream.dstream().register(); + dStream.foreachRDD(new VoidFunction>>() { + @Override + public void call(JavaRDD> rdd) throws Exception { + rdd.foreach(TranslationUtils.>emptyVoidFunction()); + } + }); } @Override public void setName(String name) { // ignore } + } From 21700da27e1a796b2d8ead4228b8eb15befcaafc Mon Sep 17 00:00:00 2001 From: Tibor Kiss Date: Tue, 7 Mar 2017 04:00:24 -0800 Subject: [PATCH 38/46] [BEAM-1635] TypeError in AfterWatermark class's __repr__ method --- sdks/python/apache_beam/transforms/trigger.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/transforms/trigger.py b/sdks/python/apache_beam/transforms/trigger.py index 8c238736824f..69a35ae820a1 100644 --- a/sdks/python/apache_beam/transforms/trigger.py +++ b/sdks/python/apache_beam/transforms/trigger.py @@ -237,7 +237,7 @@ def __repr__(self): if self.early: qualifiers.append('early=%s' % self.early) if self.late: - qualifiers.append('late=%s', self.late) + qualifiers.append('late=%s' % self.late) return 'AfterWatermark(%s)' % ', '.join(qualifiers) def is_late(self, context): From fb30c8f1da31f5cf08d7323194b41b1a417dfaf2 Mon Sep 17 00:00:00 2001 From: Tibor Kiss Date: Fri, 24 Feb 2017 14:30:54 +0100 Subject: [PATCH 39/46] [BEAM-1546] Specify exact version for Python in the SDK --- sdks/python/apache_beam/__init__.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/__init__.py b/sdks/python/apache_beam/__init__.py index ce382b8bc6aa..77c89adc2f2f 100644 --- a/sdks/python/apache_beam/__init__.py +++ b/sdks/python/apache_beam/__init__.py @@ -66,10 +66,10 @@ import sys -if sys.version_info.major != 2: +if not (sys.version_info[0] == 2 and sys.version_info[1] == 7): raise RuntimeError( 'Dataflow SDK for Python is supported only on Python 2.7. ' - 'It is not supported on Python [%s].' % sys.version) + 'It is not supported on Python [%s].' % sys.version_info) # pylint: disable=wrong-import-position import apache_beam.internal.pickler From 33f7082b0f05c086e09a83cd98559bde3b70482d Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Mon, 6 Mar 2017 16:51:04 -0800 Subject: [PATCH 40/46] Flink: register known IOChannelFactories --- .../flink/translation/utils/SerializedPipelineOptions.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java index fe2602b3377b..390e6da341b5 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.io.Serializable; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.IOChannelUtils; /** * Encapsulates the PipelineOptions in serialized form to ship them to the cluster. @@ -52,6 +53,8 @@ public PipelineOptions getPipelineOptions() { if (pipelineOptions == null) { try { pipelineOptions = new ObjectMapper().readValue(serializedOptions, PipelineOptions.class); + + IOChannelUtils.registerIOFactoriesAllowOverride(pipelineOptions); } catch (IOException e) { throw new RuntimeException("Couldn't deserialize the PipelineOptions.", e); } @@ -59,5 +62,4 @@ public PipelineOptions getPipelineOptions() { return pipelineOptions; } - } From 5b068ecede52dd0fdd0d907a5e8337f99096d368 Mon Sep 17 00:00:00 2001 From: Tibor Kiss Date: Tue, 7 Mar 2017 04:42:34 -0800 Subject: [PATCH 41/46] [BEAM-1633] Move .tox/ directory under target/ in Python SDK --- sdks/python/run_postcommit.sh | 2 +- sdks/python/tox.ini | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/run_postcommit.sh b/sdks/python/run_postcommit.sh index 442dbae222c0..1c0bbb2c0de6 100755 --- a/sdks/python/run_postcommit.sh +++ b/sdks/python/run_postcommit.sh @@ -32,7 +32,7 @@ set -v LOCAL_PATH=$HOME/.local/bin/ # Remove any tox cache from previous workspace -rm -rf sdks/python/.tox +rm -rf sdks/python/target/.tox # INFRA does not install virtualenv pip install virtualenv --user diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 3e4f2f27c498..8d8acfa32b8b 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -17,6 +17,7 @@ [tox] envlist = py27,py27gcp,py27cython,lint,docs +toxworkdir = {toxinidir}/target/.tox [pep8] # Disable all errors and warnings except for the ones related to blank lines. From 4eb5a00f8bf45575aebbc0d6f130be6083a31c6a Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Tue, 7 Mar 2017 10:53:39 -0800 Subject: [PATCH 42/46] [maven-release-plugin] prepare branch release-0.6.0 --- pom.xml | 2 +- runners/core-construction-java/pom.xml | 4 +--- sdks/common/fn-api/pom.xml | 4 +--- sdks/common/runner-api/pom.xml | 4 +--- sdks/java/extensions/jackson/pom.xml | 4 +--- sdks/java/harness/pom.xml | 4 +--- sdks/java/javadoc/pom.xml | 10 +++++----- 7 files changed, 11 insertions(+), 21 deletions(-) diff --git a/pom.xml b/pom.xml index a37f1af4d4d0..eded6840dfd8 100644 --- a/pom.xml +++ b/pom.xml @@ -48,7 +48,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/beam.git scm:git:https://git-wip-us.apache.org/repos/asf/beam.git https://git-wip-us.apache.org/repos/asf?p=beam.git;a=summary - HEAD + release-0.6.0 diff --git a/runners/core-construction-java/pom.xml b/runners/core-construction-java/pom.xml index 868f74324de3..b602f5dca8f0 100644 --- a/runners/core-construction-java/pom.xml +++ b/runners/core-construction-java/pom.xml @@ -17,9 +17,7 @@ ~ limitations under the License. --> - + 4.0.0 diff --git a/sdks/common/fn-api/pom.xml b/sdks/common/fn-api/pom.xml index 1f6193ffe58c..5a41d9e365b9 100644 --- a/sdks/common/fn-api/pom.xml +++ b/sdks/common/fn-api/pom.xml @@ -15,9 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 jar diff --git a/sdks/common/runner-api/pom.xml b/sdks/common/runner-api/pom.xml index 8eaeb8e321af..9c6de1ebfad1 100644 --- a/sdks/common/runner-api/pom.xml +++ b/sdks/common/runner-api/pom.xml @@ -15,9 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 jar diff --git a/sdks/java/extensions/jackson/pom.xml b/sdks/java/extensions/jackson/pom.xml index be5c9537da95..1dfbd72d6fa9 100644 --- a/sdks/java/extensions/jackson/pom.xml +++ b/sdks/java/extensions/jackson/pom.xml @@ -15,9 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 diff --git a/sdks/java/harness/pom.xml b/sdks/java/harness/pom.xml index 3abe70b171b1..80b01ca88695 100644 --- a/sdks/java/harness/pom.xml +++ b/sdks/java/harness/pom.xml @@ -15,9 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 jar diff --git a/sdks/java/javadoc/pom.xml b/sdks/java/javadoc/pom.xml index 243dae5d448d..b785c986dbdd 100644 --- a/sdks/java/javadoc/pom.xml +++ b/sdks/java/javadoc/pom.xml @@ -254,12 +254,12 @@ package - - - - + + + + - + From b74e186546eafb910a013ec338cd3402878e40cf Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Tue, 7 Mar 2017 10:54:00 -0800 Subject: [PATCH 43/46] [maven-release-plugin] prepare for next development iteration --- examples/java/pom.xml | 2 +- examples/java8/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 4 ++-- runners/apex/pom.xml | 2 +- runners/core-construction-java/pom.xml | 2 +- runners/core-java/pom.xml | 2 +- runners/direct-java/pom.xml | 2 +- runners/flink/examples/pom.xml | 2 +- runners/flink/pom.xml | 2 +- runners/flink/runner/pom.xml | 2 +- runners/google-cloud-dataflow-java/pom.xml | 2 +- runners/pom.xml | 2 +- runners/spark/pom.xml | 2 +- sdks/common/fn-api/pom.xml | 2 +- sdks/common/pom.xml | 2 +- sdks/common/runner-api/pom.xml | 2 +- sdks/java/build-tools/pom.xml | 2 +- sdks/java/core/pom.xml | 2 +- sdks/java/extensions/jackson/pom.xml | 2 +- sdks/java/extensions/join-library/pom.xml | 2 +- sdks/java/extensions/pom.xml | 2 +- sdks/java/extensions/sorter/pom.xml | 2 +- sdks/java/harness/pom.xml | 2 +- sdks/java/io/elasticsearch/pom.xml | 2 +- sdks/java/io/google-cloud-platform/pom.xml | 2 +- sdks/java/io/hadoop-common/pom.xml | 2 +- sdks/java/io/hbase/pom.xml | 2 +- sdks/java/io/hdfs/pom.xml | 2 +- sdks/java/io/jdbc/pom.xml | 2 +- sdks/java/io/jms/pom.xml | 2 +- sdks/java/io/kafka/pom.xml | 2 +- sdks/java/io/kinesis/pom.xml | 2 +- sdks/java/io/mongodb/pom.xml | 2 +- sdks/java/io/mqtt/pom.xml | 2 +- sdks/java/io/pom.xml | 2 +- sdks/java/java8tests/pom.xml | 2 +- sdks/java/javadoc/pom.xml | 2 +- sdks/java/maven-archetypes/examples-java8/pom.xml | 2 +- sdks/java/maven-archetypes/examples/pom.xml | 2 +- sdks/java/maven-archetypes/pom.xml | 2 +- sdks/java/maven-archetypes/starter/pom.xml | 2 +- sdks/java/pom.xml | 2 +- sdks/pom.xml | 2 +- sdks/python/pom.xml | 2 +- 45 files changed, 46 insertions(+), 46 deletions(-) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index 9da814b1388d..ed11e6abe9d3 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-examples-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/examples/java8/pom.xml b/examples/java8/pom.xml index 580a15421bfc..f5a0db7a199c 100644 --- a/examples/java8/pom.xml +++ b/examples/java8/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-examples-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 550578b122bd..c89e389c706b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index eded6840dfd8..c3b847648f76 100644 --- a/pom.xml +++ b/pom.xml @@ -34,7 +34,7 @@ http://beam.apache.org/ 2016 - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT @@ -48,7 +48,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/beam.git scm:git:https://git-wip-us.apache.org/repos/asf/beam.git https://git-wip-us.apache.org/repos/asf?p=beam.git;a=summary - release-0.6.0 + HEAD diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index f5fe4bced3dc..01dea22f45e0 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/runners/core-construction-java/pom.xml b/runners/core-construction-java/pom.xml index b602f5dca8f0..685b86674fc0 100644 --- a/runners/core-construction-java/pom.xml +++ b/runners/core-construction-java/pom.xml @@ -24,7 +24,7 @@ beam-runners-parent org.apache.beam - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/runners/core-java/pom.xml b/runners/core-java/pom.xml index 28ab0eef9ec4..53b5094ff3c8 100644 --- a/runners/core-java/pom.xml +++ b/runners/core-java/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml index ced9cd625df7..e437526b14a0 100644 --- a/runners/direct-java/pom.xml +++ b/runners/direct-java/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/runners/flink/examples/pom.xml b/runners/flink/examples/pom.xml index 1d426bd0b385..661ed432b087 100644 --- a/runners/flink/examples/pom.xml +++ b/runners/flink/examples/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-flink-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index 0030f612f473..dcf5ff770059 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index 13d5b107f0a7..f2c2d0181745 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-flink-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index fdd088f55ab1..4645e541909e 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/runners/pom.xml b/runners/pom.xml index 3f74f7b9d9c9..fb090ad3e6a5 100644 --- a/runners/pom.xml +++ b/runners/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 0d4c413e4790..ebd987d37a94 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/common/fn-api/pom.xml b/sdks/common/fn-api/pom.xml index 5a41d9e365b9..e9253c221c57 100644 --- a/sdks/common/fn-api/pom.xml +++ b/sdks/common/fn-api/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-common-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/common/pom.xml b/sdks/common/pom.xml index 55db1818b4c5..c90e6f8e6217 100644 --- a/sdks/common/pom.xml +++ b/sdks/common/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/common/runner-api/pom.xml b/sdks/common/runner-api/pom.xml index 9c6de1ebfad1..199fbb398a7c 100644 --- a/sdks/common/runner-api/pom.xml +++ b/sdks/common/runner-api/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-common-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/build-tools/pom.xml b/sdks/java/build-tools/pom.xml index 545f3942a581..bc4cff7b6032 100644 --- a/sdks/java/build-tools/pom.xml +++ b/sdks/java/build-tools/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../../../pom.xml diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index 4f8955096325..e50b6d8243bc 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/extensions/jackson/pom.xml b/sdks/java/extensions/jackson/pom.xml index 1dfbd72d6fa9..1e0dcedcaf1a 100644 --- a/sdks/java/extensions/jackson/pom.xml +++ b/sdks/java/extensions/jackson/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-extensions-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/extensions/join-library/pom.xml b/sdks/java/extensions/join-library/pom.xml index 728c6f7fe1a1..721f378d9b45 100644 --- a/sdks/java/extensions/join-library/pom.xml +++ b/sdks/java/extensions/join-library/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-extensions-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/extensions/pom.xml b/sdks/java/extensions/pom.xml index 26d92dee7bd2..74aba0f9970a 100644 --- a/sdks/java/extensions/pom.xml +++ b/sdks/java/extensions/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/extensions/sorter/pom.xml b/sdks/java/extensions/sorter/pom.xml index 9d03ba287d29..3dce782b452d 100644 --- a/sdks/java/extensions/sorter/pom.xml +++ b/sdks/java/extensions/sorter/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-extensions-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/harness/pom.xml b/sdks/java/harness/pom.xml index 80b01ca88695..84eef5e621e7 100644 --- a/sdks/java/harness/pom.xml +++ b/sdks/java/harness/pom.xml @@ -23,7 +23,7 @@ org.apache.beam beam-sdks-java-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/elasticsearch/pom.xml b/sdks/java/io/elasticsearch/pom.xml index 3279dfdcd3c6..91a7ae650b70 100644 --- a/sdks/java/io/elasticsearch/pom.xml +++ b/sdks/java/io/elasticsearch/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index 66a42075e317..0caf882a07c7 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/hadoop-common/pom.xml b/sdks/java/io/hadoop-common/pom.xml index fcd984fb4d57..ad31ded166c6 100644 --- a/sdks/java/io/hadoop-common/pom.xml +++ b/sdks/java/io/hadoop-common/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/hbase/pom.xml b/sdks/java/io/hbase/pom.xml index dfcca7a801ce..14ac04c6f85b 100644 --- a/sdks/java/io/hbase/pom.xml +++ b/sdks/java/io/hbase/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/hdfs/pom.xml b/sdks/java/io/hdfs/pom.xml index f3a1a27bb9b9..1ab41b3eaf69 100644 --- a/sdks/java/io/hdfs/pom.xml +++ b/sdks/java/io/hdfs/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/jdbc/pom.xml b/sdks/java/io/jdbc/pom.xml index fd5c52b6b339..3efeb80762d4 100644 --- a/sdks/java/io/jdbc/pom.xml +++ b/sdks/java/io/jdbc/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/jms/pom.xml b/sdks/java/io/jms/pom.xml index cc7fb279b70f..dd9909e1da46 100644 --- a/sdks/java/io/jms/pom.xml +++ b/sdks/java/io/jms/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/kafka/pom.xml b/sdks/java/io/kafka/pom.xml index d5ffe637edf3..4703e37dde92 100644 --- a/sdks/java/io/kafka/pom.xml +++ b/sdks/java/io/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/kinesis/pom.xml b/sdks/java/io/kinesis/pom.xml index 51383689c935..7ed2f955920e 100644 --- a/sdks/java/io/kinesis/pom.xml +++ b/sdks/java/io/kinesis/pom.xml @@ -21,7 +21,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/mongodb/pom.xml b/sdks/java/io/mongodb/pom.xml index 7deae14cd06a..014e1b6be3bf 100644 --- a/sdks/java/io/mongodb/pom.xml +++ b/sdks/java/io/mongodb/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/mqtt/pom.xml b/sdks/java/io/mqtt/pom.xml index 8b1d8486fa93..dcfcaa80ec56 100644 --- a/sdks/java/io/mqtt/pom.xml +++ b/sdks/java/io/mqtt/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/pom.xml b/sdks/java/io/pom.xml index 8d5b69b359e8..64424c5d907f 100644 --- a/sdks/java/io/pom.xml +++ b/sdks/java/io/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/java8tests/pom.xml b/sdks/java/java8tests/pom.xml index fda1d639ddb5..e7c6cc0a68fe 100644 --- a/sdks/java/java8tests/pom.xml +++ b/sdks/java/java8tests/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/javadoc/pom.xml b/sdks/java/javadoc/pom.xml index b785c986dbdd..840a359c2648 100644 --- a/sdks/java/javadoc/pom.xml +++ b/sdks/java/javadoc/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../../../pom.xml diff --git a/sdks/java/maven-archetypes/examples-java8/pom.xml b/sdks/java/maven-archetypes/examples-java8/pom.xml index 2632d6db5010..01f7270bf810 100644 --- a/sdks/java/maven-archetypes/examples-java8/pom.xml +++ b/sdks/java/maven-archetypes/examples-java8/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-maven-archetypes-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/maven-archetypes/examples/pom.xml b/sdks/java/maven-archetypes/examples/pom.xml index 09e542876603..3483b2a60d27 100644 --- a/sdks/java/maven-archetypes/examples/pom.xml +++ b/sdks/java/maven-archetypes/examples/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-maven-archetypes-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/maven-archetypes/pom.xml b/sdks/java/maven-archetypes/pom.xml index 194e5bd4916b..78e6f0849a1a 100644 --- a/sdks/java/maven-archetypes/pom.xml +++ b/sdks/java/maven-archetypes/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/maven-archetypes/starter/pom.xml b/sdks/java/maven-archetypes/starter/pom.xml index 092995a5d74c..dcda9e00fe42 100644 --- a/sdks/java/maven-archetypes/starter/pom.xml +++ b/sdks/java/maven-archetypes/starter/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-maven-archetypes-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/pom.xml b/sdks/java/pom.xml index a09a6be9f296..7ca61090e7b2 100644 --- a/sdks/java/pom.xml +++ b/sdks/java/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/pom.xml b/sdks/pom.xml index f130816c0a95..9ce658589978 100644 --- a/sdks/pom.xml +++ b/sdks/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml diff --git a/sdks/python/pom.xml b/sdks/python/pom.xml index 98b7fa33a013..0d1fdb401ea7 100644 --- a/sdks/python/pom.xml +++ b/sdks/python/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-parent - 0.6.0-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml From 0694344ee03520bae24a2b484a28927579e2bb7e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Mon, 6 Mar 2017 09:13:31 +0100 Subject: [PATCH 44/46] Change Json parsing from gson to jackson for ElasticsearchIO --- sdks/java/io/elasticsearch/pom.xml | 7 +- .../sdk/io/elasticsearch/ElasticsearchIO.java | 102 +++++++++--------- 2 files changed, 52 insertions(+), 57 deletions(-) diff --git a/sdks/java/io/elasticsearch/pom.xml b/sdks/java/io/elasticsearch/pom.xml index 91a7ae650b70..eecfe6b45bdf 100644 --- a/sdks/java/io/elasticsearch/pom.xml +++ b/sdks/java/io/elasticsearch/pom.xml @@ -47,9 +47,8 @@ - com.google.code.gson - gson - 2.6.2 + com.fasterxml.jackson.core + jackson-databind @@ -116,7 +115,7 @@ - org.apache.commons + commons-io commons-io 1.3.2 test 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 5073834986b9..b08cb24a021b 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 @@ -20,17 +20,13 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.auto.value.AutoValue; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; -import com.google.gson.Gson; -import com.google.gson.JsonArray; -import com.google.gson.JsonElement; -import com.google.gson.JsonObject; import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; import java.io.Serializable; import java.net.MalformedURLException; import java.net.URL; @@ -38,11 +34,11 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.ListIterator; import java.util.Map; import java.util.NoSuchElementException; -import java.util.Set; import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; @@ -140,11 +136,10 @@ public static Write write() { private ElasticsearchIO() {} - private static JsonObject parseResponse(Response response) throws IOException { - InputStream content = response.getEntity().getContent(); - InputStreamReader inputStreamReader = new InputStreamReader(content, "UTF-8"); - JsonObject jsonObject = new Gson().fromJson(inputStreamReader, JsonObject.class); - return jsonObject; + private static final ObjectMapper mapper = new ObjectMapper(); + + private static JsonNode parseResponse(Response response) throws IOException { + return mapper.readValue(response.getEntity().getContent(), JsonNode.class); } /** A POJO describing a connection configuration to Elasticsearch. */ @@ -428,23 +423,24 @@ public List> splitIntoBundles( // But, as each shard (replica or primary) is responsible for only one part of the data, // there will be no duplicate. - JsonObject statsJson = getStats(true); - JsonObject shardsJson = + JsonNode statsJson = getStats(true); + JsonNode shardsJson = statsJson - .getAsJsonObject("indices") - .getAsJsonObject(spec.getConnectionConfiguration().getIndex()) - .getAsJsonObject("shards"); - Set> shards = shardsJson.entrySet(); - for (Map.Entry shardJson : shards) { + .path("indices") + .path(spec.getConnectionConfiguration().getIndex()) + .path("shards"); + + Iterator> shards = shardsJson.fields(); + while (shards.hasNext()) { + Map.Entry shardJson = shards.next(); String shardId = shardJson.getKey(); - JsonArray value = (JsonArray) shardJson.getValue(); + JsonNode value = (JsonNode) shardJson.getValue(); boolean isPrimaryShard = value - .get(0) - .getAsJsonObject() - .getAsJsonObject("routing") - .getAsJsonPrimitive("primary") - .getAsBoolean(); + .path(0) + .path("routing") + .path("primary") + .asBoolean(); if (isPrimaryShard) { sources.add(new BoundedElasticsearchSource(spec, shardId)); } @@ -463,14 +459,14 @@ public long getEstimatedSizeBytes(PipelineOptions options) throws IOException { // NB: Elasticsearch 5.x now provides the slice API. // (https://www.elastic.co/guide/en/elasticsearch/reference/5.0/search-request-scroll.html // #sliced-scroll) - JsonObject statsJson = getStats(false); - JsonObject indexStats = + JsonNode statsJson = getStats(false); + JsonNode indexStats = statsJson - .getAsJsonObject("indices") - .getAsJsonObject(spec.getConnectionConfiguration().getIndex()) - .getAsJsonObject("primaries"); - JsonObject store = indexStats.getAsJsonObject("store"); - return store.getAsJsonPrimitive("size_in_bytes").getAsLong(); + .path("indices") + .path(spec.getConnectionConfiguration().getIndex()) + .path("primaries"); + JsonNode store = indexStats.path("store"); + return store.path("size_in_bytes").asLong(); } @Override @@ -494,7 +490,7 @@ public Coder getDefaultOutputCoder() { return StringUtf8Coder.of(); } - private JsonObject getStats(boolean shardLevel) throws IOException { + private JsonNode getStats(boolean shardLevel) throws IOException { HashMap params = new HashMap<>(); if (shardLevel) { params.put("level", "shards"); @@ -544,13 +540,13 @@ public boolean start() throws IOException { HttpEntity queryEntity = new NStringEntity(query, ContentType.APPLICATION_JSON); response = restClient.performRequest("GET", endPoint, params, queryEntity, new BasicHeader("", "")); - JsonObject searchResult = parseResponse(response); + JsonNode searchResult = parseResponse(response); updateScrollId(searchResult); return readNextBatchAndReturnFirstDocument(searchResult); } - private void updateScrollId(JsonObject searchResult) { - scrollId = searchResult.getAsJsonPrimitive("_scroll_id").getAsString(); + private void updateScrollId(JsonNode searchResult) { + scrollId = searchResult.path("_scroll_id").asText(); } @Override @@ -571,15 +567,15 @@ public boolean advance() throws IOException { Collections.emptyMap(), scrollEntity, new BasicHeader("", "")); - JsonObject searchResult = parseResponse(response); + JsonNode searchResult = parseResponse(response); updateScrollId(searchResult); return readNextBatchAndReturnFirstDocument(searchResult); } } - private boolean readNextBatchAndReturnFirstDocument(JsonObject searchResult) { + private boolean readNextBatchAndReturnFirstDocument(JsonNode searchResult) { //stop if no more data - JsonArray hits = searchResult.getAsJsonObject("hits").getAsJsonArray("hits"); + JsonNode hits = searchResult.path("hits").path("hits"); if (hits.size() == 0) { current = null; batchIterator = null; @@ -587,8 +583,8 @@ private boolean readNextBatchAndReturnFirstDocument(JsonObject searchResult) { } // list behind iterator is empty List batch = new ArrayList<>(); - for (JsonElement hit : hits) { - String document = hit.getAsJsonObject().getAsJsonObject("_source").toString(); + for (JsonNode hit : hits) { + String document = hit.path("_source").toString(); batch.add(document); } batchIterator = batch.listIterator(); @@ -780,26 +776,26 @@ public void finishBundle(Context context) throws Exception { Collections.emptyMap(), requestBody, new BasicHeader("", "")); - JsonObject searchResult = parseResponse(response); - boolean errors = searchResult.getAsJsonPrimitive("errors").getAsBoolean(); + JsonNode searchResult = parseResponse(response); + boolean errors = searchResult.path("errors").asBoolean(); if (errors) { StringBuilder errorMessages = new StringBuilder( "Error writing to Elasticsearch, some elements could not be inserted:"); - JsonArray items = searchResult.getAsJsonArray("items"); + JsonNode items = searchResult.path("items"); //some items present in bulk might have errors, concatenate error messages - for (JsonElement item : items) { - JsonObject creationObject = item.getAsJsonObject().getAsJsonObject("create"); - JsonObject error = creationObject.getAsJsonObject("error"); + for (JsonNode item : items) { + JsonNode creationObject = item.path("create"); + JsonNode error = creationObject.get("error"); if (error != null) { - String type = error.getAsJsonPrimitive("type").getAsString(); - String reason = error.getAsJsonPrimitive("reason").getAsString(); - String docId = creationObject.getAsJsonPrimitive("_id").getAsString(); + String type = error.path("type").asText(); + String reason = error.path("reason").asText(); + String docId = creationObject.path("_id").asText(); errorMessages.append(String.format("%nDocument id %s: %s (%s)", docId, reason, type)); - JsonObject causedBy = error.getAsJsonObject("caused_by"); + JsonNode causedBy = error.get("caused_by"); if (causedBy != null) { - String cbReason = causedBy.getAsJsonPrimitive("reason").getAsString(); - String cbType = causedBy.getAsJsonPrimitive("type").getAsString(); + String cbReason = causedBy.path("reason").asText(); + String cbType = causedBy.path("type").asText(); errorMessages.append(String.format("%nCaused by: %s (%s)", cbReason, cbType)); } } From 11e3029ed38ce4b6090b54ed2337df6c69796f39 Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Tue, 7 Mar 2017 13:17:22 -0800 Subject: [PATCH 45/46] Update python SDK version to the next version. --- sdks/python/apache_beam/version.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/version.py b/sdks/python/apache_beam/version.py index 12509fbdd841..e5d61a132aa6 100644 --- a/sdks/python/apache_beam/version.py +++ b/sdks/python/apache_beam/version.py @@ -21,7 +21,7 @@ import re -__version__ = '0.6.0.dev' +__version__ = '0.7.0.dev' # The following utilities are legacy code from the Maven integration; From 7927f6507f26e23208b9243b9d2a6f539e37e82c Mon Sep 17 00:00:00 2001 From: Mark Liu Date: Tue, 7 Mar 2017 14:26:37 -0800 Subject: [PATCH 46/46] [BEAM-1646] Remove duplicated bigquery dependency --- sdks/python/setup.py | 1 - 1 file changed, 1 deletion(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index cf210d9392a6..8b3e4687dfb4 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -87,7 +87,6 @@ def get_version(): 'avro>=1.7.7,<2.0.0', 'crcmod>=1.7,<2.0', 'dill==0.2.6', - 'google-cloud-bigquery>=0.22.1,<1.0.0', 'httplib2>=0.8,<0.10', 'mock>=1.0.1,<3.0.0', 'oauth2client>=2.0.1,<4.0.0',