You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ke...@apache.org on 2016/04/29 23:55:59 UTC
[01/17] incubator-beam git commit: Move InProcessRunner to its own
module
Repository: incubator-beam
Updated Branches:
refs/heads/master bba4c64d3 -> b9116ac42
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactoryTest.java
deleted file mode 100644
index dfd857e..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactoryTest.java
+++ /dev/null
@@ -1,324 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-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.ParDo;
-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.OutputTimeFns;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.common.CounterSet;
-import org.apache.beam.sdk.util.state.BagState;
-import org.apache.beam.sdk.util.state.StateNamespace;
-import org.apache.beam.sdk.util.state.StateNamespaces;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.apache.beam.sdk.util.state.StateTags;
-import org.apache.beam.sdk.util.state.WatermarkHoldState;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.TupleTag;
-
-import org.hamcrest.Matchers;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.Serializable;
-
-/**
- * Tests for {@link ParDoSingleEvaluatorFactory}.
- */
-@RunWith(JUnit4.class)
-public class ParDoSingleEvaluatorFactoryTest implements Serializable {
- private transient BundleFactory bundleFactory = InProcessBundleFactory.create();
-
- @Test
- public void testParDoInMemoryTransformEvaluator() throws Exception {
- TestPipeline p = TestPipeline.create();
-
- PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
- PCollection<Integer> collection =
- input.apply(
- ParDo.of(
- new DoFn<String, Integer>() {
- @Override
- public void processElement(ProcessContext c) {
- c.output(c.element().length());
- }
- }));
- CommittedBundle<String> inputBundle =
- bundleFactory.createRootBundle(input).commit(Instant.now());
-
- InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
- UncommittedBundle<Integer> outputBundle = bundleFactory.createRootBundle(collection);
- when(evaluationContext.createBundle(inputBundle, collection)).thenReturn(outputBundle);
- InProcessExecutionContext executionContext =
- new InProcessExecutionContext(null, null, null, null);
- when(evaluationContext.getExecutionContext(collection.getProducingTransformInternal(), null))
- .thenReturn(executionContext);
- CounterSet counters = new CounterSet();
- when(evaluationContext.createCounterSet()).thenReturn(counters);
-
- org.apache.beam.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
- new ParDoSingleEvaluatorFactory()
- .forApplication(
- collection.getProducingTransformInternal(), inputBundle, evaluationContext);
-
- evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
- evaluator.processElement(
- WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
- evaluator.processElement(
- WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
-
- InProcessTransformResult result = evaluator.finishBundle();
- assertThat(result.getOutputBundles(), Matchers.<UncommittedBundle<?>>contains(outputBundle));
- assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
- assertThat(result.getCounters(), equalTo(counters));
-
- assertThat(
- outputBundle.commit(Instant.now()).getElements(),
- Matchers.<WindowedValue<Integer>>containsInAnyOrder(
- WindowedValue.valueInGlobalWindow(3),
- WindowedValue.timestampedValueInGlobalWindow(4, new Instant(1000)),
- WindowedValue.valueInGlobalWindow(5, PaneInfo.ON_TIME_AND_ONLY_FIRING)));
- }
-
- @Test
- public void testSideOutputToUndeclaredSideOutputSucceeds() throws Exception {
- TestPipeline p = TestPipeline.create();
-
- PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
- final TupleTag<Integer> sideOutputTag = new TupleTag<Integer>() {};
- PCollection<Integer> collection =
- input.apply(
- ParDo.of(
- new DoFn<String, Integer>() {
- @Override
- public void processElement(ProcessContext c) {
- c.sideOutput(sideOutputTag, c.element().length());
- }
- }));
- CommittedBundle<String> inputBundle =
- bundleFactory.createRootBundle(input).commit(Instant.now());
-
- InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
- UncommittedBundle<Integer> outputBundle = bundleFactory.createRootBundle(collection);
- when(evaluationContext.createBundle(inputBundle, collection)).thenReturn(outputBundle);
- InProcessExecutionContext executionContext =
- new InProcessExecutionContext(null, null, null, null);
- when(evaluationContext.getExecutionContext(collection.getProducingTransformInternal(), null))
- .thenReturn(executionContext);
- CounterSet counters = new CounterSet();
- when(evaluationContext.createCounterSet()).thenReturn(counters);
-
- TransformEvaluator<String> evaluator =
- new ParDoSingleEvaluatorFactory()
- .forApplication(
- collection.getProducingTransformInternal(), inputBundle, evaluationContext);
-
- evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
- evaluator.processElement(
- WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
- evaluator.processElement(
- WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
-
- InProcessTransformResult result = evaluator.finishBundle();
- assertThat(
- result.getOutputBundles(), Matchers.<UncommittedBundle<?>>containsInAnyOrder(outputBundle));
- assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
- assertThat(result.getCounters(), equalTo(counters));
- }
-
- @Test
- public void finishBundleWithStatePutsStateInResult() throws Exception {
- TestPipeline p = TestPipeline.create();
-
- PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
-
- final StateTag<Object, WatermarkHoldState<BoundedWindow>> watermarkTag =
- StateTags.watermarkStateInternal("myId", OutputTimeFns.outputAtEarliestInputTimestamp());
- final StateTag<Object, BagState<String>> bagTag = StateTags.bag("myBag", StringUtf8Coder.of());
- final StateNamespace windowNs =
- StateNamespaces.window(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE);
- ParDo.Bound<String, KV<String, Integer>> pardo =
- ParDo.of(
- new DoFn<String, KV<String, Integer>>() {
- @Override
- public void processElement(ProcessContext c) {
- c.windowingInternals()
- .stateInternals()
- .state(StateNamespaces.global(), watermarkTag)
- .add(new Instant(124443L - c.element().length()));
- c.windowingInternals()
- .stateInternals()
- .state(
- StateNamespaces.window(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE),
- bagTag)
- .add(c.element());
- }
- });
- PCollection<KV<String, Integer>> mainOutput = input.apply(pardo);
-
- CommittedBundle<String> inputBundle =
- bundleFactory.createRootBundle(input).commit(Instant.now());
-
- InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
- UncommittedBundle<KV<String, Integer>> mainOutputBundle =
- bundleFactory.createRootBundle(mainOutput);
-
- when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
-
- InProcessExecutionContext executionContext =
- new InProcessExecutionContext(null, "myKey", null, null);
- when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
- .thenReturn(executionContext);
- CounterSet counters = new CounterSet();
- when(evaluationContext.createCounterSet()).thenReturn(counters);
-
- org.apache.beam.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
- new ParDoSingleEvaluatorFactory()
- .forApplication(
- mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
-
- evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
- evaluator.processElement(
- WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
- evaluator.processElement(
- WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
-
- InProcessTransformResult result = evaluator.finishBundle();
- assertThat(result.getWatermarkHold(), equalTo(new Instant(124438L)));
- assertThat(result.getState(), not(nullValue()));
- assertThat(
- result.getState().state(StateNamespaces.global(), watermarkTag).read(),
- equalTo(new Instant(124438L)));
- assertThat(
- result.getState().state(windowNs, bagTag).read(),
- containsInAnyOrder("foo", "bara", "bazam"));
- }
-
- @Test
- public void finishBundleWithStateAndTimersPutsTimersInResult() throws Exception {
- TestPipeline p = TestPipeline.create();
-
- PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
-
- final TimerData addedTimer =
- TimerData.of(
- StateNamespaces.window(
- IntervalWindow.getCoder(),
- new IntervalWindow(
- new Instant(0).plus(Duration.standardMinutes(5)),
- new Instant(1)
- .plus(Duration.standardMinutes(5))
- .plus(Duration.standardHours(1)))),
- new Instant(54541L),
- TimeDomain.EVENT_TIME);
- final TimerData deletedTimer =
- TimerData.of(
- StateNamespaces.window(
- IntervalWindow.getCoder(),
- new IntervalWindow(new Instant(0), new Instant(0).plus(Duration.standardHours(1)))),
- new Instant(3400000),
- TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
-
- ParDo.Bound<String, KV<String, Integer>> pardo =
- ParDo.of(
- new DoFn<String, KV<String, Integer>>() {
- @Override
- public void processElement(ProcessContext c) {
- c.windowingInternals().stateInternals();
- c.windowingInternals()
- .timerInternals()
- .setTimer(
- TimerData.of(
- StateNamespaces.window(
- IntervalWindow.getCoder(),
- new IntervalWindow(
- new Instant(0).plus(Duration.standardMinutes(5)),
- new Instant(1)
- .plus(Duration.standardMinutes(5))
- .plus(Duration.standardHours(1)))),
- new Instant(54541L),
- TimeDomain.EVENT_TIME));
- c.windowingInternals()
- .timerInternals()
- .deleteTimer(
- TimerData.of(
- StateNamespaces.window(
- IntervalWindow.getCoder(),
- new IntervalWindow(
- new Instant(0),
- new Instant(0).plus(Duration.standardHours(1)))),
- new Instant(3400000),
- TimeDomain.SYNCHRONIZED_PROCESSING_TIME));
- }
- });
- PCollection<KV<String, Integer>> mainOutput = input.apply(pardo);
-
- CommittedBundle<String> inputBundle =
- bundleFactory.createRootBundle(input).commit(Instant.now());
-
- InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
- UncommittedBundle<KV<String, Integer>> mainOutputBundle =
- bundleFactory.createRootBundle(mainOutput);
-
- when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
-
- InProcessExecutionContext executionContext =
- new InProcessExecutionContext(null, "myKey", null, null);
- when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
- .thenReturn(executionContext);
- CounterSet counters = new CounterSet();
- when(evaluationContext.createCounterSet()).thenReturn(counters);
-
- TransformEvaluator<String> evaluator =
- new ParDoSingleEvaluatorFactory()
- .forApplication(
- mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
-
- evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
-
- InProcessTransformResult result = evaluator.finishBundle();
- assertThat(
- result.getTimerUpdate(),
- equalTo(
- TimerUpdate.builder("myKey").setTimer(addedTimer).deletedTimer(deletedTimer).build()));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactoryTest.java
deleted file mode 100644
index 239ce27..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactoryTest.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.theInstance;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.io.TextIOTest;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-
-import org.hamcrest.Matchers;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.File;
-
-/**
- * Tests for {@link TextIOShardedWriteFactory}.
- */
-@RunWith(JUnit4.class)
-public class TextIOShardedWriteFactoryTest {
- @Rule public TemporaryFolder tmp = new TemporaryFolder();
- private TextIOShardedWriteFactory factory;
-
- @Before
- public void setup() {
- factory = new TextIOShardedWriteFactory();
- }
-
- @Test
- public void originalWithoutShardingReturnsOriginal() throws Exception {
- File file = tmp.newFile("foo");
- PTransform<PCollection<String>, PDone> original =
- TextIO.Write.to(file.getAbsolutePath()).withoutSharding();
- PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
- assertThat(overridden, theInstance(original));
- }
-
- @Test
- public void originalShardingNotSpecifiedReturnsOriginal() throws Exception {
- File file = tmp.newFile("foo");
- PTransform<PCollection<String>, PDone> original = TextIO.Write.to(file.getAbsolutePath());
- PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
- assertThat(overridden, theInstance(original));
- }
-
- @Test
- public void originalShardedToOneReturnsExplicitlySharded() throws Exception {
- File file = tmp.newFile("foo");
- TextIO.Write.Bound<String> original =
- TextIO.Write.to(file.getAbsolutePath()).withNumShards(1);
- PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
- assertThat(overridden, not(Matchers.<PTransform<PCollection<String>, PDone>>equalTo(original)));
-
- TestPipeline p = TestPipeline.create();
- String[] elems = new String[] {"foo", "bar", "baz"};
- p.apply(Create.<String>of(elems)).apply(overridden);
-
- file.delete();
-
- p.run();
- TextIOTest.assertOutputFiles(
- elems, StringUtf8Coder.of(), 1, tmp, "foo", original.getShardNameTemplate());
- }
-
- @Test
- public void originalShardedToManyReturnsExplicitlySharded() throws Exception {
- File file = tmp.newFile("foo");
- TextIO.Write.Bound<String> original = TextIO.Write.to(file.getAbsolutePath()).withNumShards(3);
- PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
- assertThat(overridden, not(Matchers.<PTransform<PCollection<String>, PDone>>equalTo(original)));
-
- TestPipeline p = TestPipeline.create();
- String[] elems = new String[] {"foo", "bar", "baz", "spam", "ham", "eggs"};
- p.apply(Create.<String>of(elems)).apply(overridden);
-
- file.delete();
- p.run();
- TextIOTest.assertOutputFiles(
- elems, StringUtf8Coder.of(), 3, tmp, "foo", original.getShardNameTemplate());
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServicesTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServicesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServicesTest.java
deleted file mode 100644
index 33dbbdc..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServicesTest.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.any;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.not;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.verify;
-
-import com.google.common.util.concurrent.MoreExecutors;
-
-import org.hamcrest.Matchers;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-
-/**
- * Tests for {@link TransformExecutorServices}.
- */
-@RunWith(JUnit4.class)
-public class TransformExecutorServicesTest {
- @Rule public ExpectedException thrown = ExpectedException.none();
-
- private ExecutorService executorService;
- private Map<TransformExecutor<?>, Boolean> scheduled;
-
- @Before
- public void setup() {
- executorService = MoreExecutors.newDirectExecutorService();
- scheduled = new ConcurrentHashMap<>();
- }
-
- @Test
- public void parallelScheduleMultipleSchedulesBothImmediately() {
- @SuppressWarnings("unchecked")
- TransformExecutor<Object> first = mock(TransformExecutor.class);
- @SuppressWarnings("unchecked")
- TransformExecutor<Object> second = mock(TransformExecutor.class);
-
- TransformExecutorService parallel =
- TransformExecutorServices.parallel(executorService, scheduled);
- parallel.schedule(first);
- parallel.schedule(second);
-
- verify(first).call();
- verify(second).call();
- assertThat(
- scheduled,
- Matchers.allOf(
- Matchers.<TransformExecutor<?>, Boolean>hasEntry(first, true),
- Matchers.<TransformExecutor<?>, Boolean>hasEntry(second, true)));
-
- parallel.complete(first);
- assertThat(scheduled, Matchers.<TransformExecutor<?>, Boolean>hasEntry(second, true));
- assertThat(
- scheduled,
- not(
- Matchers.<TransformExecutor<?>, Boolean>hasEntry(
- Matchers.<TransformExecutor<?>>equalTo(first), any(Boolean.class))));
- parallel.complete(second);
- assertThat(scheduled.isEmpty(), is(true));
- }
-
- @Test
- public void serialScheduleTwoWaitsForFirstToComplete() {
- @SuppressWarnings("unchecked")
- TransformExecutor<Object> first = mock(TransformExecutor.class);
- @SuppressWarnings("unchecked")
- TransformExecutor<Object> second = mock(TransformExecutor.class);
-
- TransformExecutorService serial = TransformExecutorServices.serial(executorService, scheduled);
- serial.schedule(first);
- verify(first).call();
-
- serial.schedule(second);
- verify(second, never()).call();
-
- assertThat(scheduled, Matchers.<TransformExecutor<?>, Boolean>hasEntry(first, true));
- assertThat(
- scheduled,
- not(
- Matchers.<TransformExecutor<?>, Boolean>hasEntry(
- Matchers.<TransformExecutor<?>>equalTo(second), any(Boolean.class))));
-
- serial.complete(first);
- verify(second).call();
- assertThat(scheduled, Matchers.<TransformExecutor<?>, Boolean>hasEntry(second, true));
- assertThat(
- scheduled,
- not(
- Matchers.<TransformExecutor<?>, Boolean>hasEntry(
- Matchers.<TransformExecutor<?>>equalTo(first), any(Boolean.class))));
-
- serial.complete(second);
- }
-
- @Test
- public void serialCompleteNotExecutingTaskThrows() {
- @SuppressWarnings("unchecked")
- TransformExecutor<Object> first = mock(TransformExecutor.class);
- @SuppressWarnings("unchecked")
- TransformExecutor<Object> second = mock(TransformExecutor.class);
-
- TransformExecutorService serial = TransformExecutorServices.serial(executorService, scheduled);
- serial.schedule(first);
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage("unexpected currently executing");
-
- serial.complete(second);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorTest.java
deleted file mode 100644
index 31cb29a..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorTest.java
+++ /dev/null
@@ -1,538 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.isA;
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.coders.ByteArrayCoder;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-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.WithKeys;
-import org.apache.beam.sdk.util.IllegalMutationException;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.common.util.concurrent.MoreExecutors;
-
-import org.hamcrest.Matchers;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-/**
- * Tests for {@link TransformExecutor}.
- */
-@RunWith(JUnit4.class)
-public class TransformExecutorTest {
- @Rule public ExpectedException thrown = ExpectedException.none();
- private PCollection<String> created;
- private PCollection<KV<Integer, String>> downstream;
-
- private CountDownLatch evaluatorCompleted;
-
- private RegisteringCompletionCallback completionCallback;
- private TransformExecutorService transformEvaluationState;
- private BundleFactory bundleFactory;
- @Mock private InProcessEvaluationContext evaluationContext;
- @Mock private TransformEvaluatorRegistry registry;
- private Map<TransformExecutor<?>, Boolean> scheduled;
-
- @Before
- public void setup() {
- MockitoAnnotations.initMocks(this);
-
- bundleFactory = InProcessBundleFactory.create();
-
- scheduled = new HashMap<>();
- transformEvaluationState =
- TransformExecutorServices.parallel(MoreExecutors.newDirectExecutorService(), scheduled);
-
- evaluatorCompleted = new CountDownLatch(1);
- completionCallback = new RegisteringCompletionCallback(evaluatorCompleted);
-
- TestPipeline p = TestPipeline.create();
- created = p.apply(Create.of("foo", "spam", "third"));
- downstream = created.apply(WithKeys.<Integer, String>of(3));
- }
-
- @Test
- public void callWithNullInputBundleFinishesBundleAndCompletes() throws Exception {
- final InProcessTransformResult result =
- StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
- final AtomicBoolean finishCalled = new AtomicBoolean(false);
- TransformEvaluator<Object> evaluator =
- new TransformEvaluator<Object>() {
- @Override
- public void processElement(WindowedValue<Object> element) throws Exception {
- throw new IllegalArgumentException("Shouldn't be called");
- }
-
- @Override
- public InProcessTransformResult finishBundle() throws Exception {
- finishCalled.set(true);
- return result;
- }
- };
-
- when(registry.forApplication(created.getProducingTransformInternal(), null, evaluationContext))
- .thenReturn(evaluator);
-
- TransformExecutor<Object> executor =
- TransformExecutor.create(
- registry,
- Collections.<ModelEnforcementFactory>emptyList(),
- evaluationContext,
- null,
- created.getProducingTransformInternal(),
- completionCallback,
- transformEvaluationState);
- executor.call();
-
- assertThat(finishCalled.get(), is(true));
- assertThat(completionCallback.handledResult, equalTo(result));
- assertThat(completionCallback.handledThrowable, is(nullValue()));
- assertThat(scheduled, not(Matchers.<TransformExecutor<?>>hasKey(executor)));
- }
-
- @Test
- public void inputBundleProcessesEachElementFinishesAndCompletes() throws Exception {
- final InProcessTransformResult result =
- StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build();
- final Collection<WindowedValue<String>> elementsProcessed = new ArrayList<>();
- TransformEvaluator<String> evaluator =
- new TransformEvaluator<String>() {
- @Override
- public void processElement(WindowedValue<String> element) throws Exception {
- elementsProcessed.add(element);
- return;
- }
-
- @Override
- public InProcessTransformResult finishBundle() throws Exception {
- return result;
- }
- };
-
- WindowedValue<String> foo = WindowedValue.valueInGlobalWindow("foo");
- WindowedValue<String> spam = WindowedValue.valueInGlobalWindow("spam");
- WindowedValue<String> third = WindowedValue.valueInGlobalWindow("third");
- CommittedBundle<String> inputBundle =
- bundleFactory.createRootBundle(created).add(foo).add(spam).add(third).commit(Instant.now());
- when(
- registry.<String>forApplication(
- downstream.getProducingTransformInternal(), inputBundle, evaluationContext))
- .thenReturn(evaluator);
-
- TransformExecutor<String> executor =
- TransformExecutor.create(
- registry,
- Collections.<ModelEnforcementFactory>emptyList(),
- evaluationContext,
- inputBundle,
- downstream.getProducingTransformInternal(),
- completionCallback,
- transformEvaluationState);
-
- Executors.newSingleThreadExecutor().submit(executor);
-
- evaluatorCompleted.await();
-
- assertThat(elementsProcessed, containsInAnyOrder(spam, third, foo));
- assertThat(completionCallback.handledResult, equalTo(result));
- assertThat(completionCallback.handledThrowable, is(nullValue()));
- assertThat(scheduled, not(Matchers.<TransformExecutor<?>>hasKey(executor)));
- }
-
- @Test
- public void processElementThrowsExceptionCallsback() throws Exception {
- final InProcessTransformResult result =
- StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build();
- final Exception exception = new Exception();
- TransformEvaluator<String> evaluator =
- new TransformEvaluator<String>() {
- @Override
- public void processElement(WindowedValue<String> element) throws Exception {
- throw exception;
- }
-
- @Override
- public InProcessTransformResult finishBundle() throws Exception {
- return result;
- }
- };
-
- WindowedValue<String> foo = WindowedValue.valueInGlobalWindow("foo");
- CommittedBundle<String> inputBundle =
- bundleFactory.createRootBundle(created).add(foo).commit(Instant.now());
- when(
- registry.<String>forApplication(
- downstream.getProducingTransformInternal(), inputBundle, evaluationContext))
- .thenReturn(evaluator);
-
- TransformExecutor<String> executor =
- TransformExecutor.create(
- registry,
- Collections.<ModelEnforcementFactory>emptyList(),
- evaluationContext,
- inputBundle,
- downstream.getProducingTransformInternal(),
- completionCallback,
- transformEvaluationState);
- Executors.newSingleThreadExecutor().submit(executor);
-
- evaluatorCompleted.await();
-
- assertThat(completionCallback.handledResult, is(nullValue()));
- assertThat(completionCallback.handledThrowable, Matchers.<Throwable>equalTo(exception));
- assertThat(scheduled, not(Matchers.<TransformExecutor<?>>hasKey(executor)));
- }
-
- @Test
- public void finishBundleThrowsExceptionCallsback() throws Exception {
- final Exception exception = new Exception();
- TransformEvaluator<String> evaluator =
- new TransformEvaluator<String>() {
- @Override
- public void processElement(WindowedValue<String> element) throws Exception {}
-
- @Override
- public InProcessTransformResult finishBundle() throws Exception {
- throw exception;
- }
- };
-
- CommittedBundle<String> inputBundle =
- bundleFactory.createRootBundle(created).commit(Instant.now());
- when(
- registry.<String>forApplication(
- downstream.getProducingTransformInternal(), inputBundle, evaluationContext))
- .thenReturn(evaluator);
-
- TransformExecutor<String> executor =
- TransformExecutor.create(
- registry,
- Collections.<ModelEnforcementFactory>emptyList(),
- evaluationContext,
- inputBundle,
- downstream.getProducingTransformInternal(),
- completionCallback,
- transformEvaluationState);
- Executors.newSingleThreadExecutor().submit(executor);
-
- evaluatorCompleted.await();
-
- assertThat(completionCallback.handledResult, is(nullValue()));
- assertThat(completionCallback.handledThrowable, Matchers.<Throwable>equalTo(exception));
- assertThat(scheduled, not(Matchers.<TransformExecutor<?>>hasKey(executor)));
- }
-
- @Test
- public void duringCallGetThreadIsNonNull() throws Exception {
- final InProcessTransformResult result =
- StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build();
- final CountDownLatch testLatch = new CountDownLatch(1);
- final CountDownLatch evaluatorLatch = new CountDownLatch(1);
- TransformEvaluator<Object> evaluator =
- new TransformEvaluator<Object>() {
- @Override
- public void processElement(WindowedValue<Object> element) throws Exception {
- throw new IllegalArgumentException("Shouldn't be called");
- }
-
- @Override
- public InProcessTransformResult finishBundle() throws Exception {
- testLatch.countDown();
- evaluatorLatch.await();
- return result;
- }
- };
-
- when(registry.forApplication(created.getProducingTransformInternal(), null, evaluationContext))
- .thenReturn(evaluator);
-
- TransformExecutor<String> executor =
- TransformExecutor.create(
- registry,
- Collections.<ModelEnforcementFactory>emptyList(),
- evaluationContext,
- null,
- created.getProducingTransformInternal(),
- completionCallback,
- transformEvaluationState);
-
- Executors.newSingleThreadExecutor().submit(executor);
- testLatch.await();
- assertThat(executor.getThread(), not(nullValue()));
-
- // Finish the execution so everything can get closed down cleanly.
- evaluatorLatch.countDown();
- }
-
- @Test
- public void callWithEnforcementAppliesEnforcement() throws Exception {
- final InProcessTransformResult result =
- StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build();
-
- TransformEvaluator<Object> evaluator =
- new TransformEvaluator<Object>() {
- @Override
- public void processElement(WindowedValue<Object> element) throws Exception {
- }
-
- @Override
- public InProcessTransformResult finishBundle() throws Exception {
- return result;
- }
- };
-
- WindowedValue<String> fooElem = WindowedValue.valueInGlobalWindow("foo");
- WindowedValue<String> barElem = WindowedValue.valueInGlobalWindow("bar");
- CommittedBundle<String> inputBundle =
- bundleFactory.createRootBundle(created).add(fooElem).add(barElem).commit(Instant.now());
- when(
- registry.forApplication(
- downstream.getProducingTransformInternal(), inputBundle, evaluationContext))
- .thenReturn(evaluator);
-
- TestEnforcementFactory enforcement = new TestEnforcementFactory();
- TransformExecutor<String> executor =
- TransformExecutor.create(
- registry,
- Collections.<ModelEnforcementFactory>singleton(enforcement),
- evaluationContext,
- inputBundle,
- downstream.getProducingTransformInternal(),
- completionCallback,
- transformEvaluationState);
-
- executor.call();
- TestEnforcement<?> testEnforcement = enforcement.instance;
- assertThat(
- testEnforcement.beforeElements,
- Matchers.<WindowedValue<?>>containsInAnyOrder(barElem, fooElem));
- assertThat(
- testEnforcement.afterElements,
- Matchers.<WindowedValue<?>>containsInAnyOrder(barElem, fooElem));
- assertThat(testEnforcement.finishedBundles, contains(result));
- }
-
- @Test
- public void callWithEnforcementThrowsOnFinishPropagates() throws Exception {
- PCollection<byte[]> pcBytes =
- created.apply(
- new PTransform<PCollection<String>, PCollection<byte[]>>() {
- @Override
- public PCollection<byte[]> apply(PCollection<String> input) {
- return PCollection.<byte[]>createPrimitiveOutputInternal(
- input.getPipeline(), input.getWindowingStrategy(), input.isBounded())
- .setCoder(ByteArrayCoder.of());
- }
- });
-
- final InProcessTransformResult result =
- StepTransformResult.withoutHold(pcBytes.getProducingTransformInternal()).build();
- final CountDownLatch testLatch = new CountDownLatch(1);
- final CountDownLatch evaluatorLatch = new CountDownLatch(1);
-
- TransformEvaluator<Object> evaluator =
- new TransformEvaluator<Object>() {
- @Override
- public void processElement(WindowedValue<Object> element) throws Exception {}
-
- @Override
- public InProcessTransformResult finishBundle() throws Exception {
- testLatch.countDown();
- evaluatorLatch.await();
- return result;
- }
- };
-
- WindowedValue<byte[]> fooBytes = WindowedValue.valueInGlobalWindow("foo".getBytes());
- CommittedBundle<byte[]> inputBundle =
- bundleFactory.createRootBundle(pcBytes).add(fooBytes).commit(Instant.now());
- when(
- registry.forApplication(
- pcBytes.getProducingTransformInternal(), inputBundle, evaluationContext))
- .thenReturn(evaluator);
-
- TransformExecutor<byte[]> executor =
- TransformExecutor.create(
- registry,
- Collections.<ModelEnforcementFactory>singleton(ImmutabilityEnforcementFactory.create()),
- evaluationContext,
- inputBundle,
- pcBytes.getProducingTransformInternal(),
- completionCallback,
- transformEvaluationState);
-
- Future<InProcessTransformResult> task = Executors.newSingleThreadExecutor().submit(executor);
- testLatch.await();
- fooBytes.getValue()[0] = 'b';
- evaluatorLatch.countDown();
-
- thrown.expectCause(isA(IllegalMutationException.class));
- task.get();
- }
-
- @Test
- public void callWithEnforcementThrowsOnElementPropagates() throws Exception {
- PCollection<byte[]> pcBytes =
- created.apply(
- new PTransform<PCollection<String>, PCollection<byte[]>>() {
- @Override
- public PCollection<byte[]> apply(PCollection<String> input) {
- return PCollection.<byte[]>createPrimitiveOutputInternal(
- input.getPipeline(), input.getWindowingStrategy(), input.isBounded())
- .setCoder(ByteArrayCoder.of());
- }
- });
-
- final InProcessTransformResult result =
- StepTransformResult.withoutHold(pcBytes.getProducingTransformInternal()).build();
- final CountDownLatch testLatch = new CountDownLatch(1);
- final CountDownLatch evaluatorLatch = new CountDownLatch(1);
-
- TransformEvaluator<Object> evaluator =
- new TransformEvaluator<Object>() {
- @Override
- public void processElement(WindowedValue<Object> element) throws Exception {
- testLatch.countDown();
- evaluatorLatch.await();
- }
-
- @Override
- public InProcessTransformResult finishBundle() throws Exception {
- return result;
- }
- };
-
- WindowedValue<byte[]> fooBytes = WindowedValue.valueInGlobalWindow("foo".getBytes());
- CommittedBundle<byte[]> inputBundle =
- bundleFactory.createRootBundle(pcBytes).add(fooBytes).commit(Instant.now());
- when(
- registry.forApplication(
- pcBytes.getProducingTransformInternal(), inputBundle, evaluationContext))
- .thenReturn(evaluator);
-
- TransformExecutor<byte[]> executor =
- TransformExecutor.create(
- registry,
- Collections.<ModelEnforcementFactory>singleton(ImmutabilityEnforcementFactory.create()),
- evaluationContext,
- inputBundle,
- pcBytes.getProducingTransformInternal(),
- completionCallback,
- transformEvaluationState);
-
- Future<InProcessTransformResult> task = Executors.newSingleThreadExecutor().submit(executor);
- testLatch.await();
- fooBytes.getValue()[0] = 'b';
- evaluatorLatch.countDown();
-
- thrown.expectCause(isA(IllegalMutationException.class));
- task.get();
- }
-
- private static class RegisteringCompletionCallback implements CompletionCallback {
- private InProcessTransformResult handledResult = null;
- private Throwable handledThrowable = null;
- private final CountDownLatch onMethod;
-
- private RegisteringCompletionCallback(CountDownLatch onMethod) {
- this.onMethod = onMethod;
- }
-
- @Override
- public CommittedResult handleResult(
- CommittedBundle<?> inputBundle, InProcessTransformResult result) {
- handledResult = result;
- onMethod.countDown();
- return CommittedResult.create(result, Collections.<CommittedBundle<?>>emptyList());
- }
-
- @Override
- public void handleThrowable(CommittedBundle<?> inputBundle, Throwable t) {
- handledThrowable = t;
- onMethod.countDown();
- }
- }
-
- private static class TestEnforcementFactory implements ModelEnforcementFactory {
- private TestEnforcement<?> instance;
- @Override
- public <T> TestEnforcement<T> forBundle(
- CommittedBundle<T> input, AppliedPTransform<?, ?, ?> consumer) {
- TestEnforcement<T> newEnforcement = new TestEnforcement<>();
- instance = newEnforcement;
- return newEnforcement;
- }
- }
-
- private static class TestEnforcement<T> implements ModelEnforcement<T> {
- private final List<WindowedValue<T>> beforeElements = new ArrayList<>();
- private final List<WindowedValue<T>> afterElements = new ArrayList<>();
- private final List<InProcessTransformResult> finishedBundles = new ArrayList<>();
-
- @Override
- public void beforeElement(WindowedValue<T> element) {
- beforeElements.add(element);
- }
-
- @Override
- public void afterElement(WindowedValue<T> element) {
- afterElements.add(element);
- }
-
- @Override
- public void afterFinish(
- CommittedBundle<T> input,
- InProcessTransformResult result,
- Iterable<? extends CommittedBundle<?>> outputs) {
- finishedBundles.add(result);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactoryTest.java
deleted file mode 100644
index 82657c0..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactoryTest.java
+++ /dev/null
@@ -1,334 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.hasSize;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.BigEndianLongCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.io.CountingSource;
-import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.io.UnboundedSource;
-import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.common.collect.ImmutableList;
-
-import org.hamcrest.Matchers;
-import org.joda.time.DateTime;
-import org.joda.time.Instant;
-import org.joda.time.ReadableInstant;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Arrays;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-import javax.annotation.Nullable;
-/**
- * Tests for {@link UnboundedReadEvaluatorFactory}.
- */
-@RunWith(JUnit4.class)
-public class UnboundedReadEvaluatorFactoryTest {
- private PCollection<Long> longs;
- private TransformEvaluatorFactory factory;
- private InProcessEvaluationContext context;
- private UncommittedBundle<Long> output;
-
- private BundleFactory bundleFactory = InProcessBundleFactory.create();
-
- @Before
- public void setup() {
- UnboundedSource<Long, ?> source =
- CountingSource.unboundedWithTimestampFn(new LongToInstantFn());
- TestPipeline p = TestPipeline.create();
- longs = p.apply(Read.from(source));
-
- factory = new UnboundedReadEvaluatorFactory();
- context = mock(InProcessEvaluationContext.class);
- output = bundleFactory.createRootBundle(longs);
- when(context.createRootBundle(longs)).thenReturn(output);
- }
-
- @Test
- public void unboundedSourceInMemoryTransformEvaluatorProducesElements() throws Exception {
- TransformEvaluator<?> evaluator =
- factory.forApplication(longs.getProducingTransformInternal(), null, context);
-
- InProcessTransformResult result = evaluator.finishBundle();
- assertThat(
- result.getWatermarkHold(), Matchers.<ReadableInstant>lessThan(DateTime.now().toInstant()));
- assertThat(
- output.commit(Instant.now()).getElements(),
- containsInAnyOrder(
- tgw(1L), tgw(2L), tgw(4L), tgw(8L), tgw(9L), tgw(7L), tgw(6L), tgw(5L), tgw(3L),
- tgw(0L)));
- }
-
- /**
- * Demonstrate that multiple sequential creations will produce additional elements if the source
- * can provide them.
- */
- @Test
- public void unboundedSourceInMemoryTransformEvaluatorMultipleSequentialCalls() throws Exception {
- TransformEvaluator<?> evaluator =
- factory.forApplication(longs.getProducingTransformInternal(), null, context);
-
- InProcessTransformResult result = evaluator.finishBundle();
- assertThat(
- result.getWatermarkHold(), Matchers.<ReadableInstant>lessThan(DateTime.now().toInstant()));
- assertThat(
- output.commit(Instant.now()).getElements(),
- containsInAnyOrder(
- tgw(1L), tgw(2L), tgw(4L), tgw(8L), tgw(9L), tgw(7L), tgw(6L), tgw(5L), tgw(3L),
- tgw(0L)));
-
- UncommittedBundle<Long> secondOutput = bundleFactory.createRootBundle(longs);
- when(context.createRootBundle(longs)).thenReturn(secondOutput);
- TransformEvaluator<?> secondEvaluator =
- factory.forApplication(longs.getProducingTransformInternal(), null, context);
- InProcessTransformResult secondResult = secondEvaluator.finishBundle();
- assertThat(
- secondResult.getWatermarkHold(),
- Matchers.<ReadableInstant>lessThan(DateTime.now().toInstant()));
- assertThat(
- secondOutput.commit(Instant.now()).getElements(),
- containsInAnyOrder(tgw(11L), tgw(12L), tgw(14L), tgw(18L), tgw(19L), tgw(17L), tgw(16L),
- tgw(15L), tgw(13L), tgw(10L)));
- }
-
- @Test
- public void boundedSourceEvaluatorClosesReader() throws Exception {
- TestUnboundedSource<Long> source =
- new TestUnboundedSource<>(BigEndianLongCoder.of(), 1L, 2L, 3L);
-
- TestPipeline p = TestPipeline.create();
- PCollection<Long> pcollection = p.apply(Read.from(source));
- AppliedPTransform<?, ?, ?> sourceTransform = pcollection.getProducingTransformInternal();
-
- UncommittedBundle<Long> output = bundleFactory.createRootBundle(pcollection);
- when(context.createRootBundle(pcollection)).thenReturn(output);
-
- TransformEvaluator<?> evaluator = factory.forApplication(sourceTransform, null, context);
- evaluator.finishBundle();
- CommittedBundle<Long> committed = output.commit(Instant.now());
- assertThat(ImmutableList.copyOf(committed.getElements()), hasSize(3));
- assertThat(TestUnboundedSource.readerClosedCount, equalTo(1));
- }
-
- @Test
- public void boundedSourceEvaluatorNoElementsClosesReader() throws Exception {
- TestUnboundedSource<Long> source = new TestUnboundedSource<>(BigEndianLongCoder.of());
-
- TestPipeline p = TestPipeline.create();
- PCollection<Long> pcollection = p.apply(Read.from(source));
- AppliedPTransform<?, ?, ?> sourceTransform = pcollection.getProducingTransformInternal();
-
- UncommittedBundle<Long> output = bundleFactory.createRootBundle(pcollection);
- when(context.createRootBundle(pcollection)).thenReturn(output);
-
- TransformEvaluator<?> evaluator = factory.forApplication(sourceTransform, null, context);
- evaluator.finishBundle();
- CommittedBundle<Long> committed = output.commit(Instant.now());
- assertThat(committed.getElements(), emptyIterable());
- assertThat(TestUnboundedSource.readerClosedCount, equalTo(1));
- }
-
- // TODO: Once the source is split into multiple sources before evaluating, this test will have to
- // be updated.
- /**
- * Demonstrate that only a single unfinished instance of TransformEvaluator can be created at a
- * time, with other calls returning an empty evaluator.
- */
- @Test
- public void unboundedSourceWithMultipleSimultaneousEvaluatorsIndependent() throws Exception {
- UncommittedBundle<Long> secondOutput = bundleFactory.createRootBundle(longs);
-
- TransformEvaluator<?> evaluator =
- factory.forApplication(longs.getProducingTransformInternal(), null, context);
-
- TransformEvaluator<?> secondEvaluator =
- factory.forApplication(longs.getProducingTransformInternal(), null, context);
-
- InProcessTransformResult secondResult = secondEvaluator.finishBundle();
- InProcessTransformResult result = evaluator.finishBundle();
-
- assertThat(
- result.getWatermarkHold(), Matchers.<ReadableInstant>lessThan(DateTime.now().toInstant()));
- assertThat(
- output.commit(Instant.now()).getElements(),
- containsInAnyOrder(
- tgw(1L), tgw(2L), tgw(4L), tgw(8L), tgw(9L), tgw(7L), tgw(6L), tgw(5L), tgw(3L),
- tgw(0L)));
-
- assertThat(secondResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
- assertThat(secondOutput.commit(Instant.now()).getElements(), emptyIterable());
- }
-
- /**
- * A terse alias for producing timestamped longs in the {@link GlobalWindow}, where
- * the timestamp is the epoch offset by the value of the element.
- */
- private static WindowedValue<Long> tgw(Long elem) {
- return WindowedValue.timestampedValueInGlobalWindow(elem, new Instant(elem));
- }
-
- private static class LongToInstantFn implements SerializableFunction<Long, Instant> {
- @Override
- public Instant apply(Long input) {
- return new Instant(input);
- }
- }
-
- private static class TestUnboundedSource<T> extends UnboundedSource<T, TestCheckpointMark> {
- static int readerClosedCount;
- private final Coder<T> coder;
- private final List<T> elems;
-
- public TestUnboundedSource(Coder<T> coder, T... elems) {
- readerClosedCount = 0;
- this.coder = coder;
- this.elems = Arrays.asList(elems);
- }
-
- @Override
- public List<? extends UnboundedSource<T, TestCheckpointMark>> generateInitialSplits(
- int desiredNumSplits, PipelineOptions options) throws Exception {
- return ImmutableList.of(this);
- }
-
- @Override
- public UnboundedSource.UnboundedReader<T> createReader(
- PipelineOptions options, TestCheckpointMark checkpointMark) {
- return new TestUnboundedReader(elems);
- }
-
- @Override
- @Nullable
- public Coder<TestCheckpointMark> getCheckpointMarkCoder() {
- return new TestCheckpointMark.Coder();
- }
-
- @Override
- public void validate() {}
-
- @Override
- public Coder<T> getDefaultOutputCoder() {
- return coder;
- }
-
- private class TestUnboundedReader extends UnboundedReader<T> {
- private final List<T> elems;
- private int index;
-
- public TestUnboundedReader(List<T> elems) {
- this.elems = elems;
- this.index = -1;
- }
-
- @Override
- public boolean start() throws IOException {
- return advance();
- }
-
- @Override
- public boolean advance() throws IOException {
- if (index + 1 < elems.size()) {
- index++;
- return true;
- }
- return false;
- }
-
- @Override
- public Instant getWatermark() {
- return Instant.now();
- }
-
- @Override
- public CheckpointMark getCheckpointMark() {
- return new TestCheckpointMark();
- }
-
- @Override
- public UnboundedSource<T, ?> getCurrentSource() {
- TestUnboundedSource<T> source = TestUnboundedSource.this;
- return source;
- }
-
- @Override
- public T getCurrent() throws NoSuchElementException {
- return elems.get(index);
- }
-
- @Override
- public Instant getCurrentTimestamp() throws NoSuchElementException {
- return Instant.now();
- }
-
- @Override
- public void close() throws IOException {
- readerClosedCount++;
- }
- }
- }
-
- private static class TestCheckpointMark implements CheckpointMark {
- @Override
- public void finalizeCheckpoint() throws IOException {}
-
- public static class Coder extends AtomicCoder<TestCheckpointMark> {
- @Override
- public void encode(
- TestCheckpointMark value,
- OutputStream outStream,
- org.apache.beam.sdk.coders.Coder.Context context)
- throws CoderException, IOException {}
-
- @Override
- public TestCheckpointMark decode(
- InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
- throws CoderException, IOException {
- return new TestCheckpointMark();
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactoryTest.java
deleted file mode 100644
index 05346dc..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactoryTest.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VoidCoder;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.PCollectionViewWriter;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.Values;
-import org.apache.beam.sdk.transforms.View.CreatePCollectionView;
-import org.apache.beam.sdk.transforms.WithKeys;
-import org.apache.beam.sdk.util.PCollectionViews;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionView;
-
-import com.google.common.collect.ImmutableList;
-
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link ViewEvaluatorFactory}.
- */
-@RunWith(JUnit4.class)
-public class ViewEvaluatorFactoryTest {
- private BundleFactory bundleFactory = InProcessBundleFactory.create();
-
- @Test
- public void testInMemoryEvaluator() throws Exception {
- TestPipeline p = TestPipeline.create();
-
- PCollection<String> input = p.apply(Create.of("foo", "bar"));
- CreatePCollectionView<String, Iterable<String>> createView =
- CreatePCollectionView.of(
- PCollectionViews.iterableView(p, input.getWindowingStrategy(), StringUtf8Coder.of()));
- PCollection<Iterable<String>> concat =
- input.apply(WithKeys.<Void, String>of((Void) null))
- .setCoder(KvCoder.of(VoidCoder.of(), StringUtf8Coder.of()))
- .apply(GroupByKey.<Void, String>create())
- .apply(Values.<Iterable<String>>create());
- PCollectionView<Iterable<String>> view =
- concat.apply(new ViewEvaluatorFactory.WriteView<>(createView));
-
- InProcessEvaluationContext context = mock(InProcessEvaluationContext.class);
- TestViewWriter<String, Iterable<String>> viewWriter = new TestViewWriter<>();
- when(context.createPCollectionViewWriter(concat, view)).thenReturn(viewWriter);
-
- CommittedBundle<String> inputBundle =
- bundleFactory.createRootBundle(input).commit(Instant.now());
- TransformEvaluator<Iterable<String>> evaluator =
- new ViewEvaluatorFactory()
- .forApplication(view.getProducingTransformInternal(), inputBundle, context);
-
- evaluator.processElement(
- WindowedValue.<Iterable<String>>valueInGlobalWindow(ImmutableList.of("foo", "bar")));
- assertThat(viewWriter.latest, nullValue());
-
- evaluator.finishBundle();
- assertThat(
- viewWriter.latest,
- containsInAnyOrder(
- WindowedValue.valueInGlobalWindow("foo"), WindowedValue.valueInGlobalWindow("bar")));
- }
-
- private static class TestViewWriter<ElemT, ViewT> implements PCollectionViewWriter<ElemT, ViewT> {
- private Iterable<WindowedValue<ElemT>> latest;
-
- @Override
- public void add(Iterable<WindowedValue<ElemT>> values) {
- latest = values;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutorTest.java
deleted file mode 100644
index 3b36bc5..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutorTest.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.equalTo;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-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.WindowFn;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.values.PCollection;
-
-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 java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Tests for {@link WatermarkCallbackExecutor}.
- */
-@RunWith(JUnit4.class)
-public class WatermarkCallbackExecutorTest {
- private WatermarkCallbackExecutor executor = WatermarkCallbackExecutor.create();
- private AppliedPTransform<?, ?, ?> create;
- private AppliedPTransform<?, ?, ?> sum;
-
- @Before
- public void setup() {
- TestPipeline p = TestPipeline.create();
- PCollection<Integer> created = p.apply(Create.of(1, 2, 3));
- create = created.getProducingTransformInternal();
- sum = created.apply(Sum.integersGlobally()).getProducingTransformInternal();
- }
-
- @Test
- public void onGuaranteedFiringFiresAfterTrigger() throws Exception {
- CountDownLatch latch = new CountDownLatch(1);
- executor.callOnGuaranteedFiring(
- create,
- GlobalWindow.INSTANCE,
- WindowingStrategy.globalDefault(),
- new CountDownLatchCallback(latch));
-
- executor.fireForWatermark(create, BoundedWindow.TIMESTAMP_MAX_VALUE);
- assertThat(latch.await(500, TimeUnit.MILLISECONDS), equalTo(true));
- }
-
- @Test
- public void multipleCallbacksShouldFireFires() throws Exception {
- CountDownLatch latch = new CountDownLatch(2);
- WindowFn<Object, IntervalWindow> windowFn = FixedWindows.of(Duration.standardMinutes(10));
- IntervalWindow window =
- new IntervalWindow(new Instant(0L), new Instant(0L).plus(Duration.standardMinutes(10)));
- executor.callOnGuaranteedFiring(
- create, window, WindowingStrategy.of(windowFn), new CountDownLatchCallback(latch));
- executor.callOnGuaranteedFiring(
- create, window, WindowingStrategy.of(windowFn), new CountDownLatchCallback(latch));
-
- executor.fireForWatermark(create, new Instant(0L).plus(Duration.standardMinutes(10)));
- assertThat(latch.await(500, TimeUnit.MILLISECONDS), equalTo(true));
- }
-
- @Test
- public void noCallbacksShouldFire() throws Exception {
- CountDownLatch latch = new CountDownLatch(1);
- WindowFn<Object, IntervalWindow> windowFn = FixedWindows.of(Duration.standardMinutes(10));
- IntervalWindow window =
- new IntervalWindow(new Instant(0L), new Instant(0L).plus(Duration.standardMinutes(10)));
- executor.callOnGuaranteedFiring(
- create, window, WindowingStrategy.of(windowFn), new CountDownLatchCallback(latch));
-
- executor.fireForWatermark(create, new Instant(0L).plus(Duration.standardMinutes(5)));
- assertThat(latch.await(500, TimeUnit.MILLISECONDS), equalTo(false));
- }
-
- @Test
- public void unrelatedStepShouldNotFire() throws Exception {
- CountDownLatch latch = new CountDownLatch(1);
- WindowFn<Object, IntervalWindow> windowFn = FixedWindows.of(Duration.standardMinutes(10));
- IntervalWindow window =
- new IntervalWindow(new Instant(0L), new Instant(0L).plus(Duration.standardMinutes(10)));
- executor.callOnGuaranteedFiring(
- sum, window, WindowingStrategy.of(windowFn), new CountDownLatchCallback(latch));
-
- executor.fireForWatermark(create, new Instant(0L).plus(Duration.standardMinutes(20)));
- assertThat(latch.await(500, TimeUnit.MILLISECONDS), equalTo(false));
- }
-
- private static class CountDownLatchCallback implements Runnable {
- private final CountDownLatch latch;
-
- public CountDownLatchCallback(CountDownLatch latch) {
- this.latch = latch;
- }
-
- @Override
- public void run() {
- latch.countDown();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactoryTest.java
deleted file mode 100644
index d41825d..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactoryTest.java
+++ /dev/null
@@ -1,222 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-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.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.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.transforms.windowing.Window.Bound;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-
-import org.hamcrest.Matchers;
-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 WindowEvaluatorFactory}.
- */
-@RunWith(JUnit4.class)
-public class WindowEvaluatorFactoryTest {
- private static final Instant EPOCH = new Instant(0);
-
- private PCollection<Long> input;
- private WindowEvaluatorFactory factory;
-
- @Mock private InProcessEvaluationContext evaluationContext;
-
- private BundleFactory bundleFactory;
-
- private WindowedValue<Long> first =
- WindowedValue.timestampedValueInGlobalWindow(3L, new Instant(2L));
- private WindowedValue<Long> second =
- WindowedValue.timestampedValueInGlobalWindow(
- Long.valueOf(1L), EPOCH.plus(Duration.standardDays(3)));
- private WindowedValue<Long> third =
- WindowedValue.of(
- Long.valueOf(2L),
- new Instant(-10L),
- new IntervalWindow(new Instant(-100), EPOCH),
- PaneInfo.NO_FIRING);
-
- @Before
- public void setup() {
- MockitoAnnotations.initMocks(this);
- TestPipeline p = TestPipeline.create();
- input = p.apply(Create.of(1L, 2L, 3L));
-
- bundleFactory = InProcessBundleFactory.create();
- factory = new WindowEvaluatorFactory();
- }
-
- @Test
- public void nullWindowFunSucceeds() throws Exception {
- Bound<Long> transform =
- Window.<Long>triggering(
- AfterWatermark.pastEndOfWindow().withEarlyFirings(AfterPane.elementCountAtLeast(1)))
- .accumulatingFiredPanes();
- PCollection<Long> triggering = input.apply(transform);
-
- CommittedBundle<Long> inputBundle = createInputBundle();
-
- UncommittedBundle<Long> outputBundle = createOutputBundle(triggering, inputBundle);
-
- InProcessTransformResult result = runEvaluator(triggering, inputBundle, transform);
-
- assertThat(
- Iterables.getOnlyElement(result.getOutputBundles()),
- Matchers.<UncommittedBundle<?>>equalTo(outputBundle));
- CommittedBundle<Long> committed = outputBundle.commit(Instant.now());
- assertThat(committed.getElements(), containsInAnyOrder(third, first, second));
- }
-
- @Test
- public void singleWindowFnSucceeds() throws Exception {
- Duration windowDuration = Duration.standardDays(7);
- Bound<Long> transform = Window.<Long>into(FixedWindows.of(windowDuration));
- PCollection<Long> windowed = input.apply(transform);
-
- CommittedBundle<Long> inputBundle = createInputBundle();
-
- UncommittedBundle<Long> outputBundle = createOutputBundle(windowed, inputBundle);
-
- BoundedWindow firstSecondWindow = new IntervalWindow(EPOCH, EPOCH.plus(windowDuration));
- BoundedWindow thirdWindow = new IntervalWindow(EPOCH.minus(windowDuration), EPOCH);
-
- InProcessTransformResult result = runEvaluator(windowed, inputBundle, transform);
-
- assertThat(
- Iterables.getOnlyElement(result.getOutputBundles()),
- Matchers.<UncommittedBundle<?>>equalTo(outputBundle));
- CommittedBundle<Long> committed = outputBundle.commit(Instant.now());
-
- WindowedValue<Long> expectedNewFirst =
- WindowedValue.of(3L, new Instant(2L), firstSecondWindow, PaneInfo.NO_FIRING);
- WindowedValue<Long> expectedNewSecond =
- WindowedValue.of(
- 1L, EPOCH.plus(Duration.standardDays(3)), firstSecondWindow, PaneInfo.NO_FIRING);
- WindowedValue<Long> expectedNewThird =
- WindowedValue.of(2L, new Instant(-10L), thirdWindow, PaneInfo.NO_FIRING);
- assertThat(
- committed.getElements(),
- containsInAnyOrder(expectedNewFirst, expectedNewSecond, expectedNewThird));
- }
-
- @Test
- public void multipleWindowsWindowFnSucceeds() throws Exception {
- Duration windowDuration = Duration.standardDays(6);
- Duration slidingBy = Duration.standardDays(3);
- Bound<Long> transform = Window.into(SlidingWindows.of(windowDuration).every(slidingBy));
- PCollection<Long> windowed = input.apply(transform);
-
- CommittedBundle<Long> inputBundle = createInputBundle();
- UncommittedBundle<Long> outputBundle = createOutputBundle(windowed, inputBundle);
-
- InProcessTransformResult result = runEvaluator(windowed, inputBundle, transform);
-
- assertThat(
- Iterables.getOnlyElement(result.getOutputBundles()),
- Matchers.<UncommittedBundle<?>>equalTo(outputBundle));
- CommittedBundle<Long> committed = outputBundle.commit(Instant.now());
-
- BoundedWindow w1 = new IntervalWindow(EPOCH, EPOCH.plus(windowDuration));
- BoundedWindow w2 =
- new IntervalWindow(EPOCH.plus(slidingBy), EPOCH.plus(slidingBy).plus(windowDuration));
- BoundedWindow wMinus1 = new IntervalWindow(EPOCH.minus(windowDuration), EPOCH);
- BoundedWindow wMinusSlide =
- new IntervalWindow(EPOCH.minus(windowDuration).plus(slidingBy), EPOCH.plus(slidingBy));
-
- WindowedValue<Long> expectedFirst =
- WindowedValue.of(
- first.getValue(),
- first.getTimestamp(),
- ImmutableSet.of(w1, wMinusSlide),
- PaneInfo.NO_FIRING);
- WindowedValue<Long> expectedSecond =
- WindowedValue.of(
- second.getValue(), second.getTimestamp(), ImmutableSet.of(w1, w2), PaneInfo.NO_FIRING);
- WindowedValue<Long> expectedThird =
- WindowedValue.of(
- third.getValue(),
- third.getTimestamp(),
- ImmutableSet.of(wMinus1, wMinusSlide),
- PaneInfo.NO_FIRING);
-
- assertThat(
- committed.getElements(), containsInAnyOrder(expectedFirst, expectedSecond, expectedThird));
- }
-
- private CommittedBundle<Long> createInputBundle() {
- CommittedBundle<Long> inputBundle =
- bundleFactory
- .createRootBundle(input)
- .add(first)
- .add(second)
- .add(third)
- .commit(Instant.now());
- return inputBundle;
- }
-
- private UncommittedBundle<Long> createOutputBundle(
- PCollection<Long> output, CommittedBundle<Long> inputBundle) {
- UncommittedBundle<Long> outputBundle = bundleFactory.createBundle(inputBundle, output);
- when(evaluationContext.createBundle(inputBundle, output)).thenReturn(outputBundle);
- return outputBundle;
- }
-
- private InProcessTransformResult runEvaluator(
- PCollection<Long> windowed,
- CommittedBundle<Long> inputBundle,
- Window.Bound<Long> windowTransform /* Required while Window.Bound is a composite */)
- throws Exception {
- TransformEvaluator<Long> evaluator =
- factory.forApplication(
- AppliedPTransform.of("Window", input, windowed, windowTransform),
- inputBundle,
- evaluationContext);
-
- evaluator.processElement(first);
- evaluator.processElement(second);
- evaluator.processElement(third);
- InProcessTransformResult result = evaluator.finishBundle();
- return result;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/pom.xml b/sdks/java/pom.xml
index a8972c2..54c841e 100644
--- a/sdks/java/pom.xml
+++ b/sdks/java/pom.xml
@@ -56,5 +56,4 @@
</modules>
</profile>
</profiles>
-
</project>
[13/17] incubator-beam git commit: Move InProcessRunner to its own
module
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoInProcessEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoInProcessEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoInProcessEvaluator.java
new file mode 100644
index 0000000..1c51738
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoInProcessEvaluator.java
@@ -0,0 +1,173 @@
+/*
+ * 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.direct.InProcessExecutionContext.InProcessStepContext;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.util.DoFnRunner;
+import org.apache.beam.sdk.util.DoFnRunners;
+import org.apache.beam.sdk.util.DoFnRunners.OutputManager;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+class ParDoInProcessEvaluator<T> implements TransformEvaluator<T> {
+ public static <InputT, OutputT> ParDoInProcessEvaluator<InputT> create(
+ InProcessEvaluationContext evaluationContext,
+ CommittedBundle<InputT> inputBundle,
+ AppliedPTransform<PCollection<InputT>, ?, ?> application,
+ DoFn<InputT, OutputT> fn,
+ List<PCollectionView<?>> sideInputs,
+ TupleTag<OutputT> mainOutputTag,
+ List<TupleTag<?>> sideOutputTags,
+ Map<TupleTag<?>, PCollection<?>> outputs) {
+ InProcessExecutionContext executionContext =
+ evaluationContext.getExecutionContext(application, inputBundle.getKey());
+ String stepName = evaluationContext.getStepName(application);
+ InProcessStepContext stepContext =
+ executionContext.getOrCreateStepContext(stepName, stepName);
+
+ CounterSet counters = evaluationContext.createCounterSet();
+
+ Map<TupleTag<?>, UncommittedBundle<?>> outputBundles = new HashMap<>();
+ for (Map.Entry<TupleTag<?>, PCollection<?>> outputEntry : outputs.entrySet()) {
+ outputBundles.put(
+ outputEntry.getKey(),
+ evaluationContext.createBundle(inputBundle, outputEntry.getValue()));
+ }
+
+ DoFnRunner<InputT, OutputT> runner =
+ DoFnRunners.createDefault(
+ evaluationContext.getPipelineOptions(),
+ SerializableUtils.clone(fn),
+ evaluationContext.createSideInputReader(sideInputs),
+ BundleOutputManager.create(outputBundles),
+ mainOutputTag,
+ sideOutputTags,
+ stepContext,
+ counters.getAddCounterMutator(),
+ application.getInput().getWindowingStrategy());
+
+ try {
+ runner.startBundle();
+ } catch (Exception e) {
+ throw UserCodeException.wrap(e);
+ }
+
+ return new ParDoInProcessEvaluator<>(
+ runner, application, counters, outputBundles.values(), stepContext);
+ }
+
+ ////////////////////////////////////////////////////////////////////////////////////////////////
+
+ private final DoFnRunner<T, ?> fnRunner;
+ private final AppliedPTransform<PCollection<T>, ?, ?> transform;
+ private final CounterSet counters;
+ private final Collection<UncommittedBundle<?>> outputBundles;
+ private final InProcessStepContext stepContext;
+
+ private ParDoInProcessEvaluator(
+ DoFnRunner<T, ?> fnRunner,
+ AppliedPTransform<PCollection<T>, ?, ?> transform,
+ CounterSet counters,
+ Collection<UncommittedBundle<?>> outputBundles,
+ InProcessStepContext stepContext) {
+ this.fnRunner = fnRunner;
+ this.transform = transform;
+ this.counters = counters;
+ this.outputBundles = outputBundles;
+ this.stepContext = stepContext;
+ }
+
+ @Override
+ public void processElement(WindowedValue<T> element) {
+ try {
+ fnRunner.processElement(element);
+ } catch (Exception e) {
+ throw UserCodeException.wrap(e);
+ }
+ }
+
+ @Override
+ public InProcessTransformResult finishBundle() {
+ try {
+ fnRunner.finishBundle();
+ } catch (Exception e) {
+ throw UserCodeException.wrap(e);
+ }
+ StepTransformResult.Builder resultBuilder;
+ CopyOnAccessInMemoryStateInternals<?> state = stepContext.commitState();
+ if (state != null) {
+ resultBuilder =
+ StepTransformResult.withHold(transform, state.getEarliestWatermarkHold())
+ .withState(state);
+ } else {
+ resultBuilder = StepTransformResult.withoutHold(transform);
+ }
+ return resultBuilder
+ .addOutput(outputBundles)
+ .withTimerUpdate(stepContext.getTimerUpdate())
+ .withCounters(counters)
+ .build();
+ }
+
+ static class BundleOutputManager implements OutputManager {
+ private final Map<TupleTag<?>, UncommittedBundle<?>> bundles;
+ private final Map<TupleTag<?>, List<?>> undeclaredOutputs;
+
+ public static BundleOutputManager create(Map<TupleTag<?>, UncommittedBundle<?>> outputBundles) {
+ return new BundleOutputManager(outputBundles);
+ }
+
+ private BundleOutputManager(Map<TupleTag<?>, UncommittedBundle<?>> bundles) {
+ this.bundles = bundles;
+ undeclaredOutputs = new HashMap<>();
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
+ @SuppressWarnings("rawtypes")
+ UncommittedBundle bundle = bundles.get(tag);
+ if (bundle == null) {
+ List undeclaredContents = undeclaredOutputs.get(tag);
+ if (undeclaredContents == null) {
+ undeclaredContents = new ArrayList<T>();
+ undeclaredOutputs.put(tag, undeclaredContents);
+ }
+ undeclaredContents.add(output);
+ } else {
+ bundle.add(output);
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
new file mode 100644
index 0000000..ae8ac6f
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
@@ -0,0 +1,64 @@
+/*
+ * 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.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo.BoundMulti;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+
+import java.util.Map;
+
+/**
+ * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
+ * {@link BoundMulti} primitive {@link PTransform}.
+ */
+class ParDoMultiEvaluatorFactory implements TransformEvaluatorFactory {
+ @Override
+ public <T> TransformEvaluator<T> forApplication(
+ AppliedPTransform<?, ?, ?> application,
+ CommittedBundle<?> inputBundle,
+ InProcessEvaluationContext evaluationContext) {
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ TransformEvaluator<T> evaluator =
+ createMultiEvaluator((AppliedPTransform) application, inputBundle, evaluationContext);
+ return evaluator;
+ }
+
+ private static <InT, OuT> ParDoInProcessEvaluator<InT> createMultiEvaluator(
+ AppliedPTransform<PCollection<InT>, PCollectionTuple, BoundMulti<InT, OuT>> application,
+ CommittedBundle<InT> inputBundle,
+ InProcessEvaluationContext evaluationContext) {
+ Map<TupleTag<?>, PCollection<?>> outputs = application.getOutput().getAll();
+ DoFn<InT, OuT> fn = application.getTransform().getFn();
+
+ return ParDoInProcessEvaluator.create(
+ evaluationContext,
+ inputBundle,
+ application,
+ fn,
+ application.getTransform().getSideInputs(),
+ application.getTransform().getMainOutputTag(),
+ application.getTransform().getSideOutputTags().getAll(),
+ outputs);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
new file mode 100644
index 0000000..989ae51
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo.Bound;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TupleTag;
+
+import com.google.common.collect.ImmutableMap;
+
+import java.util.Collections;
+
+/**
+ * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
+ * {@link Bound ParDo.Bound} primitive {@link PTransform}.
+ */
+class ParDoSingleEvaluatorFactory implements TransformEvaluatorFactory {
+ @Override
+ public <T> TransformEvaluator<T> forApplication(
+ final AppliedPTransform<?, ?, ?> application,
+ CommittedBundle<?> inputBundle,
+ InProcessEvaluationContext evaluationContext) {
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ TransformEvaluator<T> evaluator =
+ createSingleEvaluator((AppliedPTransform) application, inputBundle, evaluationContext);
+ return evaluator;
+ }
+
+ private static <InputT, OutputT> ParDoInProcessEvaluator<InputT> createSingleEvaluator(
+ @SuppressWarnings("rawtypes") AppliedPTransform<PCollection<InputT>, PCollection<OutputT>,
+ Bound<InputT, OutputT>> application,
+ CommittedBundle<InputT> inputBundle, InProcessEvaluationContext evaluationContext) {
+ TupleTag<OutputT> mainOutputTag = new TupleTag<>("out");
+
+ return ParDoInProcessEvaluator.create(
+ evaluationContext,
+ inputBundle,
+ application,
+ application.getTransform().getFn(),
+ application.getTransform().getSideInputs(),
+ mainOutputTag,
+ Collections.<TupleTag<?>>emptyList(),
+ ImmutableMap.<TupleTag<?>, PCollection<?>>of(mainOutputTag, application.getOutput()));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java
new file mode 100644
index 0000000..aef62b2
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java
@@ -0,0 +1,49 @@
+/*
+ * 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.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.util.WindowedValue;
+
+class PassthroughTransformEvaluator<InputT> implements TransformEvaluator<InputT> {
+ public static <InputT> PassthroughTransformEvaluator<InputT> create(
+ AppliedPTransform<?, ?, ?> transform, UncommittedBundle<InputT> output) {
+ return new PassthroughTransformEvaluator<>(transform, output);
+ }
+
+ private final AppliedPTransform<?, ?, ?> transform;
+ private final UncommittedBundle<InputT> output;
+
+ private PassthroughTransformEvaluator(
+ AppliedPTransform<?, ?, ?> transform, UncommittedBundle<InputT> output) {
+ this.transform = transform;
+ this.output = output;
+ }
+
+ @Override
+ public void processElement(WindowedValue<InputT> element) throws Exception {
+ output.add(element);
+ }
+
+ @Override
+ public InProcessTransformResult finishBundle() throws Exception {
+ return StepTransformResult.withoutHold(transform).addOutput(output).build();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ShardControlledWrite.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ShardControlledWrite.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ShardControlledWrite.java
new file mode 100644
index 0000000..4687f85
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ShardControlledWrite.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.Partition;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PDone;
+
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ * A write that explicitly controls its number of output shards.
+ */
+abstract class ShardControlledWrite<InputT>
+ extends ForwardingPTransform<PCollection<InputT>, PDone> {
+ @Override
+ public PDone apply(PCollection<InputT> input) {
+ int numShards = getNumShards();
+ checkArgument(
+ numShards >= 1,
+ "%s should only be applied if the output has a controlled number of shards (> 1); got %s",
+ getClass().getSimpleName(),
+ getNumShards());
+ PCollectionList<InputT> shards =
+ input.apply(
+ "PartitionInto" + numShards + "Shards",
+ Partition.of(getNumShards(), new RandomSeedPartitionFn<InputT>()));
+ for (int i = 0; i < shards.size(); i++) {
+ PCollection<InputT> shard = shards.get(i);
+ PTransform<? super PCollection<InputT>, PDone> writeShard = getSingleShardTransform(i);
+ shard.apply(String.format("%s(Shard:%s)", writeShard.getName(), i), writeShard);
+ }
+ return PDone.in(input.getPipeline());
+ }
+
+ /**
+ * Returns the number of shards this {@link PTransform} should write to.
+ */
+ abstract int getNumShards();
+
+ /**
+ * Returns a {@link PTransform} that performs a write to the shard with the specified shard
+ * number.
+ *
+ * <p>This method will be called n times, where n is the value of {@link #getNumShards()}, for
+ * shard numbers {@code [0...n)}.
+ */
+ abstract PTransform<? super PCollection<InputT>, PDone> getSingleShardTransform(int shardNum);
+
+ private static class RandomSeedPartitionFn<T> implements Partition.PartitionFn<T> {
+ int nextPartition = -1;
+ @Override
+ public int partitionFor(T elem, int numPartitions) {
+ if (nextPartition < 0) {
+ nextPartition = ThreadLocalRandom.current().nextInt(numPartitions);
+ }
+ nextPartition++;
+ nextPartition %= numPartitions;
+ return nextPartition;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepAndKey.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepAndKey.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepAndKey.java
new file mode 100644
index 0000000..1c7cf6c
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepAndKey.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+
+import com.google.common.base.MoreObjects;
+
+import java.util.Objects;
+
+/**
+ * A (Step, Key) pair. This is useful as a map key or cache key for things that are available
+ * per-step in a keyed manner (e.g. State).
+ */
+final class StepAndKey {
+ private final AppliedPTransform<?, ?, ?> step;
+ private final Object key;
+
+ /**
+ * Create a new {@link StepAndKey} with the provided step and key.
+ */
+ public static StepAndKey of(AppliedPTransform<?, ?, ?> step, Object key) {
+ return new StepAndKey(step, key);
+ }
+
+ private StepAndKey(AppliedPTransform<?, ?, ?> step, Object key) {
+ this.step = step;
+ this.key = key;
+ }
+
+ @Override
+ public String toString() {
+ return MoreObjects.toStringHelper(StepAndKey.class)
+ .add("step", step.getFullName())
+ .add("key", key)
+ .toString();
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(step, key);
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (other == this) {
+ return true;
+ } else if (!(other instanceof StepAndKey)) {
+ return false;
+ } else {
+ StepAndKey that = (StepAndKey) other;
+ return Objects.equals(this.step, that.step)
+ && Objects.equals(this.key, that.key);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java
new file mode 100644
index 0000000..46e7d04
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java
@@ -0,0 +1,165 @@
+/*
+ * 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 com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
+
+import com.google.common.base.MoreObjects;
+import com.google.common.collect.ImmutableList;
+
+import org.joda.time.Instant;
+
+import java.util.Collection;
+
+import javax.annotation.Nullable;
+
+/**
+ * An immutable {@link InProcessTransformResult}.
+ */
+public class StepTransformResult implements InProcessTransformResult {
+ private final AppliedPTransform<?, ?, ?> transform;
+ private final Iterable<? extends UncommittedBundle<?>> bundles;
+ @Nullable private final CopyOnAccessInMemoryStateInternals<?> state;
+ private final TimerUpdate timerUpdate;
+ @Nullable private final CounterSet counters;
+ private final Instant watermarkHold;
+
+ private StepTransformResult(
+ AppliedPTransform<?, ?, ?> transform,
+ Iterable<? extends UncommittedBundle<?>> outputBundles,
+ CopyOnAccessInMemoryStateInternals<?> state,
+ TimerUpdate timerUpdate,
+ CounterSet counters,
+ Instant watermarkHold) {
+ this.transform = checkNotNull(transform);
+ this.bundles = checkNotNull(outputBundles);
+ this.state = state;
+ this.timerUpdate = checkNotNull(timerUpdate);
+ this.counters = counters;
+ this.watermarkHold = checkNotNull(watermarkHold);
+ }
+
+ @Override
+ public Iterable<? extends UncommittedBundle<?>> getOutputBundles() {
+ return bundles;
+ }
+
+ @Override
+ public CounterSet getCounters() {
+ return counters;
+ }
+
+ @Override
+ public AppliedPTransform<?, ?, ?> getTransform() {
+ return transform;
+ }
+
+ @Override
+ public Instant getWatermarkHold() {
+ return watermarkHold;
+ }
+
+ @Nullable
+ @Override
+ public CopyOnAccessInMemoryStateInternals<?> getState() {
+ return state;
+ }
+
+ @Override
+ public TimerUpdate getTimerUpdate() {
+ return timerUpdate;
+ }
+
+ public static Builder withHold(AppliedPTransform<?, ?, ?> transform, Instant watermarkHold) {
+ return new Builder(transform, watermarkHold);
+ }
+
+ public static Builder withoutHold(AppliedPTransform<?, ?, ?> transform) {
+ return new Builder(transform, BoundedWindow.TIMESTAMP_MAX_VALUE);
+ }
+
+ @Override
+ public String toString() {
+ return MoreObjects.toStringHelper(StepTransformResult.class)
+ .add("transform", transform)
+ .toString();
+ }
+
+ /**
+ * A builder for creating instances of {@link StepTransformResult}.
+ */
+ public static class Builder {
+ private final AppliedPTransform<?, ?, ?> transform;
+ private final ImmutableList.Builder<UncommittedBundle<?>> bundlesBuilder;
+ private CopyOnAccessInMemoryStateInternals<?> state;
+ private TimerUpdate timerUpdate;
+ private CounterSet counters;
+ private final Instant watermarkHold;
+
+ private Builder(AppliedPTransform<?, ?, ?> transform, Instant watermarkHold) {
+ this.transform = transform;
+ this.watermarkHold = watermarkHold;
+ this.bundlesBuilder = ImmutableList.builder();
+ this.timerUpdate = TimerUpdate.builder(null).build();
+ }
+
+ public StepTransformResult build() {
+ return new StepTransformResult(
+ transform,
+ bundlesBuilder.build(),
+ state,
+ timerUpdate,
+ counters,
+ watermarkHold);
+ }
+
+ public Builder withCounters(CounterSet counters) {
+ this.counters = counters;
+ return this;
+ }
+
+ public Builder withState(CopyOnAccessInMemoryStateInternals<?> state) {
+ this.state = state;
+ return this;
+ }
+
+ public Builder withTimerUpdate(TimerUpdate timerUpdate) {
+ this.timerUpdate = timerUpdate;
+ return this;
+ }
+
+ public Builder addOutput(
+ UncommittedBundle<?> outputBundle, UncommittedBundle<?>... outputBundles) {
+ bundlesBuilder.add(outputBundle);
+ bundlesBuilder.add(outputBundles);
+ return this;
+ }
+
+ public Builder addOutput(Collection<UncommittedBundle<?>> outputBundles) {
+ bundlesBuilder.addAll(outputBundles);
+ return this;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TextIOShardedWriteFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TextIOShardedWriteFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TextIOShardedWriteFactory.java
new file mode 100644
index 0000000..be1bf18
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TextIOShardedWriteFactory.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.TextIO.Write.Bound;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.IOChannelUtils;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
+class TextIOShardedWriteFactory implements PTransformOverrideFactory {
+
+ @Override
+ public <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
+ PTransform<InputT, OutputT> transform) {
+ if (transform instanceof TextIO.Write.Bound) {
+ @SuppressWarnings("unchecked")
+ TextIO.Write.Bound<InputT> originalWrite = (TextIO.Write.Bound<InputT>) transform;
+ if (originalWrite.getNumShards() > 1
+ || (originalWrite.getNumShards() == 1
+ && !"".equals(originalWrite.getShardNameTemplate()))) {
+ @SuppressWarnings("unchecked")
+ PTransform<InputT, OutputT> override =
+ (PTransform<InputT, OutputT>) new TextIOShardedWrite<InputT>(originalWrite);
+ return override;
+ }
+ }
+ return transform;
+ }
+
+ private static class TextIOShardedWrite<InputT> extends ShardControlledWrite<InputT> {
+ private final TextIO.Write.Bound<InputT> initial;
+
+ private TextIOShardedWrite(Bound<InputT> initial) {
+ this.initial = initial;
+ }
+
+ @Override
+ int getNumShards() {
+ return initial.getNumShards();
+ }
+
+ @Override
+ PTransform<PCollection<InputT>, PDone> getSingleShardTransform(int shardNum) {
+ String shardName =
+ IOChannelUtils.constructName(
+ initial.getFilenamePrefix(),
+ initial.getShardTemplate(),
+ initial.getFilenameSuffix(),
+ shardNum,
+ getNumShards());
+ return TextIO.Write.withCoder(initial.getCoder()).to(shardName).withoutSharding();
+ }
+
+ @Override
+ protected PTransform<PCollection<InputT>, PDone> delegate() {
+ return initial;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluator.java
new file mode 100644
index 0000000..ba9815b
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluator.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 org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.util.WindowedValue;
+
+/**
+ * An evaluator of a specific application of a transform. Will be used for at least one
+ * {@link CommittedBundle}.
+ *
+ * @param <InputT> the type of elements that will be passed to {@link #processElement}
+ */
+public interface TransformEvaluator<InputT> {
+ /**
+ * Process an element in the input {@link CommittedBundle}.
+ *
+ * @param element the element to process
+ */
+ void processElement(WindowedValue<InputT> element) throws Exception;
+
+ /**
+ * Finish processing the bundle of this {@link TransformEvaluator}.
+ *
+ * After {@link #finishBundle()} is called, the {@link TransformEvaluator} will not be reused,
+ * and no more elements will be processed.
+ *
+ * @return an {@link InProcessTransformResult} containing the results of this bundle evaluation.
+ */
+ InProcessTransformResult finishBundle() throws Exception;
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java
new file mode 100644
index 0000000..8f8d84c
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+
+import javax.annotation.Nullable;
+
+/**
+ * A factory for creating instances of {@link TransformEvaluator} for the application of a
+ * {@link PTransform}.
+ */
+public interface TransformEvaluatorFactory {
+ /**
+ * Create a new {@link TransformEvaluator} for the application of the {@link PTransform}.
+ *
+ * Any work that must be done before input elements are processed (such as calling
+ * {@link DoFn#startBundle(DoFn.Context)}) must be done before the {@link TransformEvaluator} is
+ * made available to the caller.
+ *
+ * @throws Exception whenever constructing the underlying evaluator throws an exception
+ */
+ <InputT> TransformEvaluator<InputT> forApplication(
+ AppliedPTransform<?, ?, ?> application, @Nullable CommittedBundle<?> inputBundle,
+ InProcessEvaluationContext evaluationContext) throws Exception;
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..f449731
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Flatten.FlattenPCollectionList;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.Window;
+
+import com.google.common.collect.ImmutableMap;
+
+import java.util.Map;
+
+import javax.annotation.Nullable;
+
+/**
+ * A {@link TransformEvaluatorFactory} that delegates to primitive {@link TransformEvaluatorFactory}
+ * implementations based on the type of {@link PTransform} of the application.
+ */
+class TransformEvaluatorRegistry implements TransformEvaluatorFactory {
+ public static TransformEvaluatorRegistry defaultRegistry() {
+ @SuppressWarnings("rawtypes")
+ ImmutableMap<Class<? extends PTransform>, TransformEvaluatorFactory> primitives =
+ ImmutableMap.<Class<? extends PTransform>, TransformEvaluatorFactory>builder()
+ .put(Read.Bounded.class, new BoundedReadEvaluatorFactory())
+ .put(Read.Unbounded.class, new UnboundedReadEvaluatorFactory())
+ .put(ParDo.Bound.class, new ParDoSingleEvaluatorFactory())
+ .put(ParDo.BoundMulti.class, new ParDoMultiEvaluatorFactory())
+ .put(
+ GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly.class,
+ new GroupByKeyEvaluatorFactory())
+ .put(FlattenPCollectionList.class, new FlattenEvaluatorFactory())
+ .put(ViewEvaluatorFactory.WriteView.class, new ViewEvaluatorFactory())
+ .put(Window.Bound.class, new WindowEvaluatorFactory())
+ .build();
+ return new TransformEvaluatorRegistry(primitives);
+ }
+
+ // the TransformEvaluatorFactories can construct instances of all generic types of transform,
+ // so all instances of a primitive can be handled with the same evaluator factory.
+ @SuppressWarnings("rawtypes")
+ private final Map<Class<? extends PTransform>, TransformEvaluatorFactory> factories;
+
+ private TransformEvaluatorRegistry(
+ @SuppressWarnings("rawtypes")
+ Map<Class<? extends PTransform>, TransformEvaluatorFactory> factories) {
+ this.factories = factories;
+ }
+
+ @Override
+ public <InputT> TransformEvaluator<InputT> forApplication(
+ AppliedPTransform<?, ?, ?> application,
+ @Nullable CommittedBundle<?> inputBundle,
+ InProcessEvaluationContext evaluationContext)
+ throws Exception {
+ TransformEvaluatorFactory factory = factories.get(application.getTransform().getClass());
+ return factory.forApplication(application, inputBundle, evaluationContext);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java
new file mode 100644
index 0000000..8346e89
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.util.WindowedValue;
+
+import com.google.common.base.Throwables;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicReference;
+
+import javax.annotation.Nullable;
+
+/**
+ * A {@link Callable} responsible for constructing a {@link TransformEvaluator} from a
+ * {@link TransformEvaluatorFactory} and evaluating it on some bundle of input, and registering
+ * the result using a registered {@link CompletionCallback}.
+ *
+ * <p>A {@link TransformExecutor} that is currently executing also provides access to the thread
+ * that it is being executed on.
+ */
+class TransformExecutor<T> implements Callable<InProcessTransformResult> {
+ public static <T> TransformExecutor<T> create(
+ TransformEvaluatorFactory factory,
+ Iterable<? extends ModelEnforcementFactory> modelEnforcements,
+ InProcessEvaluationContext evaluationContext,
+ CommittedBundle<T> inputBundle,
+ AppliedPTransform<?, ?, ?> transform,
+ CompletionCallback completionCallback,
+ TransformExecutorService transformEvaluationState) {
+ return new TransformExecutor<>(
+ factory,
+ modelEnforcements,
+ evaluationContext,
+ inputBundle,
+ transform,
+ completionCallback,
+ transformEvaluationState);
+ }
+
+ private final TransformEvaluatorFactory evaluatorFactory;
+ private final Iterable<? extends ModelEnforcementFactory> modelEnforcements;
+
+ private final InProcessEvaluationContext evaluationContext;
+
+ /** The transform that will be evaluated. */
+ private final AppliedPTransform<?, ?, ?> transform;
+ /** The inputs this {@link TransformExecutor} will deliver to the transform. */
+ private final CommittedBundle<T> inputBundle;
+
+ private final CompletionCallback onComplete;
+ private final TransformExecutorService transformEvaluationState;
+
+ private final AtomicReference<Thread> thread;
+
+ private TransformExecutor(
+ TransformEvaluatorFactory factory,
+ Iterable<? extends ModelEnforcementFactory> modelEnforcements,
+ InProcessEvaluationContext evaluationContext,
+ CommittedBundle<T> inputBundle,
+ AppliedPTransform<?, ?, ?> transform,
+ CompletionCallback completionCallback,
+ TransformExecutorService transformEvaluationState) {
+ this.evaluatorFactory = factory;
+ this.modelEnforcements = modelEnforcements;
+ this.evaluationContext = evaluationContext;
+
+ this.inputBundle = inputBundle;
+ this.transform = transform;
+
+ this.onComplete = completionCallback;
+
+ this.transformEvaluationState = transformEvaluationState;
+ this.thread = new AtomicReference<>();
+ }
+
+ @Override
+ public InProcessTransformResult call() {
+ checkState(
+ thread.compareAndSet(null, Thread.currentThread()),
+ "Tried to execute %s for %s on thread %s, but is already executing on thread %s",
+ TransformExecutor.class.getSimpleName(),
+ transform.getFullName(),
+ Thread.currentThread(),
+ thread.get());
+ try {
+ Collection<ModelEnforcement<T>> enforcements = new ArrayList<>();
+ for (ModelEnforcementFactory enforcementFactory : modelEnforcements) {
+ ModelEnforcement<T> enforcement = enforcementFactory.forBundle(inputBundle, transform);
+ enforcements.add(enforcement);
+ }
+ TransformEvaluator<T> evaluator =
+ evaluatorFactory.forApplication(transform, inputBundle, evaluationContext);
+
+ processElements(evaluator, enforcements);
+
+ InProcessTransformResult result = finishBundle(evaluator, enforcements);
+ return result;
+ } catch (Throwable t) {
+ onComplete.handleThrowable(inputBundle, t);
+ throw Throwables.propagate(t);
+ } finally {
+ transformEvaluationState.complete(this);
+ }
+ }
+
+ /**
+ * Processes all the elements in the input bundle using the transform evaluator, applying any
+ * necessary {@link ModelEnforcement ModelEnforcements}.
+ */
+ private void processElements(
+ TransformEvaluator<T> evaluator, Collection<ModelEnforcement<T>> enforcements)
+ throws Exception {
+ if (inputBundle != null) {
+ for (WindowedValue<T> value : inputBundle.getElements()) {
+ for (ModelEnforcement<T> enforcement : enforcements) {
+ enforcement.beforeElement(value);
+ }
+
+ evaluator.processElement(value);
+
+ for (ModelEnforcement<T> enforcement : enforcements) {
+ enforcement.afterElement(value);
+ }
+ }
+ }
+ }
+
+ /**
+ * Finishes processing the input bundle and commit the result using the
+ * {@link CompletionCallback}, applying any {@link ModelEnforcement} if necessary.
+ *
+ * @return the {@link InProcessTransformResult} produced by
+ * {@link TransformEvaluator#finishBundle()}
+ */
+ private InProcessTransformResult finishBundle(
+ TransformEvaluator<T> evaluator, Collection<ModelEnforcement<T>> enforcements)
+ throws Exception {
+ InProcessTransformResult result = evaluator.finishBundle();
+ CommittedResult outputs = onComplete.handleResult(inputBundle, result);
+ for (ModelEnforcement<T> enforcement : enforcements) {
+ enforcement.afterFinish(inputBundle, result, outputs.getOutputs());
+ }
+ return result;
+ }
+
+ /**
+ * If this {@link TransformExecutor} is currently executing, return the thread it is executing in.
+ * Otherwise, return null.
+ */
+ @Nullable
+ public Thread getThread() {
+ return thread.get();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorService.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorService.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorService.java
new file mode 100644
index 0000000..837b858
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorService.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+/**
+ * Schedules and completes {@link TransformExecutor TransformExecutors}, controlling concurrency as
+ * appropriate for the {@link StepAndKey} the executor exists for.
+ */
+interface TransformExecutorService {
+ /**
+ * Schedule the provided work to be eventually executed.
+ */
+ void schedule(TransformExecutor<?> work);
+
+ /**
+ * Finish executing the provided work. This may cause additional
+ * {@link TransformExecutor TransformExecutors} to be evaluated.
+ */
+ void complete(TransformExecutor<?> completed);
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorServices.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorServices.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorServices.java
new file mode 100644
index 0000000..087b7c2
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorServices.java
@@ -0,0 +1,154 @@
+/*
+ * 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 com.google.common.base.MoreObjects;
+
+import java.util.Map;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Static factory methods for constructing instances of {@link TransformExecutorService}.
+ */
+final class TransformExecutorServices {
+ private TransformExecutorServices() {
+ // Do not instantiate
+ }
+
+ /**
+ * Returns an EvaluationState that evaluates {@link TransformExecutor TransformExecutors} in
+ * parallel.
+ */
+ public static TransformExecutorService parallel(
+ ExecutorService executor, Map<TransformExecutor<?>, Boolean> scheduled) {
+ return new ParallelEvaluationState(executor, scheduled);
+ }
+
+ /**
+ * Returns an EvaluationState that evaluates {@link TransformExecutor TransformExecutors} in
+ * serial.
+ */
+ public static TransformExecutorService serial(
+ ExecutorService executor, Map<TransformExecutor<?>, Boolean> scheduled) {
+ return new SerialEvaluationState(executor, scheduled);
+ }
+
+ /**
+ * A {@link TransformExecutorService} with unlimited parallelism. Any {@link TransformExecutor}
+ * scheduled will be immediately submitted to the {@link ExecutorService}.
+ *
+ * <p>A principal use of this is for the evaluation of an unkeyed Step. Unkeyed computations are
+ * processed in parallel.
+ */
+ private static class ParallelEvaluationState implements TransformExecutorService {
+ private final ExecutorService executor;
+ private final Map<TransformExecutor<?>, Boolean> scheduled;
+
+ private ParallelEvaluationState(
+ ExecutorService executor, Map<TransformExecutor<?>, Boolean> scheduled) {
+ this.executor = executor;
+ this.scheduled = scheduled;
+ }
+
+ @Override
+ public void schedule(TransformExecutor<?> work) {
+ executor.submit(work);
+ scheduled.put(work, true);
+ }
+
+ @Override
+ public void complete(TransformExecutor<?> completed) {
+ scheduled.remove(completed);
+ }
+ }
+
+ /**
+ * A {@link TransformExecutorService} with a single work queue. Any {@link TransformExecutor}
+ * scheduled will be placed on the work queue. Only one item of work will be submitted to the
+ * {@link ExecutorService} at any time.
+ *
+ * <p>A principal use of this is for the serial evaluation of a (Step, Key) pair.
+ * Keyed computations are processed serially per step.
+ */
+ private static class SerialEvaluationState implements TransformExecutorService {
+ private final ExecutorService executor;
+ private final Map<TransformExecutor<?>, Boolean> scheduled;
+
+ private AtomicReference<TransformExecutor<?>> currentlyEvaluating;
+ private final Queue<TransformExecutor<?>> workQueue;
+
+ private SerialEvaluationState(
+ ExecutorService executor, Map<TransformExecutor<?>, Boolean> scheduled) {
+ this.scheduled = scheduled;
+ this.executor = executor;
+ this.currentlyEvaluating = new AtomicReference<>();
+ this.workQueue = new ConcurrentLinkedQueue<>();
+ }
+
+ /**
+ * Schedules the work, adding it to the work queue if there is a bundle currently being
+ * evaluated and scheduling it immediately otherwise.
+ */
+ @Override
+ public void schedule(TransformExecutor<?> work) {
+ workQueue.offer(work);
+ updateCurrentlyEvaluating();
+ }
+
+ @Override
+ public void complete(TransformExecutor<?> completed) {
+ if (!currentlyEvaluating.compareAndSet(completed, null)) {
+ throw new IllegalStateException(
+ "Finished work "
+ + completed
+ + " but could not complete due to unexpected currently executing "
+ + currentlyEvaluating.get());
+ }
+ scheduled.remove(completed);
+ updateCurrentlyEvaluating();
+ }
+
+ private void updateCurrentlyEvaluating() {
+ if (currentlyEvaluating.get() == null) {
+ // Only synchronize if we need to update what's currently evaluating
+ synchronized (this) {
+ TransformExecutor<?> newWork = workQueue.poll();
+ if (newWork != null) {
+ if (currentlyEvaluating.compareAndSet(null, newWork)) {
+ scheduled.put(newWork, true);
+ executor.submit(newWork);
+ } else {
+ workQueue.offer(newWork);
+ }
+ }
+ }
+ }
+ }
+
+ @Override
+ public String toString() {
+ return MoreObjects.toStringHelper(SerialEvaluationState.class)
+ .add("currentlyEvaluating", currentlyEvaluating)
+ .add("workQueue", workQueue)
+ .toString();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java
new file mode 100644
index 0000000..7a95c9f
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java
@@ -0,0 +1,177 @@
+/*
+ * 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.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.io.Read.Unbounded;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
+import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+import java.io.IOException;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ConcurrentMap;
+
+import javax.annotation.Nullable;
+
+/**
+ * A {@link TransformEvaluatorFactory} that produces {@link TransformEvaluator TransformEvaluators}
+ * for the {@link Unbounded Read.Unbounded} primitive {@link PTransform}.
+ */
+class UnboundedReadEvaluatorFactory implements TransformEvaluatorFactory {
+ /*
+ * An evaluator for a Source is stateful, to ensure the CheckpointMark is properly persisted.
+ * Evaluators are cached here to ensure that the checkpoint mark is appropriately reused
+ * and any splits are honored.
+ */
+ private final ConcurrentMap<EvaluatorKey, Queue<? extends UnboundedReadEvaluator<?>>>
+ sourceEvaluators = new ConcurrentHashMap<>();
+
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ @Override
+ public <InputT> TransformEvaluator<InputT> forApplication(AppliedPTransform<?, ?, ?> application,
+ @Nullable CommittedBundle<?> inputBundle, InProcessEvaluationContext evaluationContext) {
+ return getTransformEvaluator((AppliedPTransform) application, evaluationContext);
+ }
+
+ private <OutputT> TransformEvaluator<?> getTransformEvaluator(
+ final AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform,
+ final InProcessEvaluationContext evaluationContext) {
+ UnboundedReadEvaluator<?> currentEvaluator =
+ getTransformEvaluatorQueue(transform, evaluationContext).poll();
+ if (currentEvaluator == null) {
+ return EmptyTransformEvaluator.create(transform);
+ }
+ return currentEvaluator;
+ }
+
+ /**
+ * Get the queue of {@link TransformEvaluator TransformEvaluators} that produce elements for the
+ * provided application of {@link Unbounded Read.Unbounded}, initializing it if required.
+ *
+ * <p>This method is thread-safe, and will only produce new evaluators if no other invocation has
+ * already done so.
+ */
+ @SuppressWarnings("unchecked")
+ private <OutputT> Queue<UnboundedReadEvaluator<OutputT>> getTransformEvaluatorQueue(
+ final AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform,
+ final InProcessEvaluationContext evaluationContext) {
+ // Key by the application and the context the evaluation is occurring in (which call to
+ // Pipeline#run).
+ EvaluatorKey key = new EvaluatorKey(transform, evaluationContext);
+ @SuppressWarnings("unchecked")
+ Queue<UnboundedReadEvaluator<OutputT>> evaluatorQueue =
+ (Queue<UnboundedReadEvaluator<OutputT>>) sourceEvaluators.get(key);
+ if (evaluatorQueue == null) {
+ evaluatorQueue = new ConcurrentLinkedQueue<>();
+ if (sourceEvaluators.putIfAbsent(key, evaluatorQueue) == null) {
+ // If no queue existed in the evaluators, add an evaluator to initialize the evaluator
+ // factory for this transform
+ UnboundedSource<OutputT, ?> source = transform.getTransform().getSource();
+ UnboundedReadEvaluator<OutputT> evaluator =
+ new UnboundedReadEvaluator<OutputT>(
+ transform, evaluationContext, source, evaluatorQueue);
+ evaluatorQueue.offer(evaluator);
+ } else {
+ // otherwise return the existing Queue that arrived before us
+ evaluatorQueue = (Queue<UnboundedReadEvaluator<OutputT>>) sourceEvaluators.get(key);
+ }
+ }
+ return evaluatorQueue;
+ }
+
+ /**
+ * A {@link UnboundedReadEvaluator} produces elements from an underlying {@link UnboundedSource},
+ * discarding all input elements. Within the call to {@link #finishBundle()}, the evaluator
+ * creates the {@link UnboundedReader} and consumes some currently available input.
+ *
+ * <p>Calls to {@link UnboundedReadEvaluator} are not internally thread-safe, and should only be
+ * used by a single thread at a time. Each {@link UnboundedReadEvaluator} maintains its own
+ * checkpoint, and constructs its reader from the current checkpoint in each call to
+ * {@link #finishBundle()}.
+ */
+ private static class UnboundedReadEvaluator<OutputT> implements TransformEvaluator<Object> {
+ private static final int ARBITRARY_MAX_ELEMENTS = 10;
+ private final AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform;
+ private final InProcessEvaluationContext evaluationContext;
+ private final Queue<UnboundedReadEvaluator<OutputT>> evaluatorQueue;
+ /**
+ * The source being read from by this {@link UnboundedReadEvaluator}. This may not be the same
+ * source as derived from {@link #transform} due to splitting.
+ */
+ private final UnboundedSource<OutputT, ?> source;
+ private CheckpointMark checkpointMark;
+
+ public UnboundedReadEvaluator(
+ AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform,
+ InProcessEvaluationContext evaluationContext,
+ UnboundedSource<OutputT, ?> source,
+ Queue<UnboundedReadEvaluator<OutputT>> evaluatorQueue) {
+ this.transform = transform;
+ this.evaluationContext = evaluationContext;
+ this.evaluatorQueue = evaluatorQueue;
+ this.source = source;
+ this.checkpointMark = null;
+ }
+
+ @Override
+ public void processElement(WindowedValue<Object> element) {}
+
+ @Override
+ public InProcessTransformResult finishBundle() throws IOException {
+ UncommittedBundle<OutputT> output = evaluationContext.createRootBundle(transform.getOutput());
+ try (UnboundedReader<OutputT> reader =
+ createReader(source, evaluationContext.getPipelineOptions());) {
+ int numElements = 0;
+ if (reader.start()) {
+ do {
+ output.add(
+ WindowedValue.timestampedValueInGlobalWindow(
+ reader.getCurrent(), reader.getCurrentTimestamp()));
+ numElements++;
+ } while (numElements < ARBITRARY_MAX_ELEMENTS && reader.advance());
+ }
+ checkpointMark = reader.getCheckpointMark();
+ checkpointMark.finalizeCheckpoint();
+ // TODO: When exercising create initial splits, make this the minimum watermark across all
+ // existing readers
+ StepTransformResult result =
+ StepTransformResult.withHold(transform, reader.getWatermark())
+ .addOutput(output)
+ .build();
+ evaluatorQueue.offer(this);
+ return result;
+ }
+ }
+
+ private <CheckpointMarkT extends CheckpointMark> UnboundedReader<OutputT> createReader(
+ UnboundedSource<OutputT, CheckpointMarkT> source, PipelineOptions options) {
+ @SuppressWarnings("unchecked")
+ CheckpointMarkT mark = (CheckpointMarkT) checkpointMark;
+ return source.createReader(options, mark);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java
new file mode 100644
index 0000000..ffaf3fa
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java
@@ -0,0 +1,145 @@
+/*
+ * 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.direct.InProcessPipelineRunner.PCollectionViewWriter;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.Values;
+import org.apache.beam.sdk.transforms.View.CreatePCollectionView;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
+ * {@link CreatePCollectionView} primitive {@link PTransform}.
+ *
+ * <p>The {@link ViewEvaluatorFactory} produces {@link TransformEvaluator TransformEvaluators} for
+ * the {@link WriteView} {@link PTransform}, which is part of the
+ * {@link InProcessCreatePCollectionView} composite transform. This transform is an override for the
+ * {@link CreatePCollectionView} transform that applies windowing and triggers before the view is
+ * written.
+ */
+class ViewEvaluatorFactory implements TransformEvaluatorFactory {
+ @Override
+ public <T> TransformEvaluator<T> forApplication(
+ AppliedPTransform<?, ?, ?> application,
+ InProcessPipelineRunner.CommittedBundle<?> inputBundle,
+ InProcessEvaluationContext evaluationContext) {
+ @SuppressWarnings({"cast", "unchecked", "rawtypes"})
+ TransformEvaluator<T> evaluator = createEvaluator(
+ (AppliedPTransform) application, evaluationContext);
+ return evaluator;
+ }
+
+ private <InT, OuT> TransformEvaluator<Iterable<InT>> createEvaluator(
+ final AppliedPTransform<PCollection<Iterable<InT>>, PCollectionView<OuT>, WriteView<InT, OuT>>
+ application,
+ InProcessEvaluationContext context) {
+ PCollection<Iterable<InT>> input = application.getInput();
+ final PCollectionViewWriter<InT, OuT> writer =
+ context.createPCollectionViewWriter(input, application.getOutput());
+ return new TransformEvaluator<Iterable<InT>>() {
+ private final List<WindowedValue<InT>> elements = new ArrayList<>();
+
+ @Override
+ public void processElement(WindowedValue<Iterable<InT>> element) {
+ for (InT input : element.getValue()) {
+ elements.add(element.withValue(input));
+ }
+ }
+
+ @Override
+ public InProcessTransformResult finishBundle() {
+ writer.add(elements);
+ return StepTransformResult.withoutHold(application).build();
+ }
+ };
+ }
+
+ public static class InProcessViewOverrideFactory implements PTransformOverrideFactory {
+ @Override
+ public <InputT extends PInput, OutputT extends POutput>
+ PTransform<InputT, OutputT> override(PTransform<InputT, OutputT> transform) {
+ if (transform instanceof CreatePCollectionView) {
+
+ }
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ PTransform<InputT, OutputT> createView =
+ (PTransform<InputT, OutputT>)
+ new InProcessCreatePCollectionView<>((CreatePCollectionView) transform);
+ return createView;
+ }
+ }
+
+ /**
+ * An in-process override for {@link CreatePCollectionView}.
+ */
+ private static class InProcessCreatePCollectionView<ElemT, ViewT>
+ extends ForwardingPTransform<PCollection<ElemT>, PCollectionView<ViewT>> {
+ private final CreatePCollectionView<ElemT, ViewT> og;
+
+ private InProcessCreatePCollectionView(CreatePCollectionView<ElemT, ViewT> og) {
+ this.og = og;
+ }
+
+ @Override
+ public PCollectionView<ViewT> apply(PCollection<ElemT> input) {
+ return input.apply(WithKeys.<Void, ElemT>of((Void) null))
+ .setCoder(KvCoder.of(VoidCoder.of(), input.getCoder()))
+ .apply(GroupByKey.<Void, ElemT>create())
+ .apply(Values.<Iterable<ElemT>>create())
+ .apply(new WriteView<ElemT, ViewT>(og));
+ }
+
+ @Override
+ protected PTransform<PCollection<ElemT>, PCollectionView<ViewT>> delegate() {
+ return og;
+ }
+ }
+
+ /**
+ * An in-process implementation of the {@link CreatePCollectionView} primitive.
+ *
+ * This implementation requires the input {@link PCollection} to be an iterable, which is provided
+ * to {@link PCollectionView#fromIterableInternal(Iterable)}.
+ */
+ public static final class WriteView<ElemT, ViewT>
+ extends PTransform<PCollection<Iterable<ElemT>>, PCollectionView<ViewT>> {
+ private final CreatePCollectionView<ElemT, ViewT> og;
+
+ WriteView(CreatePCollectionView<ElemT, ViewT> og) {
+ this.og = og;
+ }
+
+ @Override
+ public PCollectionView<ViewT> apply(PCollection<Iterable<ElemT>> input) {
+ return og.getView();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java
new file mode 100644
index 0000000..4a3a517
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowingStrategy;
+
+import com.google.common.collect.ComparisonChain;
+import com.google.common.collect.Ordering;
+
+import org.joda.time.Instant;
+
+import java.util.PriorityQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * Executes callbacks that occur based on the progression of the watermark per-step.
+ *
+ * <p>Callbacks are registered by calls to
+ * {@link #callOnGuaranteedFiring(AppliedPTransform, BoundedWindow, WindowingStrategy, Runnable)},
+ * and are executed after a call to {@link #fireForWatermark(AppliedPTransform, Instant)} with the
+ * same {@link AppliedPTransform} and a watermark sufficient to ensure that the trigger for the
+ * windowing strategy would have been produced.
+ *
+ * <p>NOTE: {@link WatermarkCallbackExecutor} does not track the latest observed watermark for any
+ * {@link AppliedPTransform} - any call to
+ * {@link #callOnGuaranteedFiring(AppliedPTransform, BoundedWindow, WindowingStrategy, Runnable)}
+ * that could have potentially already fired should be followed by a call to
+ * {@link #fireForWatermark(AppliedPTransform, Instant)} for the same transform with the current
+ * value of the watermark.
+ */
+class WatermarkCallbackExecutor {
+ /**
+ * Create a new {@link WatermarkCallbackExecutor}.
+ */
+ public static WatermarkCallbackExecutor create() {
+ return new WatermarkCallbackExecutor();
+ }
+
+ private final ConcurrentMap<AppliedPTransform<?, ?, ?>, PriorityQueue<WatermarkCallback>>
+ callbacks;
+ private final ExecutorService executor;
+
+ private WatermarkCallbackExecutor() {
+ this.callbacks = new ConcurrentHashMap<>();
+ this.executor = Executors.newSingleThreadExecutor();
+ }
+
+ /**
+ * Execute the provided {@link Runnable} after the next call to
+ * {@link #fireForWatermark(AppliedPTransform, Instant)} where the window is guaranteed to have
+ * produced output.
+ */
+ public void callOnGuaranteedFiring(
+ AppliedPTransform<?, ?, ?> step,
+ BoundedWindow window,
+ WindowingStrategy<?, ?> windowingStrategy,
+ Runnable runnable) {
+ WatermarkCallback callback =
+ WatermarkCallback.onGuaranteedFiring(window, windowingStrategy, runnable);
+
+ PriorityQueue<WatermarkCallback> callbackQueue = callbacks.get(step);
+ if (callbackQueue == null) {
+ callbackQueue = new PriorityQueue<>(11, new CallbackOrdering());
+ if (callbacks.putIfAbsent(step, callbackQueue) != null) {
+ callbackQueue = callbacks.get(step);
+ }
+ }
+
+ synchronized (callbackQueue) {
+ callbackQueue.offer(callback);
+ }
+ }
+
+ /**
+ * Schedule all pending callbacks that must have produced output by the time of the provided
+ * watermark.
+ */
+ public void fireForWatermark(AppliedPTransform<?, ?, ?> step, Instant watermark) {
+ PriorityQueue<WatermarkCallback> callbackQueue = callbacks.get(step);
+ if (callbackQueue == null) {
+ return;
+ }
+ synchronized (callbackQueue) {
+ while (!callbackQueue.isEmpty() && callbackQueue.peek().shouldFire(watermark)) {
+ executor.submit(callbackQueue.poll().getCallback());
+ }
+ }
+ }
+
+ private static class WatermarkCallback {
+ public static <W extends BoundedWindow> WatermarkCallback onGuaranteedFiring(
+ BoundedWindow window, WindowingStrategy<?, W> strategy, Runnable callback) {
+ @SuppressWarnings("unchecked")
+ Instant firingAfter =
+ strategy.getTrigger().getSpec().getWatermarkThatGuaranteesFiring((W) window);
+ return new WatermarkCallback(firingAfter, callback);
+ }
+
+ private final Instant fireAfter;
+ private final Runnable callback;
+
+ private WatermarkCallback(Instant fireAfter, Runnable callback) {
+ this.fireAfter = fireAfter;
+ this.callback = callback;
+ }
+
+ public boolean shouldFire(Instant currentWatermark) {
+ return currentWatermark.isAfter(fireAfter)
+ || currentWatermark.equals(BoundedWindow.TIMESTAMP_MAX_VALUE);
+ }
+
+ public Runnable getCallback() {
+ return callback;
+ }
+ }
+
+ private static class CallbackOrdering extends Ordering<WatermarkCallback> {
+ @Override
+ public int compare(WatermarkCallback left, WatermarkCallback right) {
+ return ComparisonChain.start()
+ .compare(left.fireAfter, right.fireAfter)
+ .compare(left.callback, right.callback, Ordering.arbitrary())
+ .result();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..628f94d
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java
@@ -0,0 +1,131 @@
+/*
+ * 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.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+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;
+
+import org.joda.time.Instant;
+
+import java.util.Collection;
+
+import javax.annotation.Nullable;
+
+/**
+ * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
+ * {@link Bound Window.Bound} primitive {@link PTransform}.
+ */
+class WindowEvaluatorFactory implements TransformEvaluatorFactory {
+
+ @Override
+ public <InputT> TransformEvaluator<InputT> forApplication(
+ AppliedPTransform<?, ?, ?> application,
+ @Nullable CommittedBundle<?> inputBundle,
+ InProcessEvaluationContext evaluationContext)
+ throws Exception {
+ return createTransformEvaluator(
+ (AppliedPTransform) application, inputBundle, evaluationContext);
+ }
+
+ private <InputT> TransformEvaluator<InputT> createTransformEvaluator(
+ AppliedPTransform<PCollection<InputT>, PCollection<InputT>, Window.Bound<InputT>> transform,
+ CommittedBundle<?> inputBundle,
+ InProcessEvaluationContext evaluationContext) {
+ WindowFn<? super InputT, ?> fn = transform.getTransform().getWindowFn();
+ UncommittedBundle<InputT> outputBundle =
+ evaluationContext.createBundle(inputBundle, transform.getOutput());
+ if (fn == null) {
+ return PassthroughTransformEvaluator.create(transform, outputBundle);
+ }
+ return new WindowIntoEvaluator<>(transform, fn, outputBundle);
+ }
+
+ private static class WindowIntoEvaluator<InputT> implements TransformEvaluator<InputT> {
+ private final AppliedPTransform<PCollection<InputT>, PCollection<InputT>, Window.Bound<InputT>>
+ transform;
+ private final WindowFn<InputT, ?> windowFn;
+ private final UncommittedBundle<InputT> outputBundle;
+
+ @SuppressWarnings("unchecked")
+ public WindowIntoEvaluator(
+ AppliedPTransform<PCollection<InputT>, PCollection<InputT>, Window.Bound<InputT>> transform,
+ WindowFn<? super InputT, ?> windowFn,
+ UncommittedBundle<InputT> outputBundle) {
+ this.outputBundle = outputBundle;
+ this.transform = transform;
+ // Safe contravariant cast
+ this.windowFn = (WindowFn<InputT, ?>) windowFn;
+ }
+
+ @Override
+ public void processElement(WindowedValue<InputT> element) throws Exception {
+ Collection<? extends BoundedWindow> windows = assignWindows(windowFn, element);
+ outputBundle.add(
+ WindowedValue.<InputT>of(
+ element.getValue(), element.getTimestamp(), windows, PaneInfo.NO_FIRING));
+ }
+
+ private <W extends BoundedWindow> Collection<? extends BoundedWindow> assignWindows(
+ WindowFn<InputT, W> windowFn, WindowedValue<InputT> element) throws Exception {
+ WindowFn<InputT, W>.AssignContext assignContext =
+ new InProcessAssignContext<>(windowFn, element);
+ Collection<? extends BoundedWindow> windows = windowFn.assignWindows(assignContext);
+ return windows;
+ }
+
+ @Override
+ public InProcessTransformResult finishBundle() throws Exception {
+ return StepTransformResult.withoutHold(transform).addOutput(outputBundle).build();
+ }
+ }
+
+ private static class InProcessAssignContext<InputT, W extends BoundedWindow>
+ extends WindowFn<InputT, W>.AssignContext {
+ private final WindowedValue<InputT> value;
+
+ public InProcessAssignContext(WindowFn<InputT, W> fn, WindowedValue<InputT> value) {
+ fn.super();
+ this.value = value;
+ }
+
+ @Override
+ public InputT element() {
+ return value.getValue();
+ }
+
+ @Override
+ public Instant timestamp() {
+ return value.getTimestamp();
+ }
+
+ @Override
+ public Collection<? extends BoundedWindow> windows() {
+ return value.getWindows();
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactoryTest.java
new file mode 100644
index 0000000..d290a4b
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactoryTest.java
@@ -0,0 +1,112 @@
+/*
+ * 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.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.theInstance;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.sdk.io.AvroIO;
+import org.apache.beam.sdk.io.AvroIOTest;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+
+import org.hamcrest.Matchers;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.File;
+
+/**
+ * Tests for {@link AvroIOShardedWriteFactory}.
+ */
+@RunWith(JUnit4.class)
+public class AvroIOShardedWriteFactoryTest {
+
+ @Rule public TemporaryFolder tmp = new TemporaryFolder();
+ private AvroIOShardedWriteFactory factory;
+
+ @Before
+ public void setup() {
+ factory = new AvroIOShardedWriteFactory();
+ }
+
+ @Test
+ public void originalWithoutShardingReturnsOriginal() throws Exception {
+ File file = tmp.newFile("foo");
+ PTransform<PCollection<String>, PDone> original =
+ AvroIO.Write.withSchema(String.class).to(file.getAbsolutePath()).withoutSharding();
+ PTransform<PCollection<String>, PDone> overridden = factory.override(original);
+
+ assertThat(overridden, theInstance(original));
+ }
+
+ @Test
+ public void originalShardingNotSpecifiedReturnsOriginal() throws Exception {
+ File file = tmp.newFile("foo");
+ PTransform<PCollection<String>, PDone> original =
+ AvroIO.Write.withSchema(String.class).to(file.getAbsolutePath());
+ PTransform<PCollection<String>, PDone> overridden = factory.override(original);
+
+ assertThat(overridden, theInstance(original));
+ }
+
+ @Test
+ public void originalShardedToOneReturnsExplicitlySharded() throws Exception {
+ File file = tmp.newFile("foo");
+ AvroIO.Write.Bound<String> original =
+ AvroIO.Write.withSchema(String.class).to(file.getAbsolutePath()).withNumShards(1);
+ PTransform<PCollection<String>, PDone> overridden = factory.override(original);
+
+ assertThat(overridden, not(Matchers.<PTransform<PCollection<String>, PDone>>equalTo(original)));
+
+ TestPipeline p = TestPipeline.create();
+ String[] elems = new String[] {"foo", "bar", "baz"};
+ p.apply(Create.<String>of(elems)).apply(overridden);
+
+ file.delete();
+
+ p.run();
+ AvroIOTest.assertTestOutputs(elems, 1, file.getAbsolutePath(), original.getShardNameTemplate());
+ }
+
+ @Test
+ public void originalShardedToManyReturnsExplicitlySharded() throws Exception {
+ File file = tmp.newFile("foo");
+ AvroIO.Write.Bound<String> original =
+ AvroIO.Write.withSchema(String.class).to(file.getAbsolutePath()).withNumShards(3);
+ PTransform<PCollection<String>, PDone> overridden = factory.override(original);
+
+ assertThat(overridden, not(Matchers.<PTransform<PCollection<String>, PDone>>equalTo(original)));
+
+ TestPipeline p = TestPipeline.create();
+ String[] elems = new String[] {"foo", "bar", "baz", "spam", "ham", "eggs"};
+ p.apply(Create.<String>of(elems)).apply(overridden);
+
+ file.delete();
+ p.run();
+ AvroIOTest.assertTestOutputs(elems, 3, file.getAbsolutePath(), original.getShardNameTemplate());
+ }
+}
[03/17] incubator-beam git commit: Move InProcessRunner to its own
module
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManagerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManagerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManagerTest.java
deleted file mode 100644
index 3c4503f..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManagerTest.java
+++ /dev/null
@@ -1,1168 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.FiredTimers;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate.TimerUpdateBuilder;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.Filter;
-import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.WithKeys;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.state.StateNamespaces;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionList;
-import org.apache.beam.sdk.values.PValue;
-import org.apache.beam.sdk.values.TimestampedValue;
-
-import com.google.common.collect.ImmutableList;
-
-import org.hamcrest.BaseMatcher;
-import org.hamcrest.Description;
-import org.hamcrest.Matcher;
-import org.joda.time.Instant;
-import org.joda.time.ReadableInstant;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Tests for {@link InMemoryWatermarkManager}.
- */
-@RunWith(JUnit4.class)
-public class InMemoryWatermarkManagerTest implements Serializable {
- private transient MockClock clock;
-
- private transient PCollection<Integer> createdInts;
-
- private transient PCollection<Integer> filtered;
- private transient PCollection<Integer> filteredTimesTwo;
- private transient PCollection<KV<String, Integer>> keyed;
-
- private transient PCollection<Integer> intsToFlatten;
- private transient PCollection<Integer> flattened;
-
- private transient InMemoryWatermarkManager manager;
- private transient BundleFactory bundleFactory;
-
- @Before
- public void setup() {
- TestPipeline p = TestPipeline.create();
-
- createdInts = p.apply("createdInts", Create.of(1, 2, 3));
-
- filtered = createdInts.apply("filtered", Filter.greaterThan(1));
- filteredTimesTwo = filtered.apply("timesTwo", ParDo.of(new DoFn<Integer, Integer>() {
- @Override
- public void processElement(DoFn<Integer, Integer>.ProcessContext c) throws Exception {
- c.output(c.element() * 2);
- }
- }));
-
- keyed = createdInts.apply("keyed", WithKeys.<String, Integer>of("MyKey"));
-
- intsToFlatten = p.apply("intsToFlatten", Create.of(-1, 256, 65535));
- PCollectionList<Integer> preFlatten = PCollectionList.of(createdInts).and(intsToFlatten);
- flattened = preFlatten.apply("flattened", Flatten.<Integer>pCollections());
-
- Collection<AppliedPTransform<?, ?, ?>> rootTransforms =
- ImmutableList.<AppliedPTransform<?, ?, ?>>of(
- createdInts.getProducingTransformInternal(),
- intsToFlatten.getProducingTransformInternal());
-
- Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers = new HashMap<>();
- consumers.put(
- createdInts,
- ImmutableList.<AppliedPTransform<?, ?, ?>>of(filtered.getProducingTransformInternal(),
- keyed.getProducingTransformInternal(), flattened.getProducingTransformInternal()));
- consumers.put(
- filtered,
- Collections.<AppliedPTransform<?, ?, ?>>singleton(
- filteredTimesTwo.getProducingTransformInternal()));
- consumers.put(filteredTimesTwo, Collections.<AppliedPTransform<?, ?, ?>>emptyList());
- consumers.put(keyed, Collections.<AppliedPTransform<?, ?, ?>>emptyList());
-
- consumers.put(
- intsToFlatten,
- Collections.<AppliedPTransform<?, ?, ?>>singleton(
- flattened.getProducingTransformInternal()));
- consumers.put(flattened, Collections.<AppliedPTransform<?, ?, ?>>emptyList());
-
- clock = MockClock.fromInstant(new Instant(1000));
-
- manager = InMemoryWatermarkManager.create(clock, rootTransforms, consumers);
- bundleFactory = InProcessBundleFactory.create();
- }
-
- /**
- * Demonstrates that getWatermark, when called on an {@link AppliedPTransform} that has not
- * processed any elements, returns the {@link BoundedWindow#TIMESTAMP_MIN_VALUE}.
- */
- @Test
- public void getWatermarkForUntouchedTransform() {
- TransformWatermarks watermarks =
- manager.getWatermarks(createdInts.getProducingTransformInternal());
-
- assertThat(watermarks.getInputWatermark(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
- assertThat(watermarks.getOutputWatermark(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
- }
-
- /**
- * Demonstrates that getWatermark for a transform that consumes no input uses the Watermark
- * Hold value provided to it as the output watermark.
- */
- @Test
- public void getWatermarkForUpdatedSourceTransform() {
- CommittedBundle<Integer> output = multiWindowedBundle(createdInts, 1);
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>singleton(output), new Instant(8000L));
- TransformWatermarks updatedSourceWatermark =
- manager.getWatermarks(createdInts.getProducingTransformInternal());
-
- assertThat(updatedSourceWatermark.getOutputWatermark(), equalTo(new Instant(8000L)));
- }
-
- /**
- * Demonstrates that getWatermark for a transform that takes multiple inputs is held to the
- * minimum watermark across all of its inputs.
- */
- @Test
- public void getWatermarkForMultiInputTransform() {
- CommittedBundle<Integer> secondPcollectionBundle = multiWindowedBundle(intsToFlatten, -1);
-
- manager.updateWatermarks(null, intsToFlatten.getProducingTransformInternal(),
- TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(secondPcollectionBundle),
- BoundedWindow.TIMESTAMP_MAX_VALUE);
-
- // We didn't do anything for the first source, so we shouldn't have progressed the watermark
- TransformWatermarks firstSourceWatermark =
- manager.getWatermarks(createdInts.getProducingTransformInternal());
- assertThat(
- firstSourceWatermark.getOutputWatermark(),
- not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
-
- // the Second Source output all of the elements so it should be done (with a watermark at the
- // end of time).
- TransformWatermarks secondSourceWatermark =
- manager.getWatermarks(intsToFlatten.getProducingTransformInternal());
- assertThat(
- secondSourceWatermark.getOutputWatermark(),
- not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-
- // We haven't consumed anything yet, so our watermark should be at the beginning of time
- TransformWatermarks transformWatermark =
- manager.getWatermarks(flattened.getProducingTransformInternal());
- assertThat(
- transformWatermark.getInputWatermark(), not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
- assertThat(
- transformWatermark.getOutputWatermark(), not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
-
- CommittedBundle<Integer> flattenedBundleSecondCreate = multiWindowedBundle(flattened, -1);
- // We have finished processing the bundle from the second PCollection, but we haven't consumed
- // anything from the first PCollection yet; so our watermark shouldn't advance
- manager.updateWatermarks(secondPcollectionBundle, flattened.getProducingTransformInternal(),
- TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(flattenedBundleSecondCreate),
- null);
- TransformWatermarks transformAfterProcessing =
- manager.getWatermarks(flattened.getProducingTransformInternal());
- manager.updateWatermarks(secondPcollectionBundle, flattened.getProducingTransformInternal(),
- TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(flattenedBundleSecondCreate),
- null);
- assertThat(
- transformAfterProcessing.getInputWatermark(),
- not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
- assertThat(
- transformAfterProcessing.getOutputWatermark(),
- not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
-
- Instant firstCollectionTimestamp = new Instant(10000);
- CommittedBundle<Integer> firstPcollectionBundle =
- timestampedBundle(createdInts, TimestampedValue.<Integer>of(5, firstCollectionTimestamp));
- // the source is done, but elements are still buffered. The source output watermark should be
- // past the end of the global window
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>singleton(firstPcollectionBundle),
- new Instant(Long.MAX_VALUE));
- TransformWatermarks firstSourceWatermarks =
- manager.getWatermarks(createdInts.getProducingTransformInternal());
- assertThat(
- firstSourceWatermarks.getOutputWatermark(),
- not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-
- // We still haven't consumed any of the first source's input, so the watermark should still not
- // progress
- TransformWatermarks flattenAfterSourcesProduced =
- manager.getWatermarks(flattened.getProducingTransformInternal());
- assertThat(
- flattenAfterSourcesProduced.getInputWatermark(), not(laterThan(firstCollectionTimestamp)));
- assertThat(
- flattenAfterSourcesProduced.getOutputWatermark(), not(laterThan(firstCollectionTimestamp)));
-
- // We have buffered inputs, but since the PCollection has all of the elements (has a WM past the
- // end of the global window), we should have a watermark equal to the min among buffered
- // elements
- TransformWatermarks withBufferedElements =
- manager.getWatermarks(flattened.getProducingTransformInternal());
- assertThat(withBufferedElements.getInputWatermark(), equalTo(firstCollectionTimestamp));
- assertThat(withBufferedElements.getOutputWatermark(), equalTo(firstCollectionTimestamp));
-
- CommittedBundle<?> completedFlattenBundle =
- bundleFactory.createRootBundle(flattened).commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
- manager.updateWatermarks(firstPcollectionBundle, flattened.getProducingTransformInternal(),
- TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(completedFlattenBundle),
- null);
- TransformWatermarks afterConsumingAllInput =
- manager.getWatermarks(flattened.getProducingTransformInternal());
- assertThat(
- afterConsumingAllInput.getInputWatermark(),
- not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
- assertThat(
- afterConsumingAllInput.getOutputWatermark(),
- not(laterThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
- }
-
- /**
- * Demonstrates that pending elements are independent among
- * {@link AppliedPTransform AppliedPTransforms} that consume the same input {@link PCollection}.
- */
- @Test
- public void getWatermarkForMultiConsumedCollection() {
- CommittedBundle<Integer> createdBundle = timestampedBundle(createdInts,
- TimestampedValue.of(1, new Instant(1_000_000L)), TimestampedValue.of(2, new Instant(1234L)),
- TimestampedValue.of(3, new Instant(-1000L)));
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>singleton(createdBundle), new Instant(Long.MAX_VALUE));
- TransformWatermarks createdAfterProducing =
- manager.getWatermarks(createdInts.getProducingTransformInternal());
- assertThat(
- createdAfterProducing.getOutputWatermark(),
- not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-
- CommittedBundle<KV<String, Integer>> keyBundle =
- timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), new Instant(1_000_000L)),
- TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)),
- TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L)));
- manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(),
- TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(keyBundle), null);
- TransformWatermarks keyedWatermarks =
- manager.getWatermarks(keyed.getProducingTransformInternal());
- assertThat(
- keyedWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
- assertThat(
- keyedWatermarks.getOutputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-
- TransformWatermarks filteredWatermarks =
- manager.getWatermarks(filtered.getProducingTransformInternal());
- assertThat(filteredWatermarks.getInputWatermark(), not(laterThan(new Instant(-1000L))));
- assertThat(filteredWatermarks.getOutputWatermark(), not(laterThan(new Instant(-1000L))));
-
- CommittedBundle<Integer> filteredBundle =
- timestampedBundle(filtered, TimestampedValue.of(2, new Instant(1234L)));
- manager.updateWatermarks(createdBundle, filtered.getProducingTransformInternal(),
- TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(filteredBundle), null);
- TransformWatermarks filteredProcessedWatermarks =
- manager.getWatermarks(filtered.getProducingTransformInternal());
- assertThat(
- filteredProcessedWatermarks.getInputWatermark(),
- not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
- assertThat(
- filteredProcessedWatermarks.getOutputWatermark(),
- not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
- }
-
- /**
- * Demonstrates that the watermark of an {@link AppliedPTransform} is held to the provided
- * watermark hold.
- */
- @Test
- public void updateWatermarkWithWatermarkHolds() {
- CommittedBundle<Integer> createdBundle = timestampedBundle(createdInts,
- TimestampedValue.of(1, new Instant(1_000_000L)), TimestampedValue.of(2, new Instant(1234L)),
- TimestampedValue.of(3, new Instant(-1000L)));
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>singleton(createdBundle), new Instant(Long.MAX_VALUE));
-
- CommittedBundle<KV<String, Integer>> keyBundle =
- timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), new Instant(1_000_000L)),
- TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)),
- TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L)));
- manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(),
- TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(keyBundle),
- new Instant(500L));
- TransformWatermarks keyedWatermarks =
- manager.getWatermarks(keyed.getProducingTransformInternal());
- assertThat(
- keyedWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
- assertThat(keyedWatermarks.getOutputWatermark(), not(laterThan(new Instant(500L))));
- }
-
- /**
- * Demonstrates that the watermark of an {@link AppliedPTransform} is held to the provided
- * watermark hold.
- */
- @Test
- public void updateWatermarkWithKeyedWatermarkHolds() {
- CommittedBundle<Integer> firstKeyBundle =
- bundleFactory.createKeyedBundle(null, "Odd", createdInts)
- .add(WindowedValue.timestampedValueInGlobalWindow(1, new Instant(1_000_000L)))
- .add(WindowedValue.timestampedValueInGlobalWindow(3, new Instant(-1000L)))
- .commit(clock.now());
-
- CommittedBundle<Integer> secondKeyBundle =
- bundleFactory.createKeyedBundle(null, "Even", createdInts)
- .add(WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1234L)))
- .commit(clock.now());
-
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- ImmutableList.of(firstKeyBundle, secondKeyBundle), BoundedWindow.TIMESTAMP_MAX_VALUE);
-
- manager.updateWatermarks(firstKeyBundle, filtered.getProducingTransformInternal(),
- TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(), new Instant(-1000L));
- manager.updateWatermarks(secondKeyBundle, filtered.getProducingTransformInternal(),
- TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(), new Instant(1234L));
-
- TransformWatermarks filteredWatermarks =
- manager.getWatermarks(filtered.getProducingTransformInternal());
- assertThat(
- filteredWatermarks.getInputWatermark(),
- not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
- assertThat(filteredWatermarks.getOutputWatermark(), not(laterThan(new Instant(-1000L))));
-
- CommittedBundle<Integer> fauxFirstKeyTimerBundle =
- bundleFactory.createKeyedBundle(null, "Odd", createdInts).commit(clock.now());
- manager.updateWatermarks(fauxFirstKeyTimerBundle, filtered.getProducingTransformInternal(),
- TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(),
- BoundedWindow.TIMESTAMP_MAX_VALUE);
-
- assertThat(filteredWatermarks.getOutputWatermark(), equalTo(new Instant(1234L)));
-
- CommittedBundle<Integer> fauxSecondKeyTimerBundle =
- bundleFactory.createKeyedBundle(null, "Even", createdInts).commit(clock.now());
- manager.updateWatermarks(fauxSecondKeyTimerBundle, filtered.getProducingTransformInternal(),
- TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(), new Instant(5678L));
- assertThat(filteredWatermarks.getOutputWatermark(), equalTo(new Instant(5678L)));
-
- manager.updateWatermarks(fauxSecondKeyTimerBundle, filtered.getProducingTransformInternal(),
- TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(),
- BoundedWindow.TIMESTAMP_MAX_VALUE);
- assertThat(
- filteredWatermarks.getOutputWatermark(),
- not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
- }
-
- /**
- * Demonstrates that updated output watermarks are monotonic in the presence of late data, when
- * called on an {@link AppliedPTransform} that consumes no input.
- */
- @Test
- public void updateOutputWatermarkShouldBeMonotonic() {
- CommittedBundle<?> firstInput =
- bundleFactory.createRootBundle(createdInts).commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>singleton(firstInput), new Instant(0L));
- TransformWatermarks firstWatermarks =
- manager.getWatermarks(createdInts.getProducingTransformInternal());
- assertThat(firstWatermarks.getOutputWatermark(), equalTo(new Instant(0L)));
-
- CommittedBundle<?> secondInput =
- bundleFactory.createRootBundle(createdInts).commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>singleton(secondInput), new Instant(-250L));
- TransformWatermarks secondWatermarks =
- manager.getWatermarks(createdInts.getProducingTransformInternal());
- assertThat(secondWatermarks.getOutputWatermark(), not(earlierThan(new Instant(0L))));
- }
-
- /**
- * Demonstrates that updated output watermarks are monotonic in the presence of watermark holds
- * that become earlier than a previous watermark hold.
- */
- @Test
- public void updateWatermarkWithHoldsShouldBeMonotonic() {
- CommittedBundle<Integer> createdBundle = timestampedBundle(createdInts,
- TimestampedValue.of(1, new Instant(1_000_000L)), TimestampedValue.of(2, new Instant(1234L)),
- TimestampedValue.of(3, new Instant(-1000L)));
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>singleton(createdBundle), new Instant(Long.MAX_VALUE));
-
- CommittedBundle<KV<String, Integer>> keyBundle =
- timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), new Instant(1_000_000L)),
- TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)),
- TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L)));
- manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(),
- TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(keyBundle),
- new Instant(500L));
- TransformWatermarks keyedWatermarks =
- manager.getWatermarks(keyed.getProducingTransformInternal());
- assertThat(
- keyedWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
- assertThat(keyedWatermarks.getOutputWatermark(), not(laterThan(new Instant(500L))));
- Instant oldOutputWatermark = keyedWatermarks.getOutputWatermark();
-
- TransformWatermarks updatedWatermarks =
- manager.getWatermarks(keyed.getProducingTransformInternal());
- assertThat(
- updatedWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
- // We added a hold prior to the old watermark; we shouldn't progress (due to the earlier hold)
- // but the watermark is monotonic and should not backslide to the new, earlier hold
- assertThat(updatedWatermarks.getOutputWatermark(), equalTo(oldOutputWatermark));
- }
-
- /**
- * Demonstrates that updateWatermarks in the presence of late data is monotonic.
- */
- @Test
- public void updateWatermarkWithLateData() {
- Instant sourceWatermark = new Instant(1_000_000L);
- CommittedBundle<Integer> createdBundle = timestampedBundle(createdInts,
- TimestampedValue.of(1, sourceWatermark), TimestampedValue.of(2, new Instant(1234L)));
-
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>singleton(createdBundle), sourceWatermark);
-
- CommittedBundle<KV<String, Integer>> keyBundle =
- timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), sourceWatermark),
- TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)));
-
- // Finish processing the on-time data. The watermarks should progress to be equal to the source
- manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(),
- TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(keyBundle), null);
- TransformWatermarks onTimeWatermarks =
- manager.getWatermarks(keyed.getProducingTransformInternal());
- assertThat(onTimeWatermarks.getInputWatermark(), equalTo(sourceWatermark));
- assertThat(onTimeWatermarks.getOutputWatermark(), equalTo(sourceWatermark));
-
- CommittedBundle<Integer> lateDataBundle =
- timestampedBundle(createdInts, TimestampedValue.of(3, new Instant(-1000L)));
- // the late data arrives in a downstream PCollection after its watermark has advanced past it;
- // we don't advance the watermark past the current watermark until we've consumed the late data
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>singleton(lateDataBundle), new Instant(2_000_000L));
- TransformWatermarks bufferedLateWm =
- manager.getWatermarks(createdInts.getProducingTransformInternal());
- assertThat(bufferedLateWm.getOutputWatermark(), equalTo(new Instant(2_000_000L)));
-
- // The input watermark should be held to its previous value (not advanced due to late data; not
- // moved backwards in the presence of watermarks due to monotonicity).
- TransformWatermarks lateDataBufferedWatermark =
- manager.getWatermarks(keyed.getProducingTransformInternal());
- assertThat(lateDataBufferedWatermark.getInputWatermark(), not(earlierThan(sourceWatermark)));
- assertThat(lateDataBufferedWatermark.getOutputWatermark(), not(earlierThan(sourceWatermark)));
-
- CommittedBundle<KV<String, Integer>> lateKeyedBundle =
- timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L)));
- manager.updateWatermarks(lateDataBundle, keyed.getProducingTransformInternal(),
- TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(lateKeyedBundle), null);
- }
-
- public void updateWatermarkWithDifferentWindowedValueInstances() {
- manager.updateWatermarks(
- null,
- createdInts.getProducingTransformInternal(),
- TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>singleton(
- bundleFactory
- .createRootBundle(createdInts)
- .add(WindowedValue.valueInGlobalWindow(1))
- .commit(Instant.now())),
- BoundedWindow.TIMESTAMP_MAX_VALUE);
-
- manager.updateWatermarks(
- bundleFactory
- .createRootBundle(createdInts)
- .add(WindowedValue.valueInGlobalWindow(1))
- .commit(Instant.now()),
- keyed.getProducingTransformInternal(),
- TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>emptyList(),
- null);
- TransformWatermarks onTimeWatermarks =
- manager.getWatermarks(keyed.getProducingTransformInternal());
- assertThat(onTimeWatermarks.getInputWatermark(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
- }
-
- /**
- * Demonstrates that after watermarks of an upstream transform are updated, but no output has been
- * produced, the watermarks of a downstream process are advanced.
- */
- @Test
- public void getWatermarksAfterOnlyEmptyOutput() {
- CommittedBundle<Integer> emptyCreateOutput = multiWindowedBundle(createdInts);
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>singleton(emptyCreateOutput),
- BoundedWindow.TIMESTAMP_MAX_VALUE);
- TransformWatermarks updatedSourceWatermarks =
- manager.getWatermarks(createdInts.getProducingTransformInternal());
-
- assertThat(
- updatedSourceWatermarks.getOutputWatermark(),
- not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-
- TransformWatermarks finishedFilterWatermarks =
- manager.getWatermarks(filtered.getProducingTransformInternal());
- assertThat(
- finishedFilterWatermarks.getInputWatermark(),
- not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
- assertThat(
- finishedFilterWatermarks.getOutputWatermark(),
- not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
- }
-
- /**
- * Demonstrates that after watermarks of an upstream transform are updated, but no output has been
- * produced, and the downstream transform has a watermark hold, the watermark is held to the hold.
- */
- @Test
- public void getWatermarksAfterHoldAndEmptyOutput() {
- CommittedBundle<Integer> firstCreateOutput = multiWindowedBundle(createdInts, 1, 2);
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>singleton(firstCreateOutput), new Instant(12_000L));
-
- CommittedBundle<Integer> firstFilterOutput = multiWindowedBundle(filtered);
- manager.updateWatermarks(firstCreateOutput, filtered.getProducingTransformInternal(),
- TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(firstFilterOutput),
- new Instant(10_000L));
- TransformWatermarks firstFilterWatermarks =
- manager.getWatermarks(filtered.getProducingTransformInternal());
- assertThat(firstFilterWatermarks.getInputWatermark(), not(earlierThan(new Instant(12_000L))));
- assertThat(firstFilterWatermarks.getOutputWatermark(), not(laterThan(new Instant(10_000L))));
-
- CommittedBundle<Integer> emptyCreateOutput = multiWindowedBundle(createdInts);
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>singleton(emptyCreateOutput),
- BoundedWindow.TIMESTAMP_MAX_VALUE);
- TransformWatermarks updatedSourceWatermarks =
- manager.getWatermarks(createdInts.getProducingTransformInternal());
-
- assertThat(
- updatedSourceWatermarks.getOutputWatermark(),
- not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-
- TransformWatermarks finishedFilterWatermarks =
- manager.getWatermarks(filtered.getProducingTransformInternal());
- assertThat(
- finishedFilterWatermarks.getInputWatermark(),
- not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
- assertThat(finishedFilterWatermarks.getOutputWatermark(), not(laterThan(new Instant(10_000L))));
- }
-
- @Test
- public void getSynchronizedProcessingTimeInputWatermarksHeldToPendingBundles() {
- TransformWatermarks watermarks =
- manager.getWatermarks(createdInts.getProducingTransformInternal());
- assertThat(watermarks.getSynchronizedProcessingInputTime(), equalTo(clock.now()));
- assertThat(
- watermarks.getSynchronizedProcessingOutputTime(),
- equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
-
- TransformWatermarks filteredWatermarks =
- manager.getWatermarks(filtered.getProducingTransformInternal());
- // Non-root processing watermarks don't progress until data has been processed
- assertThat(
- filteredWatermarks.getSynchronizedProcessingInputTime(),
- not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
- assertThat(
- filteredWatermarks.getSynchronizedProcessingOutputTime(),
- not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
-
- CommittedBundle<Integer> createOutput =
- bundleFactory.createRootBundle(createdInts).commit(new Instant(1250L));
-
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>singleton(createOutput), BoundedWindow.TIMESTAMP_MAX_VALUE);
- TransformWatermarks createAfterUpdate =
- manager.getWatermarks(createdInts.getProducingTransformInternal());
- assertThat(createAfterUpdate.getSynchronizedProcessingInputTime(), equalTo(clock.now()));
- assertThat(createAfterUpdate.getSynchronizedProcessingOutputTime(), equalTo(clock.now()));
-
- TransformWatermarks filterAfterProduced =
- manager.getWatermarks(filtered.getProducingTransformInternal());
- assertThat(
- filterAfterProduced.getSynchronizedProcessingInputTime(), not(laterThan(clock.now())));
- assertThat(
- filterAfterProduced.getSynchronizedProcessingOutputTime(), not(laterThan(clock.now())));
-
- clock.set(new Instant(1500L));
- assertThat(createAfterUpdate.getSynchronizedProcessingInputTime(), equalTo(clock.now()));
- assertThat(createAfterUpdate.getSynchronizedProcessingOutputTime(), equalTo(clock.now()));
- assertThat(
- filterAfterProduced.getSynchronizedProcessingInputTime(),
- not(laterThan(new Instant(1250L))));
- assertThat(
- filterAfterProduced.getSynchronizedProcessingOutputTime(),
- not(laterThan(new Instant(1250L))));
-
- CommittedBundle<?> filterOutputBundle =
- bundleFactory.createRootBundle(intsToFlatten).commit(new Instant(1250L));
- manager.updateWatermarks(createOutput, filtered.getProducingTransformInternal(),
- TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(filterOutputBundle),
- BoundedWindow.TIMESTAMP_MAX_VALUE);
- TransformWatermarks filterAfterConsumed =
- manager.getWatermarks(filtered.getProducingTransformInternal());
- assertThat(
- filterAfterConsumed.getSynchronizedProcessingInputTime(),
- not(laterThan(createAfterUpdate.getSynchronizedProcessingOutputTime())));
- assertThat(
- filterAfterConsumed.getSynchronizedProcessingOutputTime(),
- not(laterThan(filterAfterConsumed.getSynchronizedProcessingInputTime())));
- }
-
- /**
- * Demonstrates that the Synchronized Processing Time output watermark cannot progress past
- * pending timers in the same set. This propagates to all downstream SynchronizedProcessingTimes.
- *
- * <p>Also demonstrate that the result is monotonic.
- */
- // @Test
- public void getSynchronizedProcessingTimeOutputHeldToPendingTimers() {
- CommittedBundle<Integer> createdBundle = multiWindowedBundle(createdInts, 1, 2, 4, 8);
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>singleton(createdBundle), new Instant(1248L));
-
- TransformWatermarks filteredWms =
- manager.getWatermarks(filtered.getProducingTransformInternal());
- TransformWatermarks filteredDoubledWms =
- manager.getWatermarks(filteredTimesTwo.getProducingTransformInternal());
- Instant initialFilteredWm = filteredWms.getSynchronizedProcessingOutputTime();
- Instant initialFilteredDoubledWm = filteredDoubledWms.getSynchronizedProcessingOutputTime();
-
- CommittedBundle<Integer> filteredBundle = multiWindowedBundle(filtered, 2, 8);
- TimerData pastTimer =
- TimerData.of(StateNamespaces.global(), new Instant(250L), TimeDomain.PROCESSING_TIME);
- TimerData futureTimer =
- TimerData.of(StateNamespaces.global(), new Instant(4096L), TimeDomain.PROCESSING_TIME);
- TimerUpdate timers =
- TimerUpdate.builder("key").setTimer(pastTimer).setTimer(futureTimer).build();
- manager.updateWatermarks(createdBundle, filtered.getProducingTransformInternal(), timers,
- Collections.<CommittedBundle<?>>singleton(filteredBundle),
- BoundedWindow.TIMESTAMP_MAX_VALUE);
- Instant startTime = clock.now();
- clock.set(startTime.plus(250L));
- // We're held based on the past timer
- assertThat(filteredWms.getSynchronizedProcessingOutputTime(), not(laterThan(startTime)));
- assertThat(filteredDoubledWms.getSynchronizedProcessingOutputTime(), not(laterThan(startTime)));
- // And we're monotonic
- assertThat(
- filteredWms.getSynchronizedProcessingOutputTime(), not(earlierThan(initialFilteredWm)));
- assertThat(
- filteredDoubledWms.getSynchronizedProcessingOutputTime(),
- not(earlierThan(initialFilteredDoubledWm)));
-
- Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> firedTimers =
- manager.extractFiredTimers();
- assertThat(
- firedTimers.get(filtered.getProducingTransformInternal())
- .get("key")
- .getTimers(TimeDomain.PROCESSING_TIME),
- contains(pastTimer));
- // Our timer has fired, but has not been completed, so it holds our synchronized processing WM
- assertThat(filteredWms.getSynchronizedProcessingOutputTime(), not(laterThan(startTime)));
- assertThat(filteredDoubledWms.getSynchronizedProcessingOutputTime(), not(laterThan(startTime)));
-
- CommittedBundle<Integer> filteredTimerBundle =
- bundleFactory
- .createKeyedBundle(null, "key", filtered)
- .commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
- CommittedBundle<Integer> filteredTimerResult =
- bundleFactory.createKeyedBundle(null, "key", filteredTimesTwo)
- .commit(filteredWms.getSynchronizedProcessingOutputTime());
- // Complete the processing time timer
- manager.updateWatermarks(filteredTimerBundle, filtered.getProducingTransformInternal(),
- TimerUpdate.builder("key")
- .withCompletedTimers(Collections.<TimerData>singleton(pastTimer))
- .build(),
- Collections.<CommittedBundle<?>>singleton(filteredTimerResult),
- BoundedWindow.TIMESTAMP_MAX_VALUE);
-
- clock.set(startTime.plus(500L));
- assertThat(filteredWms.getSynchronizedProcessingOutputTime(), not(laterThan(clock.now())));
- // filtered should be held to the time at which the filteredTimerResult fired
- assertThat(
- filteredDoubledWms.getSynchronizedProcessingOutputTime(),
- not(earlierThan(filteredTimerResult.getSynchronizedProcessingOutputWatermark())));
-
- manager.updateWatermarks(filteredTimerResult, filteredTimesTwo.getProducingTransformInternal(),
- TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(),
- BoundedWindow.TIMESTAMP_MAX_VALUE);
- assertThat(filteredDoubledWms.getSynchronizedProcessingOutputTime(), equalTo(clock.now()));
-
- clock.set(new Instant(Long.MAX_VALUE));
- assertThat(filteredWms.getSynchronizedProcessingOutputTime(), equalTo(new Instant(4096)));
- assertThat(
- filteredDoubledWms.getSynchronizedProcessingOutputTime(), equalTo(new Instant(4096)));
- }
-
- /**
- * Demonstrates that if any earlier processing holds appear in the synchronized processing time
- * output hold the result is monotonic.
- */
- @Test
- public void getSynchronizedProcessingTimeOutputTimeIsMonotonic() {
- Instant startTime = clock.now();
- TransformWatermarks watermarks =
- manager.getWatermarks(createdInts.getProducingTransformInternal());
- assertThat(watermarks.getSynchronizedProcessingInputTime(), equalTo(startTime));
-
- TransformWatermarks filteredWatermarks =
- manager.getWatermarks(filtered.getProducingTransformInternal());
- // Non-root processing watermarks don't progress until data has been processed
- assertThat(
- filteredWatermarks.getSynchronizedProcessingInputTime(),
- not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
- assertThat(
- filteredWatermarks.getSynchronizedProcessingOutputTime(),
- not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
-
- CommittedBundle<Integer> createOutput =
- bundleFactory.createRootBundle(createdInts).commit(new Instant(1250L));
-
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>singleton(createOutput), BoundedWindow.TIMESTAMP_MAX_VALUE);
- TransformWatermarks createAfterUpdate =
- manager.getWatermarks(createdInts.getProducingTransformInternal());
- assertThat(createAfterUpdate.getSynchronizedProcessingInputTime(), not(laterThan(clock.now())));
- assertThat(
- createAfterUpdate.getSynchronizedProcessingOutputTime(), not(laterThan(clock.now())));
-
- CommittedBundle<Integer> createSecondOutput =
- bundleFactory.createRootBundle(createdInts).commit(new Instant(750L));
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>singleton(createSecondOutput),
- BoundedWindow.TIMESTAMP_MAX_VALUE);
-
- assertThat(createAfterUpdate.getSynchronizedProcessingOutputTime(), equalTo(clock.now()));
- }
-
- @Test
- public void synchronizedProcessingInputTimeIsHeldToUpstreamProcessingTimeTimers() {
- CommittedBundle<Integer> created = multiWindowedBundle(createdInts, 1, 2, 3);
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>singleton(created), new Instant(40_900L));
-
- CommittedBundle<Integer> filteredBundle = multiWindowedBundle(filtered, 2, 4);
- Instant upstreamHold = new Instant(2048L);
- TimerData upstreamProcessingTimer =
- TimerData.of(StateNamespaces.global(), upstreamHold, TimeDomain.PROCESSING_TIME);
- manager.updateWatermarks(created, filtered.getProducingTransformInternal(),
- TimerUpdate.builder("key").setTimer(upstreamProcessingTimer).build(),
- Collections.<CommittedBundle<?>>singleton(filteredBundle),
- BoundedWindow.TIMESTAMP_MAX_VALUE);
-
- TransformWatermarks downstreamWms =
- manager.getWatermarks(filteredTimesTwo.getProducingTransformInternal());
- assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(clock.now()));
-
- clock.set(BoundedWindow.TIMESTAMP_MAX_VALUE);
- assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(upstreamHold));
-
- manager.extractFiredTimers();
- // Pending processing time timers that have been fired but aren't completed hold the
- // synchronized processing time
- assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(upstreamHold));
-
- CommittedBundle<Integer> otherCreated = multiWindowedBundle(createdInts, 4, 8, 12);
- manager.updateWatermarks(otherCreated, filtered.getProducingTransformInternal(),
- TimerUpdate.builder("key")
- .withCompletedTimers(Collections.singleton(upstreamProcessingTimer))
- .build(),
- Collections.<CommittedBundle<?>>emptyList(), BoundedWindow.TIMESTAMP_MAX_VALUE);
-
- assertThat(downstreamWms.getSynchronizedProcessingInputTime(), not(earlierThan(clock.now())));
- }
-
- @Test
- public void synchronizedProcessingInputTimeIsHeldToPendingBundleTimes() {
- CommittedBundle<Integer> created = multiWindowedBundle(createdInts, 1, 2, 3);
- manager.updateWatermarks(
- null,
- createdInts.getProducingTransformInternal(),
- TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>singleton(created),
- new Instant(29_919_235L));
-
- Instant upstreamHold = new Instant(2048L);
- CommittedBundle<Integer> filteredBundle =
- bundleFactory.createKeyedBundle(created, "key", filtered).commit(upstreamHold);
- manager.updateWatermarks(
- created,
- filtered.getProducingTransformInternal(),
- TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>singleton(filteredBundle),
- BoundedWindow.TIMESTAMP_MAX_VALUE);
-
- TransformWatermarks downstreamWms =
- manager.getWatermarks(filteredTimesTwo.getProducingTransformInternal());
- assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(clock.now()));
-
- clock.set(BoundedWindow.TIMESTAMP_MAX_VALUE);
- assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(upstreamHold));
- }
-
- @Test
- public void extractFiredTimersReturnsFiredEventTimeTimers() {
- Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> initialTimers =
- manager.extractFiredTimers();
- // Watermarks haven't advanced
- assertThat(initialTimers.entrySet(), emptyIterable());
-
- // Advance WM of keyed past the first timer, but ahead of the second and third
- CommittedBundle<Integer> createdBundle = multiWindowedBundle(filtered);
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.singleton(createdBundle), new Instant(1500L));
-
- TimerData earliestTimer =
- TimerData.of(StateNamespaces.global(), new Instant(1000), TimeDomain.EVENT_TIME);
- TimerData middleTimer =
- TimerData.of(StateNamespaces.global(), new Instant(5000L), TimeDomain.EVENT_TIME);
- TimerData lastTimer =
- TimerData.of(StateNamespaces.global(), new Instant(10000L), TimeDomain.EVENT_TIME);
- Object key = new Object();
- TimerUpdate update =
- TimerUpdate.builder(key)
- .setTimer(earliestTimer)
- .setTimer(middleTimer)
- .setTimer(lastTimer)
- .build();
-
- manager.updateWatermarks(
- createdBundle,
- filtered.getProducingTransformInternal(),
- update,
- Collections.<CommittedBundle<?>>singleton(multiWindowedBundle(intsToFlatten)),
- new Instant(1000L));
-
- Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> firstTransformFiredTimers =
- manager.extractFiredTimers();
- assertThat(
- firstTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
- Map<Object, FiredTimers> firstFilteredTimers =
- firstTransformFiredTimers.get(filtered.getProducingTransformInternal());
- assertThat(firstFilteredTimers.get(key), not(nullValue()));
- FiredTimers firstFired = firstFilteredTimers.get(key);
- assertThat(firstFired.getTimers(TimeDomain.EVENT_TIME), contains(earliestTimer));
-
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>emptyList(), new Instant(50_000L));
- Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> secondTransformFiredTimers =
- manager.extractFiredTimers();
- assertThat(
- secondTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
- Map<Object, FiredTimers> secondFilteredTimers =
- secondTransformFiredTimers.get(filtered.getProducingTransformInternal());
- assertThat(secondFilteredTimers.get(key), not(nullValue()));
- FiredTimers secondFired = secondFilteredTimers.get(key);
- // Contains, in order, middleTimer and then lastTimer
- assertThat(secondFired.getTimers(TimeDomain.EVENT_TIME), contains(middleTimer, lastTimer));
- }
-
- @Test
- public void extractFiredTimersReturnsFiredProcessingTimeTimers() {
- Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> initialTimers =
- manager.extractFiredTimers();
- // Watermarks haven't advanced
- assertThat(initialTimers.entrySet(), emptyIterable());
-
- // Advance WM of keyed past the first timer, but ahead of the second and third
- CommittedBundle<Integer> createdBundle = multiWindowedBundle(filtered);
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.singleton(createdBundle), new Instant(1500L));
-
- TimerData earliestTimer =
- TimerData.of(StateNamespaces.global(), new Instant(999L), TimeDomain.PROCESSING_TIME);
- TimerData middleTimer =
- TimerData.of(StateNamespaces.global(), new Instant(5000L), TimeDomain.PROCESSING_TIME);
- TimerData lastTimer =
- TimerData.of(StateNamespaces.global(), new Instant(10000L), TimeDomain.PROCESSING_TIME);
- Object key = new Object();
- TimerUpdate update =
- TimerUpdate.builder(key)
- .setTimer(lastTimer)
- .setTimer(earliestTimer)
- .setTimer(middleTimer)
- .build();
-
- manager.updateWatermarks(
- createdBundle,
- filtered.getProducingTransformInternal(),
- update,
- Collections.<CommittedBundle<?>>singleton(multiWindowedBundle(intsToFlatten)),
- new Instant(1000L));
-
- Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> firstTransformFiredTimers =
- manager.extractFiredTimers();
- assertThat(
- firstTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
- Map<Object, FiredTimers> firstFilteredTimers =
- firstTransformFiredTimers.get(filtered.getProducingTransformInternal());
- assertThat(firstFilteredTimers.get(key), not(nullValue()));
- FiredTimers firstFired = firstFilteredTimers.get(key);
- assertThat(firstFired.getTimers(TimeDomain.PROCESSING_TIME), contains(earliestTimer));
-
- clock.set(new Instant(50_000L));
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>emptyList(), new Instant(50_000L));
- Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> secondTransformFiredTimers =
- manager.extractFiredTimers();
- assertThat(
- secondTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
- Map<Object, FiredTimers> secondFilteredTimers =
- secondTransformFiredTimers.get(filtered.getProducingTransformInternal());
- assertThat(secondFilteredTimers.get(key), not(nullValue()));
- FiredTimers secondFired = secondFilteredTimers.get(key);
- // Contains, in order, middleTimer and then lastTimer
- assertThat(secondFired.getTimers(TimeDomain.PROCESSING_TIME), contains(middleTimer, lastTimer));
- }
-
- @Test
- public void extractFiredTimersReturnsFiredSynchronizedProcessingTimeTimers() {
- Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> initialTimers =
- manager.extractFiredTimers();
- // Watermarks haven't advanced
- assertThat(initialTimers.entrySet(), emptyIterable());
-
- // Advance WM of keyed past the first timer, but ahead of the second and third
- CommittedBundle<Integer> createdBundle = multiWindowedBundle(filtered);
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.singleton(createdBundle), new Instant(1500L));
-
- TimerData earliestTimer = TimerData.of(
- StateNamespaces.global(), new Instant(999L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
- TimerData middleTimer = TimerData.of(
- StateNamespaces.global(), new Instant(5000L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
- TimerData lastTimer = TimerData.of(
- StateNamespaces.global(), new Instant(10000L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
- Object key = new Object();
- TimerUpdate update =
- TimerUpdate.builder(key)
- .setTimer(lastTimer)
- .setTimer(earliestTimer)
- .setTimer(middleTimer)
- .build();
-
- manager.updateWatermarks(
- createdBundle,
- filtered.getProducingTransformInternal(),
- update,
- Collections.<CommittedBundle<?>>singleton(multiWindowedBundle(intsToFlatten)),
- new Instant(1000L));
-
- Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> firstTransformFiredTimers =
- manager.extractFiredTimers();
- assertThat(
- firstTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
- Map<Object, FiredTimers> firstFilteredTimers =
- firstTransformFiredTimers.get(filtered.getProducingTransformInternal());
- assertThat(firstFilteredTimers.get(key), not(nullValue()));
- FiredTimers firstFired = firstFilteredTimers.get(key);
- assertThat(
- firstFired.getTimers(TimeDomain.SYNCHRONIZED_PROCESSING_TIME), contains(earliestTimer));
-
- clock.set(new Instant(50_000L));
- manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
- Collections.<CommittedBundle<?>>emptyList(), new Instant(50_000L));
- Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> secondTransformFiredTimers =
- manager.extractFiredTimers();
- assertThat(
- secondTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
- Map<Object, FiredTimers> secondFilteredTimers =
- secondTransformFiredTimers.get(filtered.getProducingTransformInternal());
- assertThat(secondFilteredTimers.get(key), not(nullValue()));
- FiredTimers secondFired = secondFilteredTimers.get(key);
- // Contains, in order, middleTimer and then lastTimer
- assertThat(
- secondFired.getTimers(TimeDomain.SYNCHRONIZED_PROCESSING_TIME),
- contains(middleTimer, lastTimer));
- }
-
- @Test
- public void timerUpdateBuilderBuildAddsAllAddedTimers() {
- TimerData set = TimerData.of(StateNamespaces.global(), new Instant(10L), TimeDomain.EVENT_TIME);
- TimerData deleted =
- TimerData.of(StateNamespaces.global(), new Instant(24L), TimeDomain.PROCESSING_TIME);
- TimerData completedOne = TimerData.of(
- StateNamespaces.global(), new Instant(1024L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
- TimerData completedTwo =
- TimerData.of(StateNamespaces.global(), new Instant(2048L), TimeDomain.EVENT_TIME);
-
- TimerUpdate update =
- TimerUpdate.builder("foo")
- .withCompletedTimers(ImmutableList.of(completedOne, completedTwo))
- .setTimer(set)
- .deletedTimer(deleted)
- .build();
-
- assertThat(update.getCompletedTimers(), containsInAnyOrder(completedOne, completedTwo));
- assertThat(update.getSetTimers(), contains(set));
- assertThat(update.getDeletedTimers(), contains(deleted));
- }
-
- @Test
- public void timerUpdateBuilderWithSetThenDeleteHasOnlyDeleted() {
- TimerUpdateBuilder builder = TimerUpdate.builder(null);
- TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
-
- TimerUpdate built = builder.setTimer(timer).deletedTimer(timer).build();
-
- assertThat(built.getSetTimers(), emptyIterable());
- assertThat(built.getDeletedTimers(), contains(timer));
- }
-
- @Test
- public void timerUpdateBuilderWithDeleteThenSetHasOnlySet() {
- TimerUpdateBuilder builder = TimerUpdate.builder(null);
- TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
-
- TimerUpdate built = builder.deletedTimer(timer).setTimer(timer).build();
-
- assertThat(built.getSetTimers(), contains(timer));
- assertThat(built.getDeletedTimers(), emptyIterable());
- }
-
- @Test
- public void timerUpdateBuilderWithSetAfterBuildNotAddedToBuilt() {
- TimerUpdateBuilder builder = TimerUpdate.builder(null);
- TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
-
- TimerUpdate built = builder.build();
- builder.setTimer(timer);
- assertThat(built.getSetTimers(), emptyIterable());
- builder.build();
- assertThat(built.getSetTimers(), emptyIterable());
- }
-
- @Test
- public void timerUpdateBuilderWithDeleteAfterBuildNotAddedToBuilt() {
- TimerUpdateBuilder builder = TimerUpdate.builder(null);
- TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
-
- TimerUpdate built = builder.build();
- builder.deletedTimer(timer);
- assertThat(built.getDeletedTimers(), emptyIterable());
- builder.build();
- assertThat(built.getDeletedTimers(), emptyIterable());
- }
-
- @Test
- public void timerUpdateBuilderWithCompletedAfterBuildNotAddedToBuilt() {
- TimerUpdateBuilder builder = TimerUpdate.builder(null);
- TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
-
- TimerUpdate built = builder.build();
- builder.withCompletedTimers(ImmutableList.of(timer));
- assertThat(built.getCompletedTimers(), emptyIterable());
- builder.build();
- assertThat(built.getCompletedTimers(), emptyIterable());
- }
-
- @Test
- public void timerUpdateWithCompletedTimersNotAddedToExisting() {
- TimerUpdateBuilder builder = TimerUpdate.builder(null);
- TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
-
- TimerUpdate built = builder.build();
- assertThat(built.getCompletedTimers(), emptyIterable());
- assertThat(
- built.withCompletedTimers(ImmutableList.of(timer)).getCompletedTimers(), contains(timer));
- assertThat(built.getCompletedTimers(), emptyIterable());
- }
-
- private static Matcher<Instant> earlierThan(final Instant laterInstant) {
- return new BaseMatcher<Instant>() {
- @Override
- public boolean matches(Object item) {
- ReadableInstant instant = (ReadableInstant) item;
- return instant.isBefore(laterInstant);
- }
-
- @Override
- public void describeTo(Description description) {
- description.appendText("earlier than ").appendValue(laterInstant);
- }
- };
- }
-
- private static Matcher<Instant> laterThan(final Instant shouldBeEarlier) {
- return new BaseMatcher<Instant>() {
- @Override
- public boolean matches(Object item) {
- ReadableInstant instant = (ReadableInstant) item;
- return instant.isAfter(shouldBeEarlier);
- }
-
- @Override
- public void describeTo(Description description) {
- description.appendText("later than ").appendValue(shouldBeEarlier);
- }
- };
- }
-
- @SafeVarargs
- private final <T> CommittedBundle<T> timestampedBundle(
- PCollection<T> pc, TimestampedValue<T>... values) {
- UncommittedBundle<T> bundle = bundleFactory.createRootBundle(pc);
- for (TimestampedValue<T> value : values) {
- bundle.add(
- WindowedValue.timestampedValueInGlobalWindow(value.getValue(), value.getTimestamp()));
- }
- return bundle.commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
- }
-
- @SafeVarargs
- private final <T> CommittedBundle<T> multiWindowedBundle(PCollection<T> pc, T... values) {
- UncommittedBundle<T> bundle = bundleFactory.createRootBundle(pc);
- Collection<BoundedWindow> windows =
- ImmutableList.of(
- GlobalWindow.INSTANCE,
- new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(0)));
- for (T value : values) {
- bundle.add(
- WindowedValue.of(value, BoundedWindow.TIMESTAMP_MIN_VALUE, windows, PaneInfo.NO_FIRING));
- }
- return bundle.commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactoryTest.java
deleted file mode 100644
index 3c7c2c6..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactoryTest.java
+++ /dev/null
@@ -1,223 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.WithKeys;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.common.collect.ImmutableList;
-
-import org.hamcrest.Matcher;
-import org.hamcrest.Matchers;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-
-/**
- * Tests for {@link InProcessBundleFactory}.
- */
-@RunWith(JUnit4.class)
-public class InProcessBundleFactoryTest {
- @Rule public ExpectedException thrown = ExpectedException.none();
-
- private InProcessBundleFactory bundleFactory = InProcessBundleFactory.create();
-
- private PCollection<Integer> created;
- private PCollection<KV<String, Integer>> downstream;
-
- @Before
- public void setup() {
- TestPipeline p = TestPipeline.create();
- created = p.apply(Create.of(1, 2, 3));
- downstream = created.apply(WithKeys.<String, Integer>of("foo"));
- }
-
- @Test
- public void createRootBundleShouldCreateWithNullKey() {
- PCollection<Integer> pcollection = TestPipeline.create().apply(Create.of(1));
-
- UncommittedBundle<Integer> inFlightBundle = bundleFactory.createRootBundle(pcollection);
-
- CommittedBundle<Integer> bundle = inFlightBundle.commit(Instant.now());
-
- assertThat(bundle.getKey(), nullValue());
- }
-
- private void createKeyedBundle(Object key) {
- PCollection<Integer> pcollection = TestPipeline.create().apply(Create.of(1));
-
- UncommittedBundle<Integer> inFlightBundle =
- bundleFactory.createKeyedBundle(null, key, pcollection);
-
- CommittedBundle<Integer> bundle = inFlightBundle.commit(Instant.now());
- assertThat(bundle.getKey(), equalTo(key));
- }
-
- @Test
- public void keyedWithNullKeyShouldCreateKeyedBundle() {
- createKeyedBundle(null);
- }
-
- @Test
- public void keyedWithKeyShouldCreateKeyedBundle() {
- createKeyedBundle(new Object());
- }
-
- private <T> CommittedBundle<T>
- afterCommitGetElementsShouldHaveAddedElements(Iterable<WindowedValue<T>> elems) {
- PCollection<T> pcollection = TestPipeline.create().apply(Create.<T>of());
-
- UncommittedBundle<T> bundle = bundleFactory.createRootBundle(pcollection);
- Collection<Matcher<? super WindowedValue<T>>> expectations = new ArrayList<>();
- for (WindowedValue<T> elem : elems) {
- bundle.add(elem);
- expectations.add(equalTo(elem));
- }
- Matcher<Iterable<? extends WindowedValue<T>>> containsMatcher =
- Matchers.<WindowedValue<T>>containsInAnyOrder(expectations);
- CommittedBundle<T> committed = bundle.commit(Instant.now());
- assertThat(committed.getElements(), containsMatcher);
-
- return committed;
- }
-
- @Test
- public void getElementsBeforeAddShouldReturnEmptyIterable() {
- afterCommitGetElementsShouldHaveAddedElements(Collections.<WindowedValue<Integer>>emptyList());
- }
-
- @Test
- public void getElementsAfterAddShouldReturnAddedElements() {
- WindowedValue<Integer> firstValue = WindowedValue.valueInGlobalWindow(1);
- WindowedValue<Integer> secondValue =
- WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1000L));
-
- afterCommitGetElementsShouldHaveAddedElements(ImmutableList.of(firstValue, secondValue));
- }
-
- @SuppressWarnings("unchecked")
- @Test
- public void withElementsShouldReturnIndependentBundle() {
- WindowedValue<Integer> firstValue = WindowedValue.valueInGlobalWindow(1);
- WindowedValue<Integer> secondValue =
- WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1000L));
-
- CommittedBundle<Integer> committed =
- afterCommitGetElementsShouldHaveAddedElements(ImmutableList.of(firstValue, secondValue));
-
- WindowedValue<Integer> firstReplacement =
- WindowedValue.of(
- 9,
- new Instant(2048L),
- new IntervalWindow(new Instant(2044L), Instant.now()),
- PaneInfo.NO_FIRING);
- WindowedValue<Integer> secondReplacement =
- WindowedValue.timestampedValueInGlobalWindow(-1, Instant.now());
- CommittedBundle<Integer> withed =
- committed.withElements(ImmutableList.of(firstReplacement, secondReplacement));
-
- assertThat(withed.getElements(), containsInAnyOrder(firstReplacement, secondReplacement));
- assertThat(committed.getElements(), containsInAnyOrder(firstValue, secondValue));
- assertThat(withed.getKey(), equalTo(committed.getKey()));
- assertThat(withed.getPCollection(), equalTo(committed.getPCollection()));
- assertThat(
- withed.getSynchronizedProcessingOutputWatermark(),
- equalTo(committed.getSynchronizedProcessingOutputWatermark()));
- }
-
- @Test
- public void addAfterCommitShouldThrowException() {
- PCollection<Integer> pcollection = TestPipeline.create().apply(Create.<Integer>of());
-
- UncommittedBundle<Integer> bundle = bundleFactory.createRootBundle(pcollection);
- bundle.add(WindowedValue.valueInGlobalWindow(1));
- CommittedBundle<Integer> firstCommit = bundle.commit(Instant.now());
- assertThat(firstCommit.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(1)));
-
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage("3");
- thrown.expectMessage("committed");
-
- bundle.add(WindowedValue.valueInGlobalWindow(3));
- }
-
- @Test
- public void commitAfterCommitShouldThrowException() {
- PCollection<Integer> pcollection = TestPipeline.create().apply(Create.<Integer>of());
-
- UncommittedBundle<Integer> bundle = bundleFactory.createRootBundle(pcollection);
- bundle.add(WindowedValue.valueInGlobalWindow(1));
- CommittedBundle<Integer> firstCommit = bundle.commit(Instant.now());
- assertThat(firstCommit.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(1)));
-
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage("committed");
-
- bundle.commit(Instant.now());
- }
-
- @Test
- public void createBundleUnkeyedResultUnkeyed() {
- CommittedBundle<KV<String, Integer>> newBundle =
- bundleFactory
- .createBundle(bundleFactory.createRootBundle(created).commit(Instant.now()), downstream)
- .commit(Instant.now());
- }
-
- @Test
- public void createBundleKeyedResultPropagatesKey() {
- CommittedBundle<KV<String, Integer>> newBundle =
- bundleFactory
- .createBundle(
- bundleFactory.createKeyedBundle(null, "foo", created).commit(Instant.now()),
- downstream)
- .commit(Instant.now());
- assertThat(newBundle.getKey(), Matchers.<Object>equalTo("foo"));
- }
-
- @Test
- public void createKeyedBundleKeyed() {
- CommittedBundle<KV<String, Integer>> keyedBundle =
- bundleFactory
- .createKeyedBundle(
- bundleFactory.createRootBundle(created).commit(Instant.now()), "foo", downstream)
- .commit(Instant.now());
- assertThat(keyedBundle.getKey(), Matchers.<Object>equalTo("foo"));
- }
-}
[04/17] incubator-beam git commit: Move InProcessRunner to its own
module
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactoryTest.java
deleted file mode 100644
index 8ed2684..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactoryTest.java
+++ /dev/null
@@ -1,257 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.isA;
-
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Count;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.util.UserCodeException;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-import org.joda.time.Instant;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Collections;
-
-/**
- * Tests for {@link EncodabilityEnforcementFactory}.
- */
-@RunWith(JUnit4.class)
-public class EncodabilityEnforcementFactoryTest {
- @Rule public ExpectedException thrown = ExpectedException.none();
- private EncodabilityEnforcementFactory factory = EncodabilityEnforcementFactory.create();
- private BundleFactory bundleFactory = InProcessBundleFactory.create();
-
- @Test
- public void encodeFailsThrows() {
- WindowedValue<Record> record = WindowedValue.valueInGlobalWindow(new Record());
-
- ModelEnforcement<Record> enforcement = createEnforcement(new RecordNoEncodeCoder(), record);
-
- thrown.expect(UserCodeException.class);
- thrown.expectCause(isA(CoderException.class));
- thrown.expectMessage("Encode not allowed");
- enforcement.beforeElement(record);
- }
-
- @Test
- public void decodeFailsThrows() {
- WindowedValue<Record> record = WindowedValue.valueInGlobalWindow(new Record());
-
- ModelEnforcement<Record> enforcement = createEnforcement(new RecordNoDecodeCoder(), record);
-
- thrown.expect(UserCodeException.class);
- thrown.expectCause(isA(CoderException.class));
- thrown.expectMessage("Decode not allowed");
- enforcement.beforeElement(record);
- }
-
- @Test
- public void consistentWithEqualsStructuralValueNotEqualThrows() {
- WindowedValue<Record> record =
- WindowedValue.<Record>valueInGlobalWindow(
- new Record() {
- @Override
- public String toString() {
- return "OriginalRecord";
- }
- });
-
- ModelEnforcement<Record> enforcement =
- createEnforcement(new RecordStructuralValueCoder(), record);
-
- thrown.expect(UserCodeException.class);
- thrown.expectCause(isA(IllegalArgumentException.class));
- thrown.expectMessage("does not maintain structural value equality");
- thrown.expectMessage(RecordStructuralValueCoder.class.getSimpleName());
- thrown.expectMessage("OriginalRecord");
- enforcement.beforeElement(record);
- }
-
- @Test
- public void notConsistentWithEqualsStructuralValueNotEqualSucceeds() {
- TestPipeline p = TestPipeline.create();
- PCollection<Record> unencodable =
- p.apply(
- Create.of(new Record())
- .withCoder(new RecordNotConsistentWithEqualsStructuralValueCoder()));
- AppliedPTransform<?, ?, ?> consumer =
- unencodable.apply(Count.<Record>globally()).getProducingTransformInternal();
-
- WindowedValue<Record> record = WindowedValue.<Record>valueInGlobalWindow(new Record());
-
- CommittedBundle<Record> input =
- bundleFactory.createRootBundle(unencodable).add(record).commit(Instant.now());
- ModelEnforcement<Record> enforcement = factory.forBundle(input, consumer);
-
- enforcement.beforeElement(record);
- enforcement.afterElement(record);
- enforcement.afterFinish(
- input,
- StepTransformResult.withoutHold(consumer).build(),
- Collections.<CommittedBundle<?>>emptyList());
- }
-
- private <T> ModelEnforcement<T> createEnforcement(Coder<T> coder, WindowedValue<T> record) {
- TestPipeline p = TestPipeline.create();
- PCollection<T> unencodable = p.apply(Create.<T>of().withCoder(coder));
- AppliedPTransform<?, ?, ?> consumer =
- unencodable.apply(Count.<T>globally()).getProducingTransformInternal();
- CommittedBundle<T> input =
- bundleFactory.createRootBundle(unencodable).add(record).commit(Instant.now());
- ModelEnforcement<T> enforcement = factory.forBundle(input, consumer);
- return enforcement;
- }
-
- @Test
- public void structurallyEqualResultsSucceeds() {
- TestPipeline p = TestPipeline.create();
- PCollection<Integer> unencodable = p.apply(Create.of(1).withCoder(VarIntCoder.of()));
- AppliedPTransform<?, ?, ?> consumer =
- unencodable.apply(Count.<Integer>globally()).getProducingTransformInternal();
-
- WindowedValue<Integer> value = WindowedValue.valueInGlobalWindow(1);
-
- CommittedBundle<Integer> input =
- bundleFactory.createRootBundle(unencodable).add(value).commit(Instant.now());
- ModelEnforcement<Integer> enforcement = factory.forBundle(input, consumer);
-
- enforcement.beforeElement(value);
- enforcement.afterElement(value);
- enforcement.afterFinish(
- input,
- StepTransformResult.withoutHold(consumer).build(),
- Collections.<CommittedBundle<?>>emptyList());
- }
-
- private static class Record {}
- private static class RecordNoEncodeCoder extends AtomicCoder<Record> {
-
- @Override
- public void encode(
- Record value,
- OutputStream outStream,
- org.apache.beam.sdk.coders.Coder.Context context)
- throws CoderException, IOException {
- throw new CoderException("Encode not allowed");
- }
-
- @Override
- public Record decode(
- InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
- throws CoderException, IOException {
- return null;
- }
- }
-
- private static class RecordNoDecodeCoder extends AtomicCoder<Record> {
- @Override
- public void encode(
- Record value,
- OutputStream outStream,
- org.apache.beam.sdk.coders.Coder.Context context)
- throws CoderException, IOException {}
-
- @Override
- public Record decode(
- InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
- throws CoderException, IOException {
- throw new CoderException("Decode not allowed");
- }
- }
-
- private static class RecordStructuralValueCoder extends AtomicCoder<Record> {
- @Override
- public void encode(
- Record value,
- OutputStream outStream,
- org.apache.beam.sdk.coders.Coder.Context context)
- throws CoderException, IOException {}
-
- @Override
- public Record decode(
- InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
- throws CoderException, IOException {
- return new Record() {
- @Override
- public String toString() {
- return "DecodedRecord";
- }
- };
- }
-
- @Override
- public boolean consistentWithEquals() {
- return true;
- }
-
- @Override
- public Object structuralValue(Record value) {
- return value;
- }
- }
-
- private static class RecordNotConsistentWithEqualsStructuralValueCoder
- extends AtomicCoder<Record> {
- @Override
- public void encode(
- Record value,
- OutputStream outStream,
- org.apache.beam.sdk.coders.Coder.Context context)
- throws CoderException, IOException {}
-
- @Override
- public Record decode(
- InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
- throws CoderException, IOException {
- return new Record() {
- @Override
- public String toString() {
- return "DecodedRecord";
- }
- };
- }
-
- @Override
- public boolean consistentWithEquals() {
- return false;
- }
-
- @Override
- public Object structuralValue(Record value) {
- return value;
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactoryTest.java
deleted file mode 100644
index 5c1da14..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactoryTest.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionList;
-
-import org.hamcrest.Matchers;
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link FlattenEvaluatorFactory}.
- */
-@RunWith(JUnit4.class)
-public class FlattenEvaluatorFactoryTest {
- private BundleFactory bundleFactory = InProcessBundleFactory.create();
- @Test
- public void testFlattenInMemoryEvaluator() throws Exception {
- TestPipeline p = TestPipeline.create();
- PCollection<Integer> left = p.apply("left", Create.of(1, 2, 4));
- PCollection<Integer> right = p.apply("right", Create.of(-1, 2, -4));
- PCollectionList<Integer> list = PCollectionList.of(left).and(right);
-
- PCollection<Integer> flattened = list.apply(Flatten.<Integer>pCollections());
-
- CommittedBundle<Integer> leftBundle =
- bundleFactory.createRootBundle(left).commit(Instant.now());
- CommittedBundle<Integer> rightBundle =
- bundleFactory.createRootBundle(right).commit(Instant.now());
-
- InProcessEvaluationContext context = mock(InProcessEvaluationContext.class);
-
- UncommittedBundle<Integer> flattenedLeftBundle = bundleFactory.createRootBundle(flattened);
- UncommittedBundle<Integer> flattenedRightBundle = bundleFactory.createRootBundle(flattened);
-
- when(context.createBundle(leftBundle, flattened)).thenReturn(flattenedLeftBundle);
- when(context.createBundle(rightBundle, flattened)).thenReturn(flattenedRightBundle);
-
- FlattenEvaluatorFactory factory = new FlattenEvaluatorFactory();
- TransformEvaluator<Integer> leftSideEvaluator =
- factory.forApplication(flattened.getProducingTransformInternal(), leftBundle, context);
- TransformEvaluator<Integer> rightSideEvaluator =
- factory.forApplication(
- flattened.getProducingTransformInternal(),
- rightBundle,
- context);
-
- leftSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(1));
- rightSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(-1));
- leftSideEvaluator.processElement(
- WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1024)));
- leftSideEvaluator.processElement(WindowedValue.valueInEmptyWindows(4, PaneInfo.NO_FIRING));
- rightSideEvaluator.processElement(
- WindowedValue.valueInEmptyWindows(2, PaneInfo.ON_TIME_AND_ONLY_FIRING));
- rightSideEvaluator.processElement(
- WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096)));
-
- InProcessTransformResult rightSideResult = rightSideEvaluator.finishBundle();
- InProcessTransformResult leftSideResult = leftSideEvaluator.finishBundle();
-
- assertThat(
- rightSideResult.getOutputBundles(),
- Matchers.<UncommittedBundle<?>>contains(flattenedRightBundle));
- assertThat(
- rightSideResult.getTransform(),
- Matchers.<AppliedPTransform<?, ?, ?>>equalTo(flattened.getProducingTransformInternal()));
- assertThat(
- leftSideResult.getOutputBundles(),
- Matchers.<UncommittedBundle<?>>contains(flattenedLeftBundle));
- assertThat(
- leftSideResult.getTransform(),
- Matchers.<AppliedPTransform<?, ?, ?>>equalTo(flattened.getProducingTransformInternal()));
-
- assertThat(
- flattenedLeftBundle.commit(Instant.now()).getElements(),
- containsInAnyOrder(
- WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1024)),
- WindowedValue.valueInEmptyWindows(4, PaneInfo.NO_FIRING),
- WindowedValue.valueInGlobalWindow(1)));
- assertThat(
- flattenedRightBundle.commit(Instant.now()).getElements(),
- containsInAnyOrder(
- WindowedValue.valueInEmptyWindows(2, PaneInfo.ON_TIME_AND_ONLY_FIRING),
- WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096)),
- WindowedValue.valueInGlobalWindow(-1)));
- }
-
- @Test
- public void testFlattenInMemoryEvaluatorWithEmptyPCollectionList() throws Exception {
- TestPipeline p = TestPipeline.create();
- PCollectionList<Integer> list = PCollectionList.empty(p);
-
- PCollection<Integer> flattened = list.apply(Flatten.<Integer>pCollections());
-
- InProcessEvaluationContext context = mock(InProcessEvaluationContext.class);
-
- FlattenEvaluatorFactory factory = new FlattenEvaluatorFactory();
- TransformEvaluator<Integer> emptyEvaluator =
- factory.forApplication(flattened.getProducingTransformInternal(), null, context);
-
- InProcessTransformResult leftSideResult = emptyEvaluator.finishBundle();
-
- assertThat(leftSideResult.getOutputBundles(), emptyIterable());
- assertThat(
- leftSideResult.getTransform(),
- Matchers.<AppliedPTransform<?, ?, ?>>equalTo(flattened.getProducingTransformInternal()));
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransformTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransformTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransformTest.java
deleted file mode 100644
index 366dfc5..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransformTest.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.equalTo;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.sdk.values.PCollection;
-
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-/**
- * Tests for {@link ForwardingPTransform}.
- */
-@RunWith(JUnit4.class)
-public class ForwardingPTransformTest {
- @Rule public ExpectedException thrown = ExpectedException.none();
-
- @Mock private PTransform<PCollection<Integer>, PCollection<String>> delegate;
-
- private ForwardingPTransform<PCollection<Integer>, PCollection<String>> forwarding;
-
- @Before
- public void setup() {
- MockitoAnnotations.initMocks(this);
- forwarding =
- new ForwardingPTransform<PCollection<Integer>, PCollection<String>>() {
- @Override
- protected PTransform<PCollection<Integer>, PCollection<String>> delegate() {
- return delegate;
- }
- };
- }
-
- @Test
- public void applyDelegates() {
- @SuppressWarnings("unchecked")
- PCollection<Integer> collection = mock(PCollection.class);
- @SuppressWarnings("unchecked")
- PCollection<String> output = mock(PCollection.class);
- when(delegate.apply(collection)).thenReturn(output);
- PCollection<String> result = forwarding.apply(collection);
- assertThat(result, equalTo(output));
- }
-
- @Test
- public void getNameDelegates() {
- String name = "My_forwardingptransform-name;for!thisTest";
- when(delegate.getName()).thenReturn(name);
- assertThat(forwarding.getName(), equalTo(name));
- }
-
- @Test
- public void validateDelegates() {
- @SuppressWarnings("unchecked")
- PCollection<Integer> input = mock(PCollection.class);
- doThrow(RuntimeException.class).when(delegate).validate(input);
-
- thrown.expect(RuntimeException.class);
- forwarding.validate(input);
- }
-
- @Test
- public void getDefaultOutputCoderDelegates() throws Exception {
- @SuppressWarnings("unchecked")
- PCollection<Integer> input = mock(PCollection.class);
- @SuppressWarnings("unchecked")
- PCollection<String> output = mock(PCollection.class);
- @SuppressWarnings("unchecked")
- Coder<String> outputCoder = mock(Coder.class);
-
- when(delegate.getDefaultOutputCoder(input, output)).thenReturn(outputCoder);
- assertThat(forwarding.getDefaultOutputCoder(input, output), equalTo(outputCoder));
- }
-
- @Test
- public void populateDisplayDataDelegates() {
- DisplayData.Builder builder = mock(DisplayData.Builder.class);
- doThrow(RuntimeException.class).when(delegate).populateDisplayData(builder);
-
- thrown.expect(RuntimeException.class);
- forwarding.populateDisplayData(builder);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactoryTest.java
deleted file mode 100644
index b7ce169..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactoryTest.java
+++ /dev/null
@@ -1,183 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.contains;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.ReifyTimestampsAndWindows;
-import org.apache.beam.sdk.util.KeyedWorkItem;
-import org.apache.beam.sdk.util.KeyedWorkItems;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.common.collect.HashMultiset;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Multiset;
-
-import org.hamcrest.BaseMatcher;
-import org.hamcrest.Description;
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link GroupByKeyEvaluatorFactory}.
- */
-@RunWith(JUnit4.class)
-public class GroupByKeyEvaluatorFactoryTest {
- private BundleFactory bundleFactory = InProcessBundleFactory.create();
-
- @Test
- public void testInMemoryEvaluator() throws Exception {
- TestPipeline p = TestPipeline.create();
- KV<String, Integer> firstFoo = KV.of("foo", -1);
- KV<String, Integer> secondFoo = KV.of("foo", 1);
- KV<String, Integer> thirdFoo = KV.of("foo", 3);
- KV<String, Integer> firstBar = KV.of("bar", 22);
- KV<String, Integer> secondBar = KV.of("bar", 12);
- KV<String, Integer> firstBaz = KV.of("baz", Integer.MAX_VALUE);
- PCollection<KV<String, Integer>> values =
- p.apply(Create.of(firstFoo, firstBar, secondFoo, firstBaz, secondBar, thirdFoo));
- PCollection<KV<String, WindowedValue<Integer>>> kvs =
- values.apply(new ReifyTimestampsAndWindows<String, Integer>());
- PCollection<KeyedWorkItem<String, Integer>> groupedKvs =
- kvs.apply(new GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly<String, Integer>());
-
- CommittedBundle<KV<String, WindowedValue<Integer>>> inputBundle =
- bundleFactory.createRootBundle(kvs).commit(Instant.now());
- InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
-
- UncommittedBundle<KeyedWorkItem<String, Integer>> fooBundle =
- bundleFactory.createKeyedBundle(null, "foo", groupedKvs);
- UncommittedBundle<KeyedWorkItem<String, Integer>> barBundle =
- bundleFactory.createKeyedBundle(null, "bar", groupedKvs);
- UncommittedBundle<KeyedWorkItem<String, Integer>> bazBundle =
- bundleFactory.createKeyedBundle(null, "baz", groupedKvs);
-
- when(evaluationContext.createKeyedBundle(inputBundle, "foo", groupedKvs)).thenReturn(fooBundle);
- when(evaluationContext.createKeyedBundle(inputBundle, "bar", groupedKvs)).thenReturn(barBundle);
- when(evaluationContext.createKeyedBundle(inputBundle, "baz", groupedKvs)).thenReturn(bazBundle);
-
- // The input to a GroupByKey is assumed to be a KvCoder
- @SuppressWarnings("unchecked")
- Coder<String> keyCoder =
- ((KvCoder<String, WindowedValue<Integer>>) kvs.getCoder()).getKeyCoder();
- TransformEvaluator<KV<String, WindowedValue<Integer>>> evaluator =
- new GroupByKeyEvaluatorFactory()
- .forApplication(
- groupedKvs.getProducingTransformInternal(), inputBundle, evaluationContext);
-
- evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(firstFoo)));
- evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(secondFoo)));
- evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(thirdFoo)));
- evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(firstBar)));
- evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(secondBar)));
- evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(firstBaz)));
-
- evaluator.finishBundle();
-
- assertThat(
- fooBundle.commit(Instant.now()).getElements(),
- contains(
- new KeyedWorkItemMatcher<String, Integer>(
- KeyedWorkItems.elementsWorkItem(
- "foo",
- ImmutableSet.of(
- WindowedValue.valueInGlobalWindow(-1),
- WindowedValue.valueInGlobalWindow(1),
- WindowedValue.valueInGlobalWindow(3))),
- keyCoder)));
- assertThat(
- barBundle.commit(Instant.now()).getElements(),
- contains(
- new KeyedWorkItemMatcher<String, Integer>(
- KeyedWorkItems.elementsWorkItem(
- "bar",
- ImmutableSet.of(
- WindowedValue.valueInGlobalWindow(12),
- WindowedValue.valueInGlobalWindow(22))),
- keyCoder)));
- assertThat(
- bazBundle.commit(Instant.now()).getElements(),
- contains(
- new KeyedWorkItemMatcher<String, Integer>(
- KeyedWorkItems.elementsWorkItem(
- "baz",
- ImmutableSet.of(WindowedValue.valueInGlobalWindow(Integer.MAX_VALUE))),
- keyCoder)));
- }
-
- private <K, V> KV<K, WindowedValue<V>> gwValue(KV<K, V> kv) {
- return KV.of(kv.getKey(), WindowedValue.valueInGlobalWindow(kv.getValue()));
- }
-
- private static class KeyedWorkItemMatcher<K, V>
- extends BaseMatcher<WindowedValue<KeyedWorkItem<K, V>>> {
- private final KeyedWorkItem<K, V> myWorkItem;
- private final Coder<K> keyCoder;
-
- public KeyedWorkItemMatcher(KeyedWorkItem<K, V> myWorkItem, Coder<K> keyCoder) {
- this.myWorkItem = myWorkItem;
- this.keyCoder = keyCoder;
- }
-
- @Override
- public boolean matches(Object item) {
- if (item == null || !(item instanceof WindowedValue)) {
- return false;
- }
- WindowedValue<KeyedWorkItem<K, V>> that = (WindowedValue<KeyedWorkItem<K, V>>) item;
- Multiset<WindowedValue<V>> myValues = HashMultiset.create();
- Multiset<WindowedValue<V>> thatValues = HashMultiset.create();
- for (WindowedValue<V> value : myWorkItem.elementsIterable()) {
- myValues.add(value);
- }
- for (WindowedValue<V> value : that.getValue().elementsIterable()) {
- thatValues.add(value);
- }
- try {
- return myValues.equals(thatValues)
- && keyCoder
- .structuralValue(myWorkItem.key())
- .equals(keyCoder.structuralValue(that.getValue().key()));
- } catch (Exception e) {
- return false;
- }
- }
-
- @Override
- public void describeTo(Description description) {
- description
- .appendText("KeyedWorkItem<K, V> containing key ")
- .appendValue(myWorkItem.key())
- .appendText(" and values ")
- .appendValueList("[", ", ", "]", myWorkItem.elementsIterable());
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactoryTest.java
deleted file mode 100644
index 386eacc..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactoryTest.java
+++ /dev/null
@@ -1,220 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.isA;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.coders.ByteArrayCoder;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-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.ParDo;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.IllegalMutationException;
-import org.apache.beam.sdk.util.UserCodeException;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link ImmutabilityCheckingBundleFactory}.
- */
-@RunWith(JUnit4.class)
-public class ImmutabilityCheckingBundleFactoryTest {
- @Rule public ExpectedException thrown = ExpectedException.none();
- private ImmutabilityCheckingBundleFactory factory;
- private PCollection<byte[]> created;
- private PCollection<byte[]> transformed;
-
- @Before
- public void setup() {
- TestPipeline p = TestPipeline.create();
- created = p.apply(Create.<byte[]>of().withCoder(ByteArrayCoder.of()));
- transformed = created.apply(ParDo.of(new IdentityDoFn<byte[]>()));
- factory = ImmutabilityCheckingBundleFactory.create(InProcessBundleFactory.create());
- }
-
- @Test
- public void noMutationRootBundleSucceeds() {
- UncommittedBundle<byte[]> root = factory.createRootBundle(created);
- byte[] array = new byte[] {0, 1, 2};
- root.add(WindowedValue.valueInGlobalWindow(array));
- CommittedBundle<byte[]> committed = root.commit(Instant.now());
-
- assertThat(
- committed.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(array)));
- }
-
- @Test
- public void noMutationKeyedBundleSucceeds() {
- CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
- UncommittedBundle<byte[]> keyed = factory.createKeyedBundle(root, "mykey", transformed);
-
- WindowedValue<byte[]> windowedArray =
- WindowedValue.of(
- new byte[] {4, 8, 12},
- new Instant(891L),
- new IntervalWindow(new Instant(0), new Instant(1000)),
- PaneInfo.ON_TIME_AND_ONLY_FIRING);
- keyed.add(windowedArray);
-
- CommittedBundle<byte[]> committed = keyed.commit(Instant.now());
- assertThat(committed.getElements(), containsInAnyOrder(windowedArray));
- }
-
- @Test
- public void noMutationCreateBundleSucceeds() {
- CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
- UncommittedBundle<byte[]> intermediate = factory.createBundle(root, transformed);
-
- WindowedValue<byte[]> windowedArray =
- WindowedValue.of(
- new byte[] {4, 8, 12},
- new Instant(891L),
- new IntervalWindow(new Instant(0), new Instant(1000)),
- PaneInfo.ON_TIME_AND_ONLY_FIRING);
- intermediate.add(windowedArray);
-
- CommittedBundle<byte[]> committed = intermediate.commit(Instant.now());
- assertThat(committed.getElements(), containsInAnyOrder(windowedArray));
- }
-
- @Test
- public void mutationBeforeAddRootBundleSucceeds() {
- UncommittedBundle<byte[]> root = factory.createRootBundle(created);
- byte[] array = new byte[] {0, 1, 2};
- array[1] = 2;
- root.add(WindowedValue.valueInGlobalWindow(array));
- CommittedBundle<byte[]> committed = root.commit(Instant.now());
-
- assertThat(
- committed.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(array)));
- }
-
- @Test
- public void mutationBeforeAddKeyedBundleSucceeds() {
- CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
- UncommittedBundle<byte[]> keyed = factory.createKeyedBundle(root, "mykey", transformed);
-
- byte[] array = new byte[] {4, 8, 12};
- array[0] = Byte.MAX_VALUE;
- WindowedValue<byte[]> windowedArray =
- WindowedValue.of(
- array,
- new Instant(891L),
- new IntervalWindow(new Instant(0), new Instant(1000)),
- PaneInfo.ON_TIME_AND_ONLY_FIRING);
- keyed.add(windowedArray);
-
- CommittedBundle<byte[]> committed = keyed.commit(Instant.now());
- assertThat(committed.getElements(), containsInAnyOrder(windowedArray));
- }
-
- @Test
- public void mutationBeforeAddCreateBundleSucceeds() {
- CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
- UncommittedBundle<byte[]> intermediate = factory.createBundle(root, transformed);
-
- byte[] array = new byte[] {4, 8, 12};
- WindowedValue<byte[]> windowedArray =
- WindowedValue.of(
- array,
- new Instant(891L),
- new IntervalWindow(new Instant(0), new Instant(1000)),
- PaneInfo.ON_TIME_AND_ONLY_FIRING);
- array[2] = -3;
- intermediate.add(windowedArray);
-
- CommittedBundle<byte[]> committed = intermediate.commit(Instant.now());
- assertThat(committed.getElements(), containsInAnyOrder(windowedArray));
- }
-
- @Test
- public void mutationAfterAddRootBundleThrows() {
- UncommittedBundle<byte[]> root = factory.createRootBundle(created);
- byte[] array = new byte[] {0, 1, 2};
- root.add(WindowedValue.valueInGlobalWindow(array));
-
- array[1] = 2;
- thrown.expect(UserCodeException.class);
- thrown.expectCause(isA(IllegalMutationException.class));
- thrown.expectMessage("Values must not be mutated in any way after being output");
- CommittedBundle<byte[]> committed = root.commit(Instant.now());
- }
-
- @Test
- public void mutationAfterAddKeyedBundleThrows() {
- CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
- UncommittedBundle<byte[]> keyed = factory.createKeyedBundle(root, "mykey", transformed);
-
- byte[] array = new byte[] {4, 8, 12};
- WindowedValue<byte[]> windowedArray =
- WindowedValue.of(
- array,
- new Instant(891L),
- new IntervalWindow(new Instant(0), new Instant(1000)),
- PaneInfo.ON_TIME_AND_ONLY_FIRING);
- keyed.add(windowedArray);
-
- array[0] = Byte.MAX_VALUE;
- thrown.expect(UserCodeException.class);
- thrown.expectCause(isA(IllegalMutationException.class));
- thrown.expectMessage("Values must not be mutated in any way after being output");
- CommittedBundle<byte[]> committed = keyed.commit(Instant.now());
- }
-
- @Test
- public void mutationAfterAddCreateBundleThrows() {
- CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
- UncommittedBundle<byte[]> intermediate = factory.createBundle(root, transformed);
-
- byte[] array = new byte[] {4, 8, 12};
- WindowedValue<byte[]> windowedArray =
- WindowedValue.of(
- array,
- new Instant(891L),
- new IntervalWindow(new Instant(0), new Instant(1000)),
- PaneInfo.ON_TIME_AND_ONLY_FIRING);
- intermediate.add(windowedArray);
-
- array[2] = -3;
- thrown.expect(UserCodeException.class);
- thrown.expectCause(isA(IllegalMutationException.class));
- thrown.expectMessage("Values must not be mutated in any way after being output");
- CommittedBundle<byte[]> committed = intermediate.commit(Instant.now());
- }
-
- private static class IdentityDoFn<T> extends DoFn<T, T> {
- @Override
- public void processElement(DoFn<T, T>.ProcessContext c) throws Exception {
- c.output(c.element());
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactoryTest.java
deleted file mode 100644
index 16633ed..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactoryTest.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Count;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.util.IllegalMutationException;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.Serializable;
-import java.util.Collections;
-
-/**
- * Tests for {@link ImmutabilityEnforcementFactory}.
- */
-@RunWith(JUnit4.class)
-public class ImmutabilityEnforcementFactoryTest implements Serializable {
- @Rule public transient ExpectedException thrown = ExpectedException.none();
- private transient ImmutabilityEnforcementFactory factory;
- private transient BundleFactory bundleFactory;
- private transient PCollection<byte[]> pcollection;
- private transient AppliedPTransform<?, ?, ?> consumer;
-
- @Before
- public void setup() {
- factory = new ImmutabilityEnforcementFactory();
- bundleFactory = InProcessBundleFactory.create();
- TestPipeline p = TestPipeline.create();
- pcollection =
- p.apply(Create.of("foo".getBytes(), "spamhameggs".getBytes()))
- .apply(
- ParDo.of(
- new DoFn<byte[], byte[]>() {
- @Override
- public void processElement(DoFn<byte[], byte[]>.ProcessContext c)
- throws Exception {
- c.element()[0] = 'b';
- }
- }));
- consumer = pcollection.apply(Count.<byte[]>globally()).getProducingTransformInternal();
- }
-
- @Test
- public void unchangedSucceeds() {
- WindowedValue<byte[]> element = WindowedValue.valueInGlobalWindow("bar".getBytes());
- CommittedBundle<byte[]> elements =
- bundleFactory.createRootBundle(pcollection).add(element).commit(Instant.now());
-
- ModelEnforcement<byte[]> enforcement = factory.forBundle(elements, consumer);
- enforcement.beforeElement(element);
- enforcement.afterElement(element);
- enforcement.afterFinish(
- elements,
- StepTransformResult.withoutHold(consumer).build(),
- Collections.<CommittedBundle<?>>emptyList());
- }
-
- @Test
- public void mutatedDuringProcessElementThrows() {
- WindowedValue<byte[]> element = WindowedValue.valueInGlobalWindow("bar".getBytes());
- CommittedBundle<byte[]> elements =
- bundleFactory.createRootBundle(pcollection).add(element).commit(Instant.now());
-
- ModelEnforcement<byte[]> enforcement = factory.forBundle(elements, consumer);
- enforcement.beforeElement(element);
- element.getValue()[0] = 'f';
- thrown.expect(IllegalMutationException.class);
- thrown.expectMessage(consumer.getFullName());
- thrown.expectMessage("illegaly mutated");
- thrown.expectMessage("Input values must not be mutated");
- enforcement.afterElement(element);
- enforcement.afterFinish(
- elements,
- StepTransformResult.withoutHold(consumer).build(),
- Collections.<CommittedBundle<?>>emptyList());
- }
-
- @Test
- public void mutatedAfterProcessElementFails() {
-
- WindowedValue<byte[]> element = WindowedValue.valueInGlobalWindow("bar".getBytes());
- CommittedBundle<byte[]> elements =
- bundleFactory.createRootBundle(pcollection).add(element).commit(Instant.now());
-
- ModelEnforcement<byte[]> enforcement = factory.forBundle(elements, consumer);
- enforcement.beforeElement(element);
- enforcement.afterElement(element);
-
- element.getValue()[0] = 'f';
- thrown.expect(IllegalMutationException.class);
- thrown.expectMessage(consumer.getFullName());
- thrown.expectMessage("illegaly mutated");
- thrown.expectMessage("Input values must not be mutated");
- enforcement.afterFinish(
- elements,
- StepTransformResult.withoutHold(consumer).build(),
- Collections.<CommittedBundle<?>>emptyList());
- }
-}
[11/17] incubator-beam git commit: Move InProcessRunner to its own
module
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InMemoryWatermarkManagerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InMemoryWatermarkManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InMemoryWatermarkManagerTest.java
new file mode 100644
index 0000000..2880ade
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InMemoryWatermarkManagerTest.java
@@ -0,0 +1,1168 @@
+/*
+ * 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.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.emptyIterable;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.FiredTimers;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate.TimerUpdateBuilder;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TransformWatermarks;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Filter;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+import com.google.common.collect.ImmutableList;
+
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
+import org.hamcrest.Matcher;
+import org.joda.time.Instant;
+import org.joda.time.ReadableInstant;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Tests for {@link InMemoryWatermarkManager}.
+ */
+@RunWith(JUnit4.class)
+public class InMemoryWatermarkManagerTest implements Serializable {
+ private transient MockClock clock;
+
+ private transient PCollection<Integer> createdInts;
+
+ private transient PCollection<Integer> filtered;
+ private transient PCollection<Integer> filteredTimesTwo;
+ private transient PCollection<KV<String, Integer>> keyed;
+
+ private transient PCollection<Integer> intsToFlatten;
+ private transient PCollection<Integer> flattened;
+
+ private transient InMemoryWatermarkManager manager;
+ private transient BundleFactory bundleFactory;
+
+ @Before
+ public void setup() {
+ TestPipeline p = TestPipeline.create();
+
+ createdInts = p.apply("createdInts", Create.of(1, 2, 3));
+
+ filtered = createdInts.apply("filtered", Filter.greaterThan(1));
+ filteredTimesTwo = filtered.apply("timesTwo", ParDo.of(new DoFn<Integer, Integer>() {
+ @Override
+ public void processElement(DoFn<Integer, Integer>.ProcessContext c) throws Exception {
+ c.output(c.element() * 2);
+ }
+ }));
+
+ keyed = createdInts.apply("keyed", WithKeys.<String, Integer>of("MyKey"));
+
+ intsToFlatten = p.apply("intsToFlatten", Create.of(-1, 256, 65535));
+ PCollectionList<Integer> preFlatten = PCollectionList.of(createdInts).and(intsToFlatten);
+ flattened = preFlatten.apply("flattened", Flatten.<Integer>pCollections());
+
+ Collection<AppliedPTransform<?, ?, ?>> rootTransforms =
+ ImmutableList.<AppliedPTransform<?, ?, ?>>of(
+ createdInts.getProducingTransformInternal(),
+ intsToFlatten.getProducingTransformInternal());
+
+ Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers = new HashMap<>();
+ consumers.put(
+ createdInts,
+ ImmutableList.<AppliedPTransform<?, ?, ?>>of(filtered.getProducingTransformInternal(),
+ keyed.getProducingTransformInternal(), flattened.getProducingTransformInternal()));
+ consumers.put(
+ filtered,
+ Collections.<AppliedPTransform<?, ?, ?>>singleton(
+ filteredTimesTwo.getProducingTransformInternal()));
+ consumers.put(filteredTimesTwo, Collections.<AppliedPTransform<?, ?, ?>>emptyList());
+ consumers.put(keyed, Collections.<AppliedPTransform<?, ?, ?>>emptyList());
+
+ consumers.put(
+ intsToFlatten,
+ Collections.<AppliedPTransform<?, ?, ?>>singleton(
+ flattened.getProducingTransformInternal()));
+ consumers.put(flattened, Collections.<AppliedPTransform<?, ?, ?>>emptyList());
+
+ clock = MockClock.fromInstant(new Instant(1000));
+
+ manager = InMemoryWatermarkManager.create(clock, rootTransforms, consumers);
+ bundleFactory = InProcessBundleFactory.create();
+ }
+
+ /**
+ * Demonstrates that getWatermark, when called on an {@link AppliedPTransform} that has not
+ * processed any elements, returns the {@link BoundedWindow#TIMESTAMP_MIN_VALUE}.
+ */
+ @Test
+ public void getWatermarkForUntouchedTransform() {
+ TransformWatermarks watermarks =
+ manager.getWatermarks(createdInts.getProducingTransformInternal());
+
+ assertThat(watermarks.getInputWatermark(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
+ assertThat(watermarks.getOutputWatermark(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
+ }
+
+ /**
+ * Demonstrates that getWatermark for a transform that consumes no input uses the Watermark
+ * Hold value provided to it as the output watermark.
+ */
+ @Test
+ public void getWatermarkForUpdatedSourceTransform() {
+ CommittedBundle<Integer> output = multiWindowedBundle(createdInts, 1);
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>singleton(output), new Instant(8000L));
+ TransformWatermarks updatedSourceWatermark =
+ manager.getWatermarks(createdInts.getProducingTransformInternal());
+
+ assertThat(updatedSourceWatermark.getOutputWatermark(), equalTo(new Instant(8000L)));
+ }
+
+ /**
+ * Demonstrates that getWatermark for a transform that takes multiple inputs is held to the
+ * minimum watermark across all of its inputs.
+ */
+ @Test
+ public void getWatermarkForMultiInputTransform() {
+ CommittedBundle<Integer> secondPcollectionBundle = multiWindowedBundle(intsToFlatten, -1);
+
+ manager.updateWatermarks(null, intsToFlatten.getProducingTransformInternal(),
+ TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(secondPcollectionBundle),
+ BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+ // We didn't do anything for the first source, so we shouldn't have progressed the watermark
+ TransformWatermarks firstSourceWatermark =
+ manager.getWatermarks(createdInts.getProducingTransformInternal());
+ assertThat(
+ firstSourceWatermark.getOutputWatermark(),
+ not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
+
+ // the Second Source output all of the elements so it should be done (with a watermark at the
+ // end of time).
+ TransformWatermarks secondSourceWatermark =
+ manager.getWatermarks(intsToFlatten.getProducingTransformInternal());
+ assertThat(
+ secondSourceWatermark.getOutputWatermark(),
+ not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+
+ // We haven't consumed anything yet, so our watermark should be at the beginning of time
+ TransformWatermarks transformWatermark =
+ manager.getWatermarks(flattened.getProducingTransformInternal());
+ assertThat(
+ transformWatermark.getInputWatermark(), not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
+ assertThat(
+ transformWatermark.getOutputWatermark(), not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
+
+ CommittedBundle<Integer> flattenedBundleSecondCreate = multiWindowedBundle(flattened, -1);
+ // We have finished processing the bundle from the second PCollection, but we haven't consumed
+ // anything from the first PCollection yet; so our watermark shouldn't advance
+ manager.updateWatermarks(secondPcollectionBundle, flattened.getProducingTransformInternal(),
+ TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(flattenedBundleSecondCreate),
+ null);
+ TransformWatermarks transformAfterProcessing =
+ manager.getWatermarks(flattened.getProducingTransformInternal());
+ manager.updateWatermarks(secondPcollectionBundle, flattened.getProducingTransformInternal(),
+ TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(flattenedBundleSecondCreate),
+ null);
+ assertThat(
+ transformAfterProcessing.getInputWatermark(),
+ not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
+ assertThat(
+ transformAfterProcessing.getOutputWatermark(),
+ not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
+
+ Instant firstCollectionTimestamp = new Instant(10000);
+ CommittedBundle<Integer> firstPcollectionBundle =
+ timestampedBundle(createdInts, TimestampedValue.<Integer>of(5, firstCollectionTimestamp));
+ // the source is done, but elements are still buffered. The source output watermark should be
+ // past the end of the global window
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>singleton(firstPcollectionBundle),
+ new Instant(Long.MAX_VALUE));
+ TransformWatermarks firstSourceWatermarks =
+ manager.getWatermarks(createdInts.getProducingTransformInternal());
+ assertThat(
+ firstSourceWatermarks.getOutputWatermark(),
+ not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+
+ // We still haven't consumed any of the first source's input, so the watermark should still not
+ // progress
+ TransformWatermarks flattenAfterSourcesProduced =
+ manager.getWatermarks(flattened.getProducingTransformInternal());
+ assertThat(
+ flattenAfterSourcesProduced.getInputWatermark(), not(laterThan(firstCollectionTimestamp)));
+ assertThat(
+ flattenAfterSourcesProduced.getOutputWatermark(), not(laterThan(firstCollectionTimestamp)));
+
+ // We have buffered inputs, but since the PCollection has all of the elements (has a WM past the
+ // end of the global window), we should have a watermark equal to the min among buffered
+ // elements
+ TransformWatermarks withBufferedElements =
+ manager.getWatermarks(flattened.getProducingTransformInternal());
+ assertThat(withBufferedElements.getInputWatermark(), equalTo(firstCollectionTimestamp));
+ assertThat(withBufferedElements.getOutputWatermark(), equalTo(firstCollectionTimestamp));
+
+ CommittedBundle<?> completedFlattenBundle =
+ bundleFactory.createRootBundle(flattened).commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
+ manager.updateWatermarks(firstPcollectionBundle, flattened.getProducingTransformInternal(),
+ TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(completedFlattenBundle),
+ null);
+ TransformWatermarks afterConsumingAllInput =
+ manager.getWatermarks(flattened.getProducingTransformInternal());
+ assertThat(
+ afterConsumingAllInput.getInputWatermark(),
+ not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+ assertThat(
+ afterConsumingAllInput.getOutputWatermark(),
+ not(laterThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+ }
+
+ /**
+ * Demonstrates that pending elements are independent among
+ * {@link AppliedPTransform AppliedPTransforms} that consume the same input {@link PCollection}.
+ */
+ @Test
+ public void getWatermarkForMultiConsumedCollection() {
+ CommittedBundle<Integer> createdBundle = timestampedBundle(createdInts,
+ TimestampedValue.of(1, new Instant(1_000_000L)), TimestampedValue.of(2, new Instant(1234L)),
+ TimestampedValue.of(3, new Instant(-1000L)));
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>singleton(createdBundle), new Instant(Long.MAX_VALUE));
+ TransformWatermarks createdAfterProducing =
+ manager.getWatermarks(createdInts.getProducingTransformInternal());
+ assertThat(
+ createdAfterProducing.getOutputWatermark(),
+ not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+
+ CommittedBundle<KV<String, Integer>> keyBundle =
+ timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), new Instant(1_000_000L)),
+ TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)),
+ TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L)));
+ manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(),
+ TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(keyBundle), null);
+ TransformWatermarks keyedWatermarks =
+ manager.getWatermarks(keyed.getProducingTransformInternal());
+ assertThat(
+ keyedWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+ assertThat(
+ keyedWatermarks.getOutputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+
+ TransformWatermarks filteredWatermarks =
+ manager.getWatermarks(filtered.getProducingTransformInternal());
+ assertThat(filteredWatermarks.getInputWatermark(), not(laterThan(new Instant(-1000L))));
+ assertThat(filteredWatermarks.getOutputWatermark(), not(laterThan(new Instant(-1000L))));
+
+ CommittedBundle<Integer> filteredBundle =
+ timestampedBundle(filtered, TimestampedValue.of(2, new Instant(1234L)));
+ manager.updateWatermarks(createdBundle, filtered.getProducingTransformInternal(),
+ TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(filteredBundle), null);
+ TransformWatermarks filteredProcessedWatermarks =
+ manager.getWatermarks(filtered.getProducingTransformInternal());
+ assertThat(
+ filteredProcessedWatermarks.getInputWatermark(),
+ not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+ assertThat(
+ filteredProcessedWatermarks.getOutputWatermark(),
+ not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+ }
+
+ /**
+ * Demonstrates that the watermark of an {@link AppliedPTransform} is held to the provided
+ * watermark hold.
+ */
+ @Test
+ public void updateWatermarkWithWatermarkHolds() {
+ CommittedBundle<Integer> createdBundle = timestampedBundle(createdInts,
+ TimestampedValue.of(1, new Instant(1_000_000L)), TimestampedValue.of(2, new Instant(1234L)),
+ TimestampedValue.of(3, new Instant(-1000L)));
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>singleton(createdBundle), new Instant(Long.MAX_VALUE));
+
+ CommittedBundle<KV<String, Integer>> keyBundle =
+ timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), new Instant(1_000_000L)),
+ TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)),
+ TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L)));
+ manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(),
+ TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(keyBundle),
+ new Instant(500L));
+ TransformWatermarks keyedWatermarks =
+ manager.getWatermarks(keyed.getProducingTransformInternal());
+ assertThat(
+ keyedWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+ assertThat(keyedWatermarks.getOutputWatermark(), not(laterThan(new Instant(500L))));
+ }
+
+ /**
+ * Demonstrates that the watermark of an {@link AppliedPTransform} is held to the provided
+ * watermark hold.
+ */
+ @Test
+ public void updateWatermarkWithKeyedWatermarkHolds() {
+ CommittedBundle<Integer> firstKeyBundle =
+ bundleFactory.createKeyedBundle(null, "Odd", createdInts)
+ .add(WindowedValue.timestampedValueInGlobalWindow(1, new Instant(1_000_000L)))
+ .add(WindowedValue.timestampedValueInGlobalWindow(3, new Instant(-1000L)))
+ .commit(clock.now());
+
+ CommittedBundle<Integer> secondKeyBundle =
+ bundleFactory.createKeyedBundle(null, "Even", createdInts)
+ .add(WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1234L)))
+ .commit(clock.now());
+
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ ImmutableList.of(firstKeyBundle, secondKeyBundle), BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+ manager.updateWatermarks(firstKeyBundle, filtered.getProducingTransformInternal(),
+ TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(), new Instant(-1000L));
+ manager.updateWatermarks(secondKeyBundle, filtered.getProducingTransformInternal(),
+ TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(), new Instant(1234L));
+
+ TransformWatermarks filteredWatermarks =
+ manager.getWatermarks(filtered.getProducingTransformInternal());
+ assertThat(
+ filteredWatermarks.getInputWatermark(),
+ not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+ assertThat(filteredWatermarks.getOutputWatermark(), not(laterThan(new Instant(-1000L))));
+
+ CommittedBundle<Integer> fauxFirstKeyTimerBundle =
+ bundleFactory.createKeyedBundle(null, "Odd", createdInts).commit(clock.now());
+ manager.updateWatermarks(fauxFirstKeyTimerBundle, filtered.getProducingTransformInternal(),
+ TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(),
+ BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+ assertThat(filteredWatermarks.getOutputWatermark(), equalTo(new Instant(1234L)));
+
+ CommittedBundle<Integer> fauxSecondKeyTimerBundle =
+ bundleFactory.createKeyedBundle(null, "Even", createdInts).commit(clock.now());
+ manager.updateWatermarks(fauxSecondKeyTimerBundle, filtered.getProducingTransformInternal(),
+ TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(), new Instant(5678L));
+ assertThat(filteredWatermarks.getOutputWatermark(), equalTo(new Instant(5678L)));
+
+ manager.updateWatermarks(fauxSecondKeyTimerBundle, filtered.getProducingTransformInternal(),
+ TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(),
+ BoundedWindow.TIMESTAMP_MAX_VALUE);
+ assertThat(
+ filteredWatermarks.getOutputWatermark(),
+ not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+ }
+
+ /**
+ * Demonstrates that updated output watermarks are monotonic in the presence of late data, when
+ * called on an {@link AppliedPTransform} that consumes no input.
+ */
+ @Test
+ public void updateOutputWatermarkShouldBeMonotonic() {
+ CommittedBundle<?> firstInput =
+ bundleFactory.createRootBundle(createdInts).commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>singleton(firstInput), new Instant(0L));
+ TransformWatermarks firstWatermarks =
+ manager.getWatermarks(createdInts.getProducingTransformInternal());
+ assertThat(firstWatermarks.getOutputWatermark(), equalTo(new Instant(0L)));
+
+ CommittedBundle<?> secondInput =
+ bundleFactory.createRootBundle(createdInts).commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>singleton(secondInput), new Instant(-250L));
+ TransformWatermarks secondWatermarks =
+ manager.getWatermarks(createdInts.getProducingTransformInternal());
+ assertThat(secondWatermarks.getOutputWatermark(), not(earlierThan(new Instant(0L))));
+ }
+
+ /**
+ * Demonstrates that updated output watermarks are monotonic in the presence of watermark holds
+ * that become earlier than a previous watermark hold.
+ */
+ @Test
+ public void updateWatermarkWithHoldsShouldBeMonotonic() {
+ CommittedBundle<Integer> createdBundle = timestampedBundle(createdInts,
+ TimestampedValue.of(1, new Instant(1_000_000L)), TimestampedValue.of(2, new Instant(1234L)),
+ TimestampedValue.of(3, new Instant(-1000L)));
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>singleton(createdBundle), new Instant(Long.MAX_VALUE));
+
+ CommittedBundle<KV<String, Integer>> keyBundle =
+ timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), new Instant(1_000_000L)),
+ TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)),
+ TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L)));
+ manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(),
+ TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(keyBundle),
+ new Instant(500L));
+ TransformWatermarks keyedWatermarks =
+ manager.getWatermarks(keyed.getProducingTransformInternal());
+ assertThat(
+ keyedWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+ assertThat(keyedWatermarks.getOutputWatermark(), not(laterThan(new Instant(500L))));
+ Instant oldOutputWatermark = keyedWatermarks.getOutputWatermark();
+
+ TransformWatermarks updatedWatermarks =
+ manager.getWatermarks(keyed.getProducingTransformInternal());
+ assertThat(
+ updatedWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+ // We added a hold prior to the old watermark; we shouldn't progress (due to the earlier hold)
+ // but the watermark is monotonic and should not backslide to the new, earlier hold
+ assertThat(updatedWatermarks.getOutputWatermark(), equalTo(oldOutputWatermark));
+ }
+
+ /**
+ * Demonstrates that updateWatermarks in the presence of late data is monotonic.
+ */
+ @Test
+ public void updateWatermarkWithLateData() {
+ Instant sourceWatermark = new Instant(1_000_000L);
+ CommittedBundle<Integer> createdBundle = timestampedBundle(createdInts,
+ TimestampedValue.of(1, sourceWatermark), TimestampedValue.of(2, new Instant(1234L)));
+
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>singleton(createdBundle), sourceWatermark);
+
+ CommittedBundle<KV<String, Integer>> keyBundle =
+ timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), sourceWatermark),
+ TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)));
+
+ // Finish processing the on-time data. The watermarks should progress to be equal to the source
+ manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(),
+ TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(keyBundle), null);
+ TransformWatermarks onTimeWatermarks =
+ manager.getWatermarks(keyed.getProducingTransformInternal());
+ assertThat(onTimeWatermarks.getInputWatermark(), equalTo(sourceWatermark));
+ assertThat(onTimeWatermarks.getOutputWatermark(), equalTo(sourceWatermark));
+
+ CommittedBundle<Integer> lateDataBundle =
+ timestampedBundle(createdInts, TimestampedValue.of(3, new Instant(-1000L)));
+ // the late data arrives in a downstream PCollection after its watermark has advanced past it;
+ // we don't advance the watermark past the current watermark until we've consumed the late data
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>singleton(lateDataBundle), new Instant(2_000_000L));
+ TransformWatermarks bufferedLateWm =
+ manager.getWatermarks(createdInts.getProducingTransformInternal());
+ assertThat(bufferedLateWm.getOutputWatermark(), equalTo(new Instant(2_000_000L)));
+
+ // The input watermark should be held to its previous value (not advanced due to late data; not
+ // moved backwards in the presence of watermarks due to monotonicity).
+ TransformWatermarks lateDataBufferedWatermark =
+ manager.getWatermarks(keyed.getProducingTransformInternal());
+ assertThat(lateDataBufferedWatermark.getInputWatermark(), not(earlierThan(sourceWatermark)));
+ assertThat(lateDataBufferedWatermark.getOutputWatermark(), not(earlierThan(sourceWatermark)));
+
+ CommittedBundle<KV<String, Integer>> lateKeyedBundle =
+ timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L)));
+ manager.updateWatermarks(lateDataBundle, keyed.getProducingTransformInternal(),
+ TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(lateKeyedBundle), null);
+ }
+
+ public void updateWatermarkWithDifferentWindowedValueInstances() {
+ manager.updateWatermarks(
+ null,
+ createdInts.getProducingTransformInternal(),
+ TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>singleton(
+ bundleFactory
+ .createRootBundle(createdInts)
+ .add(WindowedValue.valueInGlobalWindow(1))
+ .commit(Instant.now())),
+ BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+ manager.updateWatermarks(
+ bundleFactory
+ .createRootBundle(createdInts)
+ .add(WindowedValue.valueInGlobalWindow(1))
+ .commit(Instant.now()),
+ keyed.getProducingTransformInternal(),
+ TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>emptyList(),
+ null);
+ TransformWatermarks onTimeWatermarks =
+ manager.getWatermarks(keyed.getProducingTransformInternal());
+ assertThat(onTimeWatermarks.getInputWatermark(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
+ }
+
+ /**
+ * Demonstrates that after watermarks of an upstream transform are updated, but no output has been
+ * produced, the watermarks of a downstream process are advanced.
+ */
+ @Test
+ public void getWatermarksAfterOnlyEmptyOutput() {
+ CommittedBundle<Integer> emptyCreateOutput = multiWindowedBundle(createdInts);
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>singleton(emptyCreateOutput),
+ BoundedWindow.TIMESTAMP_MAX_VALUE);
+ TransformWatermarks updatedSourceWatermarks =
+ manager.getWatermarks(createdInts.getProducingTransformInternal());
+
+ assertThat(
+ updatedSourceWatermarks.getOutputWatermark(),
+ not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+
+ TransformWatermarks finishedFilterWatermarks =
+ manager.getWatermarks(filtered.getProducingTransformInternal());
+ assertThat(
+ finishedFilterWatermarks.getInputWatermark(),
+ not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+ assertThat(
+ finishedFilterWatermarks.getOutputWatermark(),
+ not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+ }
+
+ /**
+ * Demonstrates that after watermarks of an upstream transform are updated, but no output has been
+ * produced, and the downstream transform has a watermark hold, the watermark is held to the hold.
+ */
+ @Test
+ public void getWatermarksAfterHoldAndEmptyOutput() {
+ CommittedBundle<Integer> firstCreateOutput = multiWindowedBundle(createdInts, 1, 2);
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>singleton(firstCreateOutput), new Instant(12_000L));
+
+ CommittedBundle<Integer> firstFilterOutput = multiWindowedBundle(filtered);
+ manager.updateWatermarks(firstCreateOutput, filtered.getProducingTransformInternal(),
+ TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(firstFilterOutput),
+ new Instant(10_000L));
+ TransformWatermarks firstFilterWatermarks =
+ manager.getWatermarks(filtered.getProducingTransformInternal());
+ assertThat(firstFilterWatermarks.getInputWatermark(), not(earlierThan(new Instant(12_000L))));
+ assertThat(firstFilterWatermarks.getOutputWatermark(), not(laterThan(new Instant(10_000L))));
+
+ CommittedBundle<Integer> emptyCreateOutput = multiWindowedBundle(createdInts);
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>singleton(emptyCreateOutput),
+ BoundedWindow.TIMESTAMP_MAX_VALUE);
+ TransformWatermarks updatedSourceWatermarks =
+ manager.getWatermarks(createdInts.getProducingTransformInternal());
+
+ assertThat(
+ updatedSourceWatermarks.getOutputWatermark(),
+ not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+
+ TransformWatermarks finishedFilterWatermarks =
+ manager.getWatermarks(filtered.getProducingTransformInternal());
+ assertThat(
+ finishedFilterWatermarks.getInputWatermark(),
+ not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+ assertThat(finishedFilterWatermarks.getOutputWatermark(), not(laterThan(new Instant(10_000L))));
+ }
+
+ @Test
+ public void getSynchronizedProcessingTimeInputWatermarksHeldToPendingBundles() {
+ TransformWatermarks watermarks =
+ manager.getWatermarks(createdInts.getProducingTransformInternal());
+ assertThat(watermarks.getSynchronizedProcessingInputTime(), equalTo(clock.now()));
+ assertThat(
+ watermarks.getSynchronizedProcessingOutputTime(),
+ equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
+
+ TransformWatermarks filteredWatermarks =
+ manager.getWatermarks(filtered.getProducingTransformInternal());
+ // Non-root processing watermarks don't progress until data has been processed
+ assertThat(
+ filteredWatermarks.getSynchronizedProcessingInputTime(),
+ not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
+ assertThat(
+ filteredWatermarks.getSynchronizedProcessingOutputTime(),
+ not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
+
+ CommittedBundle<Integer> createOutput =
+ bundleFactory.createRootBundle(createdInts).commit(new Instant(1250L));
+
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>singleton(createOutput), BoundedWindow.TIMESTAMP_MAX_VALUE);
+ TransformWatermarks createAfterUpdate =
+ manager.getWatermarks(createdInts.getProducingTransformInternal());
+ assertThat(createAfterUpdate.getSynchronizedProcessingInputTime(), equalTo(clock.now()));
+ assertThat(createAfterUpdate.getSynchronizedProcessingOutputTime(), equalTo(clock.now()));
+
+ TransformWatermarks filterAfterProduced =
+ manager.getWatermarks(filtered.getProducingTransformInternal());
+ assertThat(
+ filterAfterProduced.getSynchronizedProcessingInputTime(), not(laterThan(clock.now())));
+ assertThat(
+ filterAfterProduced.getSynchronizedProcessingOutputTime(), not(laterThan(clock.now())));
+
+ clock.set(new Instant(1500L));
+ assertThat(createAfterUpdate.getSynchronizedProcessingInputTime(), equalTo(clock.now()));
+ assertThat(createAfterUpdate.getSynchronizedProcessingOutputTime(), equalTo(clock.now()));
+ assertThat(
+ filterAfterProduced.getSynchronizedProcessingInputTime(),
+ not(laterThan(new Instant(1250L))));
+ assertThat(
+ filterAfterProduced.getSynchronizedProcessingOutputTime(),
+ not(laterThan(new Instant(1250L))));
+
+ CommittedBundle<?> filterOutputBundle =
+ bundleFactory.createRootBundle(intsToFlatten).commit(new Instant(1250L));
+ manager.updateWatermarks(createOutput, filtered.getProducingTransformInternal(),
+ TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(filterOutputBundle),
+ BoundedWindow.TIMESTAMP_MAX_VALUE);
+ TransformWatermarks filterAfterConsumed =
+ manager.getWatermarks(filtered.getProducingTransformInternal());
+ assertThat(
+ filterAfterConsumed.getSynchronizedProcessingInputTime(),
+ not(laterThan(createAfterUpdate.getSynchronizedProcessingOutputTime())));
+ assertThat(
+ filterAfterConsumed.getSynchronizedProcessingOutputTime(),
+ not(laterThan(filterAfterConsumed.getSynchronizedProcessingInputTime())));
+ }
+
+ /**
+ * Demonstrates that the Synchronized Processing Time output watermark cannot progress past
+ * pending timers in the same set. This propagates to all downstream SynchronizedProcessingTimes.
+ *
+ * <p>Also demonstrate that the result is monotonic.
+ */
+ // @Test
+ public void getSynchronizedProcessingTimeOutputHeldToPendingTimers() {
+ CommittedBundle<Integer> createdBundle = multiWindowedBundle(createdInts, 1, 2, 4, 8);
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>singleton(createdBundle), new Instant(1248L));
+
+ TransformWatermarks filteredWms =
+ manager.getWatermarks(filtered.getProducingTransformInternal());
+ TransformWatermarks filteredDoubledWms =
+ manager.getWatermarks(filteredTimesTwo.getProducingTransformInternal());
+ Instant initialFilteredWm = filteredWms.getSynchronizedProcessingOutputTime();
+ Instant initialFilteredDoubledWm = filteredDoubledWms.getSynchronizedProcessingOutputTime();
+
+ CommittedBundle<Integer> filteredBundle = multiWindowedBundle(filtered, 2, 8);
+ TimerData pastTimer =
+ TimerData.of(StateNamespaces.global(), new Instant(250L), TimeDomain.PROCESSING_TIME);
+ TimerData futureTimer =
+ TimerData.of(StateNamespaces.global(), new Instant(4096L), TimeDomain.PROCESSING_TIME);
+ TimerUpdate timers =
+ TimerUpdate.builder("key").setTimer(pastTimer).setTimer(futureTimer).build();
+ manager.updateWatermarks(createdBundle, filtered.getProducingTransformInternal(), timers,
+ Collections.<CommittedBundle<?>>singleton(filteredBundle),
+ BoundedWindow.TIMESTAMP_MAX_VALUE);
+ Instant startTime = clock.now();
+ clock.set(startTime.plus(250L));
+ // We're held based on the past timer
+ assertThat(filteredWms.getSynchronizedProcessingOutputTime(), not(laterThan(startTime)));
+ assertThat(filteredDoubledWms.getSynchronizedProcessingOutputTime(), not(laterThan(startTime)));
+ // And we're monotonic
+ assertThat(
+ filteredWms.getSynchronizedProcessingOutputTime(), not(earlierThan(initialFilteredWm)));
+ assertThat(
+ filteredDoubledWms.getSynchronizedProcessingOutputTime(),
+ not(earlierThan(initialFilteredDoubledWm)));
+
+ Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> firedTimers =
+ manager.extractFiredTimers();
+ assertThat(
+ firedTimers.get(filtered.getProducingTransformInternal())
+ .get("key")
+ .getTimers(TimeDomain.PROCESSING_TIME),
+ contains(pastTimer));
+ // Our timer has fired, but has not been completed, so it holds our synchronized processing WM
+ assertThat(filteredWms.getSynchronizedProcessingOutputTime(), not(laterThan(startTime)));
+ assertThat(filteredDoubledWms.getSynchronizedProcessingOutputTime(), not(laterThan(startTime)));
+
+ CommittedBundle<Integer> filteredTimerBundle =
+ bundleFactory
+ .createKeyedBundle(null, "key", filtered)
+ .commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
+ CommittedBundle<Integer> filteredTimerResult =
+ bundleFactory.createKeyedBundle(null, "key", filteredTimesTwo)
+ .commit(filteredWms.getSynchronizedProcessingOutputTime());
+ // Complete the processing time timer
+ manager.updateWatermarks(filteredTimerBundle, filtered.getProducingTransformInternal(),
+ TimerUpdate.builder("key")
+ .withCompletedTimers(Collections.<TimerData>singleton(pastTimer))
+ .build(),
+ Collections.<CommittedBundle<?>>singleton(filteredTimerResult),
+ BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+ clock.set(startTime.plus(500L));
+ assertThat(filteredWms.getSynchronizedProcessingOutputTime(), not(laterThan(clock.now())));
+ // filtered should be held to the time at which the filteredTimerResult fired
+ assertThat(
+ filteredDoubledWms.getSynchronizedProcessingOutputTime(),
+ not(earlierThan(filteredTimerResult.getSynchronizedProcessingOutputWatermark())));
+
+ manager.updateWatermarks(filteredTimerResult, filteredTimesTwo.getProducingTransformInternal(),
+ TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(),
+ BoundedWindow.TIMESTAMP_MAX_VALUE);
+ assertThat(filteredDoubledWms.getSynchronizedProcessingOutputTime(), equalTo(clock.now()));
+
+ clock.set(new Instant(Long.MAX_VALUE));
+ assertThat(filteredWms.getSynchronizedProcessingOutputTime(), equalTo(new Instant(4096)));
+ assertThat(
+ filteredDoubledWms.getSynchronizedProcessingOutputTime(), equalTo(new Instant(4096)));
+ }
+
+ /**
+ * Demonstrates that if any earlier processing holds appear in the synchronized processing time
+ * output hold the result is monotonic.
+ */
+ @Test
+ public void getSynchronizedProcessingTimeOutputTimeIsMonotonic() {
+ Instant startTime = clock.now();
+ TransformWatermarks watermarks =
+ manager.getWatermarks(createdInts.getProducingTransformInternal());
+ assertThat(watermarks.getSynchronizedProcessingInputTime(), equalTo(startTime));
+
+ TransformWatermarks filteredWatermarks =
+ manager.getWatermarks(filtered.getProducingTransformInternal());
+ // Non-root processing watermarks don't progress until data has been processed
+ assertThat(
+ filteredWatermarks.getSynchronizedProcessingInputTime(),
+ not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
+ assertThat(
+ filteredWatermarks.getSynchronizedProcessingOutputTime(),
+ not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
+
+ CommittedBundle<Integer> createOutput =
+ bundleFactory.createRootBundle(createdInts).commit(new Instant(1250L));
+
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>singleton(createOutput), BoundedWindow.TIMESTAMP_MAX_VALUE);
+ TransformWatermarks createAfterUpdate =
+ manager.getWatermarks(createdInts.getProducingTransformInternal());
+ assertThat(createAfterUpdate.getSynchronizedProcessingInputTime(), not(laterThan(clock.now())));
+ assertThat(
+ createAfterUpdate.getSynchronizedProcessingOutputTime(), not(laterThan(clock.now())));
+
+ CommittedBundle<Integer> createSecondOutput =
+ bundleFactory.createRootBundle(createdInts).commit(new Instant(750L));
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>singleton(createSecondOutput),
+ BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+ assertThat(createAfterUpdate.getSynchronizedProcessingOutputTime(), equalTo(clock.now()));
+ }
+
+ @Test
+ public void synchronizedProcessingInputTimeIsHeldToUpstreamProcessingTimeTimers() {
+ CommittedBundle<Integer> created = multiWindowedBundle(createdInts, 1, 2, 3);
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>singleton(created), new Instant(40_900L));
+
+ CommittedBundle<Integer> filteredBundle = multiWindowedBundle(filtered, 2, 4);
+ Instant upstreamHold = new Instant(2048L);
+ TimerData upstreamProcessingTimer =
+ TimerData.of(StateNamespaces.global(), upstreamHold, TimeDomain.PROCESSING_TIME);
+ manager.updateWatermarks(created, filtered.getProducingTransformInternal(),
+ TimerUpdate.builder("key").setTimer(upstreamProcessingTimer).build(),
+ Collections.<CommittedBundle<?>>singleton(filteredBundle),
+ BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+ TransformWatermarks downstreamWms =
+ manager.getWatermarks(filteredTimesTwo.getProducingTransformInternal());
+ assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(clock.now()));
+
+ clock.set(BoundedWindow.TIMESTAMP_MAX_VALUE);
+ assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(upstreamHold));
+
+ manager.extractFiredTimers();
+ // Pending processing time timers that have been fired but aren't completed hold the
+ // synchronized processing time
+ assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(upstreamHold));
+
+ CommittedBundle<Integer> otherCreated = multiWindowedBundle(createdInts, 4, 8, 12);
+ manager.updateWatermarks(otherCreated, filtered.getProducingTransformInternal(),
+ TimerUpdate.builder("key")
+ .withCompletedTimers(Collections.singleton(upstreamProcessingTimer))
+ .build(),
+ Collections.<CommittedBundle<?>>emptyList(), BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+ assertThat(downstreamWms.getSynchronizedProcessingInputTime(), not(earlierThan(clock.now())));
+ }
+
+ @Test
+ public void synchronizedProcessingInputTimeIsHeldToPendingBundleTimes() {
+ CommittedBundle<Integer> created = multiWindowedBundle(createdInts, 1, 2, 3);
+ manager.updateWatermarks(
+ null,
+ createdInts.getProducingTransformInternal(),
+ TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>singleton(created),
+ new Instant(29_919_235L));
+
+ Instant upstreamHold = new Instant(2048L);
+ CommittedBundle<Integer> filteredBundle =
+ bundleFactory.createKeyedBundle(created, "key", filtered).commit(upstreamHold);
+ manager.updateWatermarks(
+ created,
+ filtered.getProducingTransformInternal(),
+ TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>singleton(filteredBundle),
+ BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+ TransformWatermarks downstreamWms =
+ manager.getWatermarks(filteredTimesTwo.getProducingTransformInternal());
+ assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(clock.now()));
+
+ clock.set(BoundedWindow.TIMESTAMP_MAX_VALUE);
+ assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(upstreamHold));
+ }
+
+ @Test
+ public void extractFiredTimersReturnsFiredEventTimeTimers() {
+ Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> initialTimers =
+ manager.extractFiredTimers();
+ // Watermarks haven't advanced
+ assertThat(initialTimers.entrySet(), emptyIterable());
+
+ // Advance WM of keyed past the first timer, but ahead of the second and third
+ CommittedBundle<Integer> createdBundle = multiWindowedBundle(filtered);
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.singleton(createdBundle), new Instant(1500L));
+
+ TimerData earliestTimer =
+ TimerData.of(StateNamespaces.global(), new Instant(1000), TimeDomain.EVENT_TIME);
+ TimerData middleTimer =
+ TimerData.of(StateNamespaces.global(), new Instant(5000L), TimeDomain.EVENT_TIME);
+ TimerData lastTimer =
+ TimerData.of(StateNamespaces.global(), new Instant(10000L), TimeDomain.EVENT_TIME);
+ Object key = new Object();
+ TimerUpdate update =
+ TimerUpdate.builder(key)
+ .setTimer(earliestTimer)
+ .setTimer(middleTimer)
+ .setTimer(lastTimer)
+ .build();
+
+ manager.updateWatermarks(
+ createdBundle,
+ filtered.getProducingTransformInternal(),
+ update,
+ Collections.<CommittedBundle<?>>singleton(multiWindowedBundle(intsToFlatten)),
+ new Instant(1000L));
+
+ Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> firstTransformFiredTimers =
+ manager.extractFiredTimers();
+ assertThat(
+ firstTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
+ Map<Object, FiredTimers> firstFilteredTimers =
+ firstTransformFiredTimers.get(filtered.getProducingTransformInternal());
+ assertThat(firstFilteredTimers.get(key), not(nullValue()));
+ FiredTimers firstFired = firstFilteredTimers.get(key);
+ assertThat(firstFired.getTimers(TimeDomain.EVENT_TIME), contains(earliestTimer));
+
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>emptyList(), new Instant(50_000L));
+ Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> secondTransformFiredTimers =
+ manager.extractFiredTimers();
+ assertThat(
+ secondTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
+ Map<Object, FiredTimers> secondFilteredTimers =
+ secondTransformFiredTimers.get(filtered.getProducingTransformInternal());
+ assertThat(secondFilteredTimers.get(key), not(nullValue()));
+ FiredTimers secondFired = secondFilteredTimers.get(key);
+ // Contains, in order, middleTimer and then lastTimer
+ assertThat(secondFired.getTimers(TimeDomain.EVENT_TIME), contains(middleTimer, lastTimer));
+ }
+
+ @Test
+ public void extractFiredTimersReturnsFiredProcessingTimeTimers() {
+ Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> initialTimers =
+ manager.extractFiredTimers();
+ // Watermarks haven't advanced
+ assertThat(initialTimers.entrySet(), emptyIterable());
+
+ // Advance WM of keyed past the first timer, but ahead of the second and third
+ CommittedBundle<Integer> createdBundle = multiWindowedBundle(filtered);
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.singleton(createdBundle), new Instant(1500L));
+
+ TimerData earliestTimer =
+ TimerData.of(StateNamespaces.global(), new Instant(999L), TimeDomain.PROCESSING_TIME);
+ TimerData middleTimer =
+ TimerData.of(StateNamespaces.global(), new Instant(5000L), TimeDomain.PROCESSING_TIME);
+ TimerData lastTimer =
+ TimerData.of(StateNamespaces.global(), new Instant(10000L), TimeDomain.PROCESSING_TIME);
+ Object key = new Object();
+ TimerUpdate update =
+ TimerUpdate.builder(key)
+ .setTimer(lastTimer)
+ .setTimer(earliestTimer)
+ .setTimer(middleTimer)
+ .build();
+
+ manager.updateWatermarks(
+ createdBundle,
+ filtered.getProducingTransformInternal(),
+ update,
+ Collections.<CommittedBundle<?>>singleton(multiWindowedBundle(intsToFlatten)),
+ new Instant(1000L));
+
+ Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> firstTransformFiredTimers =
+ manager.extractFiredTimers();
+ assertThat(
+ firstTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
+ Map<Object, FiredTimers> firstFilteredTimers =
+ firstTransformFiredTimers.get(filtered.getProducingTransformInternal());
+ assertThat(firstFilteredTimers.get(key), not(nullValue()));
+ FiredTimers firstFired = firstFilteredTimers.get(key);
+ assertThat(firstFired.getTimers(TimeDomain.PROCESSING_TIME), contains(earliestTimer));
+
+ clock.set(new Instant(50_000L));
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>emptyList(), new Instant(50_000L));
+ Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> secondTransformFiredTimers =
+ manager.extractFiredTimers();
+ assertThat(
+ secondTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
+ Map<Object, FiredTimers> secondFilteredTimers =
+ secondTransformFiredTimers.get(filtered.getProducingTransformInternal());
+ assertThat(secondFilteredTimers.get(key), not(nullValue()));
+ FiredTimers secondFired = secondFilteredTimers.get(key);
+ // Contains, in order, middleTimer and then lastTimer
+ assertThat(secondFired.getTimers(TimeDomain.PROCESSING_TIME), contains(middleTimer, lastTimer));
+ }
+
+ @Test
+ public void extractFiredTimersReturnsFiredSynchronizedProcessingTimeTimers() {
+ Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> initialTimers =
+ manager.extractFiredTimers();
+ // Watermarks haven't advanced
+ assertThat(initialTimers.entrySet(), emptyIterable());
+
+ // Advance WM of keyed past the first timer, but ahead of the second and third
+ CommittedBundle<Integer> createdBundle = multiWindowedBundle(filtered);
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.singleton(createdBundle), new Instant(1500L));
+
+ TimerData earliestTimer = TimerData.of(
+ StateNamespaces.global(), new Instant(999L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+ TimerData middleTimer = TimerData.of(
+ StateNamespaces.global(), new Instant(5000L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+ TimerData lastTimer = TimerData.of(
+ StateNamespaces.global(), new Instant(10000L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+ Object key = new Object();
+ TimerUpdate update =
+ TimerUpdate.builder(key)
+ .setTimer(lastTimer)
+ .setTimer(earliestTimer)
+ .setTimer(middleTimer)
+ .build();
+
+ manager.updateWatermarks(
+ createdBundle,
+ filtered.getProducingTransformInternal(),
+ update,
+ Collections.<CommittedBundle<?>>singleton(multiWindowedBundle(intsToFlatten)),
+ new Instant(1000L));
+
+ Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> firstTransformFiredTimers =
+ manager.extractFiredTimers();
+ assertThat(
+ firstTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
+ Map<Object, FiredTimers> firstFilteredTimers =
+ firstTransformFiredTimers.get(filtered.getProducingTransformInternal());
+ assertThat(firstFilteredTimers.get(key), not(nullValue()));
+ FiredTimers firstFired = firstFilteredTimers.get(key);
+ assertThat(
+ firstFired.getTimers(TimeDomain.SYNCHRONIZED_PROCESSING_TIME), contains(earliestTimer));
+
+ clock.set(new Instant(50_000L));
+ manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+ Collections.<CommittedBundle<?>>emptyList(), new Instant(50_000L));
+ Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> secondTransformFiredTimers =
+ manager.extractFiredTimers();
+ assertThat(
+ secondTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
+ Map<Object, FiredTimers> secondFilteredTimers =
+ secondTransformFiredTimers.get(filtered.getProducingTransformInternal());
+ assertThat(secondFilteredTimers.get(key), not(nullValue()));
+ FiredTimers secondFired = secondFilteredTimers.get(key);
+ // Contains, in order, middleTimer and then lastTimer
+ assertThat(
+ secondFired.getTimers(TimeDomain.SYNCHRONIZED_PROCESSING_TIME),
+ contains(middleTimer, lastTimer));
+ }
+
+ @Test
+ public void timerUpdateBuilderBuildAddsAllAddedTimers() {
+ TimerData set = TimerData.of(StateNamespaces.global(), new Instant(10L), TimeDomain.EVENT_TIME);
+ TimerData deleted =
+ TimerData.of(StateNamespaces.global(), new Instant(24L), TimeDomain.PROCESSING_TIME);
+ TimerData completedOne = TimerData.of(
+ StateNamespaces.global(), new Instant(1024L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+ TimerData completedTwo =
+ TimerData.of(StateNamespaces.global(), new Instant(2048L), TimeDomain.EVENT_TIME);
+
+ TimerUpdate update =
+ TimerUpdate.builder("foo")
+ .withCompletedTimers(ImmutableList.of(completedOne, completedTwo))
+ .setTimer(set)
+ .deletedTimer(deleted)
+ .build();
+
+ assertThat(update.getCompletedTimers(), containsInAnyOrder(completedOne, completedTwo));
+ assertThat(update.getSetTimers(), contains(set));
+ assertThat(update.getDeletedTimers(), contains(deleted));
+ }
+
+ @Test
+ public void timerUpdateBuilderWithSetThenDeleteHasOnlyDeleted() {
+ TimerUpdateBuilder builder = TimerUpdate.builder(null);
+ TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
+
+ TimerUpdate built = builder.setTimer(timer).deletedTimer(timer).build();
+
+ assertThat(built.getSetTimers(), emptyIterable());
+ assertThat(built.getDeletedTimers(), contains(timer));
+ }
+
+ @Test
+ public void timerUpdateBuilderWithDeleteThenSetHasOnlySet() {
+ TimerUpdateBuilder builder = TimerUpdate.builder(null);
+ TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
+
+ TimerUpdate built = builder.deletedTimer(timer).setTimer(timer).build();
+
+ assertThat(built.getSetTimers(), contains(timer));
+ assertThat(built.getDeletedTimers(), emptyIterable());
+ }
+
+ @Test
+ public void timerUpdateBuilderWithSetAfterBuildNotAddedToBuilt() {
+ TimerUpdateBuilder builder = TimerUpdate.builder(null);
+ TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
+
+ TimerUpdate built = builder.build();
+ builder.setTimer(timer);
+ assertThat(built.getSetTimers(), emptyIterable());
+ builder.build();
+ assertThat(built.getSetTimers(), emptyIterable());
+ }
+
+ @Test
+ public void timerUpdateBuilderWithDeleteAfterBuildNotAddedToBuilt() {
+ TimerUpdateBuilder builder = TimerUpdate.builder(null);
+ TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
+
+ TimerUpdate built = builder.build();
+ builder.deletedTimer(timer);
+ assertThat(built.getDeletedTimers(), emptyIterable());
+ builder.build();
+ assertThat(built.getDeletedTimers(), emptyIterable());
+ }
+
+ @Test
+ public void timerUpdateBuilderWithCompletedAfterBuildNotAddedToBuilt() {
+ TimerUpdateBuilder builder = TimerUpdate.builder(null);
+ TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
+
+ TimerUpdate built = builder.build();
+ builder.withCompletedTimers(ImmutableList.of(timer));
+ assertThat(built.getCompletedTimers(), emptyIterable());
+ builder.build();
+ assertThat(built.getCompletedTimers(), emptyIterable());
+ }
+
+ @Test
+ public void timerUpdateWithCompletedTimersNotAddedToExisting() {
+ TimerUpdateBuilder builder = TimerUpdate.builder(null);
+ TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
+
+ TimerUpdate built = builder.build();
+ assertThat(built.getCompletedTimers(), emptyIterable());
+ assertThat(
+ built.withCompletedTimers(ImmutableList.of(timer)).getCompletedTimers(), contains(timer));
+ assertThat(built.getCompletedTimers(), emptyIterable());
+ }
+
+ private static Matcher<Instant> earlierThan(final Instant laterInstant) {
+ return new BaseMatcher<Instant>() {
+ @Override
+ public boolean matches(Object item) {
+ ReadableInstant instant = (ReadableInstant) item;
+ return instant.isBefore(laterInstant);
+ }
+
+ @Override
+ public void describeTo(Description description) {
+ description.appendText("earlier than ").appendValue(laterInstant);
+ }
+ };
+ }
+
+ private static Matcher<Instant> laterThan(final Instant shouldBeEarlier) {
+ return new BaseMatcher<Instant>() {
+ @Override
+ public boolean matches(Object item) {
+ ReadableInstant instant = (ReadableInstant) item;
+ return instant.isAfter(shouldBeEarlier);
+ }
+
+ @Override
+ public void describeTo(Description description) {
+ description.appendText("later than ").appendValue(shouldBeEarlier);
+ }
+ };
+ }
+
+ @SafeVarargs
+ private final <T> CommittedBundle<T> timestampedBundle(
+ PCollection<T> pc, TimestampedValue<T>... values) {
+ UncommittedBundle<T> bundle = bundleFactory.createRootBundle(pc);
+ for (TimestampedValue<T> value : values) {
+ bundle.add(
+ WindowedValue.timestampedValueInGlobalWindow(value.getValue(), value.getTimestamp()));
+ }
+ return bundle.commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
+ }
+
+ @SafeVarargs
+ private final <T> CommittedBundle<T> multiWindowedBundle(PCollection<T> pc, T... values) {
+ UncommittedBundle<T> bundle = bundleFactory.createRootBundle(pc);
+ Collection<BoundedWindow> windows =
+ ImmutableList.of(
+ GlobalWindow.INSTANCE,
+ new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(0)));
+ for (T value : values) {
+ bundle.add(
+ WindowedValue.of(value, BoundedWindow.TIMESTAMP_MIN_VALUE, windows, PaneInfo.NO_FIRING));
+ }
+ return bundle.commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessBundleFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessBundleFactoryTest.java
new file mode 100644
index 0000000..1809dc6
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessBundleFactoryTest.java
@@ -0,0 +1,223 @@
+/*
+ * 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.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.common.collect.ImmutableList;
+
+import org.hamcrest.Matcher;
+import org.hamcrest.Matchers;
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * Tests for {@link InProcessBundleFactory}.
+ */
+@RunWith(JUnit4.class)
+public class InProcessBundleFactoryTest {
+ @Rule public ExpectedException thrown = ExpectedException.none();
+
+ private InProcessBundleFactory bundleFactory = InProcessBundleFactory.create();
+
+ private PCollection<Integer> created;
+ private PCollection<KV<String, Integer>> downstream;
+
+ @Before
+ public void setup() {
+ TestPipeline p = TestPipeline.create();
+ created = p.apply(Create.of(1, 2, 3));
+ downstream = created.apply(WithKeys.<String, Integer>of("foo"));
+ }
+
+ @Test
+ public void createRootBundleShouldCreateWithNullKey() {
+ PCollection<Integer> pcollection = TestPipeline.create().apply(Create.of(1));
+
+ UncommittedBundle<Integer> inFlightBundle = bundleFactory.createRootBundle(pcollection);
+
+ CommittedBundle<Integer> bundle = inFlightBundle.commit(Instant.now());
+
+ assertThat(bundle.getKey(), nullValue());
+ }
+
+ private void createKeyedBundle(Object key) {
+ PCollection<Integer> pcollection = TestPipeline.create().apply(Create.of(1));
+
+ UncommittedBundle<Integer> inFlightBundle =
+ bundleFactory.createKeyedBundle(null, key, pcollection);
+
+ CommittedBundle<Integer> bundle = inFlightBundle.commit(Instant.now());
+ assertThat(bundle.getKey(), equalTo(key));
+ }
+
+ @Test
+ public void keyedWithNullKeyShouldCreateKeyedBundle() {
+ createKeyedBundle(null);
+ }
+
+ @Test
+ public void keyedWithKeyShouldCreateKeyedBundle() {
+ createKeyedBundle(new Object());
+ }
+
+ private <T> CommittedBundle<T>
+ afterCommitGetElementsShouldHaveAddedElements(Iterable<WindowedValue<T>> elems) {
+ PCollection<T> pcollection = TestPipeline.create().apply(Create.<T>of());
+
+ UncommittedBundle<T> bundle = bundleFactory.createRootBundle(pcollection);
+ Collection<Matcher<? super WindowedValue<T>>> expectations = new ArrayList<>();
+ for (WindowedValue<T> elem : elems) {
+ bundle.add(elem);
+ expectations.add(equalTo(elem));
+ }
+ Matcher<Iterable<? extends WindowedValue<T>>> containsMatcher =
+ Matchers.<WindowedValue<T>>containsInAnyOrder(expectations);
+ CommittedBundle<T> committed = bundle.commit(Instant.now());
+ assertThat(committed.getElements(), containsMatcher);
+
+ return committed;
+ }
+
+ @Test
+ public void getElementsBeforeAddShouldReturnEmptyIterable() {
+ afterCommitGetElementsShouldHaveAddedElements(Collections.<WindowedValue<Integer>>emptyList());
+ }
+
+ @Test
+ public void getElementsAfterAddShouldReturnAddedElements() {
+ WindowedValue<Integer> firstValue = WindowedValue.valueInGlobalWindow(1);
+ WindowedValue<Integer> secondValue =
+ WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1000L));
+
+ afterCommitGetElementsShouldHaveAddedElements(ImmutableList.of(firstValue, secondValue));
+ }
+
+ @SuppressWarnings("unchecked")
+ @Test
+ public void withElementsShouldReturnIndependentBundle() {
+ WindowedValue<Integer> firstValue = WindowedValue.valueInGlobalWindow(1);
+ WindowedValue<Integer> secondValue =
+ WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1000L));
+
+ CommittedBundle<Integer> committed =
+ afterCommitGetElementsShouldHaveAddedElements(ImmutableList.of(firstValue, secondValue));
+
+ WindowedValue<Integer> firstReplacement =
+ WindowedValue.of(
+ 9,
+ new Instant(2048L),
+ new IntervalWindow(new Instant(2044L), Instant.now()),
+ PaneInfo.NO_FIRING);
+ WindowedValue<Integer> secondReplacement =
+ WindowedValue.timestampedValueInGlobalWindow(-1, Instant.now());
+ CommittedBundle<Integer> withed =
+ committed.withElements(ImmutableList.of(firstReplacement, secondReplacement));
+
+ assertThat(withed.getElements(), containsInAnyOrder(firstReplacement, secondReplacement));
+ assertThat(committed.getElements(), containsInAnyOrder(firstValue, secondValue));
+ assertThat(withed.getKey(), equalTo(committed.getKey()));
+ assertThat(withed.getPCollection(), equalTo(committed.getPCollection()));
+ assertThat(
+ withed.getSynchronizedProcessingOutputWatermark(),
+ equalTo(committed.getSynchronizedProcessingOutputWatermark()));
+ }
+
+ @Test
+ public void addAfterCommitShouldThrowException() {
+ PCollection<Integer> pcollection = TestPipeline.create().apply(Create.<Integer>of());
+
+ UncommittedBundle<Integer> bundle = bundleFactory.createRootBundle(pcollection);
+ bundle.add(WindowedValue.valueInGlobalWindow(1));
+ CommittedBundle<Integer> firstCommit = bundle.commit(Instant.now());
+ assertThat(firstCommit.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(1)));
+
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage("3");
+ thrown.expectMessage("committed");
+
+ bundle.add(WindowedValue.valueInGlobalWindow(3));
+ }
+
+ @Test
+ public void commitAfterCommitShouldThrowException() {
+ PCollection<Integer> pcollection = TestPipeline.create().apply(Create.<Integer>of());
+
+ UncommittedBundle<Integer> bundle = bundleFactory.createRootBundle(pcollection);
+ bundle.add(WindowedValue.valueInGlobalWindow(1));
+ CommittedBundle<Integer> firstCommit = bundle.commit(Instant.now());
+ assertThat(firstCommit.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(1)));
+
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage("committed");
+
+ bundle.commit(Instant.now());
+ }
+
+ @Test
+ public void createBundleUnkeyedResultUnkeyed() {
+ CommittedBundle<KV<String, Integer>> newBundle =
+ bundleFactory
+ .createBundle(bundleFactory.createRootBundle(created).commit(Instant.now()), downstream)
+ .commit(Instant.now());
+ }
+
+ @Test
+ public void createBundleKeyedResultPropagatesKey() {
+ CommittedBundle<KV<String, Integer>> newBundle =
+ bundleFactory
+ .createBundle(
+ bundleFactory.createKeyedBundle(null, "foo", created).commit(Instant.now()),
+ downstream)
+ .commit(Instant.now());
+ assertThat(newBundle.getKey(), Matchers.<Object>equalTo("foo"));
+ }
+
+ @Test
+ public void createKeyedBundleKeyed() {
+ CommittedBundle<KV<String, Integer>> keyedBundle =
+ bundleFactory
+ .createKeyedBundle(
+ bundleFactory.createRootBundle(created).commit(Instant.now()), "foo", downstream)
+ .commit(Instant.now());
+ assertThat(keyedBundle.getKey(), Matchers.<Object>equalTo("foo"));
+ }
+}
[14/17] incubator-beam git commit: Move InProcessRunner to its own
module
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessBundleOutputManager.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessBundleOutputManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessBundleOutputManager.java
new file mode 100644
index 0000000..f374f99
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessBundleOutputManager.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.util.DoFnRunners.OutputManager;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.TupleTag;
+
+import java.util.Map;
+
+/**
+ * An {@link OutputManager} that outputs to {@link CommittedBundle Bundles} used by the
+ * {@link InProcessPipelineRunner}.
+ */
+public class InProcessBundleOutputManager implements OutputManager {
+ private final Map<TupleTag<?>, UncommittedBundle<?>> bundles;
+
+ public static InProcessBundleOutputManager create(
+ Map<TupleTag<?>, UncommittedBundle<?>> outputBundles) {
+ return new InProcessBundleOutputManager(outputBundles);
+ }
+
+ public InProcessBundleOutputManager(Map<TupleTag<?>, UncommittedBundle<?>> bundles) {
+ this.bundles = bundles;
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
+ @SuppressWarnings("rawtypes")
+ UncommittedBundle bundle = bundles.get(tag);
+ bundle.add(output);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessEvaluationContext.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessEvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessEvaluationContext.java
new file mode 100644
index 0000000..d9a7ff0
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessEvaluationContext.java
@@ -0,0 +1,425 @@
+/*
+ * 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 com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.beam.runners.direct.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.FiredTimers;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TransformWatermarks;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.PCollectionViewWriter;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.util.ExecutionContext;
+import org.apache.beam.sdk.util.SideInputReader;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PValue;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import javax.annotation.Nullable;
+
+/**
+ * The evaluation context for a specific pipeline being executed by the
+ * {@link InProcessPipelineRunner}. Contains state shared within the execution across all
+ * transforms.
+ *
+ * <p>{@link InProcessEvaluationContext} contains shared state for an execution of the
+ * {@link InProcessPipelineRunner} that can be used while evaluating a {@link PTransform}. This
+ * consists of views into underlying state and watermark implementations, access to read and write
+ * {@link PCollectionView PCollectionViews}, and constructing {@link CounterSet CounterSets} and
+ * {@link ExecutionContext ExecutionContexts}. This includes executing callbacks asynchronously when
+ * state changes to the appropriate point (e.g. when a {@link PCollectionView} is requested and
+ * known to be empty).
+ *
+ * <p>{@link InProcessEvaluationContext} also handles results by committing finalizing bundles based
+ * on the current global state and updating the global state appropriately. This includes updating
+ * the per-{@link StepAndKey} state, updating global watermarks, and executing any callbacks that
+ * can be executed.
+ */
+class InProcessEvaluationContext {
+ /** The step name for each {@link AppliedPTransform} in the {@link Pipeline}. */
+ private final Map<AppliedPTransform<?, ?, ?>, String> stepNames;
+
+ /** The options that were used to create this {@link Pipeline}. */
+ private final InProcessPipelineOptions options;
+
+ private final BundleFactory bundleFactory;
+ /** The current processing time and event time watermarks and timers. */
+ private final InMemoryWatermarkManager watermarkManager;
+
+ /** Executes callbacks based on the progression of the watermark. */
+ private final WatermarkCallbackExecutor callbackExecutor;
+
+ /** The stateInternals of the world, by applied PTransform and key. */
+ private final ConcurrentMap<StepAndKey, CopyOnAccessInMemoryStateInternals<?>>
+ applicationStateInternals;
+
+ private final InProcessSideInputContainer sideInputContainer;
+
+ private final CounterSet mergedCounters;
+
+ public static InProcessEvaluationContext create(
+ InProcessPipelineOptions options,
+ BundleFactory bundleFactory,
+ Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
+ Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
+ Map<AppliedPTransform<?, ?, ?>, String> stepNames,
+ Collection<PCollectionView<?>> views) {
+ return new InProcessEvaluationContext(
+ options, bundleFactory, rootTransforms, valueToConsumers, stepNames, views);
+ }
+
+ private InProcessEvaluationContext(
+ InProcessPipelineOptions options,
+ BundleFactory bundleFactory,
+ Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
+ Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
+ Map<AppliedPTransform<?, ?, ?>, String> stepNames,
+ Collection<PCollectionView<?>> views) {
+ this.options = checkNotNull(options);
+ this.bundleFactory = checkNotNull(bundleFactory);
+ checkNotNull(rootTransforms);
+ checkNotNull(valueToConsumers);
+ checkNotNull(stepNames);
+ checkNotNull(views);
+ this.stepNames = stepNames;
+
+ this.watermarkManager =
+ InMemoryWatermarkManager.create(
+ NanosOffsetClock.create(), rootTransforms, valueToConsumers);
+ this.sideInputContainer = InProcessSideInputContainer.create(this, views);
+
+ this.applicationStateInternals = new ConcurrentHashMap<>();
+ this.mergedCounters = new CounterSet();
+
+ this.callbackExecutor = WatermarkCallbackExecutor.create();
+ }
+
+ /**
+ * Handle the provided {@link InProcessTransformResult}, produced after evaluating the provided
+ * {@link CommittedBundle} (potentially null, if the result of a root {@link PTransform}).
+ *
+ * <p>The result is the output of running the transform contained in the
+ * {@link InProcessTransformResult} on the contents of the provided bundle.
+ *
+ * @param completedBundle the bundle that was processed to produce the result. Potentially
+ * {@code null} if the transform that produced the result is a root
+ * transform
+ * @param completedTimers the timers that were delivered to produce the {@code completedBundle},
+ * or an empty iterable if no timers were delivered
+ * @param result the result of evaluating the input bundle
+ * @return the committed bundles contained within the handled {@code result}
+ */
+ public synchronized CommittedResult handleResult(
+ @Nullable CommittedBundle<?> completedBundle,
+ Iterable<TimerData> completedTimers,
+ InProcessTransformResult result) {
+ Iterable<? extends CommittedBundle<?>> committedBundles =
+ commitBundles(result.getOutputBundles());
+ // Update watermarks and timers
+ watermarkManager.updateWatermarks(
+ completedBundle,
+ result.getTransform(),
+ result.getTimerUpdate().withCompletedTimers(completedTimers),
+ committedBundles,
+ result.getWatermarkHold());
+ fireAllAvailableCallbacks();
+ // Update counters
+ if (result.getCounters() != null) {
+ mergedCounters.merge(result.getCounters());
+ }
+ // Update state internals
+ CopyOnAccessInMemoryStateInternals<?> theirState = result.getState();
+ if (theirState != null) {
+ CopyOnAccessInMemoryStateInternals<?> committedState = theirState.commit();
+ StepAndKey stepAndKey =
+ StepAndKey.of(
+ result.getTransform(), completedBundle == null ? null : completedBundle.getKey());
+ if (!committedState.isEmpty()) {
+ applicationStateInternals.put(stepAndKey, committedState);
+ } else {
+ applicationStateInternals.remove(stepAndKey);
+ }
+ }
+ return CommittedResult.create(result, committedBundles);
+ }
+
+ private Iterable<? extends CommittedBundle<?>> commitBundles(
+ Iterable<? extends UncommittedBundle<?>> bundles) {
+ ImmutableList.Builder<CommittedBundle<?>> completed = ImmutableList.builder();
+ for (UncommittedBundle<?> inProgress : bundles) {
+ AppliedPTransform<?, ?, ?> producing =
+ inProgress.getPCollection().getProducingTransformInternal();
+ TransformWatermarks watermarks = watermarkManager.getWatermarks(producing);
+ CommittedBundle<?> committed =
+ inProgress.commit(watermarks.getSynchronizedProcessingOutputTime());
+ // Empty bundles don't impact watermarks and shouldn't trigger downstream execution, so
+ // filter them out
+ if (!Iterables.isEmpty(committed.getElements())) {
+ completed.add(committed);
+ }
+ }
+ return completed.build();
+ }
+
+ private void fireAllAvailableCallbacks() {
+ for (AppliedPTransform<?, ?, ?> transform : stepNames.keySet()) {
+ fireAvailableCallbacks(transform);
+ }
+ }
+
+ private void fireAvailableCallbacks(AppliedPTransform<?, ?, ?> producingTransform) {
+ TransformWatermarks watermarks = watermarkManager.getWatermarks(producingTransform);
+ callbackExecutor.fireForWatermark(producingTransform, watermarks.getOutputWatermark());
+ }
+
+ /**
+ * Create a {@link UncommittedBundle} for use by a source.
+ */
+ public <T> UncommittedBundle<T> createRootBundle(PCollection<T> output) {
+ return bundleFactory.createRootBundle(output);
+ }
+
+ /**
+ * Create a {@link UncommittedBundle} whose elements belong to the specified {@link
+ * PCollection}.
+ */
+ public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output) {
+ return bundleFactory.createBundle(input, output);
+ }
+
+ /**
+ * Create a {@link UncommittedBundle} with the specified keys at the specified step. For use by
+ * {@link InProcessGroupByKeyOnly} {@link PTransform PTransforms}.
+ */
+ public <T> UncommittedBundle<T> createKeyedBundle(
+ CommittedBundle<?> input, Object key, PCollection<T> output) {
+ return bundleFactory.createKeyedBundle(input, key, output);
+ }
+
+ /**
+ * Create a {@link PCollectionViewWriter}, whose elements will be used in the provided
+ * {@link PCollectionView}.
+ */
+ public <ElemT, ViewT> PCollectionViewWriter<ElemT, ViewT> createPCollectionViewWriter(
+ PCollection<Iterable<ElemT>> input, final PCollectionView<ViewT> output) {
+ return new PCollectionViewWriter<ElemT, ViewT>() {
+ @Override
+ public void add(Iterable<WindowedValue<ElemT>> values) {
+ sideInputContainer.write(output, values);
+ }
+ };
+ }
+
+ /**
+ * Schedule a callback to be executed after output would be produced for the given window
+ * if there had been input.
+ *
+ * <p>Output would be produced when the watermark for a {@link PValue} passes the point at
+ * which the trigger for the specified window (with the specified windowing strategy) must have
+ * fired from the perspective of that {@link PValue}, as specified by the value of
+ * {@link Trigger#getWatermarkThatGuaranteesFiring(BoundedWindow)} for the trigger of the
+ * {@link WindowingStrategy}. When the callback has fired, either values will have been produced
+ * for a key in that window, the window is empty, or all elements in the window are late. The
+ * callback will be executed regardless of whether values have been produced.
+ */
+ public void scheduleAfterOutputWouldBeProduced(
+ PValue value,
+ BoundedWindow window,
+ WindowingStrategy<?, ?> windowingStrategy,
+ Runnable runnable) {
+ AppliedPTransform<?, ?, ?> producing = getProducing(value);
+ callbackExecutor.callOnGuaranteedFiring(producing, window, windowingStrategy, runnable);
+
+ fireAvailableCallbacks(lookupProducing(value));
+ }
+
+ private AppliedPTransform<?, ?, ?> getProducing(PValue value) {
+ if (value.getProducingTransformInternal() != null) {
+ return value.getProducingTransformInternal();
+ }
+ return lookupProducing(value);
+ }
+
+ private AppliedPTransform<?, ?, ?> lookupProducing(PValue value) {
+ for (AppliedPTransform<?, ?, ?> transform : stepNames.keySet()) {
+ if (transform.getOutput().equals(value) || transform.getOutput().expand().contains(value)) {
+ return transform;
+ }
+ }
+ return null;
+ }
+
+ /**
+ * Get the options used by this {@link Pipeline}.
+ */
+ public InProcessPipelineOptions getPipelineOptions() {
+ return options;
+ }
+
+ /**
+ * Get an {@link ExecutionContext} for the provided {@link AppliedPTransform} and key.
+ */
+ public InProcessExecutionContext getExecutionContext(
+ AppliedPTransform<?, ?, ?> application, Object key) {
+ StepAndKey stepAndKey = StepAndKey.of(application, key);
+ return new InProcessExecutionContext(
+ options.getClock(),
+ key,
+ (CopyOnAccessInMemoryStateInternals<Object>) applicationStateInternals.get(stepAndKey),
+ watermarkManager.getWatermarks(application));
+ }
+
+ /**
+ * Get all of the steps used in this {@link Pipeline}.
+ */
+ public Collection<AppliedPTransform<?, ?, ?>> getSteps() {
+ return stepNames.keySet();
+ }
+
+ /**
+ * Get the Step Name for the provided application.
+ */
+ public String getStepName(AppliedPTransform<?, ?, ?> application) {
+ return stepNames.get(application);
+ }
+
+ /**
+ * Returns a {@link ReadyCheckingSideInputReader} capable of reading the provided
+ * {@link PCollectionView PCollectionViews}.
+ *
+ * @param sideInputs the {@link PCollectionView PCollectionViews} the result should be able to
+ * read
+ * @return a {@link SideInputReader} that can read all of the provided {@link PCollectionView
+ * PCollectionViews}
+ */
+ public ReadyCheckingSideInputReader createSideInputReader(
+ final List<PCollectionView<?>> sideInputs) {
+ return sideInputContainer.createReaderForViews(sideInputs);
+ }
+
+ /**
+ * A {@link SideInputReader} that allows callers to check to see if a {@link PCollectionView} has
+ * had its contents set in a window.
+ */
+ static interface ReadyCheckingSideInputReader extends SideInputReader {
+ /**
+ * Returns true if the {@link PCollectionView} is ready in the provided {@link BoundedWindow}.
+ */
+ boolean isReady(PCollectionView<?> view, BoundedWindow window);
+ }
+
+ /**
+ * Create a {@link CounterSet} for this {@link Pipeline}. The {@link CounterSet} is independent
+ * of all other {@link CounterSet CounterSets} created by this call.
+ *
+ * The {@link InProcessEvaluationContext} is responsible for unifying the counters present in
+ * all created {@link CounterSet CounterSets} when the transforms that call this method
+ * complete.
+ */
+ public CounterSet createCounterSet() {
+ return new CounterSet();
+ }
+
+ /**
+ * Returns all of the counters that have been merged into this context via calls to
+ * {@link CounterSet#merge(CounterSet)}.
+ */
+ public CounterSet getCounters() {
+ return mergedCounters;
+ }
+
+ /**
+ * Extracts all timers that have been fired and have not already been extracted.
+ *
+ * <p>This is a destructive operation. Timers will only appear in the result of this method once
+ * for each time they are set.
+ */
+ public Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> extractFiredTimers() {
+ Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> fired =
+ watermarkManager.extractFiredTimers();
+ return fired;
+ }
+
+ /**
+ * Returns true if the step will not produce additional output.
+ *
+ * <p>If the provided transform produces only {@link IsBounded#BOUNDED}
+ * {@link PCollection PCollections}, returns true if the watermark is at
+ * {@link BoundedWindow#TIMESTAMP_MAX_VALUE positive infinity}.
+ *
+ * <p>If the provided transform produces any {@link IsBounded#UNBOUNDED}
+ * {@link PCollection PCollections}, returns the value of
+ * {@link InProcessPipelineOptions#isShutdownUnboundedProducersWithMaxWatermark()}.
+ */
+ public boolean isDone(AppliedPTransform<?, ?, ?> transform) {
+ // if the PTransform's watermark isn't at the max value, it isn't done
+ if (watermarkManager
+ .getWatermarks(transform)
+ .getOutputWatermark()
+ .isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE)) {
+ return false;
+ }
+ // If the PTransform has any unbounded outputs, and unbounded producers should not be shut down,
+ // the PTransform may produce additional output. It is not done.
+ for (PValue output : transform.getOutput().expand()) {
+ if (output instanceof PCollection) {
+ IsBounded bounded = ((PCollection<?>) output).isBounded();
+ if (bounded.equals(IsBounded.UNBOUNDED)
+ && !options.isShutdownUnboundedProducersWithMaxWatermark()) {
+ return false;
+ }
+ }
+ }
+ // The PTransform's watermark was at positive infinity and all of its outputs are known to be
+ // done. It is done.
+ return true;
+ }
+
+ /**
+ * Returns true if all steps are done.
+ */
+ public boolean isDone() {
+ for (AppliedPTransform<?, ?, ?> transform : stepNames.keySet()) {
+ if (!isDone(transform)) {
+ return false;
+ }
+ }
+ return true;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessExecutionContext.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessExecutionContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessExecutionContext.java
new file mode 100644
index 0000000..44d8bd9
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessExecutionContext.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TransformWatermarks;
+import org.apache.beam.sdk.util.BaseExecutionContext;
+import org.apache.beam.sdk.util.ExecutionContext;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
+
+/**
+ * Execution Context for the {@link InProcessPipelineRunner}.
+ *
+ * This implementation is not thread safe. A new {@link InProcessExecutionContext} must be created
+ * for each thread that requires it.
+ */
+class InProcessExecutionContext
+ extends BaseExecutionContext<InProcessExecutionContext.InProcessStepContext> {
+ private final Clock clock;
+ private final Object key;
+ private final CopyOnAccessInMemoryStateInternals<Object> existingState;
+ private final TransformWatermarks watermarks;
+
+ public InProcessExecutionContext(Clock clock, Object key,
+ CopyOnAccessInMemoryStateInternals<Object> existingState, TransformWatermarks watermarks) {
+ this.clock = clock;
+ this.key = key;
+ this.existingState = existingState;
+ this.watermarks = watermarks;
+ }
+
+ @Override
+ protected InProcessStepContext createStepContext(String stepName, String transformName) {
+ return new InProcessStepContext(this, stepName, transformName);
+ }
+
+ /**
+ * Step Context for the {@link InProcessPipelineRunner}.
+ */
+ public class InProcessStepContext
+ extends org.apache.beam.sdk.util.BaseExecutionContext.StepContext {
+ private CopyOnAccessInMemoryStateInternals<Object> stateInternals;
+ private InProcessTimerInternals timerInternals;
+
+ public InProcessStepContext(
+ ExecutionContext executionContext, String stepName, String transformName) {
+ super(executionContext, stepName, transformName);
+ }
+
+ @Override
+ public CopyOnAccessInMemoryStateInternals<Object> stateInternals() {
+ if (stateInternals == null) {
+ stateInternals = CopyOnAccessInMemoryStateInternals.withUnderlying(key, existingState);
+ }
+ return stateInternals;
+ }
+
+ @Override
+ public InProcessTimerInternals timerInternals() {
+ if (timerInternals == null) {
+ timerInternals =
+ InProcessTimerInternals.create(clock, watermarks, TimerUpdate.builder(key));
+ }
+ return timerInternals;
+ }
+
+ /**
+ * Commits the state of this step, and returns the committed state. If the step has not
+ * accessed any state, return null.
+ */
+ public CopyOnAccessInMemoryStateInternals<?> commitState() {
+ if (stateInternals != null) {
+ return stateInternals.commit();
+ }
+ return null;
+ }
+
+ /**
+ * Gets the timer update of the {@link TimerInternals} of this {@link InProcessStepContext},
+ * which is empty if the {@link TimerInternals} were never accessed.
+ */
+ public TimerUpdate getTimerUpdate() {
+ if (timerInternals == null) {
+ return TimerUpdate.empty();
+ }
+ return timerInternals.getTimerUpdate();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessExecutor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessExecutor.java
new file mode 100644
index 0000000..d811e1b
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessExecutor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+
+import java.util.Collection;
+
+/**
+ * An executor that schedules and executes {@link AppliedPTransform AppliedPTransforms} for both
+ * source and intermediate {@link PTransform PTransforms}.
+ */
+interface InProcessExecutor {
+ /**
+ * Starts this executor. The provided collection is the collection of root transforms to
+ * initially schedule.
+ *
+ * @param rootTransforms
+ */
+ void start(Collection<AppliedPTransform<?, ?, ?>> rootTransforms);
+
+ /**
+ * Blocks until the job being executed enters a terminal state. A job is completed after all
+ * root {@link AppliedPTransform AppliedPTransforms} have completed, and all
+ * {@link CommittedBundle Bundles} have been consumed. Jobs may also terminate abnormally.
+ *
+ * @throws Throwable whenever an executor thread throws anything, transfers the throwable to the
+ * waiting thread and rethrows it
+ */
+ void awaitCompletion() throws Throwable;
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineOptions.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineOptions.java
new file mode 100644
index 0000000..512b3bd
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineOptions.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.ApplicationNameOptions;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.Hidden;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.Validation.Required;
+import org.apache.beam.sdk.transforms.PTransform;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * Options that can be used to configure the {@link InProcessPipelineRunner}.
+ */
+public interface InProcessPipelineOptions extends PipelineOptions, ApplicationNameOptions {
+ /**
+ * Gets the {@link ExecutorServiceFactory} to use to create instances of {@link ExecutorService}
+ * to execute {@link PTransform PTransforms}.
+ *
+ * <p>Note that {@link ExecutorService ExecutorServices} returned by the factory must ensure that
+ * it cannot enter a state in which it will not schedule additional pending work unless currently
+ * scheduled work completes, as this may cause the {@link Pipeline} to cease processing.
+ *
+ * <p>Defaults to a {@link CachedThreadPoolExecutorServiceFactory}, which produces instances of
+ * {@link Executors#newCachedThreadPool()}.
+ */
+ @JsonIgnore
+ @Required
+ @Hidden
+ @Default.InstanceFactory(CachedThreadPoolExecutorServiceFactory.class)
+ ExecutorServiceFactory getExecutorServiceFactory();
+
+ void setExecutorServiceFactory(ExecutorServiceFactory executorService);
+
+ /**
+ * Gets the {@link Clock} used by this pipeline. The clock is used in place of accessing the
+ * system time when time values are required by the evaluator.
+ */
+ @Default.InstanceFactory(NanosOffsetClock.Factory.class)
+ @JsonIgnore
+ @Required
+ @Hidden
+ @Description(
+ "The processing time source used by the pipeline. When the current time is "
+ + "needed by the evaluator, the result of clock#now() is used.")
+ Clock getClock();
+
+ void setClock(Clock clock);
+
+ @Default.Boolean(false)
+ @Description(
+ "If the pipeline should shut down producers which have reached the maximum "
+ + "representable watermark. If this is set to true, a pipeline in which all PTransforms "
+ + "have reached the maximum watermark will be shut down, even if there are unbounded "
+ + "sources that could produce additional (late) data. By default, if the pipeline "
+ + "contains any unbounded PCollections, it will run until explicitly shut down.")
+ boolean isShutdownUnboundedProducersWithMaxWatermark();
+
+ void setShutdownUnboundedProducersWithMaxWatermark(boolean shutdown);
+
+ @Default.Boolean(true)
+ @Description(
+ "If the pipeline should block awaiting completion of the pipeline. If set to true, "
+ + "a call to Pipeline#run() will block until all PTransforms are complete. Otherwise, "
+ + "the Pipeline will execute asynchronously. If set to false, the completion of the "
+ + "pipeline can be awaited on by use of InProcessPipelineResult#awaitCompletion().")
+ boolean isBlockOnRun();
+
+ void setBlockOnRun(boolean b);
+
+ @Default.Boolean(true)
+ @Description(
+ "Controls whether the runner should ensure that all of the elements of every "
+ + "PCollection are not mutated. PTransforms are not permitted to mutate input elements "
+ + "at any point, or output elements after they are output.")
+ boolean isTestImmutability();
+
+ void setTestImmutability(boolean test);
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineRunner.java
new file mode 100644
index 0000000..bb8c0de
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineRunner.java
@@ -0,0 +1,370 @@
+/*
+ * 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.direct.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
+import org.apache.beam.runners.direct.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOverrideFactory;
+import org.apache.beam.runners.direct.ViewEvaluatorFactory.InProcessViewOverrideFactory;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.io.AvroIO;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.AggregatorPipelineExtractor;
+import org.apache.beam.sdk.runners.AggregatorRetrievalException;
+import org.apache.beam.sdk.runners.AggregatorValues;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.View.CreatePCollectionView;
+import org.apache.beam.sdk.util.MapAggregatorValues;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.common.Counter;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.PValue;
+
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+
+import org.joda.time.Instant;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+
+import javax.annotation.Nullable;
+
+/**
+ * An In-Memory implementation of the Dataflow Programming Model. Supports Unbounded
+ * {@link PCollection PCollections}.
+ */
+@Experimental
+public class InProcessPipelineRunner
+ extends PipelineRunner<InProcessPipelineRunner.InProcessPipelineResult> {
+ /**
+ * The default set of transform overrides to use in the {@link InProcessPipelineRunner}.
+ *
+ * <p>A transform override must have a single-argument constructor that takes an instance of the
+ * type of transform it is overriding.
+ */
+ @SuppressWarnings("rawtypes")
+ private static Map<Class<? extends PTransform>, PTransformOverrideFactory>
+ defaultTransformOverrides =
+ ImmutableMap.<Class<? extends PTransform>, PTransformOverrideFactory>builder()
+ .put(GroupByKey.class, new InProcessGroupByKeyOverrideFactory())
+ .put(CreatePCollectionView.class, new InProcessViewOverrideFactory())
+ .put(AvroIO.Write.Bound.class, new AvroIOShardedWriteFactory())
+ .put(TextIO.Write.Bound.class, new TextIOShardedWriteFactory())
+ .build();
+
+ /**
+ * Part of a {@link PCollection}. Elements are output to a bundle, which will cause them to be
+ * executed by {@link PTransform PTransforms} that consume the {@link PCollection} this bundle is
+ * a part of at a later point. This is an uncommitted bundle and can have elements added to it.
+ *
+ * @param <T> the type of elements that can be added to this bundle
+ */
+ public static interface UncommittedBundle<T> {
+ /**
+ * Returns the PCollection that the elements of this {@link UncommittedBundle} belong to.
+ */
+ PCollection<T> getPCollection();
+
+ /**
+ * Outputs an element to this bundle.
+ *
+ * @param element the element to add to this bundle
+ * @return this bundle
+ */
+ UncommittedBundle<T> add(WindowedValue<T> element);
+
+ /**
+ * Commits this {@link UncommittedBundle}, returning an immutable {@link CommittedBundle}
+ * containing all of the elements that were added to it. The {@link #add(WindowedValue)} method
+ * will throw an {@link IllegalStateException} if called after a call to commit.
+ * @param synchronizedProcessingTime the synchronized processing time at which this bundle was
+ * committed
+ */
+ CommittedBundle<T> commit(Instant synchronizedProcessingTime);
+ }
+
+ /**
+ * Part of a {@link PCollection}. Elements are output to an {@link UncommittedBundle}, which will
+ * eventually committed. Committed elements are executed by the {@link PTransform PTransforms}
+ * that consume the {@link PCollection} this bundle is
+ * a part of at a later point.
+ * @param <T> the type of elements contained within this bundle
+ */
+ public static interface CommittedBundle<T> {
+ /**
+ * Returns the PCollection that the elements of this bundle belong to.
+ */
+ PCollection<T> getPCollection();
+
+ /**
+ * Returns the (possibly null) key that was output in the most recent {@link GroupByKey} in the
+ * execution of this bundle.
+ */
+ @Nullable
+ Object getKey();
+
+ /**
+ * Returns an {@link Iterable} containing all of the elements that have been added to this
+ * {@link CommittedBundle}.
+ */
+ Iterable<WindowedValue<T>> getElements();
+
+ /**
+ * Returns the processing time output watermark at the time the producing {@link PTransform}
+ * committed this bundle. Downstream synchronized processing time watermarks cannot progress
+ * past this point before consuming this bundle.
+ *
+ * <p>This value is no greater than the earliest incomplete processing time or synchronized
+ * processing time {@link TimerData timer} at the time this bundle was committed, including any
+ * timers that fired to produce this bundle.
+ */
+ Instant getSynchronizedProcessingOutputWatermark();
+
+ /**
+ * Return a new {@link CommittedBundle} that is like this one, except calls to
+ * {@link #getElements()} will return the provided elements. This bundle is unchanged.
+ *
+ * <p>
+ * The value of the {@link #getSynchronizedProcessingOutputWatermark() synchronized processing
+ * output watermark} of the returned {@link CommittedBundle} is equal to the value returned from
+ * the current bundle. This is used to ensure a {@link PTransform} that could not complete
+ * processing on input elements properly holds the synchronized processing time to the
+ * appropriate value.
+ */
+ CommittedBundle<T> withElements(Iterable<WindowedValue<T>> elements);
+ }
+
+ /**
+ * A {@link PCollectionViewWriter} is responsible for writing contents of a {@link PCollection} to
+ * a storage mechanism that can be read from while constructing a {@link PCollectionView}.
+ * @param <ElemT> the type of elements the input {@link PCollection} contains.
+ * @param <ViewT> the type of the PCollectionView this writer writes to.
+ */
+ public static interface PCollectionViewWriter<ElemT, ViewT> {
+ void add(Iterable<WindowedValue<ElemT>> values);
+ }
+
+ ////////////////////////////////////////////////////////////////////////////////////////////////
+ private final InProcessPipelineOptions options;
+
+ public static InProcessPipelineRunner fromOptions(PipelineOptions options) {
+ return new InProcessPipelineRunner(options.as(InProcessPipelineOptions.class));
+ }
+
+ private InProcessPipelineRunner(InProcessPipelineOptions options) {
+ this.options = options;
+ }
+
+ /**
+ * Returns the {@link PipelineOptions} used to create this {@link InProcessPipelineRunner}.
+ */
+ public InProcessPipelineOptions getPipelineOptions() {
+ return options;
+ }
+
+ @Override
+ public <OutputT extends POutput, InputT extends PInput> OutputT apply(
+ PTransform<InputT, OutputT> transform, InputT input) {
+ PTransformOverrideFactory overrideFactory = defaultTransformOverrides.get(transform.getClass());
+ if (overrideFactory != null) {
+ PTransform<InputT, OutputT> customTransform = overrideFactory.override(transform);
+
+ return super.apply(customTransform, input);
+ }
+ // If there is no override, or we should not apply the override, apply the original transform
+ return super.apply(transform, input);
+ }
+
+ @Override
+ public InProcessPipelineResult run(Pipeline pipeline) {
+ ConsumerTrackingPipelineVisitor consumerTrackingVisitor = new ConsumerTrackingPipelineVisitor();
+ pipeline.traverseTopologically(consumerTrackingVisitor);
+ for (PValue unfinalized : consumerTrackingVisitor.getUnfinalizedPValues()) {
+ unfinalized.finishSpecifying();
+ }
+ @SuppressWarnings("rawtypes")
+ KeyedPValueTrackingVisitor keyedPValueVisitor =
+ KeyedPValueTrackingVisitor.create(
+ ImmutableSet.<Class<? extends PTransform>>of(
+ GroupByKey.class, InProcessGroupByKeyOnly.class));
+ pipeline.traverseTopologically(keyedPValueVisitor);
+
+ InProcessEvaluationContext context =
+ InProcessEvaluationContext.create(
+ getPipelineOptions(),
+ createBundleFactory(getPipelineOptions()),
+ consumerTrackingVisitor.getRootTransforms(),
+ consumerTrackingVisitor.getValueToConsumers(),
+ consumerTrackingVisitor.getStepNames(),
+ consumerTrackingVisitor.getViews());
+
+ // independent executor service for each run
+ ExecutorService executorService =
+ context.getPipelineOptions().getExecutorServiceFactory().create();
+ InProcessExecutor executor =
+ ExecutorServiceParallelExecutor.create(
+ executorService,
+ consumerTrackingVisitor.getValueToConsumers(),
+ keyedPValueVisitor.getKeyedPValues(),
+ TransformEvaluatorRegistry.defaultRegistry(),
+ defaultModelEnforcements(options),
+ context);
+ executor.start(consumerTrackingVisitor.getRootTransforms());
+
+ Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps =
+ new AggregatorPipelineExtractor(pipeline).getAggregatorSteps();
+ InProcessPipelineResult result =
+ new InProcessPipelineResult(executor, context, aggregatorSteps);
+ if (options.isBlockOnRun()) {
+ try {
+ result.awaitCompletion();
+ } catch (UserCodeException userException) {
+ throw new PipelineExecutionException(userException.getCause());
+ } catch (Throwable t) {
+ Throwables.propagate(t);
+ }
+ }
+ return result;
+ }
+
+ private Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>>
+ defaultModelEnforcements(InProcessPipelineOptions options) {
+ ImmutableMap.Builder<Class<? extends PTransform>, Collection<ModelEnforcementFactory>>
+ enforcements = ImmutableMap.builder();
+ Collection<ModelEnforcementFactory> parDoEnforcements = createParDoEnforcements(options);
+ enforcements.put(ParDo.Bound.class, parDoEnforcements);
+ enforcements.put(ParDo.BoundMulti.class, parDoEnforcements);
+ return enforcements.build();
+ }
+
+ private Collection<ModelEnforcementFactory> createParDoEnforcements(
+ InProcessPipelineOptions options) {
+ ImmutableList.Builder<ModelEnforcementFactory> enforcements = ImmutableList.builder();
+ if (options.isTestImmutability()) {
+ enforcements.add(ImmutabilityEnforcementFactory.create());
+ }
+ return enforcements.build();
+ }
+
+ private BundleFactory createBundleFactory(InProcessPipelineOptions pipelineOptions) {
+ BundleFactory bundleFactory = InProcessBundleFactory.create();
+ if (pipelineOptions.isTestImmutability()) {
+ bundleFactory = ImmutabilityCheckingBundleFactory.create(bundleFactory);
+ }
+ return bundleFactory;
+ }
+
+ /**
+ * The result of running a {@link Pipeline} with the {@link InProcessPipelineRunner}.
+ *
+ * Throws {@link UnsupportedOperationException} for all methods.
+ */
+ public static class InProcessPipelineResult implements PipelineResult {
+ private final InProcessExecutor executor;
+ private final InProcessEvaluationContext evaluationContext;
+ private final Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps;
+ private State state;
+
+ private InProcessPipelineResult(
+ InProcessExecutor executor,
+ InProcessEvaluationContext evaluationContext,
+ Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps) {
+ this.executor = executor;
+ this.evaluationContext = evaluationContext;
+ this.aggregatorSteps = aggregatorSteps;
+ // Only ever constructed after the executor has started.
+ this.state = State.RUNNING;
+ }
+
+ @Override
+ public State getState() {
+ return state;
+ }
+
+ @Override
+ public <T> AggregatorValues<T> getAggregatorValues(Aggregator<?, T> aggregator)
+ throws AggregatorRetrievalException {
+ CounterSet counters = evaluationContext.getCounters();
+ Collection<PTransform<?, ?>> steps = aggregatorSteps.get(aggregator);
+ Map<String, T> stepValues = new HashMap<>();
+ for (AppliedPTransform<?, ?, ?> transform : evaluationContext.getSteps()) {
+ if (steps.contains(transform.getTransform())) {
+ String stepName =
+ String.format(
+ "user-%s-%s", evaluationContext.getStepName(transform), aggregator.getName());
+ Counter<T> counter = (Counter<T>) counters.getExistingCounter(stepName);
+ if (counter != null) {
+ stepValues.put(transform.getFullName(), counter.getAggregate());
+ }
+ }
+ }
+ return new MapAggregatorValues<>(stepValues);
+ }
+
+ /**
+ * Blocks until the {@link Pipeline} execution represented by this
+ * {@link InProcessPipelineResult} is complete, returning the terminal state.
+ *
+ * <p>If the pipeline terminates abnormally by throwing an exception, this will rethrow the
+ * exception. Future calls to {@link #getState()} will return
+ * {@link org.apache.beam.sdk.PipelineResult.State#FAILED}.
+ *
+ * <p>NOTE: if the {@link Pipeline} contains an {@link IsBounded#UNBOUNDED unbounded}
+ * {@link PCollection}, and the {@link PipelineRunner} was created with
+ * {@link InProcessPipelineOptions#isShutdownUnboundedProducersWithMaxWatermark()} set to false,
+ * this method will never return.
+ *
+ * See also {@link InProcessExecutor#awaitCompletion()}.
+ */
+ public State awaitCompletion() throws Throwable {
+ if (!state.isTerminal()) {
+ try {
+ executor.awaitCompletion();
+ state = State.DONE;
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw e;
+ } catch (Throwable t) {
+ state = State.FAILED;
+ throw t;
+ }
+ }
+ return state;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessRegistrar.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessRegistrar.java
new file mode 100644
index 0000000..4a09de7
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessRegistrar.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
+
+import com.google.auto.service.AutoService;
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Contains the {@link PipelineRunnerRegistrar} and {@link PipelineOptionsRegistrar} for the
+ * {@link InProcessPipelineRunner}.
+ */
+public class InProcessRegistrar {
+ private InProcessRegistrar() {}
+ /**
+ * Registers the {@link InProcessPipelineRunner}.
+ */
+ @AutoService(PipelineRunnerRegistrar.class)
+ public static class InProcessRunner implements PipelineRunnerRegistrar {
+ @Override
+ public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
+ return ImmutableList.<Class<? extends PipelineRunner<?>>>of(InProcessPipelineRunner.class);
+ }
+ }
+
+ /**
+ * Registers the {@link InProcessPipelineOptions}.
+ */
+ @AutoService(PipelineOptionsRegistrar.class)
+ public static class InProcessOptions implements PipelineOptionsRegistrar {
+ @Override
+ public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
+ return ImmutableList.<Class<? extends PipelineOptions>>of(InProcessPipelineOptions.class);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessSideInputContainer.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessSideInputContainer.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessSideInputContainer.java
new file mode 100644
index 0000000..f4980ef
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessSideInputContainer.java
@@ -0,0 +1,271 @@
+/*
+ * 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 com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.beam.runners.direct.InProcessEvaluationContext.ReadyCheckingSideInputReader;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.PCollectionViewWindow;
+import org.apache.beam.sdk.util.SideInputReader;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.PCollectionView;
+
+import com.google.common.base.MoreObjects;
+import com.google.common.base.Throwables;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.SettableFuture;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+import javax.annotation.Nullable;
+
+/**
+ * An in-process container for {@link PCollectionView PCollectionViews}, which provides methods for
+ * constructing {@link SideInputReader SideInputReaders} which block until a side input is
+ * available and writing to a {@link PCollectionView}.
+ */
+class InProcessSideInputContainer {
+ private final InProcessEvaluationContext evaluationContext;
+ private final Collection<PCollectionView<?>> containedViews;
+ private final LoadingCache<PCollectionViewWindow<?>,
+ SettableFuture<Iterable<? extends WindowedValue<?>>>> viewByWindows;
+
+ /**
+ * Create a new {@link InProcessSideInputContainer} with the provided views and the provided
+ * context.
+ */
+ public static InProcessSideInputContainer create(
+ InProcessEvaluationContext context, Collection<PCollectionView<?>> containedViews) {
+ CacheLoader<PCollectionViewWindow<?>, SettableFuture<Iterable<? extends WindowedValue<?>>>>
+ loader = new CacheLoader<PCollectionViewWindow<?>,
+ SettableFuture<Iterable<? extends WindowedValue<?>>>>() {
+ @Override
+ public SettableFuture<Iterable<? extends WindowedValue<?>>> load(
+ PCollectionViewWindow<?> view) {
+ return SettableFuture.create();
+ }
+ };
+ LoadingCache<PCollectionViewWindow<?>, SettableFuture<Iterable<? extends WindowedValue<?>>>>
+ viewByWindows = CacheBuilder.newBuilder().build(loader);
+ return new InProcessSideInputContainer(context, containedViews, viewByWindows);
+ }
+
+ private InProcessSideInputContainer(InProcessEvaluationContext context,
+ Collection<PCollectionView<?>> containedViews,
+ LoadingCache<PCollectionViewWindow<?>, SettableFuture<Iterable<? extends WindowedValue<?>>>>
+ viewByWindows) {
+ this.evaluationContext = context;
+ this.containedViews = ImmutableSet.copyOf(containedViews);
+ this.viewByWindows = viewByWindows;
+ }
+
+ /**
+ * Return a view of this {@link InProcessSideInputContainer} that contains only the views in the
+ * provided argument. The returned {@link InProcessSideInputContainer} is unmodifiable without
+ * casting, but will change as this {@link InProcessSideInputContainer} is modified.
+ */
+ public ReadyCheckingSideInputReader createReaderForViews(
+ Collection<PCollectionView<?>> newContainedViews) {
+ if (!containedViews.containsAll(newContainedViews)) {
+ Set<PCollectionView<?>> currentlyContained = ImmutableSet.copyOf(containedViews);
+ Set<PCollectionView<?>> newRequested = ImmutableSet.copyOf(newContainedViews);
+ throw new IllegalArgumentException("Can't create a SideInputReader with unknown views "
+ + Sets.difference(newRequested, currentlyContained));
+ }
+ return new SideInputContainerSideInputReader(newContainedViews);
+ }
+
+ /**
+ * Write the provided values to the provided view.
+ *
+ * <p>The windowed values are first exploded, then for each window the pane is determined. For
+ * each window, if the pane is later than the current pane stored within this container, write
+ * all of the values to the container as the new values of the {@link PCollectionView}.
+ *
+ * <p>The provided iterable is expected to contain only a single window and pane.
+ */
+ public void write(PCollectionView<?> view, Iterable<? extends WindowedValue<?>> values) {
+ Map<BoundedWindow, Collection<WindowedValue<?>>> valuesPerWindow =
+ indexValuesByWindow(values);
+ for (Map.Entry<BoundedWindow, Collection<WindowedValue<?>>> windowValues :
+ valuesPerWindow.entrySet()) {
+ updatePCollectionViewWindowValues(view, windowValues.getKey(), windowValues.getValue());
+ }
+ }
+
+ /**
+ * Index the provided values by all {@link BoundedWindow windows} in which they appear.
+ */
+ private Map<BoundedWindow, Collection<WindowedValue<?>>> indexValuesByWindow(
+ Iterable<? extends WindowedValue<?>> values) {
+ Map<BoundedWindow, Collection<WindowedValue<?>>> valuesPerWindow = new HashMap<>();
+ for (WindowedValue<?> value : values) {
+ for (BoundedWindow window : value.getWindows()) {
+ Collection<WindowedValue<?>> windowValues = valuesPerWindow.get(window);
+ if (windowValues == null) {
+ windowValues = new ArrayList<>();
+ valuesPerWindow.put(window, windowValues);
+ }
+ windowValues.add(value);
+ }
+ }
+ return valuesPerWindow;
+ }
+
+ /**
+ * Set the value of the {@link PCollectionView} in the {@link BoundedWindow} to be based on the
+ * specified values, if the values are part of a later pane than currently exist within the
+ * {@link PCollectionViewWindow}.
+ */
+ private void updatePCollectionViewWindowValues(
+ PCollectionView<?> view, BoundedWindow window, Collection<WindowedValue<?>> windowValues) {
+ PCollectionViewWindow<?> windowedView = PCollectionViewWindow.of(view, window);
+ SettableFuture<Iterable<? extends WindowedValue<?>>> future = null;
+ try {
+ future = viewByWindows.get(windowedView);
+ if (future.isDone()) {
+ Iterator<? extends WindowedValue<?>> existingValues = future.get().iterator();
+ PaneInfo newPane = windowValues.iterator().next().getPane();
+ // The current value may have no elements, if no elements were produced for the window,
+ // but we are recieving late data.
+ if (!existingValues.hasNext()
+ || newPane.getIndex() > existingValues.next().getPane().getIndex()) {
+ viewByWindows.invalidate(windowedView);
+ viewByWindows.get(windowedView).set(windowValues);
+ }
+ } else {
+ future.set(windowValues);
+ }
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ if (future != null && !future.isDone()) {
+ future.set(Collections.<WindowedValue<?>>emptyList());
+ }
+ } catch (ExecutionException e) {
+ Throwables.propagate(e.getCause());
+ }
+ }
+
+ private final class SideInputContainerSideInputReader implements ReadyCheckingSideInputReader {
+ private final Collection<PCollectionView<?>> readerViews;
+
+ private SideInputContainerSideInputReader(Collection<PCollectionView<?>> readerViews) {
+ this.readerViews = ImmutableSet.copyOf(readerViews);
+ }
+
+ @Override
+ public boolean isReady(final PCollectionView<?> view, final BoundedWindow window) {
+ checkArgument(
+ readerViews.contains(view),
+ "Tried to check if view %s was ready in a SideInputReader that does not contain it. "
+ + "Contained views; %s",
+ view,
+ readerViews);
+ return getViewFuture(view, window).isDone();
+ }
+
+ @Override
+ @Nullable
+ public <T> T get(final PCollectionView<T> view, final BoundedWindow window) {
+ checkArgument(
+ readerViews.contains(view), "calling get(PCollectionView) with unknown view: " + view);
+ try {
+ final Future<Iterable<? extends WindowedValue<?>>> future = getViewFuture(view, window);
+ // Safe covariant cast
+ @SuppressWarnings("unchecked")
+ Iterable<WindowedValue<?>> values = (Iterable<WindowedValue<?>>) future.get();
+ return view.fromIterableInternal(values);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ return null;
+ } catch (ExecutionException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ /**
+ * Gets the future containing the contents of the provided {@link PCollectionView} in the
+ * provided {@link BoundedWindow}, setting up a callback to populate the future with empty
+ * contents if necessary.
+ */
+ private <T> Future<Iterable<? extends WindowedValue<?>>> getViewFuture(
+ final PCollectionView<T> view, final BoundedWindow window) {
+ PCollectionViewWindow<T> windowedView = PCollectionViewWindow.of(view, window);
+ final SettableFuture<Iterable<? extends WindowedValue<?>>> future =
+ viewByWindows.getUnchecked(windowedView);
+
+ WindowingStrategy<?, ?> windowingStrategy = view.getWindowingStrategyInternal();
+ evaluationContext.scheduleAfterOutputWouldBeProduced(
+ view, window, windowingStrategy, new WriteEmptyViewContents(view, window, future));
+ return future;
+ }
+
+ @Override
+ public <T> boolean contains(PCollectionView<T> view) {
+ return readerViews.contains(view);
+ }
+
+ @Override
+ public boolean isEmpty() {
+ return readerViews.isEmpty();
+ }
+ }
+
+ private static class WriteEmptyViewContents implements Runnable {
+ private final PCollectionView<?> view;
+ private final BoundedWindow window;
+ private final SettableFuture<Iterable<? extends WindowedValue<?>>> future;
+
+ private WriteEmptyViewContents(PCollectionView<?> view, BoundedWindow window,
+ SettableFuture<Iterable<? extends WindowedValue<?>>> future) {
+ this.future = future;
+ this.view = view;
+ this.window = window;
+ }
+
+ @Override
+ public void run() {
+ // The requested window has closed without producing elements, so reflect that in
+ // the PCollectionView. If set has already been called, will do nothing.
+ future.set(Collections.<WindowedValue<?>>emptyList());
+ }
+
+ @Override
+ public String toString() {
+ return MoreObjects.toStringHelper(this)
+ .add("view", view)
+ .add("window", window)
+ .toString();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessTimerInternals.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessTimerInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessTimerInternals.java
new file mode 100644
index 0000000..cd54f59
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessTimerInternals.java
@@ -0,0 +1,84 @@
+/*
+ * 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.direct.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate.TimerUpdateBuilder;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TransformWatermarks;
+import org.apache.beam.sdk.util.TimerInternals;
+
+import org.joda.time.Instant;
+
+import javax.annotation.Nullable;
+
+/**
+ * An implementation of {@link TimerInternals} where all relevant data exists in memory.
+ */
+public class InProcessTimerInternals implements TimerInternals {
+ private final Clock processingTimeClock;
+ private final TransformWatermarks watermarks;
+ private final TimerUpdateBuilder timerUpdateBuilder;
+
+ public static InProcessTimerInternals create(
+ Clock clock, TransformWatermarks watermarks, TimerUpdateBuilder timerUpdateBuilder) {
+ return new InProcessTimerInternals(clock, watermarks, timerUpdateBuilder);
+ }
+
+ private InProcessTimerInternals(
+ Clock clock, TransformWatermarks watermarks, TimerUpdateBuilder timerUpdateBuilder) {
+ this.processingTimeClock = clock;
+ this.watermarks = watermarks;
+ this.timerUpdateBuilder = timerUpdateBuilder;
+ }
+
+ @Override
+ public void setTimer(TimerData timerKey) {
+ timerUpdateBuilder.setTimer(timerKey);
+ }
+
+ @Override
+ public void deleteTimer(TimerData timerKey) {
+ timerUpdateBuilder.deletedTimer(timerKey);
+ }
+
+ public TimerUpdate getTimerUpdate() {
+ return timerUpdateBuilder.build();
+ }
+
+ @Override
+ public Instant currentProcessingTime() {
+ return processingTimeClock.now();
+ }
+
+ @Override
+ @Nullable
+ public Instant currentSynchronizedProcessingTime() {
+ return watermarks.getSynchronizedProcessingInputTime();
+ }
+
+ @Override
+ public Instant currentInputWatermarkTime() {
+ return watermarks.getInputWatermark();
+ }
+
+ @Override
+ @Nullable
+ public Instant currentOutputWatermarkTime() {
+ return watermarks.getOutputWatermark();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessTransformResult.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessTransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessTransformResult.java
new file mode 100644
index 0000000..a132c33
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessTransformResult.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
+
+import org.joda.time.Instant;
+
+import javax.annotation.Nullable;
+
+/**
+ * The result of evaluating an {@link AppliedPTransform} with a {@link TransformEvaluator}.
+ */
+public interface InProcessTransformResult {
+ /**
+ * Returns the {@link AppliedPTransform} that produced this result.
+ */
+ AppliedPTransform<?, ?, ?> getTransform();
+
+ /**
+ * Returns the {@link UncommittedBundle (uncommitted) Bundles} output by this transform. These
+ * will be committed by the evaluation context as part of completing this result.
+ */
+ Iterable<? extends UncommittedBundle<?>> getOutputBundles();
+
+ /**
+ * Returns the {@link CounterSet} used by this {@link PTransform}, or null if this transform did
+ * not use a {@link CounterSet}.
+ */
+ @Nullable CounterSet getCounters();
+
+ /**
+ * Returns the Watermark Hold for the transform at the time this result was produced.
+ *
+ * If the transform does not set any watermark hold, returns
+ * {@link BoundedWindow#TIMESTAMP_MAX_VALUE}.
+ */
+ Instant getWatermarkHold();
+
+ /**
+ * Returns the State used by the transform.
+ *
+ * If this evaluation did not access state, this may return null.
+ */
+ @Nullable
+ CopyOnAccessInMemoryStateInternals<?> getState();
+
+ /**
+ * Returns a TimerUpdateBuilder that was produced as a result of this evaluation. If the
+ * evaluation was triggered due to the delivery of one or more timers, those timers must be added
+ * to the builder before it is complete.
+ *
+ * <p>If this evaluation did not add or remove any timers, returns an empty TimerUpdate.
+ */
+ TimerUpdate getTimerUpdate();
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
new file mode 100644
index 0000000..b7c755e
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
@@ -0,0 +1,96 @@
+/*
+ * 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 com.google.common.base.Preconditions.checkState;
+
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PValue;
+
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * A pipeline visitor that tracks all keyed {@link PValue PValues}. A {@link PValue} is keyed if it
+ * is the result of a {@link PTransform} that produces keyed outputs. A {@link PTransform} that
+ * produces keyed outputs is assumed to colocate output elements that share a key.
+ *
+ * <p>All {@link GroupByKey} transforms, or their runner-specific implementation primitive, produce
+ * keyed output.
+ */
+// TODO: Handle Key-preserving transforms when appropriate and more aggressively make PTransforms
+// unkeyed
+class KeyedPValueTrackingVisitor implements PipelineVisitor {
+ @SuppressWarnings("rawtypes")
+ private final Set<Class<? extends PTransform>> producesKeyedOutputs;
+ private final Set<PValue> keyedValues;
+ private boolean finalized;
+
+ public static KeyedPValueTrackingVisitor create(
+ @SuppressWarnings("rawtypes") Set<Class<? extends PTransform>> producesKeyedOutputs) {
+ return new KeyedPValueTrackingVisitor(producesKeyedOutputs);
+ }
+
+ private KeyedPValueTrackingVisitor(
+ @SuppressWarnings("rawtypes") Set<Class<? extends PTransform>> producesKeyedOutputs) {
+ this.producesKeyedOutputs = producesKeyedOutputs;
+ this.keyedValues = new HashSet<>();
+ }
+
+ @Override
+ public void enterCompositeTransform(TransformTreeNode node) {
+ checkState(
+ !finalized,
+ "Attempted to use a %s that has already been finalized on a pipeline (visiting node %s)",
+ KeyedPValueTrackingVisitor.class.getSimpleName(),
+ node);
+ }
+
+ @Override
+ public void leaveCompositeTransform(TransformTreeNode node) {
+ checkState(
+ !finalized,
+ "Attempted to use a %s that has already been finalized on a pipeline (visiting node %s)",
+ KeyedPValueTrackingVisitor.class.getSimpleName(),
+ node);
+ if (node.isRootNode()) {
+ finalized = true;
+ } else if (producesKeyedOutputs.contains(node.getTransform().getClass())) {
+ keyedValues.addAll(node.getExpandedOutputs());
+ }
+ }
+
+ @Override
+ public void visitTransform(TransformTreeNode node) {}
+
+ @Override
+ public void visitValue(PValue value, TransformTreeNode producer) {
+ if (producesKeyedOutputs.contains(producer.getTransform().getClass())) {
+ keyedValues.addAll(value.expand());
+ }
+ }
+
+ public Set<PValue> getKeyedPValues() {
+ checkState(
+ finalized, "can't call getKeyedPValues before a Pipeline has been completely traversed");
+ return keyedValues;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java
new file mode 100644
index 0000000..cc9b6da
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * Enforcement tools that verify that executing code conforms to the model.
+ *
+ * <p>ModelEnforcement is performed on a per-element and per-bundle basis. The
+ * {@link ModelEnforcement} is provided with the input bundle as part of
+ * {@link ModelEnforcementFactory#forBundle(CommittedBundle, AppliedPTransform)}, each element
+ * before and after that element is provided to an underlying {@link TransformEvaluator}, and the
+ * output {@link InProcessTransformResult} and committed output bundles after the
+ * {@link TransformEvaluator} has completed.
+ *
+ * <p>Typically, {@link ModelEnforcement} will obtain required metadata (such as the {@link Coder}
+ * of the input {@link PCollection} on construction, and then enforce per-element behavior
+ * (such as the immutability of input elements). When the element is output or the bundle is
+ * completed, the required conditions can be enforced across all elements.
+ */
+public interface ModelEnforcement<T> {
+ /**
+ * Called before a call to {@link TransformEvaluator#processElement(WindowedValue)} on the
+ * provided {@link WindowedValue}.
+ */
+ void beforeElement(WindowedValue<T> element);
+
+ /**
+ * Called after a call to {@link TransformEvaluator#processElement(WindowedValue)} on the
+ * provided {@link WindowedValue}.
+ */
+ void afterElement(WindowedValue<T> element);
+
+ /**
+ * Called after a bundle has been completed and {@link TransformEvaluator#finishBundle()} has been
+ * called, producing the provided {@link InProcessTransformResult} and
+ * {@link CommittedBundle output bundles}.
+ */
+ void afterFinish(
+ CommittedBundle<T> input,
+ InProcessTransformResult result,
+ Iterable<? extends CommittedBundle<?>> outputs);
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcementFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcementFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcementFactory.java
new file mode 100644
index 0000000..6162ba0
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcementFactory.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+
+/**
+ * Creates {@link ModelEnforcement} instances for an {@link AppliedPTransform} on an input
+ * {@link CommittedBundle bundle}. {@link ModelEnforcement} instances are created before the
+ * {@link TransformEvaluator} is created.
+ */
+public interface ModelEnforcementFactory {
+ <T> ModelEnforcement<T> forBundle(CommittedBundle<T> input, AppliedPTransform<?, ?, ?> consumer);
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java
new file mode 100644
index 0000000..ffdee9d
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+import org.joda.time.Instant;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A {@link Clock} that uses {@link System#nanoTime()} to track the progress of time.
+ */
+public class NanosOffsetClock implements Clock {
+ private final long baseMillis;
+ private final long nanosAtBaseMillis;
+
+ public static NanosOffsetClock create() {
+ return new NanosOffsetClock();
+ }
+
+ private NanosOffsetClock() {
+ baseMillis = System.currentTimeMillis();
+ nanosAtBaseMillis = System.nanoTime();
+ }
+
+ @Override
+ public Instant now() {
+ return new Instant(
+ baseMillis + (TimeUnit.MILLISECONDS.convert(
+ System.nanoTime() - nanosAtBaseMillis, TimeUnit.NANOSECONDS)));
+ }
+
+ /**
+ * Creates instances of {@link NanosOffsetClock}.
+ */
+ public static class Factory implements DefaultValueFactory<Clock> {
+ @Override
+ public Clock create(PipelineOptions options) {
+ return new NanosOffsetClock();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PTransformOverrideFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PTransformOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PTransformOverrideFactory.java
new file mode 100644
index 0000000..81e4863
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PTransformOverrideFactory.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
+interface PTransformOverrideFactory {
+ /**
+ * Create a {@link PTransform} override for the provided {@link PTransform} if applicable.
+ * Otherwise, return the input {@link PTransform}.
+ *
+ * <p>The returned PTransform must be semantically equivalent to the input {@link PTransform}.
+ */
+ <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
+ PTransform<InputT, OutputT> transform);
+}
[02/17] incubator-beam git commit: Move InProcessRunner to its own
module
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContextTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContextTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContextTest.java
deleted file mode 100644
index d1ea51a..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContextTest.java
+++ /dev/null
@@ -1,526 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.not;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.io.CountingInput;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.FiredTimers;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import org.apache.beam.sdk.runners.inprocess.InProcessExecutionContext.InProcessStepContext;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.PCollectionViewWriter;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.View;
-import org.apache.beam.sdk.transforms.WithKeys;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
-import org.apache.beam.sdk.util.SideInputReader;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.util.common.Counter;
-import org.apache.beam.sdk.util.common.Counter.AggregationKind;
-import org.apache.beam.sdk.util.common.CounterSet;
-import org.apache.beam.sdk.util.state.BagState;
-import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-import org.apache.beam.sdk.util.state.StateNamespaces;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.apache.beam.sdk.util.state.StateTags;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollection.IsBounded;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PValue;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-
-import org.hamcrest.Matchers;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Tests for {@link InProcessEvaluationContext}.
- */
-@RunWith(JUnit4.class)
-public class InProcessEvaluationContextTest {
- private TestPipeline p;
- private InProcessEvaluationContext context;
-
- private PCollection<Integer> created;
- private PCollection<KV<String, Integer>> downstream;
- private PCollectionView<Iterable<Integer>> view;
- private PCollection<Long> unbounded;
- private Collection<AppliedPTransform<?, ?, ?>> rootTransforms;
- private Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers;
-
- private BundleFactory bundleFactory;
-
- @Before
- public void setup() {
- InProcessPipelineRunner runner =
- InProcessPipelineRunner.fromOptions(PipelineOptionsFactory.create());
-
- p = TestPipeline.create();
-
- created = p.apply(Create.of(1, 2, 3));
- downstream = created.apply(WithKeys.<String, Integer>of("foo"));
- view = created.apply(View.<Integer>asIterable());
- unbounded = p.apply(CountingInput.unbounded());
-
- ConsumerTrackingPipelineVisitor cVis = new ConsumerTrackingPipelineVisitor();
- p.traverseTopologically(cVis);
- rootTransforms = cVis.getRootTransforms();
- valueToConsumers = cVis.getValueToConsumers();
-
- bundleFactory = InProcessBundleFactory.create();
-
- context =
- InProcessEvaluationContext.create(
- runner.getPipelineOptions(),
- InProcessBundleFactory.create(),
- rootTransforms,
- valueToConsumers,
- cVis.getStepNames(),
- cVis.getViews());
- }
-
- @Test
- public void writeToViewWriterThenReadReads() {
- PCollectionViewWriter<Integer, Iterable<Integer>> viewWriter =
- context.createPCollectionViewWriter(
- PCollection.<Iterable<Integer>>createPrimitiveOutputInternal(
- p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED),
- view);
- BoundedWindow window = new TestBoundedWindow(new Instant(1024L));
- BoundedWindow second = new TestBoundedWindow(new Instant(899999L));
- WindowedValue<Integer> firstValue =
- WindowedValue.of(1, new Instant(1222), window, PaneInfo.ON_TIME_AND_ONLY_FIRING);
- WindowedValue<Integer> secondValue =
- WindowedValue.of(
- 2, new Instant(8766L), second, PaneInfo.createPane(true, false, Timing.ON_TIME, 0, 0));
- Iterable<WindowedValue<Integer>> values = ImmutableList.of(firstValue, secondValue);
- viewWriter.add(values);
-
- SideInputReader reader =
- context.createSideInputReader(ImmutableList.<PCollectionView<?>>of(view));
- assertThat(reader.get(view, window), containsInAnyOrder(1));
- assertThat(reader.get(view, second), containsInAnyOrder(2));
-
- WindowedValue<Integer> overrittenSecondValue =
- WindowedValue.of(
- 4444, new Instant(8677L), second, PaneInfo.createPane(false, true, Timing.LATE, 1, 1));
- viewWriter.add(Collections.singleton(overrittenSecondValue));
- assertThat(reader.get(view, second), containsInAnyOrder(4444));
- }
-
- @Test
- public void getExecutionContextSameStepSameKeyState() {
- InProcessExecutionContext fooContext =
- context.getExecutionContext(created.getProducingTransformInternal(), "foo");
-
- StateTag<Object, BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
-
- InProcessStepContext stepContext = fooContext.getOrCreateStepContext("s1", "s1");
- stepContext.stateInternals().state(StateNamespaces.global(), intBag).add(1);
-
- context.handleResult(
- InProcessBundleFactory.create()
- .createKeyedBundle(null, "foo", created)
- .commit(Instant.now()),
- ImmutableList.<TimerData>of(),
- StepTransformResult.withoutHold(created.getProducingTransformInternal())
- .withState(stepContext.commitState())
- .build());
-
- InProcessExecutionContext secondFooContext =
- context.getExecutionContext(created.getProducingTransformInternal(), "foo");
- assertThat(
- secondFooContext
- .getOrCreateStepContext("s1", "s1")
- .stateInternals()
- .state(StateNamespaces.global(), intBag)
- .read(),
- contains(1));
- }
-
-
- @Test
- public void getExecutionContextDifferentKeysIndependentState() {
- InProcessExecutionContext fooContext =
- context.getExecutionContext(created.getProducingTransformInternal(), "foo");
-
- StateTag<Object, BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
-
- fooContext
- .getOrCreateStepContext("s1", "s1")
- .stateInternals()
- .state(StateNamespaces.global(), intBag)
- .add(1);
-
- InProcessExecutionContext barContext =
- context.getExecutionContext(created.getProducingTransformInternal(), "bar");
- assertThat(barContext, not(equalTo(fooContext)));
- assertThat(
- barContext
- .getOrCreateStepContext("s1", "s1")
- .stateInternals()
- .state(StateNamespaces.global(), intBag)
- .read(),
- emptyIterable());
- }
-
- @Test
- public void getExecutionContextDifferentStepsIndependentState() {
- String myKey = "foo";
- InProcessExecutionContext fooContext =
- context.getExecutionContext(created.getProducingTransformInternal(), myKey);
-
- StateTag<Object, BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
-
- fooContext
- .getOrCreateStepContext("s1", "s1")
- .stateInternals()
- .state(StateNamespaces.global(), intBag)
- .add(1);
-
- InProcessExecutionContext barContext =
- context.getExecutionContext(downstream.getProducingTransformInternal(), myKey);
- assertThat(
- barContext
- .getOrCreateStepContext("s1", "s1")
- .stateInternals()
- .state(StateNamespaces.global(), intBag)
- .read(),
- emptyIterable());
- }
-
- @Test
- public void handleResultMergesCounters() {
- CounterSet counters = context.createCounterSet();
- Counter<Long> myCounter = Counter.longs("foo", AggregationKind.SUM);
- counters.addCounter(myCounter);
-
- myCounter.addValue(4L);
- InProcessTransformResult result =
- StepTransformResult.withoutHold(created.getProducingTransformInternal())
- .withCounters(counters)
- .build();
- context.handleResult(null, ImmutableList.<TimerData>of(), result);
- assertThat((Long) context.getCounters().getExistingCounter("foo").getAggregate(), equalTo(4L));
-
- CounterSet againCounters = context.createCounterSet();
- Counter<Long> myLongCounterAgain = Counter.longs("foo", AggregationKind.SUM);
- againCounters.add(myLongCounterAgain);
- myLongCounterAgain.addValue(8L);
-
- InProcessTransformResult secondResult =
- StepTransformResult.withoutHold(downstream.getProducingTransformInternal())
- .withCounters(againCounters)
- .build();
- context.handleResult(
- context.createRootBundle(created).commit(Instant.now()),
- ImmutableList.<TimerData>of(),
- secondResult);
- assertThat((Long) context.getCounters().getExistingCounter("foo").getAggregate(), equalTo(12L));
- }
-
- @Test
- public void handleResultStoresState() {
- String myKey = "foo";
- InProcessExecutionContext fooContext =
- context.getExecutionContext(downstream.getProducingTransformInternal(), myKey);
-
- StateTag<Object, BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
-
- CopyOnAccessInMemoryStateInternals<Object> state =
- fooContext.getOrCreateStepContext("s1", "s1").stateInternals();
- BagState<Integer> bag = state.state(StateNamespaces.global(), intBag);
- bag.add(1);
- bag.add(2);
- bag.add(4);
-
- InProcessTransformResult stateResult =
- StepTransformResult.withoutHold(downstream.getProducingTransformInternal())
- .withState(state)
- .build();
-
- context.handleResult(
- context.createKeyedBundle(null, myKey, created).commit(Instant.now()),
- ImmutableList.<TimerData>of(),
- stateResult);
-
- InProcessExecutionContext afterResultContext =
- context.getExecutionContext(downstream.getProducingTransformInternal(), myKey);
-
- CopyOnAccessInMemoryStateInternals<Object> afterResultState =
- afterResultContext.getOrCreateStepContext("s1", "s1").stateInternals();
- assertThat(afterResultState.state(StateNamespaces.global(), intBag).read(), contains(1, 2, 4));
- }
-
- @Test
- public void callAfterOutputMustHaveBeenProducedAfterEndOfWatermarkCallsback() throws Exception {
- final CountDownLatch callLatch = new CountDownLatch(1);
- Runnable callback =
- new Runnable() {
- @Override
- public void run() {
- callLatch.countDown();
- }
- };
-
- // Should call back after the end of the global window
- context.scheduleAfterOutputWouldBeProduced(
- downstream, GlobalWindow.INSTANCE, WindowingStrategy.globalDefault(), callback);
-
- InProcessTransformResult result =
- StepTransformResult.withHold(created.getProducingTransformInternal(), new Instant(0))
- .build();
-
- context.handleResult(null, ImmutableList.<TimerData>of(), result);
-
- // Difficult to demonstrate that we took no action in a multithreaded world; poll for a bit
- // will likely be flaky if this logic is broken
- assertThat(callLatch.await(500L, TimeUnit.MILLISECONDS), is(false));
-
- InProcessTransformResult finishedResult =
- StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
- context.handleResult(null, ImmutableList.<TimerData>of(), finishedResult);
- // Obtain the value via blocking call
- assertThat(callLatch.await(1, TimeUnit.SECONDS), is(true));
- }
-
- @Test
- public void callAfterOutputMustHaveBeenProducedAlreadyAfterCallsImmediately() throws Exception {
- InProcessTransformResult finishedResult =
- StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
- context.handleResult(null, ImmutableList.<TimerData>of(), finishedResult);
-
- final CountDownLatch callLatch = new CountDownLatch(1);
- Runnable callback =
- new Runnable() {
- @Override
- public void run() {
- callLatch.countDown();
- }
- };
- context.scheduleAfterOutputWouldBeProduced(
- downstream, GlobalWindow.INSTANCE, WindowingStrategy.globalDefault(), callback);
- assertThat(callLatch.await(1, TimeUnit.SECONDS), is(true));
- }
-
- @Test
- public void extractFiredTimersExtractsTimers() {
- InProcessTransformResult holdResult =
- StepTransformResult.withHold(created.getProducingTransformInternal(), new Instant(0))
- .build();
- context.handleResult(null, ImmutableList.<TimerData>of(), holdResult);
-
- String key = "foo";
- TimerData toFire =
- TimerData.of(StateNamespaces.global(), new Instant(100L), TimeDomain.EVENT_TIME);
- InProcessTransformResult timerResult =
- StepTransformResult.withoutHold(downstream.getProducingTransformInternal())
- .withState(CopyOnAccessInMemoryStateInternals.withUnderlying(key, null))
- .withTimerUpdate(TimerUpdate.builder(key).setTimer(toFire).build())
- .build();
-
- // haven't added any timers, must be empty
- assertThat(context.extractFiredTimers().entrySet(), emptyIterable());
- context.handleResult(
- context.createKeyedBundle(null, key, created).commit(Instant.now()),
- ImmutableList.<TimerData>of(),
- timerResult);
-
- // timer hasn't fired
- assertThat(context.extractFiredTimers().entrySet(), emptyIterable());
-
- InProcessTransformResult advanceResult =
- StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
- // Should cause the downstream timer to fire
- context.handleResult(null, ImmutableList.<TimerData>of(), advanceResult);
-
- Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> fired = context.extractFiredTimers();
- assertThat(
- fired,
- Matchers.<AppliedPTransform<?, ?, ?>>hasKey(downstream.getProducingTransformInternal()));
- Map<Object, FiredTimers> downstreamFired =
- fired.get(downstream.getProducingTransformInternal());
- assertThat(downstreamFired, Matchers.<Object>hasKey(key));
-
- FiredTimers firedForKey = downstreamFired.get(key);
- assertThat(firedForKey.getTimers(TimeDomain.PROCESSING_TIME), emptyIterable());
- assertThat(firedForKey.getTimers(TimeDomain.SYNCHRONIZED_PROCESSING_TIME), emptyIterable());
- assertThat(firedForKey.getTimers(TimeDomain.EVENT_TIME), contains(toFire));
-
- // Don't reextract timers
- assertThat(context.extractFiredTimers().entrySet(), emptyIterable());
- }
-
- @Test
- public void createBundleKeyedResultPropagatesKey() {
- CommittedBundle<KV<String, Integer>> newBundle =
- context
- .createBundle(
- bundleFactory.createKeyedBundle(null, "foo", created).commit(Instant.now()),
- downstream)
- .commit(Instant.now());
- assertThat(newBundle.getKey(), Matchers.<Object>equalTo("foo"));
- }
-
- @Test
- public void createKeyedBundleKeyed() {
- CommittedBundle<KV<String, Integer>> keyedBundle =
- context
- .createKeyedBundle(
- bundleFactory.createRootBundle(created).commit(Instant.now()), "foo", downstream)
- .commit(Instant.now());
- assertThat(keyedBundle.getKey(), Matchers.<Object>equalTo("foo"));
- }
-
- @Test
- public void isDoneWithUnboundedPCollectionAndShutdown() {
- context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(true);
- assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(false));
-
- context.handleResult(
- null,
- ImmutableList.<TimerData>of(),
- StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build());
- assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(true));
- }
-
- @Test
- public void isDoneWithUnboundedPCollectionAndNotShutdown() {
- context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(false);
- assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(false));
-
- context.handleResult(
- null,
- ImmutableList.<TimerData>of(),
- StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build());
- assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(false));
- }
-
- @Test
- public void isDoneWithOnlyBoundedPCollections() {
- context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(false);
- assertThat(context.isDone(created.getProducingTransformInternal()), is(false));
-
- context.handleResult(
- null,
- ImmutableList.<TimerData>of(),
- StepTransformResult.withoutHold(created.getProducingTransformInternal()).build());
- assertThat(context.isDone(created.getProducingTransformInternal()), is(true));
- }
-
- @Test
- public void isDoneWithPartiallyDone() {
- context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(true);
- assertThat(context.isDone(), is(false));
-
- UncommittedBundle<Integer> rootBundle = context.createRootBundle(created);
- rootBundle.add(WindowedValue.valueInGlobalWindow(1));
- CommittedResult handleResult =
- context.handleResult(
- null,
- ImmutableList.<TimerData>of(),
- StepTransformResult.withoutHold(created.getProducingTransformInternal())
- .addOutput(rootBundle)
- .build());
- @SuppressWarnings("unchecked")
- CommittedBundle<Integer> committedBundle =
- (CommittedBundle<Integer>) Iterables.getOnlyElement(handleResult.getOutputs());
- context.handleResult(
- null,
- ImmutableList.<TimerData>of(),
- StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build());
- assertThat(context.isDone(), is(false));
-
- for (AppliedPTransform<?, ?, ?> consumers : valueToConsumers.get(created)) {
- context.handleResult(
- committedBundle,
- ImmutableList.<TimerData>of(),
- StepTransformResult.withoutHold(consumers).build());
- }
- assertThat(context.isDone(), is(true));
- }
-
- @Test
- public void isDoneWithUnboundedAndNotShutdown() {
- context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(false);
- assertThat(context.isDone(), is(false));
-
- context.handleResult(
- null,
- ImmutableList.<TimerData>of(),
- StepTransformResult.withoutHold(created.getProducingTransformInternal()).build());
- context.handleResult(
- null,
- ImmutableList.<TimerData>of(),
- StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build());
- context.handleResult(
- context.createRootBundle(created).commit(Instant.now()),
- ImmutableList.<TimerData>of(),
- StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build());
- assertThat(context.isDone(), is(false));
-
- context.handleResult(
- context.createRootBundle(created).commit(Instant.now()),
- ImmutableList.<TimerData>of(),
- StepTransformResult.withoutHold(view.getProducingTransformInternal()).build());
- assertThat(context.isDone(), is(false));
- }
-
- private static class TestBoundedWindow extends BoundedWindow {
- private final Instant ts;
-
- public TestBoundedWindow(Instant ts) {
- this.ts = ts;
- }
-
- @Override
- public Instant maxTimestamp() {
- return ts;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRegistrarTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRegistrarTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRegistrarTest.java
deleted file mode 100644
index 59a96ed..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRegistrarTest.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
-import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
-import org.apache.beam.sdk.runners.inprocess.InProcessRegistrar.InProcessRunner;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.ServiceLoader;
-
-/** Tests for {@link InProcessRunner}. */
-@RunWith(JUnit4.class)
-public class InProcessPipelineRegistrarTest {
- @Test
- public void testCorrectOptionsAreReturned() {
- assertEquals(
- ImmutableList.of(InProcessPipelineOptions.class),
- new InProcessRegistrar.InProcessOptions().getPipelineOptions());
- }
-
- @Test
- public void testCorrectRunnersAreReturned() {
- assertEquals(
- ImmutableList.of(InProcessPipelineRunner.class),
- new InProcessRegistrar.InProcessRunner().getPipelineRunners());
- }
-
- @Test
- public void testServiceLoaderForOptions() {
- for (PipelineOptionsRegistrar registrar :
- Lists.newArrayList(ServiceLoader.load(PipelineOptionsRegistrar.class).iterator())) {
- if (registrar instanceof InProcessRegistrar.InProcessOptions) {
- return;
- }
- }
- fail("Expected to find " + InProcessRegistrar.InProcessOptions.class);
- }
-
- @Test
- public void testServiceLoaderForRunner() {
- for (PipelineRunnerRegistrar registrar :
- Lists.newArrayList(ServiceLoader.load(PipelineRunnerRegistrar.class).iterator())) {
- if (registrar instanceof InProcessRegistrar.InProcessRunner) {
- return;
- }
- }
- fail("Expected to find " + InProcessRegistrar.InProcessRunner.class);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunnerTest.java
deleted file mode 100644
index e9e9e36..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunnerTest.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.InProcessPipelineResult;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.transforms.Count;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.MapElements;
-import org.apache.beam.sdk.transforms.SimpleFunction;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.Serializable;
-
-/**
- * Tests for basic {@link InProcessPipelineRunner} functionality.
- */
-@RunWith(JUnit4.class)
-public class InProcessPipelineRunnerTest implements Serializable {
- @Test
- public void wordCountShouldSucceed() throws Throwable {
- Pipeline p = getPipeline();
-
- PCollection<KV<String, Long>> counts =
- p.apply(Create.of("foo", "bar", "foo", "baz", "bar", "foo"))
- .apply(MapElements.via(new SimpleFunction<String, String>() {
- @Override
- public String apply(String input) {
- return input;
- }
- }))
- .apply(Count.<String>perElement());
- PCollection<String> countStrs =
- counts.apply(MapElements.via(new SimpleFunction<KV<String, Long>, String>() {
- @Override
- public String apply(KV<String, Long> input) {
- String str = String.format("%s: %s", input.getKey(), input.getValue());
- return str;
- }
- }));
-
- PAssert.that(countStrs).containsInAnyOrder("baz: 1", "bar: 2", "foo: 3");
-
- InProcessPipelineResult result = ((InProcessPipelineResult) p.run());
- result.awaitCompletion();
- }
-
- private Pipeline getPipeline() {
- PipelineOptions opts = PipelineOptionsFactory.create();
- opts.setRunner(InProcessPipelineRunner.class);
-
- Pipeline p = Pipeline.create(opts);
- return p;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainerTest.java
deleted file mode 100644
index 03443f8..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainerTest.java
+++ /dev/null
@@ -1,496 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.hasEntry;
-import static org.hamcrest.Matchers.is;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.doAnswer;
-
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.runners.inprocess.InProcessEvaluationContext.ReadyCheckingSideInputReader;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.Mean;
-import org.apache.beam.sdk.transforms.View;
-import org.apache.beam.sdk.transforms.WithKeys;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
-import org.apache.beam.sdk.util.PCollectionViews;
-import org.apache.beam.sdk.util.SideInputReader;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionView;
-
-import com.google.common.collect.ImmutableList;
-
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.Mockito;
-import org.mockito.MockitoAnnotations;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-/**
- * Tests for {@link InProcessSideInputContainer}.
- */
-@RunWith(JUnit4.class)
-public class InProcessSideInputContainerTest {
- private static final BoundedWindow FIRST_WINDOW =
- new BoundedWindow() {
- @Override
- public Instant maxTimestamp() {
- return new Instant(789541L);
- }
-
- @Override
- public String toString() {
- return "firstWindow";
- }
- };
-
- private static final BoundedWindow SECOND_WINDOW =
- new BoundedWindow() {
- @Override
- public Instant maxTimestamp() {
- return new Instant(14564786L);
- }
-
- @Override
- public String toString() {
- return "secondWindow";
- }
- };
-
- @Rule
- public ExpectedException thrown = ExpectedException.none();
-
- @Mock
- private InProcessEvaluationContext context;
-
- private TestPipeline pipeline;
-
- private InProcessSideInputContainer container;
-
- private PCollectionView<Map<String, Integer>> mapView;
- private PCollectionView<Double> singletonView;
-
- // Not present in container.
- private PCollectionView<Iterable<Integer>> iterableView;
-
- @Before
- public void setup() {
- MockitoAnnotations.initMocks(this);
- pipeline = TestPipeline.create();
-
- PCollection<Integer> create =
- pipeline.apply("forBaseCollection", Create.<Integer>of(1, 2, 3, 4));
-
- mapView =
- create.apply("forKeyTypes", WithKeys.<String, Integer>of("foo"))
- .apply("asMapView", View.<String, Integer>asMap());
-
- singletonView = create.apply("forCombinedTypes", Mean.<Integer>globally().asSingletonView());
- iterableView = create.apply("asIterableView", View.<Integer>asIterable());
-
- container = InProcessSideInputContainer.create(
- context, ImmutableList.of(iterableView, mapView, singletonView));
- }
-
- @Test
- public void getAfterWriteReturnsPaneInWindow() throws Exception {
- WindowedValue<KV<String, Integer>> one =
- WindowedValue.of(
- KV.of("one", 1), new Instant(1L), FIRST_WINDOW, PaneInfo.ON_TIME_AND_ONLY_FIRING);
- WindowedValue<KV<String, Integer>> two =
- WindowedValue.of(
- KV.of("two", 2), new Instant(20L), FIRST_WINDOW, PaneInfo.ON_TIME_AND_ONLY_FIRING);
- container.write(mapView, ImmutableList.<WindowedValue<?>>of(one, two));
-
- Map<String, Integer> viewContents =
- container
- .createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
- .get(mapView, FIRST_WINDOW);
- assertThat(viewContents, hasEntry("one", 1));
- assertThat(viewContents, hasEntry("two", 2));
- assertThat(viewContents.size(), is(2));
- }
-
- @Test
- public void getReturnsLatestPaneInWindow() throws Exception {
- WindowedValue<KV<String, Integer>> one =
- WindowedValue.of(
- KV.of("one", 1),
- new Instant(1L),
- SECOND_WINDOW,
- PaneInfo.createPane(true, false, Timing.EARLY));
- WindowedValue<KV<String, Integer>> two =
- WindowedValue.of(
- KV.of("two", 2),
- new Instant(20L),
- SECOND_WINDOW,
- PaneInfo.createPane(true, false, Timing.EARLY));
- container.write(mapView, ImmutableList.<WindowedValue<?>>of(one, two));
-
- Map<String, Integer> viewContents =
- container
- .createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
- .get(mapView, SECOND_WINDOW);
- assertThat(viewContents, hasEntry("one", 1));
- assertThat(viewContents, hasEntry("two", 2));
- assertThat(viewContents.size(), is(2));
-
- WindowedValue<KV<String, Integer>> three =
- WindowedValue.of(
- KV.of("three", 3),
- new Instant(300L),
- SECOND_WINDOW,
- PaneInfo.createPane(false, false, Timing.EARLY, 1, -1));
- container.write(mapView, ImmutableList.<WindowedValue<?>>of(three));
-
- Map<String, Integer> overwrittenViewContents =
- container
- .createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
- .get(mapView, SECOND_WINDOW);
- assertThat(overwrittenViewContents, hasEntry("three", 3));
- assertThat(overwrittenViewContents.size(), is(1));
- }
-
- /**
- * Demonstrates that calling get() on a window that currently has no data does not return until
- * there is data in the pane.
- */
- @Test
- public void getBlocksUntilPaneAvailable() throws Exception {
- BoundedWindow window =
- new BoundedWindow() {
- @Override
- public Instant maxTimestamp() {
- return new Instant(1024L);
- }
- };
- Future<Double> singletonFuture =
- getFutureOfView(
- container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView)),
- singletonView,
- window);
-
- WindowedValue<Double> singletonValue =
- WindowedValue.of(4.75, new Instant(475L), window, PaneInfo.ON_TIME_AND_ONLY_FIRING);
-
- assertThat(singletonFuture.isDone(), is(false));
- container.write(singletonView, ImmutableList.<WindowedValue<?>>of(singletonValue));
- assertThat(singletonFuture.get(), equalTo(4.75));
- }
-
- @Test
- public void withPCollectionViewsWithPutInOriginalReturnsContents() throws Exception {
- BoundedWindow window = new BoundedWindow() {
- @Override
- public Instant maxTimestamp() {
- return new Instant(1024L);
- }
- };
- SideInputReader newReader =
- container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView));
- Future<Double> singletonFuture = getFutureOfView(newReader, singletonView, window);
-
- WindowedValue<Double> singletonValue =
- WindowedValue.of(24.125, new Instant(475L), window, PaneInfo.ON_TIME_AND_ONLY_FIRING);
-
- assertThat(singletonFuture.isDone(), is(false));
- container.write(singletonView, ImmutableList.<WindowedValue<?>>of(singletonValue));
- assertThat(singletonFuture.get(), equalTo(24.125));
- }
-
- @Test
- public void withPCollectionViewsErrorsForContainsNotInViews() {
- PCollectionView<Map<String, Iterable<String>>> newView =
- PCollectionViews.multimapView(
- pipeline,
- WindowingStrategy.globalDefault(),
- KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()));
-
- thrown.expect(IllegalArgumentException.class);
- thrown.expectMessage("with unknown views " + ImmutableList.of(newView).toString());
-
- container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(newView));
- }
-
- @Test
- public void withViewsForViewNotInContainerFails() {
- PCollectionView<Map<String, Iterable<String>>> newView =
- PCollectionViews.multimapView(
- pipeline,
- WindowingStrategy.globalDefault(),
- KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()));
-
- thrown.expect(IllegalArgumentException.class);
- thrown.expectMessage("unknown views");
- thrown.expectMessage(newView.toString());
-
- container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(newView));
- }
-
- @Test
- public void getOnReaderForViewNotInReaderFails() {
- thrown.expect(IllegalArgumentException.class);
- thrown.expectMessage("unknown view: " + iterableView.toString());
-
- container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
- .get(iterableView, GlobalWindow.INSTANCE);
- }
-
- @Test
- public void writeForMultipleElementsInDifferentWindowsSucceeds() throws Exception {
- WindowedValue<Double> firstWindowedValue =
- WindowedValue.of(
- 2.875,
- FIRST_WINDOW.maxTimestamp().minus(200L),
- FIRST_WINDOW,
- PaneInfo.ON_TIME_AND_ONLY_FIRING);
- WindowedValue<Double> secondWindowedValue =
- WindowedValue.of(
- 4.125,
- SECOND_WINDOW.maxTimestamp().minus(2_000_000L),
- SECOND_WINDOW,
- PaneInfo.ON_TIME_AND_ONLY_FIRING);
- container.write(singletonView, ImmutableList.of(firstWindowedValue, secondWindowedValue));
- assertThat(
- container
- .createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView))
- .get(singletonView, FIRST_WINDOW),
- equalTo(2.875));
- assertThat(
- container
- .createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView))
- .get(singletonView, SECOND_WINDOW),
- equalTo(4.125));
- }
-
- @Test
- public void writeForMultipleIdenticalElementsInSameWindowSucceeds() throws Exception {
- WindowedValue<Integer> firstValue =
- WindowedValue.of(
- 44,
- FIRST_WINDOW.maxTimestamp().minus(200L),
- FIRST_WINDOW,
- PaneInfo.ON_TIME_AND_ONLY_FIRING);
- WindowedValue<Integer> secondValue =
- WindowedValue.of(
- 44,
- FIRST_WINDOW.maxTimestamp().minus(200L),
- FIRST_WINDOW,
- PaneInfo.ON_TIME_AND_ONLY_FIRING);
-
- container.write(iterableView, ImmutableList.of(firstValue, secondValue));
-
- assertThat(
- container
- .createReaderForViews(ImmutableList.<PCollectionView<?>>of(iterableView))
- .get(iterableView, FIRST_WINDOW),
- contains(44, 44));
- }
-
- @Test
- public void writeForElementInMultipleWindowsSucceeds() throws Exception {
- WindowedValue<Double> multiWindowedValue =
- WindowedValue.of(
- 2.875,
- FIRST_WINDOW.maxTimestamp().minus(200L),
- ImmutableList.of(FIRST_WINDOW, SECOND_WINDOW),
- PaneInfo.ON_TIME_AND_ONLY_FIRING);
- container.write(singletonView, ImmutableList.of(multiWindowedValue));
- assertThat(
- container
- .createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView))
- .get(singletonView, FIRST_WINDOW),
- equalTo(2.875));
- assertThat(
- container
- .createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView))
- .get(singletonView, SECOND_WINDOW),
- equalTo(2.875));
- }
-
- @Test
- public void finishDoesNotOverwriteWrittenElements() throws Exception {
- WindowedValue<KV<String, Integer>> one =
- WindowedValue.of(
- KV.of("one", 1),
- new Instant(1L),
- SECOND_WINDOW,
- PaneInfo.createPane(true, false, Timing.EARLY));
- WindowedValue<KV<String, Integer>> two =
- WindowedValue.of(
- KV.of("two", 2),
- new Instant(20L),
- SECOND_WINDOW,
- PaneInfo.createPane(true, false, Timing.EARLY));
- container.write(mapView, ImmutableList.<WindowedValue<?>>of(one, two));
-
- immediatelyInvokeCallback(mapView, SECOND_WINDOW);
-
- Map<String, Integer> viewContents =
- container
- .createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
- .get(mapView, SECOND_WINDOW);
-
- assertThat(viewContents, hasEntry("one", 1));
- assertThat(viewContents, hasEntry("two", 2));
- assertThat(viewContents.size(), is(2));
- }
-
- @Test
- public void finishOnPendingViewsSetsEmptyElements() throws Exception {
- immediatelyInvokeCallback(mapView, SECOND_WINDOW);
- Future<Map<String, Integer>> mapFuture =
- getFutureOfView(
- container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView)),
- mapView,
- SECOND_WINDOW);
-
- assertThat(mapFuture.get().isEmpty(), is(true));
- }
-
- /**
- * Demonstrates that calling isReady on an empty container throws an
- * {@link IllegalArgumentException}.
- */
- @Test
- public void isReadyInEmptyReaderThrows() {
- ReadyCheckingSideInputReader reader =
- container.createReaderForViews(ImmutableList.<PCollectionView<?>>of());
- thrown.expect(IllegalArgumentException.class);
- thrown.expectMessage("does not contain");
- thrown.expectMessage(ImmutableList.of().toString());
- reader.isReady(mapView, GlobalWindow.INSTANCE);
- }
-
- /**
- * Demonstrates that calling isReady returns false until elements are written to the
- * {@link PCollectionView}, {@link BoundedWindow} pair, at which point it returns true.
- */
- @Test
- public void isReadyForSomeNotReadyViewsFalseUntilElements() {
- container.write(
- mapView,
- ImmutableList.of(
- WindowedValue.of(
- KV.of("one", 1),
- SECOND_WINDOW.maxTimestamp().minus(100L),
- SECOND_WINDOW,
- PaneInfo.ON_TIME_AND_ONLY_FIRING)));
-
- ReadyCheckingSideInputReader reader =
- container.createReaderForViews(ImmutableList.of(mapView, singletonView));
- assertThat(reader.isReady(mapView, FIRST_WINDOW), is(false));
- assertThat(reader.isReady(mapView, SECOND_WINDOW), is(true));
-
- assertThat(reader.isReady(singletonView, SECOND_WINDOW), is(false));
-
- container.write(
- mapView,
- ImmutableList.of(
- WindowedValue.of(
- KV.of("too", 2),
- FIRST_WINDOW.maxTimestamp().minus(100L),
- FIRST_WINDOW,
- PaneInfo.ON_TIME_AND_ONLY_FIRING)));
- assertThat(reader.isReady(mapView, FIRST_WINDOW), is(true));
-
- container.write(
- singletonView,
- ImmutableList.of(
- WindowedValue.of(
- 1.25,
- SECOND_WINDOW.maxTimestamp().minus(100L),
- SECOND_WINDOW,
- PaneInfo.ON_TIME_AND_ONLY_FIRING)));
- assertThat(reader.isReady(mapView, SECOND_WINDOW), is(true));
- assertThat(reader.isReady(singletonView, SECOND_WINDOW), is(true));
-
- assertThat(reader.isReady(mapView, GlobalWindow.INSTANCE), is(false));
- assertThat(reader.isReady(singletonView, GlobalWindow.INSTANCE), is(false));
- }
-
- @Test
- public void isReadyForEmptyWindowTrue() {
- immediatelyInvokeCallback(mapView, GlobalWindow.INSTANCE);
-
- ReadyCheckingSideInputReader reader =
- container.createReaderForViews(ImmutableList.of(mapView, singletonView));
- assertThat(reader.isReady(mapView, GlobalWindow.INSTANCE), is(true));
- assertThat(reader.isReady(singletonView, GlobalWindow.INSTANCE), is(false));
-
- immediatelyInvokeCallback(singletonView, GlobalWindow.INSTANCE);
- assertThat(reader.isReady(singletonView, GlobalWindow.INSTANCE), is(true));
- }
-
- /**
- * When a callAfterWindowCloses with the specified view's producing transform, window, and
- * windowing strategy is invoked, immediately execute the callback.
- */
- private void immediatelyInvokeCallback(PCollectionView<?> view, BoundedWindow window) {
- doAnswer(
- new Answer<Void>() {
- @Override
- public Void answer(InvocationOnMock invocation) throws Throwable {
- Object callback = invocation.getArguments()[3];
- Runnable callbackRunnable = (Runnable) callback;
- callbackRunnable.run();
- return null;
- }
- })
- .when(context)
- .scheduleAfterOutputWouldBeProduced(
- Mockito.eq(view),
- Mockito.eq(window),
- Mockito.eq(view.getWindowingStrategyInternal()),
- Mockito.any(Runnable.class));
- }
-
- private <ValueT> Future<ValueT> getFutureOfView(final SideInputReader myReader,
- final PCollectionView<ValueT> view, final BoundedWindow window) {
- Callable<ValueT> callable = new Callable<ValueT>() {
- @Override
- public ValueT call() throws Exception {
- return myReader.get(view, window);
- }
- };
- return Executors.newSingleThreadExecutor().submit(callable);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternalsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternalsTest.java
deleted file mode 100644
index b496981..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternalsTest.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate.TimerUpdateBuilder;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.state.StateNamespaces;
-
-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 InProcessTimerInternals}.
- */
-@RunWith(JUnit4.class)
-public class InProcessTimerInternalsTest {
- private MockClock clock;
- @Mock private TransformWatermarks watermarks;
-
- private TimerUpdateBuilder timerUpdateBuilder;
-
- private InProcessTimerInternals internals;
-
- @Before
- public void setup() {
- MockitoAnnotations.initMocks(this);
- clock = MockClock.fromInstant(new Instant(0));
-
- timerUpdateBuilder = TimerUpdate.builder(1234);
-
- internals = InProcessTimerInternals.create(clock, watermarks, timerUpdateBuilder);
- }
-
- @Test
- public void setTimerAddsToBuilder() {
- TimerData eventTimer =
- TimerData.of(StateNamespaces.global(), new Instant(20145L), TimeDomain.EVENT_TIME);
- TimerData processingTimer =
- TimerData.of(StateNamespaces.global(), new Instant(125555555L), TimeDomain.PROCESSING_TIME);
- TimerData synchronizedProcessingTimer =
- TimerData.of(
- StateNamespaces.global(),
- new Instant(98745632189L),
- TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
- internals.setTimer(eventTimer);
- internals.setTimer(processingTimer);
- internals.setTimer(synchronizedProcessingTimer);
-
- assertThat(
- internals.getTimerUpdate().getSetTimers(),
- containsInAnyOrder(eventTimer, synchronizedProcessingTimer, processingTimer));
- }
-
- @Test
- public void deleteTimerDeletesOnBuilder() {
- TimerData eventTimer =
- TimerData.of(StateNamespaces.global(), new Instant(20145L), TimeDomain.EVENT_TIME);
- TimerData processingTimer =
- TimerData.of(StateNamespaces.global(), new Instant(125555555L), TimeDomain.PROCESSING_TIME);
- TimerData synchronizedProcessingTimer =
- TimerData.of(
- StateNamespaces.global(),
- new Instant(98745632189L),
- TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
- internals.deleteTimer(eventTimer);
- internals.deleteTimer(processingTimer);
- internals.deleteTimer(synchronizedProcessingTimer);
-
- assertThat(
- internals.getTimerUpdate().getDeletedTimers(),
- containsInAnyOrder(eventTimer, synchronizedProcessingTimer, processingTimer));
- }
-
- @Test
- public void getProcessingTimeIsClockNow() {
- assertThat(internals.currentProcessingTime(), equalTo(clock.now()));
- Instant oldProcessingTime = internals.currentProcessingTime();
-
- clock.advance(Duration.standardHours(12));
-
- assertThat(internals.currentProcessingTime(), equalTo(clock.now()));
- assertThat(
- internals.currentProcessingTime(),
- equalTo(oldProcessingTime.plus(Duration.standardHours(12))));
- }
-
- @Test
- public void getSynchronizedProcessingTimeIsWatermarkSynchronizedInputTime() {
- when(watermarks.getSynchronizedProcessingInputTime()).thenReturn(new Instant(12345L));
- assertThat(internals.currentSynchronizedProcessingTime(), equalTo(new Instant(12345L)));
- }
-
- @Test
- public void getInputWatermarkTimeUsesWatermarkTime() {
- when(watermarks.getInputWatermark()).thenReturn(new Instant(8765L));
- assertThat(internals.currentInputWatermarkTime(), equalTo(new Instant(8765L)));
- }
-
- @Test
- public void getOutputWatermarkTimeUsesWatermarkTime() {
- when(watermarks.getOutputWatermark()).thenReturn(new Instant(25525L));
- assertThat(internals.currentOutputWatermarkTime(), equalTo(new Instant(25525L)));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitorTest.java
deleted file mode 100644
index b89340e..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitorTest.java
+++ /dev/null
@@ -1,192 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.hasItem;
-import static org.hamcrest.Matchers.not;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.coders.IterableCoder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.coders.VoidCoder;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-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.Keys;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.common.collect.ImmutableSet;
-
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.Collections;
-import java.util.Set;
-
-/**
- * Tests for {@link KeyedPValueTrackingVisitor}.
- */
-@RunWith(JUnit4.class)
-public class KeyedPValueTrackingVisitorTest {
- @Rule public ExpectedException thrown = ExpectedException.none();
-
- private KeyedPValueTrackingVisitor visitor;
- private Pipeline p;
-
- @Before
- public void setup() {
- PipelineOptions options = PipelineOptionsFactory.create();
-
- p = Pipeline.create(options);
- @SuppressWarnings("rawtypes")
- Set<Class<? extends PTransform>> producesKeyed =
- ImmutableSet.<Class<? extends PTransform>>of(PrimitiveKeyer.class, CompositeKeyer.class);
- visitor = KeyedPValueTrackingVisitor.create(producesKeyed);
- }
-
- @Test
- public void primitiveProducesKeyedOutputUnkeyedInputKeyedOutput() {
- PCollection<Integer> keyed =
- p.apply(Create.<Integer>of(1, 2, 3)).apply(new PrimitiveKeyer<Integer>());
-
- p.traverseTopologically(visitor);
- assertThat(visitor.getKeyedPValues(), hasItem(keyed));
- }
-
- @Test
- public void primitiveProducesKeyedOutputKeyedInputKeyedOutut() {
- PCollection<Integer> keyed =
- p.apply(Create.<Integer>of(1, 2, 3))
- .apply("firstKey", new PrimitiveKeyer<Integer>())
- .apply("secondKey", new PrimitiveKeyer<Integer>());
-
- p.traverseTopologically(visitor);
- assertThat(visitor.getKeyedPValues(), hasItem(keyed));
- }
-
- @Test
- public void compositeProducesKeyedOutputUnkeyedInputKeyedOutput() {
- PCollection<Integer> keyed =
- p.apply(Create.<Integer>of(1, 2, 3)).apply(new CompositeKeyer<Integer>());
-
- p.traverseTopologically(visitor);
- assertThat(visitor.getKeyedPValues(), hasItem(keyed));
- }
-
- @Test
- public void compositeProducesKeyedOutputKeyedInputKeyedOutut() {
- PCollection<Integer> keyed =
- p.apply(Create.<Integer>of(1, 2, 3))
- .apply("firstKey", new CompositeKeyer<Integer>())
- .apply("secondKey", new CompositeKeyer<Integer>());
-
- p.traverseTopologically(visitor);
- assertThat(visitor.getKeyedPValues(), hasItem(keyed));
- }
-
-
- @Test
- public void noInputUnkeyedOutput() {
- PCollection<KV<Integer, Iterable<Void>>> unkeyed =
- p.apply(
- Create.of(KV.<Integer, Iterable<Void>>of(-1, Collections.<Void>emptyList()))
- .withCoder(KvCoder.of(VarIntCoder.of(), IterableCoder.of(VoidCoder.of()))));
-
- p.traverseTopologically(visitor);
- assertThat(visitor.getKeyedPValues(), not(hasItem(unkeyed)));
- }
-
- @Test
- public void keyedInputNotProducesKeyedOutputUnkeyedOutput() {
- PCollection<Integer> onceKeyed =
- p.apply(Create.<Integer>of(1, 2, 3))
- .apply(new PrimitiveKeyer<Integer>())
- .apply(ParDo.of(new IdentityFn<Integer>()));
-
- p.traverseTopologically(visitor);
- assertThat(visitor.getKeyedPValues(), not(hasItem(onceKeyed)));
- }
-
- @Test
- public void unkeyedInputNotProducesKeyedOutputUnkeyedOutput() {
- PCollection<Integer> unkeyed =
- p.apply(Create.<Integer>of(1, 2, 3)).apply(ParDo.of(new IdentityFn<Integer>()));
-
- p.traverseTopologically(visitor);
- assertThat(visitor.getKeyedPValues(), not(hasItem(unkeyed)));
- }
-
- @Test
- public void traverseMultipleTimesThrows() {
- p.apply(
- Create.<KV<Integer, Void>>of(
- KV.of(1, (Void) null), KV.of(2, (Void) null), KV.of(3, (Void) null))
- .withCoder(KvCoder.of(VarIntCoder.of(), VoidCoder.of())))
- .apply(GroupByKey.<Integer, Void>create())
- .apply(Keys.<Integer>create());
-
- p.traverseTopologically(visitor);
-
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage("already been finalized");
- thrown.expectMessage(KeyedPValueTrackingVisitor.class.getSimpleName());
- p.traverseTopologically(visitor);
- }
-
- @Test
- public void getKeyedPValuesBeforeTraverseThrows() {
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage("completely traversed");
- thrown.expectMessage("getKeyedPValues");
- visitor.getKeyedPValues();
- }
-
- private static class PrimitiveKeyer<K> extends PTransform<PCollection<K>, PCollection<K>> {
- @Override
- public PCollection<K> apply(PCollection<K> input) {
- return PCollection.<K>createPrimitiveOutputInternal(
- input.getPipeline(), input.getWindowingStrategy(), input.isBounded())
- .setCoder(input.getCoder());
- }
- }
-
- private static class CompositeKeyer<K> extends PTransform<PCollection<K>, PCollection<K>> {
- @Override
- public PCollection<K> apply(PCollection<K> input) {
- return input.apply(new PrimitiveKeyer<K>()).apply(ParDo.of(new IdentityFn<K>()));
- }
- }
-
- private static class IdentityFn<K> extends DoFn<K, K> {
- @Override
- public void processElement(DoFn<K, K>.ProcessContext c) throws Exception {
- c.output(c.element());
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/MockClock.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/MockClock.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/MockClock.java
deleted file mode 100644
index 152cac4..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/MockClock.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * A clock that returns a constant value for now which can be set with calls to
- * {@link #set(Instant)}.
- *
- * <p>For uses of the {@link Clock} interface in unit tests.
- */
-public class MockClock implements Clock {
-
- private Instant now;
-
- public static MockClock fromInstant(Instant initial) {
- return new MockClock(initial);
- }
-
- private MockClock(Instant initialNow) {
- this.now = initialNow;
- }
-
- public void set(Instant newNow) {
- checkArgument(!newNow.isBefore(now), "Cannot move MockClock backwards in time from %s to %s",
- now, newNow);
- this.now = newNow;
- }
-
- public void advance(Duration duration) {
- checkArgument(
- duration.getMillis() > 0,
- "Cannot move MockClock backwards in time by duration %s",
- duration);
- set(now.plus(duration));
- }
-
- @Override
- public Instant now() {
- return now;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactoryTest.java
deleted file mode 100644
index a048e3a..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactoryTest.java
+++ /dev/null
@@ -1,431 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-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.ParDo;
-import org.apache.beam.sdk.transforms.ParDo.BoundMulti;
-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.OutputTimeFns;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.common.CounterSet;
-import org.apache.beam.sdk.util.state.BagState;
-import org.apache.beam.sdk.util.state.StateNamespace;
-import org.apache.beam.sdk.util.state.StateNamespaces;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.apache.beam.sdk.util.state.StateTags;
-import org.apache.beam.sdk.util.state.WatermarkHoldState;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.sdk.values.TupleTagList;
-
-import org.hamcrest.Matchers;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.Serializable;
-
-/**
- * Tests for {@link ParDoMultiEvaluatorFactory}.
- */
-@RunWith(JUnit4.class)
-public class ParDoMultiEvaluatorFactoryTest implements Serializable {
- private transient BundleFactory bundleFactory = InProcessBundleFactory.create();
-
- @Test
- public void testParDoMultiInMemoryTransformEvaluator() throws Exception {
- TestPipeline p = TestPipeline.create();
-
- PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
-
- TupleTag<KV<String, Integer>> mainOutputTag = new TupleTag<KV<String, Integer>>() {};
- final TupleTag<String> elementTag = new TupleTag<>();
- final TupleTag<Integer> lengthTag = new TupleTag<>();
-
- BoundMulti<String, KV<String, Integer>> pardo =
- ParDo.of(
- new DoFn<String, KV<String, Integer>>() {
- @Override
- public void processElement(ProcessContext c) {
- c.output(KV.<String, Integer>of(c.element(), c.element().length()));
- c.sideOutput(elementTag, c.element());
- c.sideOutput(lengthTag, c.element().length());
- }
- })
- .withOutputTags(mainOutputTag, TupleTagList.of(elementTag).and(lengthTag));
- PCollectionTuple outputTuple = input.apply(pardo);
-
- CommittedBundle<String> inputBundle =
- bundleFactory.createRootBundle(input).commit(Instant.now());
-
- PCollection<KV<String, Integer>> mainOutput = outputTuple.get(mainOutputTag);
- PCollection<String> elementOutput = outputTuple.get(elementTag);
- PCollection<Integer> lengthOutput = outputTuple.get(lengthTag);
-
- InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
- UncommittedBundle<KV<String, Integer>> mainOutputBundle =
- bundleFactory.createRootBundle(mainOutput);
- UncommittedBundle<String> elementOutputBundle = bundleFactory.createRootBundle(elementOutput);
- UncommittedBundle<Integer> lengthOutputBundle = bundleFactory.createRootBundle(lengthOutput);
-
- when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
- when(evaluationContext.createBundle(inputBundle, elementOutput))
- .thenReturn(elementOutputBundle);
- when(evaluationContext.createBundle(inputBundle, lengthOutput)).thenReturn(lengthOutputBundle);
-
- InProcessExecutionContext executionContext =
- new InProcessExecutionContext(null, null, null, null);
- when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
- .thenReturn(executionContext);
- CounterSet counters = new CounterSet();
- when(evaluationContext.createCounterSet()).thenReturn(counters);
-
- org.apache.beam.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
- new ParDoMultiEvaluatorFactory()
- .forApplication(
- mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
-
- evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
- evaluator.processElement(
- WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
- evaluator.processElement(
- WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
-
- InProcessTransformResult result = evaluator.finishBundle();
- assertThat(
- result.getOutputBundles(),
- Matchers.<UncommittedBundle<?>>containsInAnyOrder(
- lengthOutputBundle, mainOutputBundle, elementOutputBundle));
- assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
- assertThat(result.getCounters(), equalTo(counters));
-
- assertThat(
- mainOutputBundle.commit(Instant.now()).getElements(),
- Matchers.<WindowedValue<KV<String, Integer>>>containsInAnyOrder(
- WindowedValue.valueInGlobalWindow(KV.of("foo", 3)),
- WindowedValue.timestampedValueInGlobalWindow(KV.of("bara", 4), new Instant(1000)),
- WindowedValue.valueInGlobalWindow(
- KV.of("bazam", 5), PaneInfo.ON_TIME_AND_ONLY_FIRING)));
- assertThat(
- elementOutputBundle.commit(Instant.now()).getElements(),
- Matchers.<WindowedValue<String>>containsInAnyOrder(
- WindowedValue.valueInGlobalWindow("foo"),
- WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)),
- WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING)));
- assertThat(
- lengthOutputBundle.commit(Instant.now()).getElements(),
- Matchers.<WindowedValue<Integer>>containsInAnyOrder(
- WindowedValue.valueInGlobalWindow(3),
- WindowedValue.timestampedValueInGlobalWindow(4, new Instant(1000)),
- WindowedValue.valueInGlobalWindow(5, PaneInfo.ON_TIME_AND_ONLY_FIRING)));
- }
-
- @Test
- public void testParDoMultiUndeclaredSideOutput() throws Exception {
- TestPipeline p = TestPipeline.create();
-
- PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
-
- TupleTag<KV<String, Integer>> mainOutputTag = new TupleTag<KV<String, Integer>>() {};
- final TupleTag<String> elementTag = new TupleTag<>();
- final TupleTag<Integer> lengthTag = new TupleTag<>();
-
- BoundMulti<String, KV<String, Integer>> pardo =
- ParDo.of(
- new DoFn<String, KV<String, Integer>>() {
- @Override
- public void processElement(ProcessContext c) {
- c.output(KV.<String, Integer>of(c.element(), c.element().length()));
- c.sideOutput(elementTag, c.element());
- c.sideOutput(lengthTag, c.element().length());
- }
- })
- .withOutputTags(mainOutputTag, TupleTagList.of(elementTag));
- PCollectionTuple outputTuple = input.apply(pardo);
-
- CommittedBundle<String> inputBundle =
- bundleFactory.createRootBundle(input).commit(Instant.now());
-
- PCollection<KV<String, Integer>> mainOutput = outputTuple.get(mainOutputTag);
- PCollection<String> elementOutput = outputTuple.get(elementTag);
-
- InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
- UncommittedBundle<KV<String, Integer>> mainOutputBundle =
- bundleFactory.createRootBundle(mainOutput);
- UncommittedBundle<String> elementOutputBundle = bundleFactory.createRootBundle(elementOutput);
-
- when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
- when(evaluationContext.createBundle(inputBundle, elementOutput))
- .thenReturn(elementOutputBundle);
-
- InProcessExecutionContext executionContext =
- new InProcessExecutionContext(null, null, null, null);
- when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
- .thenReturn(executionContext);
- CounterSet counters = new CounterSet();
- when(evaluationContext.createCounterSet()).thenReturn(counters);
-
- org.apache.beam.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
- new ParDoMultiEvaluatorFactory()
- .forApplication(
- mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
-
- evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
- evaluator.processElement(
- WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
- evaluator.processElement(
- WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
-
- InProcessTransformResult result = evaluator.finishBundle();
- assertThat(
- result.getOutputBundles(),
- Matchers.<UncommittedBundle<?>>containsInAnyOrder(mainOutputBundle, elementOutputBundle));
- assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
- assertThat(result.getCounters(), equalTo(counters));
-
- assertThat(
- mainOutputBundle.commit(Instant.now()).getElements(),
- Matchers.<WindowedValue<KV<String, Integer>>>containsInAnyOrder(
- WindowedValue.valueInGlobalWindow(KV.of("foo", 3)),
- WindowedValue.timestampedValueInGlobalWindow(KV.of("bara", 4), new Instant(1000)),
- WindowedValue.valueInGlobalWindow(
- KV.of("bazam", 5), PaneInfo.ON_TIME_AND_ONLY_FIRING)));
- assertThat(
- elementOutputBundle.commit(Instant.now()).getElements(),
- Matchers.<WindowedValue<String>>containsInAnyOrder(
- WindowedValue.valueInGlobalWindow("foo"),
- WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)),
- WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING)));
- }
-
- @Test
- public void finishBundleWithStatePutsStateInResult() throws Exception {
- TestPipeline p = TestPipeline.create();
-
- PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
-
- TupleTag<KV<String, Integer>> mainOutputTag = new TupleTag<KV<String, Integer>>() {};
- final TupleTag<String> elementTag = new TupleTag<>();
-
- final StateTag<Object, WatermarkHoldState<BoundedWindow>> watermarkTag =
- StateTags.watermarkStateInternal("myId", OutputTimeFns.outputAtEndOfWindow());
- final StateTag<Object, BagState<String>> bagTag = StateTags.bag("myBag", StringUtf8Coder.of());
- final StateNamespace windowNs =
- StateNamespaces.window(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE);
- BoundMulti<String, KV<String, Integer>> pardo =
- ParDo.of(
- new DoFn<String, KV<String, Integer>>() {
- @Override
- public void processElement(ProcessContext c) {
- c.windowingInternals()
- .stateInternals()
- .state(StateNamespaces.global(), watermarkTag)
- .add(new Instant(20202L + c.element().length()));
- c.windowingInternals()
- .stateInternals()
- .state(
- StateNamespaces.window(
- GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE),
- bagTag)
- .add(c.element());
- }
- })
- .withOutputTags(mainOutputTag, TupleTagList.of(elementTag));
- PCollectionTuple outputTuple = input.apply(pardo);
-
- CommittedBundle<String> inputBundle =
- bundleFactory.createRootBundle(input).commit(Instant.now());
-
- PCollection<KV<String, Integer>> mainOutput = outputTuple.get(mainOutputTag);
- PCollection<String> elementOutput = outputTuple.get(elementTag);
-
- InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
- UncommittedBundle<KV<String, Integer>> mainOutputBundle =
- bundleFactory.createRootBundle(mainOutput);
- UncommittedBundle<String> elementOutputBundle = bundleFactory.createRootBundle(elementOutput);
-
- when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
- when(evaluationContext.createBundle(inputBundle, elementOutput))
- .thenReturn(elementOutputBundle);
-
- InProcessExecutionContext executionContext =
- new InProcessExecutionContext(null, "myKey", null, null);
- when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
- .thenReturn(executionContext);
- CounterSet counters = new CounterSet();
- when(evaluationContext.createCounterSet()).thenReturn(counters);
-
- org.apache.beam.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
- new ParDoMultiEvaluatorFactory()
- .forApplication(
- mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
-
- evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
- evaluator.processElement(
- WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
- evaluator.processElement(
- WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
-
- InProcessTransformResult result = evaluator.finishBundle();
- assertThat(
- result.getOutputBundles(),
- Matchers.<UncommittedBundle<?>>containsInAnyOrder(mainOutputBundle, elementOutputBundle));
- assertThat(result.getWatermarkHold(), equalTo(new Instant(20205L)));
- assertThat(result.getState(), not(nullValue()));
- assertThat(
- result.getState().state(StateNamespaces.global(), watermarkTag).read(),
- equalTo(new Instant(20205L)));
- assertThat(
- result.getState().state(windowNs, bagTag).read(),
- containsInAnyOrder("foo", "bara", "bazam"));
- }
-
- @Test
- public void finishBundleWithStateAndTimersPutsTimersInResult() throws Exception {
- TestPipeline p = TestPipeline.create();
-
- PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
-
- TupleTag<KV<String, Integer>> mainOutputTag = new TupleTag<KV<String, Integer>>() {};
- final TupleTag<String> elementTag = new TupleTag<>();
-
- final TimerData addedTimer =
- TimerData.of(
- StateNamespaces.window(
- IntervalWindow.getCoder(),
- new IntervalWindow(
- new Instant(0).plus(Duration.standardMinutes(5)),
- new Instant(1)
- .plus(Duration.standardMinutes(5))
- .plus(Duration.standardHours(1)))),
- new Instant(54541L),
- TimeDomain.EVENT_TIME);
- final TimerData deletedTimer =
- TimerData.of(
- StateNamespaces.window(
- IntervalWindow.getCoder(),
- new IntervalWindow(new Instant(0), new Instant(0).plus(Duration.standardHours(1)))),
- new Instant(3400000),
- TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
-
- BoundMulti<String, KV<String, Integer>> pardo =
- ParDo.of(
- new DoFn<String, KV<String, Integer>>() {
- @Override
- public void processElement(ProcessContext c) {
- c.windowingInternals().stateInternals();
- c.windowingInternals()
- .timerInternals()
- .setTimer(
- TimerData.of(
- StateNamespaces.window(
- IntervalWindow.getCoder(),
- new IntervalWindow(
- new Instant(0).plus(Duration.standardMinutes(5)),
- new Instant(1)
- .plus(Duration.standardMinutes(5))
- .plus(Duration.standardHours(1)))),
- new Instant(54541L),
- TimeDomain.EVENT_TIME));
- c.windowingInternals()
- .timerInternals()
- .deleteTimer(
- TimerData.of(
- StateNamespaces.window(
- IntervalWindow.getCoder(),
- new IntervalWindow(
- new Instant(0),
- new Instant(0).plus(Duration.standardHours(1)))),
- new Instant(3400000),
- TimeDomain.SYNCHRONIZED_PROCESSING_TIME));
- }
- })
- .withOutputTags(mainOutputTag, TupleTagList.of(elementTag));
- PCollectionTuple outputTuple = input.apply(pardo);
-
- CommittedBundle<String> inputBundle =
- bundleFactory.createRootBundle(input).commit(Instant.now());
-
- PCollection<KV<String, Integer>> mainOutput = outputTuple.get(mainOutputTag);
- PCollection<String> elementOutput = outputTuple.get(elementTag);
-
- InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
- UncommittedBundle<KV<String, Integer>> mainOutputBundle =
- bundleFactory.createRootBundle(mainOutput);
- UncommittedBundle<String> elementOutputBundle = bundleFactory.createRootBundle(elementOutput);
-
- when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
- when(evaluationContext.createBundle(inputBundle, elementOutput))
- .thenReturn(elementOutputBundle);
-
- InProcessExecutionContext executionContext =
- new InProcessExecutionContext(null, "myKey", null, null);
- when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
- .thenReturn(executionContext);
- CounterSet counters = new CounterSet();
- when(evaluationContext.createCounterSet()).thenReturn(counters);
-
- org.apache.beam.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
- new ParDoMultiEvaluatorFactory()
- .forApplication(
- mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
-
- evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
- evaluator.processElement(
- WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
- evaluator.processElement(
- WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
-
- InProcessTransformResult result = evaluator.finishBundle();
- assertThat(
- result.getTimerUpdate(),
- equalTo(
- TimerUpdate.builder("myKey")
- .setTimer(addedTimer)
- .setTimer(addedTimer)
- .setTimer(addedTimer)
- .deletedTimer(deletedTimer)
- .deletedTimer(deletedTimer)
- .deletedTimer(deletedTimer)
- .build()));
- }
-}
[07/17] incubator-beam git commit: Move InProcessRunner to its own
module
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManager.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManager.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManager.java
deleted file mode 100644
index d941cc7..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManager.java
+++ /dev/null
@@ -1,1327 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.apache.beam.sdk.util.TimerInternals;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PValue;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Function;
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ComparisonChain;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Ordering;
-import com.google.common.collect.SortedMultiset;
-import com.google.common.collect.TreeMultiset;
-
-import org.joda.time.Instant;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.EnumMap;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableSet;
-import java.util.Objects;
-import java.util.PriorityQueue;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.atomic.AtomicReference;
-
-import javax.annotation.Nullable;
-
-/**
- * Manages watermarks of {@link PCollection PCollections} and input and output watermarks of
- * {@link AppliedPTransform AppliedPTransforms} to provide event-time and completion tracking for
- * in-memory execution. {@link InMemoryWatermarkManager} is designed to update and return a
- * consistent view of watermarks in the presence of concurrent updates.
- *
- * <p>An {@link InMemoryWatermarkManager} is provided with the collection of root
- * {@link AppliedPTransform AppliedPTransforms} and a map of {@link PCollection PCollections} to
- * all the {@link AppliedPTransform AppliedPTransforms} that consume them at construction time.
- *
- * <p>Whenever a root {@link AppliedPTransform transform} produces elements, the
- * {@link InMemoryWatermarkManager} is provided with the produced elements and the output watermark
- * of the producing {@link AppliedPTransform transform}. The
- * {@link InMemoryWatermarkManager watermark manager} is responsible for computing the watermarks
- * of all {@link AppliedPTransform transforms} that consume one or more
- * {@link PCollection PCollections}.
- *
- * <p>Whenever a non-root {@link AppliedPTransform} finishes processing one or more in-flight
- * elements (referred to as the input {@link CommittedBundle bundle}), the following occurs
- * atomically:
- * <ul>
- * <li>All of the in-flight elements are removed from the collection of pending elements for the
- * {@link AppliedPTransform}.</li>
- * <li>All of the elements produced by the {@link AppliedPTransform} are added to the collection
- * of pending elements for each {@link AppliedPTransform} that consumes them.</li>
- * <li>The input watermark for the {@link AppliedPTransform} becomes the maximum value of
- * <ul>
- * <li>the previous input watermark</li>
- * <li>the minimum of
- * <ul>
- * <li>the timestamps of all currently pending elements</li>
- * <li>all input {@link PCollection} watermarks</li>
- * </ul>
- * </li>
- * </ul>
- * </li>
- * <li>The output watermark for the {@link AppliedPTransform} becomes the maximum of
- * <ul>
- * <li>the previous output watermark</li>
- * <li>the minimum of
- * <ul>
- * <li>the current input watermark</li>
- * <li>the current watermark holds</li>
- * </ul>
- * </li>
- * </ul>
- * </li>
- * <li>The watermark of the output {@link PCollection} can be advanced to the output watermark of
- * the {@link AppliedPTransform}</li>
- * <li>The watermark of all downstream {@link AppliedPTransform AppliedPTransforms} can be
- * advanced.</li>
- * </ul>
- *
- * <p>The watermark of a {@link PCollection} is equal to the output watermark of the
- * {@link AppliedPTransform} that produces it.
- *
- * <p>The watermarks for a {@link PTransform} are updated as follows when output is committed:<pre>
- * Watermark_In' = MAX(Watermark_In, MIN(U(TS_Pending), U(Watermark_InputPCollection)))
- * Watermark_Out' = MAX(Watermark_Out, MIN(Watermark_In', U(StateHold)))
- * Watermark_PCollection = Watermark_Out_ProducingPTransform
- * </pre>
- */
-public class InMemoryWatermarkManager {
- /**
- * The watermark of some {@link Pipeline} element, usually a {@link PTransform} or a
- * {@link PCollection}.
- *
- * <p>A watermark is a monotonically increasing value, which represents the point up to which the
- * system believes it has received all of the data. Data that arrives with a timestamp that is
- * before the watermark is considered late. {@link BoundedWindow#TIMESTAMP_MAX_VALUE} is a special
- * timestamp which indicates we have received all of the data and there will be no more on-time or
- * late data. This value is represented by {@link InMemoryWatermarkManager#THE_END_OF_TIME}.
- */
- private static interface Watermark {
- /**
- * Returns the current value of this watermark.
- */
- Instant get();
-
- /**
- * Refreshes the value of this watermark from its input watermarks and watermark holds.
- *
- * @return true if the value of the watermark has changed (and thus dependent watermark must
- * also be updated
- */
- WatermarkUpdate refresh();
- }
-
- /**
- * The result of computing a {@link Watermark}.
- */
- private static enum WatermarkUpdate {
- /** The watermark is later than the value at the previous time it was computed. */
- ADVANCED(true),
- /** The watermark is equal to the value at the previous time it was computed. */
- NO_CHANGE(false);
-
- private final boolean advanced;
-
- private WatermarkUpdate(boolean advanced) {
- this.advanced = advanced;
- }
-
- public boolean isAdvanced() {
- return advanced;
- }
-
- /**
- * Returns the {@link WatermarkUpdate} that is a result of combining the two watermark updates.
- *
- * If either of the input {@link WatermarkUpdate WatermarkUpdates} were advanced, the result
- * {@link WatermarkUpdate} has been advanced.
- */
- public WatermarkUpdate union(WatermarkUpdate that) {
- if (this.advanced) {
- return this;
- }
- return that;
- }
-
- /**
- * Returns the {@link WatermarkUpdate} based on the former and current
- * {@link Instant timestamps}.
- */
- public static WatermarkUpdate fromTimestamps(Instant oldTime, Instant currentTime) {
- if (currentTime.isAfter(oldTime)) {
- return ADVANCED;
- }
- return NO_CHANGE;
- }
- }
-
- /**
- * The input {@link Watermark} of an {@link AppliedPTransform}.
- *
- * <p>At any point, the value of an {@link AppliedPTransformInputWatermark} is equal to the
- * minimum watermark across all of its input {@link Watermark Watermarks}, and the minimum
- * timestamp of all of the pending elements, restricted to be monotonically increasing.
- *
- * <p>See {@link #refresh()} for more information.
- */
- private static class AppliedPTransformInputWatermark implements Watermark {
- private final Collection<? extends Watermark> inputWatermarks;
- private final SortedMultiset<WindowedValue<?>> pendingElements;
- private final Map<Object, NavigableSet<TimerData>> objectTimers;
-
- private AtomicReference<Instant> currentWatermark;
-
- public AppliedPTransformInputWatermark(Collection<? extends Watermark> inputWatermarks) {
- this.inputWatermarks = inputWatermarks;
- this.pendingElements = TreeMultiset.create(new WindowedValueByTimestampComparator());
- this.objectTimers = new HashMap<>();
- currentWatermark = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
- }
-
- @Override
- public Instant get() {
- return currentWatermark.get();
- }
-
- /**
- * {@inheritDoc}.
- *
- * <p>When refresh is called, the value of the {@link AppliedPTransformInputWatermark} becomes
- * equal to the maximum value of
- * <ul>
- * <li>the previous input watermark</li>
- * <li>the minimum of
- * <ul>
- * <li>the timestamps of all currently pending elements</li>
- * <li>all input {@link PCollection} watermarks</li>
- * </ul>
- * </li>
- * </ul>
- */
- @Override
- public synchronized WatermarkUpdate refresh() {
- Instant oldWatermark = currentWatermark.get();
- Instant minInputWatermark = BoundedWindow.TIMESTAMP_MAX_VALUE;
- for (Watermark inputWatermark : inputWatermarks) {
- minInputWatermark = INSTANT_ORDERING.min(minInputWatermark, inputWatermark.get());
- }
- if (!pendingElements.isEmpty()) {
- minInputWatermark = INSTANT_ORDERING.min(
- minInputWatermark, pendingElements.firstEntry().getElement().getTimestamp());
- }
- Instant newWatermark = INSTANT_ORDERING.max(oldWatermark, minInputWatermark);
- currentWatermark.set(newWatermark);
- return WatermarkUpdate.fromTimestamps(oldWatermark, newWatermark);
- }
-
- private synchronized void addPendingElements(Iterable<? extends WindowedValue<?>> newPending) {
- for (WindowedValue<?> pendingElement : newPending) {
- pendingElements.add(pendingElement);
- }
- }
-
- private synchronized void removePendingElements(
- Iterable<? extends WindowedValue<?>> finishedElements) {
- for (WindowedValue<?> finishedElement : finishedElements) {
- pendingElements.remove(finishedElement);
- }
- }
-
- private synchronized void updateTimers(TimerUpdate update) {
- NavigableSet<TimerData> keyTimers = objectTimers.get(update.key);
- if (keyTimers == null) {
- keyTimers = new TreeSet<>();
- objectTimers.put(update.key, keyTimers);
- }
- for (TimerData timer : update.setTimers) {
- if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) {
- keyTimers.add(timer);
- }
- }
- for (TimerData timer : update.deletedTimers) {
- if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) {
- keyTimers.remove(timer);
- }
- }
- // We don't keep references to timers that have been fired and delivered via #getFiredTimers()
- }
-
- private synchronized Map<Object, List<TimerData>> extractFiredEventTimeTimers() {
- return extractFiredTimers(currentWatermark.get(), objectTimers);
- }
-
- @Override
- public synchronized String toString() {
- return MoreObjects.toStringHelper(AppliedPTransformInputWatermark.class)
- .add("pendingElements", pendingElements)
- .add("currentWatermark", currentWatermark)
- .toString();
- }
- }
-
- /**
- * The output {@link Watermark} of an {@link AppliedPTransform}.
- *
- * <p>The value of an {@link AppliedPTransformOutputWatermark} is equal to the minimum of the
- * current watermark hold and the {@link AppliedPTransformInputWatermark} for the same
- * {@link AppliedPTransform}, restricted to be monotonically increasing. See
- * {@link #refresh()} for more information.
- */
- private static class AppliedPTransformOutputWatermark implements Watermark {
- private final Watermark inputWatermark;
- private final PerKeyHolds holds;
- private AtomicReference<Instant> currentWatermark;
-
- public AppliedPTransformOutputWatermark(AppliedPTransformInputWatermark inputWatermark) {
- this.inputWatermark = inputWatermark;
- holds = new PerKeyHolds();
- currentWatermark = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
- }
-
- public synchronized void updateHold(Object key, Instant newHold) {
- if (newHold == null) {
- holds.removeHold(key);
- } else {
- holds.updateHold(key, newHold);
- }
- }
-
- @Override
- public Instant get() {
- return currentWatermark.get();
- }
-
- /**
- * {@inheritDoc}.
- *
- * <p>When refresh is called, the value of the {@link AppliedPTransformOutputWatermark} becomes
- * equal to the maximum value of:
- * <ul>
- * <li>the previous output watermark</li>
- * <li>the minimum of
- * <ul>
- * <li>the current input watermark</li>
- * <li>the current watermark holds</li>
- * </ul>
- * </li>
- * </ul>
- */
- @Override
- public synchronized WatermarkUpdate refresh() {
- Instant oldWatermark = currentWatermark.get();
- Instant newWatermark = INSTANT_ORDERING.min(inputWatermark.get(), holds.getMinHold());
- newWatermark = INSTANT_ORDERING.max(oldWatermark, newWatermark);
- currentWatermark.set(newWatermark);
- return WatermarkUpdate.fromTimestamps(oldWatermark, newWatermark);
- }
-
- @Override
- public synchronized String toString() {
- return MoreObjects.toStringHelper(AppliedPTransformOutputWatermark.class)
- .add("holds", holds)
- .add("currentWatermark", currentWatermark)
- .toString();
- }
- }
-
- /**
- * The input {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} hold for an
- * {@link AppliedPTransform}.
- *
- * <p>At any point, the hold value of an {@link SynchronizedProcessingTimeInputWatermark} is equal
- * to the minimum across all pending bundles at the {@link AppliedPTransform} and all upstream
- * {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} watermarks. The value of the input
- * synchronized processing time at any step is equal to the maximum of:
- * <ul>
- * <li>The most recently returned synchronized processing input time
- * <li>The minimum of
- * <ul>
- * <li>The current processing time
- * <li>The current synchronized processing time input hold
- * </ul>
- * </ul>
- */
- private static class SynchronizedProcessingTimeInputWatermark implements Watermark {
- private final Collection<? extends Watermark> inputWms;
- private final Collection<CommittedBundle<?>> pendingBundles;
- private final Map<Object, NavigableSet<TimerData>> processingTimers;
- private final Map<Object, NavigableSet<TimerData>> synchronizedProcessingTimers;
-
- private final PriorityQueue<TimerData> pendingTimers;
-
- private AtomicReference<Instant> earliestHold;
-
- public SynchronizedProcessingTimeInputWatermark(Collection<? extends Watermark> inputWms) {
- this.inputWms = inputWms;
- this.pendingBundles = new HashSet<>();
- this.processingTimers = new HashMap<>();
- this.synchronizedProcessingTimers = new HashMap<>();
- this.pendingTimers = new PriorityQueue<>();
- Instant initialHold = BoundedWindow.TIMESTAMP_MAX_VALUE;
- for (Watermark wm : inputWms) {
- initialHold = INSTANT_ORDERING.min(initialHold, wm.get());
- }
- earliestHold = new AtomicReference<>(initialHold);
- }
-
- @Override
- public Instant get() {
- return earliestHold.get();
- }
-
- /**
- * {@inheritDoc}.
- *
- * <p>When refresh is called, the value of the {@link SynchronizedProcessingTimeInputWatermark}
- * becomes equal to the minimum value of
- * <ul>
- * <li>the timestamps of all currently pending bundles</li>
- * <li>all input {@link PCollection} synchronized processing time watermarks</li>
- * </ul>
- *
- * <p>Note that this value is not monotonic, but the returned value for the synchronized
- * processing time must be.
- */
- @Override
- public synchronized WatermarkUpdate refresh() {
- Instant oldHold = earliestHold.get();
- Instant minTime = THE_END_OF_TIME.get();
- for (Watermark input : inputWms) {
- minTime = INSTANT_ORDERING.min(minTime, input.get());
- }
- for (CommittedBundle<?> bundle : pendingBundles) {
- // TODO: Track elements in the bundle by the processing time they were output instead of
- // entire bundles. Requried to support arbitrarily splitting and merging bundles between
- // steps
- minTime = INSTANT_ORDERING.min(minTime, bundle.getSynchronizedProcessingOutputWatermark());
- }
- earliestHold.set(minTime);
- return WatermarkUpdate.fromTimestamps(oldHold, minTime);
- }
-
- public synchronized void addPending(CommittedBundle<?> bundle) {
- pendingBundles.add(bundle);
- }
-
- public synchronized void removePending(CommittedBundle<?> bundle) {
- pendingBundles.remove(bundle);
- }
-
- /**
- * Return the earliest timestamp of the earliest timer that has not been completed. This is
- * either the earliest timestamp across timers that have not been completed, or the earliest
- * timestamp across timers that have been delivered but have not been completed.
- */
- public synchronized Instant getEarliestTimerTimestamp() {
- Instant earliest = THE_END_OF_TIME.get();
- for (NavigableSet<TimerData> timers : processingTimers.values()) {
- if (!timers.isEmpty()) {
- earliest = INSTANT_ORDERING.min(timers.first().getTimestamp(), earliest);
- }
- }
- for (NavigableSet<TimerData> timers : synchronizedProcessingTimers.values()) {
- if (!timers.isEmpty()) {
- earliest = INSTANT_ORDERING.min(timers.first().getTimestamp(), earliest);
- }
- }
- if (!pendingTimers.isEmpty()) {
- earliest = INSTANT_ORDERING.min(pendingTimers.peek().getTimestamp(), earliest);
- }
- return earliest;
- }
-
- private synchronized void updateTimers(TimerUpdate update) {
- for (TimerData completedTimer : update.completedTimers) {
- pendingTimers.remove(completedTimer);
- }
- Map<TimeDomain, NavigableSet<TimerData>> timerMap = timerMap(update.key);
- for (TimerData addedTimer : update.setTimers) {
- NavigableSet<TimerData> timerQueue = timerMap.get(addedTimer.getDomain());
- if (timerQueue != null) {
- timerQueue.add(addedTimer);
- }
- }
- for (TimerData deletedTimer : update.deletedTimers) {
- NavigableSet<TimerData> timerQueue = timerMap.get(deletedTimer.getDomain());
- if (timerQueue != null) {
- timerQueue.remove(deletedTimer);
- }
- }
- }
-
- private synchronized Map<Object, List<TimerData>> extractFiredDomainTimers(
- TimeDomain domain, Instant firingTime) {
- Map<Object, List<TimerData>> firedTimers;
- switch (domain) {
- case PROCESSING_TIME:
- firedTimers = extractFiredTimers(firingTime, processingTimers);
- break;
- case SYNCHRONIZED_PROCESSING_TIME:
- firedTimers =
- extractFiredTimers(
- INSTANT_ORDERING.min(firingTime, earliestHold.get()),
- synchronizedProcessingTimers);
- break;
- default:
- throw new IllegalArgumentException(
- "Called getFiredTimers on a Synchronized Processing Time watermark"
- + " and gave a non-processing time domain "
- + domain);
- }
- for (Map.Entry<Object, ? extends Collection<TimerData>> firedTimer : firedTimers.entrySet()) {
- pendingTimers.addAll(firedTimer.getValue());
- }
- return firedTimers;
- }
-
- private Map<TimeDomain, NavigableSet<TimerData>> timerMap(Object key) {
- NavigableSet<TimerData> processingQueue = processingTimers.get(key);
- if (processingQueue == null) {
- processingQueue = new TreeSet<>();
- processingTimers.put(key, processingQueue);
- }
- NavigableSet<TimerData> synchronizedProcessingQueue =
- synchronizedProcessingTimers.get(key);
- if (synchronizedProcessingQueue == null) {
- synchronizedProcessingQueue = new TreeSet<>();
- synchronizedProcessingTimers.put(key, synchronizedProcessingQueue);
- }
- EnumMap<TimeDomain, NavigableSet<TimerData>> result = new EnumMap<>(TimeDomain.class);
- result.put(TimeDomain.PROCESSING_TIME, processingQueue);
- result.put(TimeDomain.SYNCHRONIZED_PROCESSING_TIME, synchronizedProcessingQueue);
- return result;
- }
-
- @Override
- public synchronized String toString() {
- return MoreObjects.toStringHelper(SynchronizedProcessingTimeInputWatermark.class)
- .add("earliestHold", earliestHold)
- .toString();
- }
- }
-
- /**
- * The output {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} hold for an
- * {@link AppliedPTransform}.
- *
- * <p>At any point, the hold value of an {@link SynchronizedProcessingTimeOutputWatermark} is
- * equal to the minimum across all incomplete timers at the {@link AppliedPTransform} and all
- * upstream {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} watermarks. The value of the output
- * synchronized processing time at any step is equal to the maximum of:
- * <ul>
- * <li>The most recently returned synchronized processing output time
- * <li>The minimum of
- * <ul>
- * <li>The current processing time
- * <li>The current synchronized processing time output hold
- * </ul>
- * </ul>
- */
- private static class SynchronizedProcessingTimeOutputWatermark implements Watermark {
- private final SynchronizedProcessingTimeInputWatermark inputWm;
- private AtomicReference<Instant> latestRefresh;
-
- public SynchronizedProcessingTimeOutputWatermark(
- SynchronizedProcessingTimeInputWatermark inputWm) {
- this.inputWm = inputWm;
- this.latestRefresh = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
- }
-
- @Override
- public Instant get() {
- return latestRefresh.get();
- }
-
- /**
- * {@inheritDoc}.
- *
- * <p>When refresh is called, the value of the {@link SynchronizedProcessingTimeOutputWatermark}
- * becomes equal to the minimum value of:
- * <ul>
- * <li>the current input watermark.
- * <li>all {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} timers that are based on the input
- * watermark.
- * <li>all {@link TimeDomain#PROCESSING_TIME} timers that are based on the input watermark.
- * </ul>
- *
- * <p>Note that this value is not monotonic, but the returned value for the synchronized
- * processing time must be.
- */
- @Override
- public synchronized WatermarkUpdate refresh() {
- // Hold the output synchronized processing time to the input watermark, which takes into
- // account buffered bundles, and the earliest pending timer, which determines what to hold
- // downstream timers to.
- Instant oldRefresh = latestRefresh.get();
- Instant newTimestamp =
- INSTANT_ORDERING.min(inputWm.get(), inputWm.getEarliestTimerTimestamp());
- latestRefresh.set(newTimestamp);
- return WatermarkUpdate.fromTimestamps(oldRefresh, newTimestamp);
- }
-
- @Override
- public synchronized String toString() {
- return MoreObjects.toStringHelper(SynchronizedProcessingTimeOutputWatermark.class)
- .add("latestRefresh", latestRefresh)
- .toString();
- }
- }
-
- /**
- * The {@code Watermark} that is after the latest time it is possible to represent in the global
- * window. This is a distinguished value representing a complete {@link PTransform}.
- */
- private static final Watermark THE_END_OF_TIME = new Watermark() {
- @Override
- public WatermarkUpdate refresh() {
- // THE_END_OF_TIME is a distinguished value that cannot be advanced.
- return WatermarkUpdate.NO_CHANGE;
- }
-
- @Override
- public Instant get() {
- return BoundedWindow.TIMESTAMP_MAX_VALUE;
- }
- };
-
- private static final Ordering<Instant> INSTANT_ORDERING = Ordering.natural();
-
- /**
- * A function that takes a WindowedValue and returns the exploded representation of that
- * {@link WindowedValue}.
- */
- private static final Function<WindowedValue<?>, ? extends Iterable<? extends WindowedValue<?>>>
- EXPLODE_WINDOWS_FN =
- new Function<WindowedValue<?>, Iterable<? extends WindowedValue<?>>>() {
- @Override
- public Iterable<? extends WindowedValue<?>> apply(WindowedValue<?> input) {
- return input.explodeWindows();
- }
- };
-
- /**
- * For each (Object, PriorityQueue) pair in the provided map, remove each Timer that is before the
- * latestTime argument and put in in the result with the same key, then remove all of the keys
- * which have no more pending timers.
- *
- * The result collection retains ordering of timers (from earliest to latest).
- */
- private static Map<Object, List<TimerData>> extractFiredTimers(
- Instant latestTime, Map<Object, NavigableSet<TimerData>> objectTimers) {
- Map<Object, List<TimerData>> result = new HashMap<>();
- Set<Object> emptyKeys = new HashSet<>();
- for (Map.Entry<Object, NavigableSet<TimerData>> pendingTimers : objectTimers.entrySet()) {
- NavigableSet<TimerData> timers = pendingTimers.getValue();
- if (!timers.isEmpty() && timers.first().getTimestamp().isBefore(latestTime)) {
- ArrayList<TimerData> keyFiredTimers = new ArrayList<>();
- result.put(pendingTimers.getKey(), keyFiredTimers);
- while (!timers.isEmpty() && timers.first().getTimestamp().isBefore(latestTime)) {
- keyFiredTimers.add(timers.first());
- timers.remove(timers.first());
- }
- }
- if (timers.isEmpty()) {
- emptyKeys.add(pendingTimers.getKey());
- }
- }
- objectTimers.keySet().removeAll(emptyKeys);
- return result;
- }
-
- ////////////////////////////////////////////////////////////////////////////////////////////////
-
- /**
- * The {@link Clock} providing the current time in the {@link TimeDomain#PROCESSING_TIME} domain.
- */
- private final Clock clock;
-
- /**
- * A map from each {@link PCollection} to all {@link AppliedPTransform PTransform applications}
- * that consume that {@link PCollection}.
- */
- private final Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers;
-
- /**
- * The input and output watermark of each {@link AppliedPTransform}.
- */
- private final Map<AppliedPTransform<?, ?, ?>, TransformWatermarks> transformToWatermarks;
-
- /**
- * Creates a new {@link InMemoryWatermarkManager}. All watermarks within the newly created
- * {@link InMemoryWatermarkManager} start at {@link BoundedWindow#TIMESTAMP_MIN_VALUE}, the
- * minimum watermark, with no watermark holds or pending elements.
- *
- * @param rootTransforms the root-level transforms of the {@link Pipeline}
- * @param consumers a mapping between each {@link PCollection} in the {@link Pipeline} to the
- * transforms that consume it as a part of their input
- */
- public static InMemoryWatermarkManager create(
- Clock clock,
- Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
- Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers) {
- return new InMemoryWatermarkManager(clock, rootTransforms, consumers);
- }
-
- private InMemoryWatermarkManager(
- Clock clock,
- Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
- Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers) {
- this.clock = clock;
- this.consumers = consumers;
-
- transformToWatermarks = new HashMap<>();
-
- for (AppliedPTransform<?, ?, ?> rootTransform : rootTransforms) {
- getTransformWatermark(rootTransform);
- }
- for (Collection<AppliedPTransform<?, ?, ?>> intermediateTransforms : consumers.values()) {
- for (AppliedPTransform<?, ?, ?> transform : intermediateTransforms) {
- getTransformWatermark(transform);
- }
- }
- }
-
- private TransformWatermarks getTransformWatermark(AppliedPTransform<?, ?, ?> transform) {
- TransformWatermarks wms = transformToWatermarks.get(transform);
- if (wms == null) {
- List<Watermark> inputCollectionWatermarks = getInputWatermarks(transform);
- AppliedPTransformInputWatermark inputWatermark =
- new AppliedPTransformInputWatermark(inputCollectionWatermarks);
- AppliedPTransformOutputWatermark outputWatermark =
- new AppliedPTransformOutputWatermark(inputWatermark);
-
- SynchronizedProcessingTimeInputWatermark inputProcessingWatermark =
- new SynchronizedProcessingTimeInputWatermark(getInputProcessingWatermarks(transform));
- SynchronizedProcessingTimeOutputWatermark outputProcessingWatermark =
- new SynchronizedProcessingTimeOutputWatermark(inputProcessingWatermark);
-
- wms =
- new TransformWatermarks(
- inputWatermark, outputWatermark, inputProcessingWatermark, outputProcessingWatermark);
- transformToWatermarks.put(transform, wms);
- }
- return wms;
- }
-
- private Collection<Watermark> getInputProcessingWatermarks(
- AppliedPTransform<?, ?, ?> transform) {
- ImmutableList.Builder<Watermark> inputWmsBuilder = ImmutableList.builder();
- Collection<? extends PValue> inputs = transform.getInput().expand();
- if (inputs.isEmpty()) {
- inputWmsBuilder.add(THE_END_OF_TIME);
- }
- for (PValue pvalue : inputs) {
- Watermark producerOutputWatermark =
- getTransformWatermark(pvalue.getProducingTransformInternal())
- .synchronizedProcessingOutputWatermark;
- inputWmsBuilder.add(producerOutputWatermark);
- }
- return inputWmsBuilder.build();
- }
-
- private List<Watermark> getInputWatermarks(AppliedPTransform<?, ?, ?> transform) {
- ImmutableList.Builder<Watermark> inputWatermarksBuilder = ImmutableList.builder();
- Collection<? extends PValue> inputs = transform.getInput().expand();
- if (inputs.isEmpty()) {
- inputWatermarksBuilder.add(THE_END_OF_TIME);
- }
- for (PValue pvalue : inputs) {
- Watermark producerOutputWatermark =
- getTransformWatermark(pvalue.getProducingTransformInternal()).outputWatermark;
- inputWatermarksBuilder.add(producerOutputWatermark);
- }
- List<Watermark> inputCollectionWatermarks = inputWatermarksBuilder.build();
- return inputCollectionWatermarks;
- }
-
- ////////////////////////////////////////////////////////////////////////////////////////////////
-
- /**
- * Gets the input and output watermarks for an {@link AppliedPTransform}. If the
- * {@link AppliedPTransform PTransform} has not processed any elements, return a watermark of
- * {@link BoundedWindow#TIMESTAMP_MIN_VALUE}.
- *
- * @return a snapshot of the input watermark and output watermark for the provided transform
- */
- public TransformWatermarks getWatermarks(AppliedPTransform<?, ?, ?> transform) {
- return transformToWatermarks.get(transform);
- }
-
- /**
- * Updates the watermarks of a transform with one or more inputs.
- *
- * <p>Each transform has two monotonically increasing watermarks: the input watermark, which can,
- * at any time, be updated to equal:
- * <pre>
- * MAX(CurrentInputWatermark, MIN(PendingElements, InputPCollectionWatermarks))
- * </pre>
- * and the output watermark, which can, at any time, be updated to equal:
- * <pre>
- * MAX(CurrentOutputWatermark, MIN(InputWatermark, WatermarkHolds))
- * </pre>.
- *
- * @param completed the input that has completed
- * @param transform the transform that has completed processing the input
- * @param outputs the bundles the transform has output
- * @param earliestHold the earliest watermark hold in the transform's state. {@code null} if there
- * is no hold
- */
- public void updateWatermarks(
- @Nullable CommittedBundle<?> completed,
- AppliedPTransform<?, ?, ?> transform,
- TimerUpdate timerUpdate,
- Iterable<? extends CommittedBundle<?>> outputs,
- @Nullable Instant earliestHold) {
- updatePending(completed, transform, timerUpdate, outputs);
- TransformWatermarks transformWms = transformToWatermarks.get(transform);
- transformWms.setEventTimeHold(completed == null ? null : completed.getKey(), earliestHold);
- refreshWatermarks(transform);
- }
-
- private void refreshWatermarks(AppliedPTransform<?, ?, ?> transform) {
- TransformWatermarks myWatermarks = transformToWatermarks.get(transform);
- WatermarkUpdate updateResult = myWatermarks.refresh();
- if (updateResult.isAdvanced()) {
- for (PValue outputPValue : transform.getOutput().expand()) {
- Collection<AppliedPTransform<?, ?, ?>> downstreamTransforms = consumers.get(outputPValue);
- if (downstreamTransforms != null) {
- for (AppliedPTransform<?, ?, ?> downstreamTransform : downstreamTransforms) {
- refreshWatermarks(downstreamTransform);
- }
- }
- }
- }
- }
-
- /**
- * Removes all of the completed Timers from the collection of pending timers, adds all new timers,
- * and removes all deleted timers. Removes all elements consumed by the input bundle from the
- * {@link PTransform PTransforms} collection of pending elements, and adds all elements produced
- * by the {@link PTransform} to the pending queue of each consumer.
- */
- private void updatePending(
- CommittedBundle<?> input,
- AppliedPTransform<?, ?, ?> transform,
- TimerUpdate timerUpdate,
- Iterable<? extends CommittedBundle<?>> outputs) {
- TransformWatermarks completedTransform = transformToWatermarks.get(transform);
- completedTransform.updateTimers(timerUpdate);
- if (input != null) {
- completedTransform.removePending(input);
- }
-
- for (CommittedBundle<?> bundle : outputs) {
- for (AppliedPTransform<?, ?, ?> consumer : consumers.get(bundle.getPCollection())) {
- TransformWatermarks watermarks = transformToWatermarks.get(consumer);
- watermarks.addPending(bundle);
- }
- }
- }
-
- /**
- * Returns a map of each {@link PTransform} that has pending timers to those timers. All of the
- * pending timers will be removed from this {@link InMemoryWatermarkManager}.
- */
- public Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> extractFiredTimers() {
- Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> allTimers = new HashMap<>();
- for (Map.Entry<AppliedPTransform<?, ?, ?>, TransformWatermarks> watermarksEntry :
- transformToWatermarks.entrySet()) {
- Map<Object, FiredTimers> keyFiredTimers = watermarksEntry.getValue().extractFiredTimers();
- if (!keyFiredTimers.isEmpty()) {
- allTimers.put(watermarksEntry.getKey(), keyFiredTimers);
- }
- }
- return allTimers;
- }
-
- /**
- * A (key, Instant) pair that holds the watermark. Holds are per-key, but the watermark is global,
- * and as such the watermark manager must track holds and the release of holds on a per-key basis.
- *
- * <p>The {@link #compareTo(KeyedHold)} method of {@link KeyedHold} is not consistent with equals,
- * as the key is arbitrarily ordered via identity, rather than object equality.
- */
- private static final class KeyedHold implements Comparable<KeyedHold> {
- private static final Ordering<Object> KEY_ORDERING = Ordering.arbitrary().nullsLast();
-
- private final Object key;
- private final Instant timestamp;
-
- /**
- * Create a new KeyedHold with the specified key and timestamp.
- */
- public static KeyedHold of(Object key, Instant timestamp) {
- return new KeyedHold(key, MoreObjects.firstNonNull(timestamp, THE_END_OF_TIME.get()));
- }
-
- private KeyedHold(Object key, Instant timestamp) {
- this.key = key;
- this.timestamp = timestamp;
- }
-
- @Override
- public int compareTo(KeyedHold that) {
- return ComparisonChain.start()
- .compare(this.timestamp, that.timestamp)
- .compare(this.key, that.key, KEY_ORDERING)
- .result();
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(timestamp, key);
- }
-
- @Override
- public boolean equals(Object other) {
- if (other == null || !(other instanceof KeyedHold)) {
- return false;
- }
- KeyedHold that = (KeyedHold) other;
- return Objects.equals(this.timestamp, that.timestamp) && Objects.equals(this.key, that.key);
- }
-
- /**
- * Get the value of this {@link KeyedHold}.
- */
- public Instant getTimestamp() {
- return timestamp;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(KeyedHold.class)
- .add("key", key)
- .add("hold", timestamp)
- .toString();
- }
- }
-
- private static class PerKeyHolds {
- private final Map<Object, KeyedHold> keyedHolds;
- private final PriorityQueue<KeyedHold> allHolds;
-
- private PerKeyHolds() {
- this.keyedHolds = new HashMap<>();
- this.allHolds = new PriorityQueue<>();
- }
-
- /**
- * Gets the minimum hold across all keys in this {@link PerKeyHolds}, or THE_END_OF_TIME if
- * there are no holds within this {@link PerKeyHolds}.
- */
- public Instant getMinHold() {
- return allHolds.isEmpty() ? THE_END_OF_TIME.get() : allHolds.peek().getTimestamp();
- }
-
- /**
- * Updates the hold of the provided key to the provided value, removing any other holds for
- * the same key.
- */
- public void updateHold(@Nullable Object key, Instant newHold) {
- removeHold(key);
- KeyedHold newKeyedHold = KeyedHold.of(key, newHold);
- keyedHolds.put(key, newKeyedHold);
- allHolds.offer(newKeyedHold);
- }
-
- /**
- * Removes the hold of the provided key.
- */
- public void removeHold(Object key) {
- KeyedHold oldHold = keyedHolds.get(key);
- if (oldHold != null) {
- allHolds.remove(oldHold);
- }
- }
- }
-
- /**
- * A reference to the input and output watermarks of an {@link AppliedPTransform}.
- */
- public class TransformWatermarks {
- private final AppliedPTransformInputWatermark inputWatermark;
- private final AppliedPTransformOutputWatermark outputWatermark;
-
- private final SynchronizedProcessingTimeInputWatermark synchronizedProcessingInputWatermark;
- private final SynchronizedProcessingTimeOutputWatermark synchronizedProcessingOutputWatermark;
-
- private Instant latestSynchronizedInputWm;
- private Instant latestSynchronizedOutputWm;
-
- private TransformWatermarks(
- AppliedPTransformInputWatermark inputWatermark,
- AppliedPTransformOutputWatermark outputWatermark,
- SynchronizedProcessingTimeInputWatermark inputSynchProcessingWatermark,
- SynchronizedProcessingTimeOutputWatermark outputSynchProcessingWatermark) {
- this.inputWatermark = inputWatermark;
- this.outputWatermark = outputWatermark;
-
- this.synchronizedProcessingInputWatermark = inputSynchProcessingWatermark;
- this.synchronizedProcessingOutputWatermark = outputSynchProcessingWatermark;
- this.latestSynchronizedInputWm = BoundedWindow.TIMESTAMP_MIN_VALUE;
- this.latestSynchronizedOutputWm = BoundedWindow.TIMESTAMP_MIN_VALUE;
- }
-
- /**
- * Returns the input watermark of the {@link AppliedPTransform}.
- */
- public Instant getInputWatermark() {
- return Preconditions.checkNotNull(inputWatermark.get());
- }
-
- /**
- * Returns the output watermark of the {@link AppliedPTransform}.
- */
- public Instant getOutputWatermark() {
- return outputWatermark.get();
- }
-
- /**
- * Returns the synchronized processing input time of the {@link AppliedPTransform}.
- *
- * <p>The returned value is guaranteed to be monotonically increasing, and outside of the
- * presence of holds, will increase as the system time progresses.
- */
- public synchronized Instant getSynchronizedProcessingInputTime() {
- latestSynchronizedInputWm = INSTANT_ORDERING.max(
- latestSynchronizedInputWm,
- INSTANT_ORDERING.min(clock.now(), synchronizedProcessingInputWatermark.get()));
- return latestSynchronizedInputWm;
- }
-
- /**
- * Returns the synchronized processing output time of the {@link AppliedPTransform}.
- *
- * <p>The returned value is guaranteed to be monotonically increasing, and outside of the
- * presence of holds, will increase as the system time progresses.
- */
- public synchronized Instant getSynchronizedProcessingOutputTime() {
- latestSynchronizedOutputWm = INSTANT_ORDERING.max(
- latestSynchronizedOutputWm,
- INSTANT_ORDERING.min(clock.now(), synchronizedProcessingOutputWatermark.get()));
- return latestSynchronizedOutputWm;
- }
-
- private WatermarkUpdate refresh() {
- inputWatermark.refresh();
- synchronizedProcessingInputWatermark.refresh();
- WatermarkUpdate eventOutputUpdate = outputWatermark.refresh();
- WatermarkUpdate syncOutputUpdate = synchronizedProcessingOutputWatermark.refresh();
- return eventOutputUpdate.union(syncOutputUpdate);
- }
-
- private void setEventTimeHold(Object key, Instant newHold) {
- outputWatermark.updateHold(key, newHold);
- }
-
- private void removePending(CommittedBundle<?> bundle) {
- inputWatermark.removePendingElements(elementsFromBundle(bundle));
- synchronizedProcessingInputWatermark.removePending(bundle);
- }
-
- private void addPending(CommittedBundle<?> bundle) {
- inputWatermark.addPendingElements(elementsFromBundle(bundle));
- synchronizedProcessingInputWatermark.addPending(bundle);
- }
-
- private Iterable<? extends WindowedValue<?>> elementsFromBundle(CommittedBundle<?> bundle) {
- return FluentIterable.from(bundle.getElements()).transformAndConcat(EXPLODE_WINDOWS_FN);
- }
-
- private Map<Object, FiredTimers> extractFiredTimers() {
- Map<Object, List<TimerData>> eventTimeTimers = inputWatermark.extractFiredEventTimeTimers();
- Map<Object, List<TimerData>> processingTimers;
- Map<Object, List<TimerData>> synchronizedTimers;
- if (inputWatermark.get().equals(BoundedWindow.TIMESTAMP_MAX_VALUE)) {
- processingTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers(
- TimeDomain.PROCESSING_TIME, BoundedWindow.TIMESTAMP_MAX_VALUE);
- synchronizedTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers(
- TimeDomain.PROCESSING_TIME, BoundedWindow.TIMESTAMP_MAX_VALUE);
- } else {
- processingTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers(
- TimeDomain.PROCESSING_TIME, clock.now());
- synchronizedTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers(
- TimeDomain.SYNCHRONIZED_PROCESSING_TIME, getSynchronizedProcessingInputTime());
- }
- Map<Object, Map<TimeDomain, List<TimerData>>> groupedTimers = new HashMap<>();
- groupFiredTimers(groupedTimers, eventTimeTimers, processingTimers, synchronizedTimers);
-
- Map<Object, FiredTimers> keyFiredTimers = new HashMap<>();
- for (Map.Entry<Object, Map<TimeDomain, List<TimerData>>> firedTimers :
- groupedTimers.entrySet()) {
- keyFiredTimers.put(firedTimers.getKey(), new FiredTimers(firedTimers.getValue()));
- }
- return keyFiredTimers;
- }
-
- @SafeVarargs
- private final void groupFiredTimers(
- Map<Object, Map<TimeDomain, List<TimerData>>> groupedToMutate,
- Map<Object, List<TimerData>>... timersToGroup) {
- for (Map<Object, List<TimerData>> subGroup : timersToGroup) {
- for (Map.Entry<Object, List<TimerData>> newTimers : subGroup.entrySet()) {
- Map<TimeDomain, List<TimerData>> grouped = groupedToMutate.get(newTimers.getKey());
- if (grouped == null) {
- grouped = new HashMap<>();
- groupedToMutate.put(newTimers.getKey(), grouped);
- }
- grouped.put(newTimers.getValue().get(0).getDomain(), newTimers.getValue());
- }
- }
- }
-
- private void updateTimers(TimerUpdate update) {
- inputWatermark.updateTimers(update);
- synchronizedProcessingInputWatermark.updateTimers(update);
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(TransformWatermarks.class)
- .add("inputWatermark", inputWatermark)
- .add("outputWatermark", outputWatermark)
- .add("inputProcessingTime", synchronizedProcessingInputWatermark)
- .add("outputProcessingTime", synchronizedProcessingOutputWatermark)
- .toString();
- }
- }
-
- /**
- * A collection of newly set, deleted, and completed timers.
- *
- * <p>setTimers and deletedTimers are collections of {@link TimerData} that have been added to the
- * {@link TimerInternals} of an executed step. completedTimers are timers that were delivered as
- * the input to the executed step.
- */
- public static class TimerUpdate {
- private final Object key;
- private final Iterable<? extends TimerData> completedTimers;
-
- private final Iterable<? extends TimerData> setTimers;
- private final Iterable<? extends TimerData> deletedTimers;
-
- /**
- * Returns a TimerUpdate for a null key with no timers.
- */
- public static TimerUpdate empty() {
- return new TimerUpdate(
- null,
- Collections.<TimerData>emptyList(),
- Collections.<TimerData>emptyList(),
- Collections.<TimerData>emptyList());
- }
-
- /**
- * Creates a new {@link TimerUpdate} builder with the provided completed timers that needs the
- * set and deleted timers to be added to it.
- */
- public static TimerUpdateBuilder builder(Object key) {
- return new TimerUpdateBuilder(key);
- }
-
- /**
- * A {@link TimerUpdate} builder that needs to be provided with set timers and deleted timers.
- */
- public static final class TimerUpdateBuilder {
- private final Object key;
- private final Collection<TimerData> completedTimers;
- private final Collection<TimerData> setTimers;
- private final Collection<TimerData> deletedTimers;
-
- private TimerUpdateBuilder(Object key) {
- this.key = key;
- this.completedTimers = new HashSet<>();
- this.setTimers = new HashSet<>();
- this.deletedTimers = new HashSet<>();
- }
-
- /**
- * Adds all of the provided timers to the collection of completed timers, and returns this
- * {@link TimerUpdateBuilder}.
- */
- public TimerUpdateBuilder withCompletedTimers(Iterable<TimerData> completedTimers) {
- Iterables.addAll(this.completedTimers, completedTimers);
- return this;
- }
-
- /**
- * Adds the provided timer to the collection of set timers, removing it from deleted timers if
- * it has previously been deleted. Returns this {@link TimerUpdateBuilder}.
- */
- public TimerUpdateBuilder setTimer(TimerData setTimer) {
- deletedTimers.remove(setTimer);
- setTimers.add(setTimer);
- return this;
- }
-
- /**
- * Adds the provided timer to the collection of deleted timers, removing it from set timers if
- * it has previously been set. Returns this {@link TimerUpdateBuilder}.
- */
- public TimerUpdateBuilder deletedTimer(TimerData deletedTimer) {
- deletedTimers.add(deletedTimer);
- setTimers.remove(deletedTimer);
- return this;
- }
-
- /**
- * Returns a new {@link TimerUpdate} with the most recently set completedTimers, setTimers,
- * and deletedTimers.
- */
- public TimerUpdate build() {
- return new TimerUpdate(
- key,
- ImmutableSet.copyOf(completedTimers),
- ImmutableSet.copyOf(setTimers),
- ImmutableSet.copyOf(deletedTimers));
- }
- }
-
- private TimerUpdate(
- Object key,
- Iterable<? extends TimerData> completedTimers,
- Iterable<? extends TimerData> setTimers,
- Iterable<? extends TimerData> deletedTimers) {
- this.key = key;
- this.completedTimers = completedTimers;
- this.setTimers = setTimers;
- this.deletedTimers = deletedTimers;
- }
-
- @VisibleForTesting
- Object getKey() {
- return key;
- }
-
- @VisibleForTesting
- Iterable<? extends TimerData> getCompletedTimers() {
- return completedTimers;
- }
-
- @VisibleForTesting
- Iterable<? extends TimerData> getSetTimers() {
- return setTimers;
- }
-
- @VisibleForTesting
- Iterable<? extends TimerData> getDeletedTimers() {
- return deletedTimers;
- }
-
- /**
- * Returns a {@link TimerUpdate} that is like this one, but with the specified completed timers.
- */
- public TimerUpdate withCompletedTimers(Iterable<TimerData> completedTimers) {
- return new TimerUpdate(this.key, completedTimers, setTimers, deletedTimers);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(key, completedTimers, setTimers, deletedTimers);
- }
-
- @Override
- public boolean equals(Object other) {
- if (other == null || !(other instanceof TimerUpdate)) {
- return false;
- }
- TimerUpdate that = (TimerUpdate) other;
- return Objects.equals(this.key, that.key)
- && Objects.equals(this.completedTimers, that.completedTimers)
- && Objects.equals(this.setTimers, that.setTimers)
- && Objects.equals(this.deletedTimers, that.deletedTimers);
- }
- }
-
- /**
- * A pair of {@link TimerData} and key which can be delivered to the appropriate
- * {@link AppliedPTransform}. A timer fires at the transform that set it with a specific key when
- * the time domain in which it lives progresses past a specified time, as determined by the
- * {@link InMemoryWatermarkManager}.
- */
- public static class FiredTimers {
- private final Map<TimeDomain, ? extends Collection<TimerData>> timers;
-
- private FiredTimers(Map<TimeDomain, ? extends Collection<TimerData>> timers) {
- this.timers = timers;
- }
-
- /**
- * Gets all of the timers that have fired within the provided {@link TimeDomain}. If no timers
- * fired within the provided domain, return an empty collection.
- *
- * <p>Timers within a {@link TimeDomain} are guaranteed to be in order of increasing timestamp.
- */
- public Collection<TimerData> getTimers(TimeDomain domain) {
- Collection<TimerData> domainTimers = timers.get(domain);
- if (domainTimers == null) {
- return Collections.emptyList();
- }
- return domainTimers;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(FiredTimers.class).add("timers", timers).toString();
- }
- }
-
- private static class WindowedValueByTimestampComparator extends Ordering<WindowedValue<?>> {
- @Override
- public int compare(WindowedValue<?> o1, WindowedValue<?> o2) {
- return ComparisonChain.start()
- .compare(o1.getTimestamp(), o2.getTimestamp())
- .result();
- }
- }
-
- public Set<AppliedPTransform<?, ?, ?>> getCompletedTransforms() {
- Set<AppliedPTransform<?, ?, ?>> result = new HashSet<>();
- for (Map.Entry<AppliedPTransform<?, ?, ?>, TransformWatermarks> wms :
- transformToWatermarks.entrySet()) {
- if (wms.getValue().getOutputWatermark().equals(THE_END_OF_TIME.get())) {
- result.add(wms.getKey());
- }
- }
- return result;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactory.java
deleted file mode 100644
index 58bc4fb..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactory.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.ImmutableList;
-
-import org.joda.time.Instant;
-
-import javax.annotation.Nullable;
-
-/**
- * A factory that produces bundles that perform no additional validation.
- */
-class InProcessBundleFactory implements BundleFactory {
- public static InProcessBundleFactory create() {
- return new InProcessBundleFactory();
- }
-
- private InProcessBundleFactory() {}
-
- @Override
- public <T> UncommittedBundle<T> createRootBundle(PCollection<T> output) {
- return InProcessBundle.create(output, null);
- }
-
- @Override
- public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output) {
- return InProcessBundle.create(output, input.getKey());
- }
-
- @Override
- public <T> UncommittedBundle<T> createKeyedBundle(
- CommittedBundle<?> input, @Nullable Object key, PCollection<T> output) {
- return InProcessBundle.create(output, key);
- }
-
- /**
- * A {@link UncommittedBundle} that buffers elements in memory.
- */
- private static final class InProcessBundle<T> implements UncommittedBundle<T> {
- private final PCollection<T> pcollection;
- @Nullable private final Object key;
- private boolean committed = false;
- private ImmutableList.Builder<WindowedValue<T>> elements;
-
- /**
- * Create a new {@link InProcessBundle} for the specified {@link PCollection}.
- */
- public static <T> InProcessBundle<T> create(PCollection<T> pcollection, @Nullable Object key) {
- return new InProcessBundle<T>(pcollection, key);
- }
-
- private InProcessBundle(PCollection<T> pcollection, Object key) {
- this.pcollection = pcollection;
- this.key = key;
- this.elements = ImmutableList.builder();
- }
-
- @Override
- public PCollection<T> getPCollection() {
- return pcollection;
- }
-
- @Override
- public InProcessBundle<T> add(WindowedValue<T> element) {
- checkState(
- !committed,
- "Can't add element %s to committed bundle in PCollection %s",
- element,
- pcollection);
- elements.add(element);
- return this;
- }
-
- @Override
- public CommittedBundle<T> commit(final Instant synchronizedCompletionTime) {
- checkState(!committed, "Can't commit already committed bundle %s", this);
- committed = true;
- final Iterable<WindowedValue<T>> committedElements = elements.build();
- return new CommittedInProcessBundle<>(
- pcollection, key, committedElements, synchronizedCompletionTime);
- }
- }
-
- private static class CommittedInProcessBundle<T> implements CommittedBundle<T> {
- public CommittedInProcessBundle(
- PCollection<T> pcollection,
- Object key,
- Iterable<WindowedValue<T>> committedElements,
- Instant synchronizedCompletionTime) {
- this.pcollection = pcollection;
- this.key = key;
- this.committedElements = committedElements;
- this.synchronizedCompletionTime = synchronizedCompletionTime;
- }
-
- private final PCollection<T> pcollection;
- private final Object key;
- private final Iterable<WindowedValue<T>> committedElements;
- private final Instant synchronizedCompletionTime;
-
- @Override
- @Nullable
- public Object getKey() {
- return key;
- }
-
- @Override
- public Iterable<WindowedValue<T>> getElements() {
- return committedElements;
- }
-
- @Override
- public PCollection<T> getPCollection() {
- return pcollection;
- }
-
- @Override
- public Instant getSynchronizedProcessingOutputWatermark() {
- return synchronizedCompletionTime;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(this)
- .omitNullValues()
- .add("pcollection", pcollection)
- .add("key", key)
- .add("elements", committedElements)
- .toString();
- }
-
- @Override
- public CommittedBundle<T> withElements(Iterable<WindowedValue<T>> elements) {
- return new CommittedInProcessBundle<>(
- pcollection, key, ImmutableList.copyOf(elements), synchronizedCompletionTime);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleOutputManager.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleOutputManager.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleOutputManager.java
deleted file mode 100644
index 4678a53..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleOutputManager.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.util.DoFnRunners.OutputManager;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.TupleTag;
-
-import java.util.Map;
-
-/**
- * An {@link OutputManager} that outputs to {@link CommittedBundle Bundles} used by the
- * {@link InProcessPipelineRunner}.
- */
-public class InProcessBundleOutputManager implements OutputManager {
- private final Map<TupleTag<?>, UncommittedBundle<?>> bundles;
-
- public static InProcessBundleOutputManager create(
- Map<TupleTag<?>, UncommittedBundle<?>> outputBundles) {
- return new InProcessBundleOutputManager(outputBundles);
- }
-
- public InProcessBundleOutputManager(Map<TupleTag<?>, UncommittedBundle<?>> bundles) {
- this.bundles = bundles;
- }
-
- @SuppressWarnings("unchecked")
- @Override
- public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
- @SuppressWarnings("rawtypes")
- UncommittedBundle bundle = bundles.get(tag);
- bundle.add(output);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContext.java
deleted file mode 100644
index 7c0dcee..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContext.java
+++ /dev/null
@@ -1,425 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.FiredTimers;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.PCollectionViewWriter;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.apache.beam.sdk.util.ExecutionContext;
-import org.apache.beam.sdk.util.SideInputReader;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.util.common.CounterSet;
-import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollection.IsBounded;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PValue;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import javax.annotation.Nullable;
-
-/**
- * The evaluation context for a specific pipeline being executed by the
- * {@link InProcessPipelineRunner}. Contains state shared within the execution across all
- * transforms.
- *
- * <p>{@link InProcessEvaluationContext} contains shared state for an execution of the
- * {@link InProcessPipelineRunner} that can be used while evaluating a {@link PTransform}. This
- * consists of views into underlying state and watermark implementations, access to read and write
- * {@link PCollectionView PCollectionViews}, and constructing {@link CounterSet CounterSets} and
- * {@link ExecutionContext ExecutionContexts}. This includes executing callbacks asynchronously when
- * state changes to the appropriate point (e.g. when a {@link PCollectionView} is requested and
- * known to be empty).
- *
- * <p>{@link InProcessEvaluationContext} also handles results by committing finalizing bundles based
- * on the current global state and updating the global state appropriately. This includes updating
- * the per-{@link StepAndKey} state, updating global watermarks, and executing any callbacks that
- * can be executed.
- */
-class InProcessEvaluationContext {
- /** The step name for each {@link AppliedPTransform} in the {@link Pipeline}. */
- private final Map<AppliedPTransform<?, ?, ?>, String> stepNames;
-
- /** The options that were used to create this {@link Pipeline}. */
- private final InProcessPipelineOptions options;
-
- private final BundleFactory bundleFactory;
- /** The current processing time and event time watermarks and timers. */
- private final InMemoryWatermarkManager watermarkManager;
-
- /** Executes callbacks based on the progression of the watermark. */
- private final WatermarkCallbackExecutor callbackExecutor;
-
- /** The stateInternals of the world, by applied PTransform and key. */
- private final ConcurrentMap<StepAndKey, CopyOnAccessInMemoryStateInternals<?>>
- applicationStateInternals;
-
- private final InProcessSideInputContainer sideInputContainer;
-
- private final CounterSet mergedCounters;
-
- public static InProcessEvaluationContext create(
- InProcessPipelineOptions options,
- BundleFactory bundleFactory,
- Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
- Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
- Map<AppliedPTransform<?, ?, ?>, String> stepNames,
- Collection<PCollectionView<?>> views) {
- return new InProcessEvaluationContext(
- options, bundleFactory, rootTransforms, valueToConsumers, stepNames, views);
- }
-
- private InProcessEvaluationContext(
- InProcessPipelineOptions options,
- BundleFactory bundleFactory,
- Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
- Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
- Map<AppliedPTransform<?, ?, ?>, String> stepNames,
- Collection<PCollectionView<?>> views) {
- this.options = checkNotNull(options);
- this.bundleFactory = checkNotNull(bundleFactory);
- checkNotNull(rootTransforms);
- checkNotNull(valueToConsumers);
- checkNotNull(stepNames);
- checkNotNull(views);
- this.stepNames = stepNames;
-
- this.watermarkManager =
- InMemoryWatermarkManager.create(
- NanosOffsetClock.create(), rootTransforms, valueToConsumers);
- this.sideInputContainer = InProcessSideInputContainer.create(this, views);
-
- this.applicationStateInternals = new ConcurrentHashMap<>();
- this.mergedCounters = new CounterSet();
-
- this.callbackExecutor = WatermarkCallbackExecutor.create();
- }
-
- /**
- * Handle the provided {@link InProcessTransformResult}, produced after evaluating the provided
- * {@link CommittedBundle} (potentially null, if the result of a root {@link PTransform}).
- *
- * <p>The result is the output of running the transform contained in the
- * {@link InProcessTransformResult} on the contents of the provided bundle.
- *
- * @param completedBundle the bundle that was processed to produce the result. Potentially
- * {@code null} if the transform that produced the result is a root
- * transform
- * @param completedTimers the timers that were delivered to produce the {@code completedBundle},
- * or an empty iterable if no timers were delivered
- * @param result the result of evaluating the input bundle
- * @return the committed bundles contained within the handled {@code result}
- */
- public synchronized CommittedResult handleResult(
- @Nullable CommittedBundle<?> completedBundle,
- Iterable<TimerData> completedTimers,
- InProcessTransformResult result) {
- Iterable<? extends CommittedBundle<?>> committedBundles =
- commitBundles(result.getOutputBundles());
- // Update watermarks and timers
- watermarkManager.updateWatermarks(
- completedBundle,
- result.getTransform(),
- result.getTimerUpdate().withCompletedTimers(completedTimers),
- committedBundles,
- result.getWatermarkHold());
- fireAllAvailableCallbacks();
- // Update counters
- if (result.getCounters() != null) {
- mergedCounters.merge(result.getCounters());
- }
- // Update state internals
- CopyOnAccessInMemoryStateInternals<?> theirState = result.getState();
- if (theirState != null) {
- CopyOnAccessInMemoryStateInternals<?> committedState = theirState.commit();
- StepAndKey stepAndKey =
- StepAndKey.of(
- result.getTransform(), completedBundle == null ? null : completedBundle.getKey());
- if (!committedState.isEmpty()) {
- applicationStateInternals.put(stepAndKey, committedState);
- } else {
- applicationStateInternals.remove(stepAndKey);
- }
- }
- return CommittedResult.create(result, committedBundles);
- }
-
- private Iterable<? extends CommittedBundle<?>> commitBundles(
- Iterable<? extends UncommittedBundle<?>> bundles) {
- ImmutableList.Builder<CommittedBundle<?>> completed = ImmutableList.builder();
- for (UncommittedBundle<?> inProgress : bundles) {
- AppliedPTransform<?, ?, ?> producing =
- inProgress.getPCollection().getProducingTransformInternal();
- TransformWatermarks watermarks = watermarkManager.getWatermarks(producing);
- CommittedBundle<?> committed =
- inProgress.commit(watermarks.getSynchronizedProcessingOutputTime());
- // Empty bundles don't impact watermarks and shouldn't trigger downstream execution, so
- // filter them out
- if (!Iterables.isEmpty(committed.getElements())) {
- completed.add(committed);
- }
- }
- return completed.build();
- }
-
- private void fireAllAvailableCallbacks() {
- for (AppliedPTransform<?, ?, ?> transform : stepNames.keySet()) {
- fireAvailableCallbacks(transform);
- }
- }
-
- private void fireAvailableCallbacks(AppliedPTransform<?, ?, ?> producingTransform) {
- TransformWatermarks watermarks = watermarkManager.getWatermarks(producingTransform);
- callbackExecutor.fireForWatermark(producingTransform, watermarks.getOutputWatermark());
- }
-
- /**
- * Create a {@link UncommittedBundle} for use by a source.
- */
- public <T> UncommittedBundle<T> createRootBundle(PCollection<T> output) {
- return bundleFactory.createRootBundle(output);
- }
-
- /**
- * Create a {@link UncommittedBundle} whose elements belong to the specified {@link
- * PCollection}.
- */
- public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output) {
- return bundleFactory.createBundle(input, output);
- }
-
- /**
- * Create a {@link UncommittedBundle} with the specified keys at the specified step. For use by
- * {@link InProcessGroupByKeyOnly} {@link PTransform PTransforms}.
- */
- public <T> UncommittedBundle<T> createKeyedBundle(
- CommittedBundle<?> input, Object key, PCollection<T> output) {
- return bundleFactory.createKeyedBundle(input, key, output);
- }
-
- /**
- * Create a {@link PCollectionViewWriter}, whose elements will be used in the provided
- * {@link PCollectionView}.
- */
- public <ElemT, ViewT> PCollectionViewWriter<ElemT, ViewT> createPCollectionViewWriter(
- PCollection<Iterable<ElemT>> input, final PCollectionView<ViewT> output) {
- return new PCollectionViewWriter<ElemT, ViewT>() {
- @Override
- public void add(Iterable<WindowedValue<ElemT>> values) {
- sideInputContainer.write(output, values);
- }
- };
- }
-
- /**
- * Schedule a callback to be executed after output would be produced for the given window
- * if there had been input.
- *
- * <p>Output would be produced when the watermark for a {@link PValue} passes the point at
- * which the trigger for the specified window (with the specified windowing strategy) must have
- * fired from the perspective of that {@link PValue}, as specified by the value of
- * {@link Trigger#getWatermarkThatGuaranteesFiring(BoundedWindow)} for the trigger of the
- * {@link WindowingStrategy}. When the callback has fired, either values will have been produced
- * for a key in that window, the window is empty, or all elements in the window are late. The
- * callback will be executed regardless of whether values have been produced.
- */
- public void scheduleAfterOutputWouldBeProduced(
- PValue value,
- BoundedWindow window,
- WindowingStrategy<?, ?> windowingStrategy,
- Runnable runnable) {
- AppliedPTransform<?, ?, ?> producing = getProducing(value);
- callbackExecutor.callOnGuaranteedFiring(producing, window, windowingStrategy, runnable);
-
- fireAvailableCallbacks(lookupProducing(value));
- }
-
- private AppliedPTransform<?, ?, ?> getProducing(PValue value) {
- if (value.getProducingTransformInternal() != null) {
- return value.getProducingTransformInternal();
- }
- return lookupProducing(value);
- }
-
- private AppliedPTransform<?, ?, ?> lookupProducing(PValue value) {
- for (AppliedPTransform<?, ?, ?> transform : stepNames.keySet()) {
- if (transform.getOutput().equals(value) || transform.getOutput().expand().contains(value)) {
- return transform;
- }
- }
- return null;
- }
-
- /**
- * Get the options used by this {@link Pipeline}.
- */
- public InProcessPipelineOptions getPipelineOptions() {
- return options;
- }
-
- /**
- * Get an {@link ExecutionContext} for the provided {@link AppliedPTransform} and key.
- */
- public InProcessExecutionContext getExecutionContext(
- AppliedPTransform<?, ?, ?> application, Object key) {
- StepAndKey stepAndKey = StepAndKey.of(application, key);
- return new InProcessExecutionContext(
- options.getClock(),
- key,
- (CopyOnAccessInMemoryStateInternals<Object>) applicationStateInternals.get(stepAndKey),
- watermarkManager.getWatermarks(application));
- }
-
- /**
- * Get all of the steps used in this {@link Pipeline}.
- */
- public Collection<AppliedPTransform<?, ?, ?>> getSteps() {
- return stepNames.keySet();
- }
-
- /**
- * Get the Step Name for the provided application.
- */
- public String getStepName(AppliedPTransform<?, ?, ?> application) {
- return stepNames.get(application);
- }
-
- /**
- * Returns a {@link ReadyCheckingSideInputReader} capable of reading the provided
- * {@link PCollectionView PCollectionViews}.
- *
- * @param sideInputs the {@link PCollectionView PCollectionViews} the result should be able to
- * read
- * @return a {@link SideInputReader} that can read all of the provided {@link PCollectionView
- * PCollectionViews}
- */
- public ReadyCheckingSideInputReader createSideInputReader(
- final List<PCollectionView<?>> sideInputs) {
- return sideInputContainer.createReaderForViews(sideInputs);
- }
-
- /**
- * A {@link SideInputReader} that allows callers to check to see if a {@link PCollectionView} has
- * had its contents set in a window.
- */
- static interface ReadyCheckingSideInputReader extends SideInputReader {
- /**
- * Returns true if the {@link PCollectionView} is ready in the provided {@link BoundedWindow}.
- */
- boolean isReady(PCollectionView<?> view, BoundedWindow window);
- }
-
- /**
- * Create a {@link CounterSet} for this {@link Pipeline}. The {@link CounterSet} is independent
- * of all other {@link CounterSet CounterSets} created by this call.
- *
- * The {@link InProcessEvaluationContext} is responsible for unifying the counters present in
- * all created {@link CounterSet CounterSets} when the transforms that call this method
- * complete.
- */
- public CounterSet createCounterSet() {
- return new CounterSet();
- }
-
- /**
- * Returns all of the counters that have been merged into this context via calls to
- * {@link CounterSet#merge(CounterSet)}.
- */
- public CounterSet getCounters() {
- return mergedCounters;
- }
-
- /**
- * Extracts all timers that have been fired and have not already been extracted.
- *
- * <p>This is a destructive operation. Timers will only appear in the result of this method once
- * for each time they are set.
- */
- public Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> extractFiredTimers() {
- Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> fired =
- watermarkManager.extractFiredTimers();
- return fired;
- }
-
- /**
- * Returns true if the step will not produce additional output.
- *
- * <p>If the provided transform produces only {@link IsBounded#BOUNDED}
- * {@link PCollection PCollections}, returns true if the watermark is at
- * {@link BoundedWindow#TIMESTAMP_MAX_VALUE positive infinity}.
- *
- * <p>If the provided transform produces any {@link IsBounded#UNBOUNDED}
- * {@link PCollection PCollections}, returns the value of
- * {@link InProcessPipelineOptions#isShutdownUnboundedProducersWithMaxWatermark()}.
- */
- public boolean isDone(AppliedPTransform<?, ?, ?> transform) {
- // if the PTransform's watermark isn't at the max value, it isn't done
- if (watermarkManager
- .getWatermarks(transform)
- .getOutputWatermark()
- .isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE)) {
- return false;
- }
- // If the PTransform has any unbounded outputs, and unbounded producers should not be shut down,
- // the PTransform may produce additional output. It is not done.
- for (PValue output : transform.getOutput().expand()) {
- if (output instanceof PCollection) {
- IsBounded bounded = ((PCollection<?>) output).isBounded();
- if (bounded.equals(IsBounded.UNBOUNDED)
- && !options.isShutdownUnboundedProducersWithMaxWatermark()) {
- return false;
- }
- }
- }
- // The PTransform's watermark was at positive infinity and all of its outputs are known to be
- // done. It is done.
- return true;
- }
-
- /**
- * Returns true if all steps are done.
- */
- public boolean isDone() {
- for (AppliedPTransform<?, ?, ?> transform : stepNames.keySet()) {
- if (!isDone(transform)) {
- return false;
- }
- }
- return true;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutionContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutionContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutionContext.java
deleted file mode 100644
index e6441cf..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutionContext.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
-import org.apache.beam.sdk.util.BaseExecutionContext;
-import org.apache.beam.sdk.util.ExecutionContext;
-import org.apache.beam.sdk.util.TimerInternals;
-import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-
-/**
- * Execution Context for the {@link InProcessPipelineRunner}.
- *
- * This implementation is not thread safe. A new {@link InProcessExecutionContext} must be created
- * for each thread that requires it.
- */
-class InProcessExecutionContext
- extends BaseExecutionContext<InProcessExecutionContext.InProcessStepContext> {
- private final Clock clock;
- private final Object key;
- private final CopyOnAccessInMemoryStateInternals<Object> existingState;
- private final TransformWatermarks watermarks;
-
- public InProcessExecutionContext(Clock clock, Object key,
- CopyOnAccessInMemoryStateInternals<Object> existingState, TransformWatermarks watermarks) {
- this.clock = clock;
- this.key = key;
- this.existingState = existingState;
- this.watermarks = watermarks;
- }
-
- @Override
- protected InProcessStepContext createStepContext(String stepName, String transformName) {
- return new InProcessStepContext(this, stepName, transformName);
- }
-
- /**
- * Step Context for the {@link InProcessPipelineRunner}.
- */
- public class InProcessStepContext
- extends org.apache.beam.sdk.util.BaseExecutionContext.StepContext {
- private CopyOnAccessInMemoryStateInternals<Object> stateInternals;
- private InProcessTimerInternals timerInternals;
-
- public InProcessStepContext(
- ExecutionContext executionContext, String stepName, String transformName) {
- super(executionContext, stepName, transformName);
- }
-
- @Override
- public CopyOnAccessInMemoryStateInternals<Object> stateInternals() {
- if (stateInternals == null) {
- stateInternals = CopyOnAccessInMemoryStateInternals.withUnderlying(key, existingState);
- }
- return stateInternals;
- }
-
- @Override
- public InProcessTimerInternals timerInternals() {
- if (timerInternals == null) {
- timerInternals =
- InProcessTimerInternals.create(clock, watermarks, TimerUpdate.builder(key));
- }
- return timerInternals;
- }
-
- /**
- * Commits the state of this step, and returns the committed state. If the step has not
- * accessed any state, return null.
- */
- public CopyOnAccessInMemoryStateInternals<?> commitState() {
- if (stateInternals != null) {
- return stateInternals.commit();
- }
- return null;
- }
-
- /**
- * Gets the timer update of the {@link TimerInternals} of this {@link InProcessStepContext},
- * which is empty if the {@link TimerInternals} were never accessed.
- */
- public TimerUpdate getTimerUpdate() {
- if (timerInternals == null) {
- return TimerUpdate.empty();
- }
- return timerInternals.getTimerUpdate();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutor.java
deleted file mode 100644
index 059aae6..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutor.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-
-import java.util.Collection;
-
-/**
- * An executor that schedules and executes {@link AppliedPTransform AppliedPTransforms} for both
- * source and intermediate {@link PTransform PTransforms}.
- */
-interface InProcessExecutor {
- /**
- * Starts this executor. The provided collection is the collection of root transforms to
- * initially schedule.
- *
- * @param rootTransforms
- */
- void start(Collection<AppliedPTransform<?, ?, ?>> rootTransforms);
-
- /**
- * Blocks until the job being executed enters a terminal state. A job is completed after all
- * root {@link AppliedPTransform AppliedPTransforms} have completed, and all
- * {@link CommittedBundle Bundles} have been consumed. Jobs may also terminate abnormally.
- *
- * @throws Throwable whenever an executor thread throws anything, transfers the throwable to the
- * waiting thread and rethrows it
- */
- void awaitCompletion() throws Throwable;
-}
[10/17] incubator-beam git commit: Move InProcessRunner to its own
module
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessEvaluationContextTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessEvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessEvaluationContextTest.java
new file mode 100644
index 0000000..59c4d8e
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessEvaluationContextTest.java
@@ -0,0 +1,526 @@
+/*
+ * 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.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.emptyIterable;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.FiredTimers;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.InProcessExecutionContext.InProcessStepContext;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.PCollectionViewWriter;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.io.CountingInput;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
+import org.apache.beam.sdk.util.SideInputReader;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.common.Counter;
+import org.apache.beam.sdk.util.common.Counter.AggregationKind;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PValue;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+
+import org.hamcrest.Matchers;
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Tests for {@link InProcessEvaluationContext}.
+ */
+@RunWith(JUnit4.class)
+public class InProcessEvaluationContextTest {
+ private TestPipeline p;
+ private InProcessEvaluationContext context;
+
+ private PCollection<Integer> created;
+ private PCollection<KV<String, Integer>> downstream;
+ private PCollectionView<Iterable<Integer>> view;
+ private PCollection<Long> unbounded;
+ private Collection<AppliedPTransform<?, ?, ?>> rootTransforms;
+ private Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers;
+
+ private BundleFactory bundleFactory;
+
+ @Before
+ public void setup() {
+ InProcessPipelineRunner runner =
+ InProcessPipelineRunner.fromOptions(PipelineOptionsFactory.create());
+
+ p = TestPipeline.create();
+
+ created = p.apply(Create.of(1, 2, 3));
+ downstream = created.apply(WithKeys.<String, Integer>of("foo"));
+ view = created.apply(View.<Integer>asIterable());
+ unbounded = p.apply(CountingInput.unbounded());
+
+ ConsumerTrackingPipelineVisitor cVis = new ConsumerTrackingPipelineVisitor();
+ p.traverseTopologically(cVis);
+ rootTransforms = cVis.getRootTransforms();
+ valueToConsumers = cVis.getValueToConsumers();
+
+ bundleFactory = InProcessBundleFactory.create();
+
+ context =
+ InProcessEvaluationContext.create(
+ runner.getPipelineOptions(),
+ InProcessBundleFactory.create(),
+ rootTransforms,
+ valueToConsumers,
+ cVis.getStepNames(),
+ cVis.getViews());
+ }
+
+ @Test
+ public void writeToViewWriterThenReadReads() {
+ PCollectionViewWriter<Integer, Iterable<Integer>> viewWriter =
+ context.createPCollectionViewWriter(
+ PCollection.<Iterable<Integer>>createPrimitiveOutputInternal(
+ p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED),
+ view);
+ BoundedWindow window = new TestBoundedWindow(new Instant(1024L));
+ BoundedWindow second = new TestBoundedWindow(new Instant(899999L));
+ WindowedValue<Integer> firstValue =
+ WindowedValue.of(1, new Instant(1222), window, PaneInfo.ON_TIME_AND_ONLY_FIRING);
+ WindowedValue<Integer> secondValue =
+ WindowedValue.of(
+ 2, new Instant(8766L), second, PaneInfo.createPane(true, false, Timing.ON_TIME, 0, 0));
+ Iterable<WindowedValue<Integer>> values = ImmutableList.of(firstValue, secondValue);
+ viewWriter.add(values);
+
+ SideInputReader reader =
+ context.createSideInputReader(ImmutableList.<PCollectionView<?>>of(view));
+ assertThat(reader.get(view, window), containsInAnyOrder(1));
+ assertThat(reader.get(view, second), containsInAnyOrder(2));
+
+ WindowedValue<Integer> overrittenSecondValue =
+ WindowedValue.of(
+ 4444, new Instant(8677L), second, PaneInfo.createPane(false, true, Timing.LATE, 1, 1));
+ viewWriter.add(Collections.singleton(overrittenSecondValue));
+ assertThat(reader.get(view, second), containsInAnyOrder(4444));
+ }
+
+ @Test
+ public void getExecutionContextSameStepSameKeyState() {
+ InProcessExecutionContext fooContext =
+ context.getExecutionContext(created.getProducingTransformInternal(), "foo");
+
+ StateTag<Object, BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
+
+ InProcessStepContext stepContext = fooContext.getOrCreateStepContext("s1", "s1");
+ stepContext.stateInternals().state(StateNamespaces.global(), intBag).add(1);
+
+ context.handleResult(
+ InProcessBundleFactory.create()
+ .createKeyedBundle(null, "foo", created)
+ .commit(Instant.now()),
+ ImmutableList.<TimerData>of(),
+ StepTransformResult.withoutHold(created.getProducingTransformInternal())
+ .withState(stepContext.commitState())
+ .build());
+
+ InProcessExecutionContext secondFooContext =
+ context.getExecutionContext(created.getProducingTransformInternal(), "foo");
+ assertThat(
+ secondFooContext
+ .getOrCreateStepContext("s1", "s1")
+ .stateInternals()
+ .state(StateNamespaces.global(), intBag)
+ .read(),
+ contains(1));
+ }
+
+
+ @Test
+ public void getExecutionContextDifferentKeysIndependentState() {
+ InProcessExecutionContext fooContext =
+ context.getExecutionContext(created.getProducingTransformInternal(), "foo");
+
+ StateTag<Object, BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
+
+ fooContext
+ .getOrCreateStepContext("s1", "s1")
+ .stateInternals()
+ .state(StateNamespaces.global(), intBag)
+ .add(1);
+
+ InProcessExecutionContext barContext =
+ context.getExecutionContext(created.getProducingTransformInternal(), "bar");
+ assertThat(barContext, not(equalTo(fooContext)));
+ assertThat(
+ barContext
+ .getOrCreateStepContext("s1", "s1")
+ .stateInternals()
+ .state(StateNamespaces.global(), intBag)
+ .read(),
+ emptyIterable());
+ }
+
+ @Test
+ public void getExecutionContextDifferentStepsIndependentState() {
+ String myKey = "foo";
+ InProcessExecutionContext fooContext =
+ context.getExecutionContext(created.getProducingTransformInternal(), myKey);
+
+ StateTag<Object, BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
+
+ fooContext
+ .getOrCreateStepContext("s1", "s1")
+ .stateInternals()
+ .state(StateNamespaces.global(), intBag)
+ .add(1);
+
+ InProcessExecutionContext barContext =
+ context.getExecutionContext(downstream.getProducingTransformInternal(), myKey);
+ assertThat(
+ barContext
+ .getOrCreateStepContext("s1", "s1")
+ .stateInternals()
+ .state(StateNamespaces.global(), intBag)
+ .read(),
+ emptyIterable());
+ }
+
+ @Test
+ public void handleResultMergesCounters() {
+ CounterSet counters = context.createCounterSet();
+ Counter<Long> myCounter = Counter.longs("foo", AggregationKind.SUM);
+ counters.addCounter(myCounter);
+
+ myCounter.addValue(4L);
+ InProcessTransformResult result =
+ StepTransformResult.withoutHold(created.getProducingTransformInternal())
+ .withCounters(counters)
+ .build();
+ context.handleResult(null, ImmutableList.<TimerData>of(), result);
+ assertThat((Long) context.getCounters().getExistingCounter("foo").getAggregate(), equalTo(4L));
+
+ CounterSet againCounters = context.createCounterSet();
+ Counter<Long> myLongCounterAgain = Counter.longs("foo", AggregationKind.SUM);
+ againCounters.add(myLongCounterAgain);
+ myLongCounterAgain.addValue(8L);
+
+ InProcessTransformResult secondResult =
+ StepTransformResult.withoutHold(downstream.getProducingTransformInternal())
+ .withCounters(againCounters)
+ .build();
+ context.handleResult(
+ context.createRootBundle(created).commit(Instant.now()),
+ ImmutableList.<TimerData>of(),
+ secondResult);
+ assertThat((Long) context.getCounters().getExistingCounter("foo").getAggregate(), equalTo(12L));
+ }
+
+ @Test
+ public void handleResultStoresState() {
+ String myKey = "foo";
+ InProcessExecutionContext fooContext =
+ context.getExecutionContext(downstream.getProducingTransformInternal(), myKey);
+
+ StateTag<Object, BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
+
+ CopyOnAccessInMemoryStateInternals<Object> state =
+ fooContext.getOrCreateStepContext("s1", "s1").stateInternals();
+ BagState<Integer> bag = state.state(StateNamespaces.global(), intBag);
+ bag.add(1);
+ bag.add(2);
+ bag.add(4);
+
+ InProcessTransformResult stateResult =
+ StepTransformResult.withoutHold(downstream.getProducingTransformInternal())
+ .withState(state)
+ .build();
+
+ context.handleResult(
+ context.createKeyedBundle(null, myKey, created).commit(Instant.now()),
+ ImmutableList.<TimerData>of(),
+ stateResult);
+
+ InProcessExecutionContext afterResultContext =
+ context.getExecutionContext(downstream.getProducingTransformInternal(), myKey);
+
+ CopyOnAccessInMemoryStateInternals<Object> afterResultState =
+ afterResultContext.getOrCreateStepContext("s1", "s1").stateInternals();
+ assertThat(afterResultState.state(StateNamespaces.global(), intBag).read(), contains(1, 2, 4));
+ }
+
+ @Test
+ public void callAfterOutputMustHaveBeenProducedAfterEndOfWatermarkCallsback() throws Exception {
+ final CountDownLatch callLatch = new CountDownLatch(1);
+ Runnable callback =
+ new Runnable() {
+ @Override
+ public void run() {
+ callLatch.countDown();
+ }
+ };
+
+ // Should call back after the end of the global window
+ context.scheduleAfterOutputWouldBeProduced(
+ downstream, GlobalWindow.INSTANCE, WindowingStrategy.globalDefault(), callback);
+
+ InProcessTransformResult result =
+ StepTransformResult.withHold(created.getProducingTransformInternal(), new Instant(0))
+ .build();
+
+ context.handleResult(null, ImmutableList.<TimerData>of(), result);
+
+ // Difficult to demonstrate that we took no action in a multithreaded world; poll for a bit
+ // will likely be flaky if this logic is broken
+ assertThat(callLatch.await(500L, TimeUnit.MILLISECONDS), is(false));
+
+ InProcessTransformResult finishedResult =
+ StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
+ context.handleResult(null, ImmutableList.<TimerData>of(), finishedResult);
+ // Obtain the value via blocking call
+ assertThat(callLatch.await(1, TimeUnit.SECONDS), is(true));
+ }
+
+ @Test
+ public void callAfterOutputMustHaveBeenProducedAlreadyAfterCallsImmediately() throws Exception {
+ InProcessTransformResult finishedResult =
+ StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
+ context.handleResult(null, ImmutableList.<TimerData>of(), finishedResult);
+
+ final CountDownLatch callLatch = new CountDownLatch(1);
+ Runnable callback =
+ new Runnable() {
+ @Override
+ public void run() {
+ callLatch.countDown();
+ }
+ };
+ context.scheduleAfterOutputWouldBeProduced(
+ downstream, GlobalWindow.INSTANCE, WindowingStrategy.globalDefault(), callback);
+ assertThat(callLatch.await(1, TimeUnit.SECONDS), is(true));
+ }
+
+ @Test
+ public void extractFiredTimersExtractsTimers() {
+ InProcessTransformResult holdResult =
+ StepTransformResult.withHold(created.getProducingTransformInternal(), new Instant(0))
+ .build();
+ context.handleResult(null, ImmutableList.<TimerData>of(), holdResult);
+
+ String key = "foo";
+ TimerData toFire =
+ TimerData.of(StateNamespaces.global(), new Instant(100L), TimeDomain.EVENT_TIME);
+ InProcessTransformResult timerResult =
+ StepTransformResult.withoutHold(downstream.getProducingTransformInternal())
+ .withState(CopyOnAccessInMemoryStateInternals.withUnderlying(key, null))
+ .withTimerUpdate(TimerUpdate.builder(key).setTimer(toFire).build())
+ .build();
+
+ // haven't added any timers, must be empty
+ assertThat(context.extractFiredTimers().entrySet(), emptyIterable());
+ context.handleResult(
+ context.createKeyedBundle(null, key, created).commit(Instant.now()),
+ ImmutableList.<TimerData>of(),
+ timerResult);
+
+ // timer hasn't fired
+ assertThat(context.extractFiredTimers().entrySet(), emptyIterable());
+
+ InProcessTransformResult advanceResult =
+ StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
+ // Should cause the downstream timer to fire
+ context.handleResult(null, ImmutableList.<TimerData>of(), advanceResult);
+
+ Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> fired = context.extractFiredTimers();
+ assertThat(
+ fired,
+ Matchers.<AppliedPTransform<?, ?, ?>>hasKey(downstream.getProducingTransformInternal()));
+ Map<Object, FiredTimers> downstreamFired =
+ fired.get(downstream.getProducingTransformInternal());
+ assertThat(downstreamFired, Matchers.<Object>hasKey(key));
+
+ FiredTimers firedForKey = downstreamFired.get(key);
+ assertThat(firedForKey.getTimers(TimeDomain.PROCESSING_TIME), emptyIterable());
+ assertThat(firedForKey.getTimers(TimeDomain.SYNCHRONIZED_PROCESSING_TIME), emptyIterable());
+ assertThat(firedForKey.getTimers(TimeDomain.EVENT_TIME), contains(toFire));
+
+ // Don't reextract timers
+ assertThat(context.extractFiredTimers().entrySet(), emptyIterable());
+ }
+
+ @Test
+ public void createBundleKeyedResultPropagatesKey() {
+ CommittedBundle<KV<String, Integer>> newBundle =
+ context
+ .createBundle(
+ bundleFactory.createKeyedBundle(null, "foo", created).commit(Instant.now()),
+ downstream)
+ .commit(Instant.now());
+ assertThat(newBundle.getKey(), Matchers.<Object>equalTo("foo"));
+ }
+
+ @Test
+ public void createKeyedBundleKeyed() {
+ CommittedBundle<KV<String, Integer>> keyedBundle =
+ context
+ .createKeyedBundle(
+ bundleFactory.createRootBundle(created).commit(Instant.now()), "foo", downstream)
+ .commit(Instant.now());
+ assertThat(keyedBundle.getKey(), Matchers.<Object>equalTo("foo"));
+ }
+
+ @Test
+ public void isDoneWithUnboundedPCollectionAndShutdown() {
+ context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(true);
+ assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(false));
+
+ context.handleResult(
+ null,
+ ImmutableList.<TimerData>of(),
+ StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build());
+ assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(true));
+ }
+
+ @Test
+ public void isDoneWithUnboundedPCollectionAndNotShutdown() {
+ context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(false);
+ assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(false));
+
+ context.handleResult(
+ null,
+ ImmutableList.<TimerData>of(),
+ StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build());
+ assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(false));
+ }
+
+ @Test
+ public void isDoneWithOnlyBoundedPCollections() {
+ context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(false);
+ assertThat(context.isDone(created.getProducingTransformInternal()), is(false));
+
+ context.handleResult(
+ null,
+ ImmutableList.<TimerData>of(),
+ StepTransformResult.withoutHold(created.getProducingTransformInternal()).build());
+ assertThat(context.isDone(created.getProducingTransformInternal()), is(true));
+ }
+
+ @Test
+ public void isDoneWithPartiallyDone() {
+ context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(true);
+ assertThat(context.isDone(), is(false));
+
+ UncommittedBundle<Integer> rootBundle = context.createRootBundle(created);
+ rootBundle.add(WindowedValue.valueInGlobalWindow(1));
+ CommittedResult handleResult =
+ context.handleResult(
+ null,
+ ImmutableList.<TimerData>of(),
+ StepTransformResult.withoutHold(created.getProducingTransformInternal())
+ .addOutput(rootBundle)
+ .build());
+ @SuppressWarnings("unchecked")
+ CommittedBundle<Integer> committedBundle =
+ (CommittedBundle<Integer>) Iterables.getOnlyElement(handleResult.getOutputs());
+ context.handleResult(
+ null,
+ ImmutableList.<TimerData>of(),
+ StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build());
+ assertThat(context.isDone(), is(false));
+
+ for (AppliedPTransform<?, ?, ?> consumers : valueToConsumers.get(created)) {
+ context.handleResult(
+ committedBundle,
+ ImmutableList.<TimerData>of(),
+ StepTransformResult.withoutHold(consumers).build());
+ }
+ assertThat(context.isDone(), is(true));
+ }
+
+ @Test
+ public void isDoneWithUnboundedAndNotShutdown() {
+ context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(false);
+ assertThat(context.isDone(), is(false));
+
+ context.handleResult(
+ null,
+ ImmutableList.<TimerData>of(),
+ StepTransformResult.withoutHold(created.getProducingTransformInternal()).build());
+ context.handleResult(
+ null,
+ ImmutableList.<TimerData>of(),
+ StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build());
+ context.handleResult(
+ context.createRootBundle(created).commit(Instant.now()),
+ ImmutableList.<TimerData>of(),
+ StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build());
+ assertThat(context.isDone(), is(false));
+
+ context.handleResult(
+ context.createRootBundle(created).commit(Instant.now()),
+ ImmutableList.<TimerData>of(),
+ StepTransformResult.withoutHold(view.getProducingTransformInternal()).build());
+ assertThat(context.isDone(), is(false));
+ }
+
+ private static class TestBoundedWindow extends BoundedWindow {
+ private final Instant ts;
+
+ public TestBoundedWindow(Instant ts) {
+ this.ts = ts;
+ }
+
+ @Override
+ public Instant maxTimestamp() {
+ return ts;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessPipelineRegistrarTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessPipelineRegistrarTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessPipelineRegistrarTest.java
new file mode 100644
index 0000000..54094c4
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessPipelineRegistrarTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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.assertEquals;
+import static org.junit.Assert.fail;
+
+import org.apache.beam.runners.direct.InProcessRegistrar.InProcessRunner;
+import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
+import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.ServiceLoader;
+
+/** Tests for {@link InProcessRunner}. */
+@RunWith(JUnit4.class)
+public class InProcessPipelineRegistrarTest {
+ @Test
+ public void testCorrectOptionsAreReturned() {
+ assertEquals(
+ ImmutableList.of(InProcessPipelineOptions.class),
+ new InProcessRegistrar.InProcessOptions().getPipelineOptions());
+ }
+
+ @Test
+ public void testCorrectRunnersAreReturned() {
+ assertEquals(
+ ImmutableList.of(InProcessPipelineRunner.class),
+ new InProcessRegistrar.InProcessRunner().getPipelineRunners());
+ }
+
+ @Test
+ public void testServiceLoaderForOptions() {
+ for (PipelineOptionsRegistrar registrar :
+ Lists.newArrayList(ServiceLoader.load(PipelineOptionsRegistrar.class).iterator())) {
+ if (registrar instanceof InProcessRegistrar.InProcessOptions) {
+ return;
+ }
+ }
+ fail("Expected to find " + InProcessRegistrar.InProcessOptions.class);
+ }
+
+ @Test
+ public void testServiceLoaderForRunner() {
+ for (PipelineRunnerRegistrar registrar :
+ Lists.newArrayList(ServiceLoader.load(PipelineRunnerRegistrar.class).iterator())) {
+ if (registrar instanceof InProcessRegistrar.InProcessRunner) {
+ return;
+ }
+ }
+ fail("Expected to find " + InProcessRegistrar.InProcessRunner.class);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessPipelineRunnerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessPipelineRunnerTest.java
new file mode 100644
index 0000000..87db39a
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessPipelineRunnerTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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.direct.InProcessPipelineRunner.InProcessPipelineResult;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.Serializable;
+
+/**
+ * Tests for basic {@link InProcessPipelineRunner} functionality.
+ */
+@RunWith(JUnit4.class)
+public class InProcessPipelineRunnerTest implements Serializable {
+ @Test
+ public void wordCountShouldSucceed() throws Throwable {
+ Pipeline p = getPipeline();
+
+ PCollection<KV<String, Long>> counts =
+ p.apply(Create.of("foo", "bar", "foo", "baz", "bar", "foo"))
+ .apply(MapElements.via(new SimpleFunction<String, String>() {
+ @Override
+ public String apply(String input) {
+ return input;
+ }
+ }))
+ .apply(Count.<String>perElement());
+ PCollection<String> countStrs =
+ counts.apply(MapElements.via(new SimpleFunction<KV<String, Long>, String>() {
+ @Override
+ public String apply(KV<String, Long> input) {
+ String str = String.format("%s: %s", input.getKey(), input.getValue());
+ return str;
+ }
+ }));
+
+ PAssert.that(countStrs).containsInAnyOrder("baz: 1", "bar: 2", "foo: 3");
+
+ InProcessPipelineResult result = ((InProcessPipelineResult) p.run());
+ result.awaitCompletion();
+ }
+
+ private Pipeline getPipeline() {
+ PipelineOptions opts = PipelineOptionsFactory.create();
+ opts.setRunner(InProcessPipelineRunner.class);
+
+ Pipeline p = Pipeline.create(opts);
+ return p;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessSideInputContainerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessSideInputContainerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessSideInputContainerTest.java
new file mode 100644
index 0000000..d8a78f2
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessSideInputContainerTest.java
@@ -0,0 +1,496 @@
+/*
+ * 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.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasEntry;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.doAnswer;
+
+import org.apache.beam.runners.direct.InProcessEvaluationContext.ReadyCheckingSideInputReader;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.Mean;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
+import org.apache.beam.sdk.util.PCollectionViews;
+import org.apache.beam.sdk.util.SideInputReader;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+
+import com.google.common.collect.ImmutableList;
+
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.MockitoAnnotations;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+/**
+ * Tests for {@link InProcessSideInputContainer}.
+ */
+@RunWith(JUnit4.class)
+public class InProcessSideInputContainerTest {
+ private static final BoundedWindow FIRST_WINDOW =
+ new BoundedWindow() {
+ @Override
+ public Instant maxTimestamp() {
+ return new Instant(789541L);
+ }
+
+ @Override
+ public String toString() {
+ return "firstWindow";
+ }
+ };
+
+ private static final BoundedWindow SECOND_WINDOW =
+ new BoundedWindow() {
+ @Override
+ public Instant maxTimestamp() {
+ return new Instant(14564786L);
+ }
+
+ @Override
+ public String toString() {
+ return "secondWindow";
+ }
+ };
+
+ @Rule
+ public ExpectedException thrown = ExpectedException.none();
+
+ @Mock
+ private InProcessEvaluationContext context;
+
+ private TestPipeline pipeline;
+
+ private InProcessSideInputContainer container;
+
+ private PCollectionView<Map<String, Integer>> mapView;
+ private PCollectionView<Double> singletonView;
+
+ // Not present in container.
+ private PCollectionView<Iterable<Integer>> iterableView;
+
+ @Before
+ public void setup() {
+ MockitoAnnotations.initMocks(this);
+ pipeline = TestPipeline.create();
+
+ PCollection<Integer> create =
+ pipeline.apply("forBaseCollection", Create.<Integer>of(1, 2, 3, 4));
+
+ mapView =
+ create.apply("forKeyTypes", WithKeys.<String, Integer>of("foo"))
+ .apply("asMapView", View.<String, Integer>asMap());
+
+ singletonView = create.apply("forCombinedTypes", Mean.<Integer>globally().asSingletonView());
+ iterableView = create.apply("asIterableView", View.<Integer>asIterable());
+
+ container = InProcessSideInputContainer.create(
+ context, ImmutableList.of(iterableView, mapView, singletonView));
+ }
+
+ @Test
+ public void getAfterWriteReturnsPaneInWindow() throws Exception {
+ WindowedValue<KV<String, Integer>> one =
+ WindowedValue.of(
+ KV.of("one", 1), new Instant(1L), FIRST_WINDOW, PaneInfo.ON_TIME_AND_ONLY_FIRING);
+ WindowedValue<KV<String, Integer>> two =
+ WindowedValue.of(
+ KV.of("two", 2), new Instant(20L), FIRST_WINDOW, PaneInfo.ON_TIME_AND_ONLY_FIRING);
+ container.write(mapView, ImmutableList.<WindowedValue<?>>of(one, two));
+
+ Map<String, Integer> viewContents =
+ container
+ .createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
+ .get(mapView, FIRST_WINDOW);
+ assertThat(viewContents, hasEntry("one", 1));
+ assertThat(viewContents, hasEntry("two", 2));
+ assertThat(viewContents.size(), is(2));
+ }
+
+ @Test
+ public void getReturnsLatestPaneInWindow() throws Exception {
+ WindowedValue<KV<String, Integer>> one =
+ WindowedValue.of(
+ KV.of("one", 1),
+ new Instant(1L),
+ SECOND_WINDOW,
+ PaneInfo.createPane(true, false, Timing.EARLY));
+ WindowedValue<KV<String, Integer>> two =
+ WindowedValue.of(
+ KV.of("two", 2),
+ new Instant(20L),
+ SECOND_WINDOW,
+ PaneInfo.createPane(true, false, Timing.EARLY));
+ container.write(mapView, ImmutableList.<WindowedValue<?>>of(one, two));
+
+ Map<String, Integer> viewContents =
+ container
+ .createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
+ .get(mapView, SECOND_WINDOW);
+ assertThat(viewContents, hasEntry("one", 1));
+ assertThat(viewContents, hasEntry("two", 2));
+ assertThat(viewContents.size(), is(2));
+
+ WindowedValue<KV<String, Integer>> three =
+ WindowedValue.of(
+ KV.of("three", 3),
+ new Instant(300L),
+ SECOND_WINDOW,
+ PaneInfo.createPane(false, false, Timing.EARLY, 1, -1));
+ container.write(mapView, ImmutableList.<WindowedValue<?>>of(three));
+
+ Map<String, Integer> overwrittenViewContents =
+ container
+ .createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
+ .get(mapView, SECOND_WINDOW);
+ assertThat(overwrittenViewContents, hasEntry("three", 3));
+ assertThat(overwrittenViewContents.size(), is(1));
+ }
+
+ /**
+ * Demonstrates that calling get() on a window that currently has no data does not return until
+ * there is data in the pane.
+ */
+ @Test
+ public void getBlocksUntilPaneAvailable() throws Exception {
+ BoundedWindow window =
+ new BoundedWindow() {
+ @Override
+ public Instant maxTimestamp() {
+ return new Instant(1024L);
+ }
+ };
+ Future<Double> singletonFuture =
+ getFutureOfView(
+ container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView)),
+ singletonView,
+ window);
+
+ WindowedValue<Double> singletonValue =
+ WindowedValue.of(4.75, new Instant(475L), window, PaneInfo.ON_TIME_AND_ONLY_FIRING);
+
+ assertThat(singletonFuture.isDone(), is(false));
+ container.write(singletonView, ImmutableList.<WindowedValue<?>>of(singletonValue));
+ assertThat(singletonFuture.get(), equalTo(4.75));
+ }
+
+ @Test
+ public void withPCollectionViewsWithPutInOriginalReturnsContents() throws Exception {
+ BoundedWindow window = new BoundedWindow() {
+ @Override
+ public Instant maxTimestamp() {
+ return new Instant(1024L);
+ }
+ };
+ SideInputReader newReader =
+ container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView));
+ Future<Double> singletonFuture = getFutureOfView(newReader, singletonView, window);
+
+ WindowedValue<Double> singletonValue =
+ WindowedValue.of(24.125, new Instant(475L), window, PaneInfo.ON_TIME_AND_ONLY_FIRING);
+
+ assertThat(singletonFuture.isDone(), is(false));
+ container.write(singletonView, ImmutableList.<WindowedValue<?>>of(singletonValue));
+ assertThat(singletonFuture.get(), equalTo(24.125));
+ }
+
+ @Test
+ public void withPCollectionViewsErrorsForContainsNotInViews() {
+ PCollectionView<Map<String, Iterable<String>>> newView =
+ PCollectionViews.multimapView(
+ pipeline,
+ WindowingStrategy.globalDefault(),
+ KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()));
+
+ thrown.expect(IllegalArgumentException.class);
+ thrown.expectMessage("with unknown views " + ImmutableList.of(newView).toString());
+
+ container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(newView));
+ }
+
+ @Test
+ public void withViewsForViewNotInContainerFails() {
+ PCollectionView<Map<String, Iterable<String>>> newView =
+ PCollectionViews.multimapView(
+ pipeline,
+ WindowingStrategy.globalDefault(),
+ KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()));
+
+ thrown.expect(IllegalArgumentException.class);
+ thrown.expectMessage("unknown views");
+ thrown.expectMessage(newView.toString());
+
+ container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(newView));
+ }
+
+ @Test
+ public void getOnReaderForViewNotInReaderFails() {
+ thrown.expect(IllegalArgumentException.class);
+ thrown.expectMessage("unknown view: " + iterableView.toString());
+
+ container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
+ .get(iterableView, GlobalWindow.INSTANCE);
+ }
+
+ @Test
+ public void writeForMultipleElementsInDifferentWindowsSucceeds() throws Exception {
+ WindowedValue<Double> firstWindowedValue =
+ WindowedValue.of(
+ 2.875,
+ FIRST_WINDOW.maxTimestamp().minus(200L),
+ FIRST_WINDOW,
+ PaneInfo.ON_TIME_AND_ONLY_FIRING);
+ WindowedValue<Double> secondWindowedValue =
+ WindowedValue.of(
+ 4.125,
+ SECOND_WINDOW.maxTimestamp().minus(2_000_000L),
+ SECOND_WINDOW,
+ PaneInfo.ON_TIME_AND_ONLY_FIRING);
+ container.write(singletonView, ImmutableList.of(firstWindowedValue, secondWindowedValue));
+ assertThat(
+ container
+ .createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView))
+ .get(singletonView, FIRST_WINDOW),
+ equalTo(2.875));
+ assertThat(
+ container
+ .createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView))
+ .get(singletonView, SECOND_WINDOW),
+ equalTo(4.125));
+ }
+
+ @Test
+ public void writeForMultipleIdenticalElementsInSameWindowSucceeds() throws Exception {
+ WindowedValue<Integer> firstValue =
+ WindowedValue.of(
+ 44,
+ FIRST_WINDOW.maxTimestamp().minus(200L),
+ FIRST_WINDOW,
+ PaneInfo.ON_TIME_AND_ONLY_FIRING);
+ WindowedValue<Integer> secondValue =
+ WindowedValue.of(
+ 44,
+ FIRST_WINDOW.maxTimestamp().minus(200L),
+ FIRST_WINDOW,
+ PaneInfo.ON_TIME_AND_ONLY_FIRING);
+
+ container.write(iterableView, ImmutableList.of(firstValue, secondValue));
+
+ assertThat(
+ container
+ .createReaderForViews(ImmutableList.<PCollectionView<?>>of(iterableView))
+ .get(iterableView, FIRST_WINDOW),
+ contains(44, 44));
+ }
+
+ @Test
+ public void writeForElementInMultipleWindowsSucceeds() throws Exception {
+ WindowedValue<Double> multiWindowedValue =
+ WindowedValue.of(
+ 2.875,
+ FIRST_WINDOW.maxTimestamp().minus(200L),
+ ImmutableList.of(FIRST_WINDOW, SECOND_WINDOW),
+ PaneInfo.ON_TIME_AND_ONLY_FIRING);
+ container.write(singletonView, ImmutableList.of(multiWindowedValue));
+ assertThat(
+ container
+ .createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView))
+ .get(singletonView, FIRST_WINDOW),
+ equalTo(2.875));
+ assertThat(
+ container
+ .createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView))
+ .get(singletonView, SECOND_WINDOW),
+ equalTo(2.875));
+ }
+
+ @Test
+ public void finishDoesNotOverwriteWrittenElements() throws Exception {
+ WindowedValue<KV<String, Integer>> one =
+ WindowedValue.of(
+ KV.of("one", 1),
+ new Instant(1L),
+ SECOND_WINDOW,
+ PaneInfo.createPane(true, false, Timing.EARLY));
+ WindowedValue<KV<String, Integer>> two =
+ WindowedValue.of(
+ KV.of("two", 2),
+ new Instant(20L),
+ SECOND_WINDOW,
+ PaneInfo.createPane(true, false, Timing.EARLY));
+ container.write(mapView, ImmutableList.<WindowedValue<?>>of(one, two));
+
+ immediatelyInvokeCallback(mapView, SECOND_WINDOW);
+
+ Map<String, Integer> viewContents =
+ container
+ .createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
+ .get(mapView, SECOND_WINDOW);
+
+ assertThat(viewContents, hasEntry("one", 1));
+ assertThat(viewContents, hasEntry("two", 2));
+ assertThat(viewContents.size(), is(2));
+ }
+
+ @Test
+ public void finishOnPendingViewsSetsEmptyElements() throws Exception {
+ immediatelyInvokeCallback(mapView, SECOND_WINDOW);
+ Future<Map<String, Integer>> mapFuture =
+ getFutureOfView(
+ container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView)),
+ mapView,
+ SECOND_WINDOW);
+
+ assertThat(mapFuture.get().isEmpty(), is(true));
+ }
+
+ /**
+ * Demonstrates that calling isReady on an empty container throws an
+ * {@link IllegalArgumentException}.
+ */
+ @Test
+ public void isReadyInEmptyReaderThrows() {
+ ReadyCheckingSideInputReader reader =
+ container.createReaderForViews(ImmutableList.<PCollectionView<?>>of());
+ thrown.expect(IllegalArgumentException.class);
+ thrown.expectMessage("does not contain");
+ thrown.expectMessage(ImmutableList.of().toString());
+ reader.isReady(mapView, GlobalWindow.INSTANCE);
+ }
+
+ /**
+ * Demonstrates that calling isReady returns false until elements are written to the
+ * {@link PCollectionView}, {@link BoundedWindow} pair, at which point it returns true.
+ */
+ @Test
+ public void isReadyForSomeNotReadyViewsFalseUntilElements() {
+ container.write(
+ mapView,
+ ImmutableList.of(
+ WindowedValue.of(
+ KV.of("one", 1),
+ SECOND_WINDOW.maxTimestamp().minus(100L),
+ SECOND_WINDOW,
+ PaneInfo.ON_TIME_AND_ONLY_FIRING)));
+
+ ReadyCheckingSideInputReader reader =
+ container.createReaderForViews(ImmutableList.of(mapView, singletonView));
+ assertThat(reader.isReady(mapView, FIRST_WINDOW), is(false));
+ assertThat(reader.isReady(mapView, SECOND_WINDOW), is(true));
+
+ assertThat(reader.isReady(singletonView, SECOND_WINDOW), is(false));
+
+ container.write(
+ mapView,
+ ImmutableList.of(
+ WindowedValue.of(
+ KV.of("too", 2),
+ FIRST_WINDOW.maxTimestamp().minus(100L),
+ FIRST_WINDOW,
+ PaneInfo.ON_TIME_AND_ONLY_FIRING)));
+ assertThat(reader.isReady(mapView, FIRST_WINDOW), is(true));
+
+ container.write(
+ singletonView,
+ ImmutableList.of(
+ WindowedValue.of(
+ 1.25,
+ SECOND_WINDOW.maxTimestamp().minus(100L),
+ SECOND_WINDOW,
+ PaneInfo.ON_TIME_AND_ONLY_FIRING)));
+ assertThat(reader.isReady(mapView, SECOND_WINDOW), is(true));
+ assertThat(reader.isReady(singletonView, SECOND_WINDOW), is(true));
+
+ assertThat(reader.isReady(mapView, GlobalWindow.INSTANCE), is(false));
+ assertThat(reader.isReady(singletonView, GlobalWindow.INSTANCE), is(false));
+ }
+
+ @Test
+ public void isReadyForEmptyWindowTrue() {
+ immediatelyInvokeCallback(mapView, GlobalWindow.INSTANCE);
+
+ ReadyCheckingSideInputReader reader =
+ container.createReaderForViews(ImmutableList.of(mapView, singletonView));
+ assertThat(reader.isReady(mapView, GlobalWindow.INSTANCE), is(true));
+ assertThat(reader.isReady(singletonView, GlobalWindow.INSTANCE), is(false));
+
+ immediatelyInvokeCallback(singletonView, GlobalWindow.INSTANCE);
+ assertThat(reader.isReady(singletonView, GlobalWindow.INSTANCE), is(true));
+ }
+
+ /**
+ * When a callAfterWindowCloses with the specified view's producing transform, window, and
+ * windowing strategy is invoked, immediately execute the callback.
+ */
+ private void immediatelyInvokeCallback(PCollectionView<?> view, BoundedWindow window) {
+ doAnswer(
+ new Answer<Void>() {
+ @Override
+ public Void answer(InvocationOnMock invocation) throws Throwable {
+ Object callback = invocation.getArguments()[3];
+ Runnable callbackRunnable = (Runnable) callback;
+ callbackRunnable.run();
+ return null;
+ }
+ })
+ .when(context)
+ .scheduleAfterOutputWouldBeProduced(
+ Mockito.eq(view),
+ Mockito.eq(window),
+ Mockito.eq(view.getWindowingStrategyInternal()),
+ Mockito.any(Runnable.class));
+ }
+
+ private <ValueT> Future<ValueT> getFutureOfView(final SideInputReader myReader,
+ final PCollectionView<ValueT> view, final BoundedWindow window) {
+ Callable<ValueT> callable = new Callable<ValueT>() {
+ @Override
+ public ValueT call() throws Exception {
+ return myReader.get(view, window);
+ }
+ };
+ return Executors.newSingleThreadExecutor().submit(callable);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessTimerInternalsTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessTimerInternalsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessTimerInternalsTest.java
new file mode 100644
index 0000000..34a8980
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessTimerInternalsTest.java
@@ -0,0 +1,133 @@
+/*
+ * 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.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate.TimerUpdateBuilder;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TransformWatermarks;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+
+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 InProcessTimerInternals}.
+ */
+@RunWith(JUnit4.class)
+public class InProcessTimerInternalsTest {
+ private MockClock clock;
+ @Mock private TransformWatermarks watermarks;
+
+ private TimerUpdateBuilder timerUpdateBuilder;
+
+ private InProcessTimerInternals internals;
+
+ @Before
+ public void setup() {
+ MockitoAnnotations.initMocks(this);
+ clock = MockClock.fromInstant(new Instant(0));
+
+ timerUpdateBuilder = TimerUpdate.builder(1234);
+
+ internals = InProcessTimerInternals.create(clock, watermarks, timerUpdateBuilder);
+ }
+
+ @Test
+ public void setTimerAddsToBuilder() {
+ TimerData eventTimer =
+ TimerData.of(StateNamespaces.global(), new Instant(20145L), TimeDomain.EVENT_TIME);
+ TimerData processingTimer =
+ TimerData.of(StateNamespaces.global(), new Instant(125555555L), TimeDomain.PROCESSING_TIME);
+ TimerData synchronizedProcessingTimer =
+ TimerData.of(
+ StateNamespaces.global(),
+ new Instant(98745632189L),
+ TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+ internals.setTimer(eventTimer);
+ internals.setTimer(processingTimer);
+ internals.setTimer(synchronizedProcessingTimer);
+
+ assertThat(
+ internals.getTimerUpdate().getSetTimers(),
+ containsInAnyOrder(eventTimer, synchronizedProcessingTimer, processingTimer));
+ }
+
+ @Test
+ public void deleteTimerDeletesOnBuilder() {
+ TimerData eventTimer =
+ TimerData.of(StateNamespaces.global(), new Instant(20145L), TimeDomain.EVENT_TIME);
+ TimerData processingTimer =
+ TimerData.of(StateNamespaces.global(), new Instant(125555555L), TimeDomain.PROCESSING_TIME);
+ TimerData synchronizedProcessingTimer =
+ TimerData.of(
+ StateNamespaces.global(),
+ new Instant(98745632189L),
+ TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+ internals.deleteTimer(eventTimer);
+ internals.deleteTimer(processingTimer);
+ internals.deleteTimer(synchronizedProcessingTimer);
+
+ assertThat(
+ internals.getTimerUpdate().getDeletedTimers(),
+ containsInAnyOrder(eventTimer, synchronizedProcessingTimer, processingTimer));
+ }
+
+ @Test
+ public void getProcessingTimeIsClockNow() {
+ assertThat(internals.currentProcessingTime(), equalTo(clock.now()));
+ Instant oldProcessingTime = internals.currentProcessingTime();
+
+ clock.advance(Duration.standardHours(12));
+
+ assertThat(internals.currentProcessingTime(), equalTo(clock.now()));
+ assertThat(
+ internals.currentProcessingTime(),
+ equalTo(oldProcessingTime.plus(Duration.standardHours(12))));
+ }
+
+ @Test
+ public void getSynchronizedProcessingTimeIsWatermarkSynchronizedInputTime() {
+ when(watermarks.getSynchronizedProcessingInputTime()).thenReturn(new Instant(12345L));
+ assertThat(internals.currentSynchronizedProcessingTime(), equalTo(new Instant(12345L)));
+ }
+
+ @Test
+ public void getInputWatermarkTimeUsesWatermarkTime() {
+ when(watermarks.getInputWatermark()).thenReturn(new Instant(8765L));
+ assertThat(internals.currentInputWatermarkTime(), equalTo(new Instant(8765L)));
+ }
+
+ @Test
+ public void getOutputWatermarkTimeUsesWatermarkTime() {
+ when(watermarks.getOutputWatermark()).thenReturn(new Instant(25525L));
+ assertThat(internals.currentOutputWatermarkTime(), equalTo(new Instant(25525L)));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java
new file mode 100644
index 0000000..24f9715
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java
@@ -0,0 +1,192 @@
+/*
+ * 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.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+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.Keys;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.common.collect.ImmutableSet;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.Collections;
+import java.util.Set;
+
+/**
+ * Tests for {@link KeyedPValueTrackingVisitor}.
+ */
+@RunWith(JUnit4.class)
+public class KeyedPValueTrackingVisitorTest {
+ @Rule public ExpectedException thrown = ExpectedException.none();
+
+ private KeyedPValueTrackingVisitor visitor;
+ private Pipeline p;
+
+ @Before
+ public void setup() {
+ PipelineOptions options = PipelineOptionsFactory.create();
+
+ p = Pipeline.create(options);
+ @SuppressWarnings("rawtypes")
+ Set<Class<? extends PTransform>> producesKeyed =
+ ImmutableSet.<Class<? extends PTransform>>of(PrimitiveKeyer.class, CompositeKeyer.class);
+ visitor = KeyedPValueTrackingVisitor.create(producesKeyed);
+ }
+
+ @Test
+ public void primitiveProducesKeyedOutputUnkeyedInputKeyedOutput() {
+ PCollection<Integer> keyed =
+ p.apply(Create.<Integer>of(1, 2, 3)).apply(new PrimitiveKeyer<Integer>());
+
+ p.traverseTopologically(visitor);
+ assertThat(visitor.getKeyedPValues(), hasItem(keyed));
+ }
+
+ @Test
+ public void primitiveProducesKeyedOutputKeyedInputKeyedOutut() {
+ PCollection<Integer> keyed =
+ p.apply(Create.<Integer>of(1, 2, 3))
+ .apply("firstKey", new PrimitiveKeyer<Integer>())
+ .apply("secondKey", new PrimitiveKeyer<Integer>());
+
+ p.traverseTopologically(visitor);
+ assertThat(visitor.getKeyedPValues(), hasItem(keyed));
+ }
+
+ @Test
+ public void compositeProducesKeyedOutputUnkeyedInputKeyedOutput() {
+ PCollection<Integer> keyed =
+ p.apply(Create.<Integer>of(1, 2, 3)).apply(new CompositeKeyer<Integer>());
+
+ p.traverseTopologically(visitor);
+ assertThat(visitor.getKeyedPValues(), hasItem(keyed));
+ }
+
+ @Test
+ public void compositeProducesKeyedOutputKeyedInputKeyedOutut() {
+ PCollection<Integer> keyed =
+ p.apply(Create.<Integer>of(1, 2, 3))
+ .apply("firstKey", new CompositeKeyer<Integer>())
+ .apply("secondKey", new CompositeKeyer<Integer>());
+
+ p.traverseTopologically(visitor);
+ assertThat(visitor.getKeyedPValues(), hasItem(keyed));
+ }
+
+
+ @Test
+ public void noInputUnkeyedOutput() {
+ PCollection<KV<Integer, Iterable<Void>>> unkeyed =
+ p.apply(
+ Create.of(KV.<Integer, Iterable<Void>>of(-1, Collections.<Void>emptyList()))
+ .withCoder(KvCoder.of(VarIntCoder.of(), IterableCoder.of(VoidCoder.of()))));
+
+ p.traverseTopologically(visitor);
+ assertThat(visitor.getKeyedPValues(), not(hasItem(unkeyed)));
+ }
+
+ @Test
+ public void keyedInputNotProducesKeyedOutputUnkeyedOutput() {
+ PCollection<Integer> onceKeyed =
+ p.apply(Create.<Integer>of(1, 2, 3))
+ .apply(new PrimitiveKeyer<Integer>())
+ .apply(ParDo.of(new IdentityFn<Integer>()));
+
+ p.traverseTopologically(visitor);
+ assertThat(visitor.getKeyedPValues(), not(hasItem(onceKeyed)));
+ }
+
+ @Test
+ public void unkeyedInputNotProducesKeyedOutputUnkeyedOutput() {
+ PCollection<Integer> unkeyed =
+ p.apply(Create.<Integer>of(1, 2, 3)).apply(ParDo.of(new IdentityFn<Integer>()));
+
+ p.traverseTopologically(visitor);
+ assertThat(visitor.getKeyedPValues(), not(hasItem(unkeyed)));
+ }
+
+ @Test
+ public void traverseMultipleTimesThrows() {
+ p.apply(
+ Create.<KV<Integer, Void>>of(
+ KV.of(1, (Void) null), KV.of(2, (Void) null), KV.of(3, (Void) null))
+ .withCoder(KvCoder.of(VarIntCoder.of(), VoidCoder.of())))
+ .apply(GroupByKey.<Integer, Void>create())
+ .apply(Keys.<Integer>create());
+
+ p.traverseTopologically(visitor);
+
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage("already been finalized");
+ thrown.expectMessage(KeyedPValueTrackingVisitor.class.getSimpleName());
+ p.traverseTopologically(visitor);
+ }
+
+ @Test
+ public void getKeyedPValuesBeforeTraverseThrows() {
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage("completely traversed");
+ thrown.expectMessage("getKeyedPValues");
+ visitor.getKeyedPValues();
+ }
+
+ private static class PrimitiveKeyer<K> extends PTransform<PCollection<K>, PCollection<K>> {
+ @Override
+ public PCollection<K> apply(PCollection<K> input) {
+ return PCollection.<K>createPrimitiveOutputInternal(
+ input.getPipeline(), input.getWindowingStrategy(), input.isBounded())
+ .setCoder(input.getCoder());
+ }
+ }
+
+ private static class CompositeKeyer<K> extends PTransform<PCollection<K>, PCollection<K>> {
+ @Override
+ public PCollection<K> apply(PCollection<K> input) {
+ return input.apply(new PrimitiveKeyer<K>()).apply(ParDo.of(new IdentityFn<K>()));
+ }
+ }
+
+ private static class IdentityFn<K> extends DoFn<K, K> {
+ @Override
+ public void processElement(DoFn<K, K>.ProcessContext c) throws Exception {
+ c.output(c.element());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/MockClock.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/MockClock.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/MockClock.java
new file mode 100644
index 0000000..11ecbff
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/MockClock.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * A clock that returns a constant value for now which can be set with calls to
+ * {@link #set(Instant)}.
+ *
+ * <p>For uses of the {@link Clock} interface in unit tests.
+ */
+public class MockClock implements Clock {
+
+ private Instant now;
+
+ public static MockClock fromInstant(Instant initial) {
+ return new MockClock(initial);
+ }
+
+ private MockClock(Instant initialNow) {
+ this.now = initialNow;
+ }
+
+ public void set(Instant newNow) {
+ checkArgument(!newNow.isBefore(now), "Cannot move MockClock backwards in time from %s to %s",
+ now, newNow);
+ this.now = newNow;
+ }
+
+ public void advance(Duration duration) {
+ checkArgument(
+ duration.getMillis() > 0,
+ "Cannot move MockClock backwards in time by duration %s",
+ duration);
+ set(now.plus(duration));
+ }
+
+ @Override
+ public Instant now() {
+ return now;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
new file mode 100644
index 0000000..cecfe01
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
@@ -0,0 +1,431 @@
+/*
+ * 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.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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.ParDo;
+import org.apache.beam.sdk.transforms.ParDo.BoundMulti;
+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.OutputTimeFns;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+import org.apache.beam.sdk.util.state.WatermarkHoldState;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+
+import org.hamcrest.Matchers;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.Serializable;
+
+/**
+ * Tests for {@link ParDoMultiEvaluatorFactory}.
+ */
+@RunWith(JUnit4.class)
+public class ParDoMultiEvaluatorFactoryTest implements Serializable {
+ private transient BundleFactory bundleFactory = InProcessBundleFactory.create();
+
+ @Test
+ public void testParDoMultiInMemoryTransformEvaluator() throws Exception {
+ TestPipeline p = TestPipeline.create();
+
+ PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
+
+ TupleTag<KV<String, Integer>> mainOutputTag = new TupleTag<KV<String, Integer>>() {};
+ final TupleTag<String> elementTag = new TupleTag<>();
+ final TupleTag<Integer> lengthTag = new TupleTag<>();
+
+ BoundMulti<String, KV<String, Integer>> pardo =
+ ParDo.of(
+ new DoFn<String, KV<String, Integer>>() {
+ @Override
+ public void processElement(ProcessContext c) {
+ c.output(KV.<String, Integer>of(c.element(), c.element().length()));
+ c.sideOutput(elementTag, c.element());
+ c.sideOutput(lengthTag, c.element().length());
+ }
+ })
+ .withOutputTags(mainOutputTag, TupleTagList.of(elementTag).and(lengthTag));
+ PCollectionTuple outputTuple = input.apply(pardo);
+
+ CommittedBundle<String> inputBundle =
+ bundleFactory.createRootBundle(input).commit(Instant.now());
+
+ PCollection<KV<String, Integer>> mainOutput = outputTuple.get(mainOutputTag);
+ PCollection<String> elementOutput = outputTuple.get(elementTag);
+ PCollection<Integer> lengthOutput = outputTuple.get(lengthTag);
+
+ InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+ UncommittedBundle<KV<String, Integer>> mainOutputBundle =
+ bundleFactory.createRootBundle(mainOutput);
+ UncommittedBundle<String> elementOutputBundle = bundleFactory.createRootBundle(elementOutput);
+ UncommittedBundle<Integer> lengthOutputBundle = bundleFactory.createRootBundle(lengthOutput);
+
+ when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
+ when(evaluationContext.createBundle(inputBundle, elementOutput))
+ .thenReturn(elementOutputBundle);
+ when(evaluationContext.createBundle(inputBundle, lengthOutput)).thenReturn(lengthOutputBundle);
+
+ InProcessExecutionContext executionContext =
+ new InProcessExecutionContext(null, null, null, null);
+ when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
+ .thenReturn(executionContext);
+ CounterSet counters = new CounterSet();
+ when(evaluationContext.createCounterSet()).thenReturn(counters);
+
+ TransformEvaluator<String> evaluator =
+ new ParDoMultiEvaluatorFactory()
+ .forApplication(
+ mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
+
+ evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
+ evaluator.processElement(
+ WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
+ evaluator.processElement(
+ WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
+
+ InProcessTransformResult result = evaluator.finishBundle();
+ assertThat(
+ result.getOutputBundles(),
+ Matchers.<UncommittedBundle<?>>containsInAnyOrder(
+ lengthOutputBundle, mainOutputBundle, elementOutputBundle));
+ assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
+ assertThat(result.getCounters(), equalTo(counters));
+
+ assertThat(
+ mainOutputBundle.commit(Instant.now()).getElements(),
+ Matchers.<WindowedValue<KV<String, Integer>>>containsInAnyOrder(
+ WindowedValue.valueInGlobalWindow(KV.of("foo", 3)),
+ WindowedValue.timestampedValueInGlobalWindow(KV.of("bara", 4), new Instant(1000)),
+ WindowedValue.valueInGlobalWindow(
+ KV.of("bazam", 5), PaneInfo.ON_TIME_AND_ONLY_FIRING)));
+ assertThat(
+ elementOutputBundle.commit(Instant.now()).getElements(),
+ Matchers.<WindowedValue<String>>containsInAnyOrder(
+ WindowedValue.valueInGlobalWindow("foo"),
+ WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)),
+ WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING)));
+ assertThat(
+ lengthOutputBundle.commit(Instant.now()).getElements(),
+ Matchers.<WindowedValue<Integer>>containsInAnyOrder(
+ WindowedValue.valueInGlobalWindow(3),
+ WindowedValue.timestampedValueInGlobalWindow(4, new Instant(1000)),
+ WindowedValue.valueInGlobalWindow(5, PaneInfo.ON_TIME_AND_ONLY_FIRING)));
+ }
+
+ @Test
+ public void testParDoMultiUndeclaredSideOutput() throws Exception {
+ TestPipeline p = TestPipeline.create();
+
+ PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
+
+ TupleTag<KV<String, Integer>> mainOutputTag = new TupleTag<KV<String, Integer>>() {};
+ final TupleTag<String> elementTag = new TupleTag<>();
+ final TupleTag<Integer> lengthTag = new TupleTag<>();
+
+ BoundMulti<String, KV<String, Integer>> pardo =
+ ParDo.of(
+ new DoFn<String, KV<String, Integer>>() {
+ @Override
+ public void processElement(ProcessContext c) {
+ c.output(KV.<String, Integer>of(c.element(), c.element().length()));
+ c.sideOutput(elementTag, c.element());
+ c.sideOutput(lengthTag, c.element().length());
+ }
+ })
+ .withOutputTags(mainOutputTag, TupleTagList.of(elementTag));
+ PCollectionTuple outputTuple = input.apply(pardo);
+
+ CommittedBundle<String> inputBundle =
+ bundleFactory.createRootBundle(input).commit(Instant.now());
+
+ PCollection<KV<String, Integer>> mainOutput = outputTuple.get(mainOutputTag);
+ PCollection<String> elementOutput = outputTuple.get(elementTag);
+
+ InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+ UncommittedBundle<KV<String, Integer>> mainOutputBundle =
+ bundleFactory.createRootBundle(mainOutput);
+ UncommittedBundle<String> elementOutputBundle = bundleFactory.createRootBundle(elementOutput);
+
+ when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
+ when(evaluationContext.createBundle(inputBundle, elementOutput))
+ .thenReturn(elementOutputBundle);
+
+ InProcessExecutionContext executionContext =
+ new InProcessExecutionContext(null, null, null, null);
+ when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
+ .thenReturn(executionContext);
+ CounterSet counters = new CounterSet();
+ when(evaluationContext.createCounterSet()).thenReturn(counters);
+
+ TransformEvaluator<String> evaluator =
+ new ParDoMultiEvaluatorFactory()
+ .forApplication(
+ mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
+
+ evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
+ evaluator.processElement(
+ WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
+ evaluator.processElement(
+ WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
+
+ InProcessTransformResult result = evaluator.finishBundle();
+ assertThat(
+ result.getOutputBundles(),
+ Matchers.<UncommittedBundle<?>>containsInAnyOrder(mainOutputBundle, elementOutputBundle));
+ assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
+ assertThat(result.getCounters(), equalTo(counters));
+
+ assertThat(
+ mainOutputBundle.commit(Instant.now()).getElements(),
+ Matchers.<WindowedValue<KV<String, Integer>>>containsInAnyOrder(
+ WindowedValue.valueInGlobalWindow(KV.of("foo", 3)),
+ WindowedValue.timestampedValueInGlobalWindow(KV.of("bara", 4), new Instant(1000)),
+ WindowedValue.valueInGlobalWindow(
+ KV.of("bazam", 5), PaneInfo.ON_TIME_AND_ONLY_FIRING)));
+ assertThat(
+ elementOutputBundle.commit(Instant.now()).getElements(),
+ Matchers.<WindowedValue<String>>containsInAnyOrder(
+ WindowedValue.valueInGlobalWindow("foo"),
+ WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)),
+ WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING)));
+ }
+
+ @Test
+ public void finishBundleWithStatePutsStateInResult() throws Exception {
+ TestPipeline p = TestPipeline.create();
+
+ PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
+
+ TupleTag<KV<String, Integer>> mainOutputTag = new TupleTag<KV<String, Integer>>() {};
+ final TupleTag<String> elementTag = new TupleTag<>();
+
+ final StateTag<Object, WatermarkHoldState<BoundedWindow>> watermarkTag =
+ StateTags.watermarkStateInternal("myId", OutputTimeFns.outputAtEndOfWindow());
+ final StateTag<Object, BagState<String>> bagTag = StateTags.bag("myBag", StringUtf8Coder.of());
+ final StateNamespace windowNs =
+ StateNamespaces.window(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE);
+ BoundMulti<String, KV<String, Integer>> pardo =
+ ParDo.of(
+ new DoFn<String, KV<String, Integer>>() {
+ @Override
+ public void processElement(ProcessContext c) {
+ c.windowingInternals()
+ .stateInternals()
+ .state(StateNamespaces.global(), watermarkTag)
+ .add(new Instant(20202L + c.element().length()));
+ c.windowingInternals()
+ .stateInternals()
+ .state(
+ StateNamespaces.window(
+ GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE),
+ bagTag)
+ .add(c.element());
+ }
+ })
+ .withOutputTags(mainOutputTag, TupleTagList.of(elementTag));
+ PCollectionTuple outputTuple = input.apply(pardo);
+
+ CommittedBundle<String> inputBundle =
+ bundleFactory.createRootBundle(input).commit(Instant.now());
+
+ PCollection<KV<String, Integer>> mainOutput = outputTuple.get(mainOutputTag);
+ PCollection<String> elementOutput = outputTuple.get(elementTag);
+
+ InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+ UncommittedBundle<KV<String, Integer>> mainOutputBundle =
+ bundleFactory.createRootBundle(mainOutput);
+ UncommittedBundle<String> elementOutputBundle = bundleFactory.createRootBundle(elementOutput);
+
+ when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
+ when(evaluationContext.createBundle(inputBundle, elementOutput))
+ .thenReturn(elementOutputBundle);
+
+ InProcessExecutionContext executionContext =
+ new InProcessExecutionContext(null, "myKey", null, null);
+ when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
+ .thenReturn(executionContext);
+ CounterSet counters = new CounterSet();
+ when(evaluationContext.createCounterSet()).thenReturn(counters);
+
+ TransformEvaluator<String> evaluator =
+ new ParDoMultiEvaluatorFactory()
+ .forApplication(
+ mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
+
+ evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
+ evaluator.processElement(
+ WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
+ evaluator.processElement(
+ WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
+
+ InProcessTransformResult result = evaluator.finishBundle();
+ assertThat(
+ result.getOutputBundles(),
+ Matchers.<UncommittedBundle<?>>containsInAnyOrder(mainOutputBundle, elementOutputBundle));
+ assertThat(result.getWatermarkHold(), equalTo(new Instant(20205L)));
+ assertThat(result.getState(), not(nullValue()));
+ assertThat(
+ result.getState().state(StateNamespaces.global(), watermarkTag).read(),
+ equalTo(new Instant(20205L)));
+ assertThat(
+ result.getState().state(windowNs, bagTag).read(),
+ containsInAnyOrder("foo", "bara", "bazam"));
+ }
+
+ @Test
+ public void finishBundleWithStateAndTimersPutsTimersInResult() throws Exception {
+ TestPipeline p = TestPipeline.create();
+
+ PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
+
+ TupleTag<KV<String, Integer>> mainOutputTag = new TupleTag<KV<String, Integer>>() {};
+ final TupleTag<String> elementTag = new TupleTag<>();
+
+ final TimerData addedTimer =
+ TimerData.of(
+ StateNamespaces.window(
+ IntervalWindow.getCoder(),
+ new IntervalWindow(
+ new Instant(0).plus(Duration.standardMinutes(5)),
+ new Instant(1)
+ .plus(Duration.standardMinutes(5))
+ .plus(Duration.standardHours(1)))),
+ new Instant(54541L),
+ TimeDomain.EVENT_TIME);
+ final TimerData deletedTimer =
+ TimerData.of(
+ StateNamespaces.window(
+ IntervalWindow.getCoder(),
+ new IntervalWindow(new Instant(0), new Instant(0).plus(Duration.standardHours(1)))),
+ new Instant(3400000),
+ TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+
+ BoundMulti<String, KV<String, Integer>> pardo =
+ ParDo.of(
+ new DoFn<String, KV<String, Integer>>() {
+ @Override
+ public void processElement(ProcessContext c) {
+ c.windowingInternals().stateInternals();
+ c.windowingInternals()
+ .timerInternals()
+ .setTimer(
+ TimerData.of(
+ StateNamespaces.window(
+ IntervalWindow.getCoder(),
+ new IntervalWindow(
+ new Instant(0).plus(Duration.standardMinutes(5)),
+ new Instant(1)
+ .plus(Duration.standardMinutes(5))
+ .plus(Duration.standardHours(1)))),
+ new Instant(54541L),
+ TimeDomain.EVENT_TIME));
+ c.windowingInternals()
+ .timerInternals()
+ .deleteTimer(
+ TimerData.of(
+ StateNamespaces.window(
+ IntervalWindow.getCoder(),
+ new IntervalWindow(
+ new Instant(0),
+ new Instant(0).plus(Duration.standardHours(1)))),
+ new Instant(3400000),
+ TimeDomain.SYNCHRONIZED_PROCESSING_TIME));
+ }
+ })
+ .withOutputTags(mainOutputTag, TupleTagList.of(elementTag));
+ PCollectionTuple outputTuple = input.apply(pardo);
+
+ CommittedBundle<String> inputBundle =
+ bundleFactory.createRootBundle(input).commit(Instant.now());
+
+ PCollection<KV<String, Integer>> mainOutput = outputTuple.get(mainOutputTag);
+ PCollection<String> elementOutput = outputTuple.get(elementTag);
+
+ InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+ UncommittedBundle<KV<String, Integer>> mainOutputBundle =
+ bundleFactory.createRootBundle(mainOutput);
+ UncommittedBundle<String> elementOutputBundle = bundleFactory.createRootBundle(elementOutput);
+
+ when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
+ when(evaluationContext.createBundle(inputBundle, elementOutput))
+ .thenReturn(elementOutputBundle);
+
+ InProcessExecutionContext executionContext =
+ new InProcessExecutionContext(null, "myKey", null, null);
+ when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
+ .thenReturn(executionContext);
+ CounterSet counters = new CounterSet();
+ when(evaluationContext.createCounterSet()).thenReturn(counters);
+
+ TransformEvaluator<String> evaluator =
+ new ParDoMultiEvaluatorFactory()
+ .forApplication(
+ mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
+
+ evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
+ evaluator.processElement(
+ WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
+ evaluator.processElement(
+ WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
+
+ InProcessTransformResult result = evaluator.finishBundle();
+ assertThat(
+ result.getTimerUpdate(),
+ equalTo(
+ TimerUpdate.builder("myKey")
+ .setTimer(addedTimer)
+ .setTimer(addedTimer)
+ .setTimer(addedTimer)
+ .deletedTimer(deletedTimer)
+ .deletedTimer(deletedTimer)
+ .deletedTimer(deletedTimer)
+ .build()));
+ }
+}
[16/17] incubator-beam git commit: Move InProcessRunner to its own
module
Posted by ke...@apache.org.
Move InProcessRunner to its own module
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/e13cacb8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/e13cacb8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/e13cacb8
Branch: refs/heads/master
Commit: e13cacb81c9c9718a45bfb7aefd839dcdfd442f6
Parents: bba4c64
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Apr 27 15:01:48 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Fri Apr 29 14:28:45 2016 -0700
----------------------------------------------------------------------
.travis.yml | 1 +
runners/direct-java/pom.xml | 400 ++++++
.../direct/AbstractModelEnforcement.java | 38 +
.../direct/AvroIOShardedWriteFactory.java | 76 +
.../direct/BoundedReadEvaluatorFactory.java | 155 ++
.../beam/runners/direct/BundleFactory.java | 49 +
.../CachedThreadPoolExecutorServiceFactory.java | 44 +
.../org/apache/beam/runners/direct/Clock.java | 30 +
.../beam/runners/direct/CommittedResult.java | 46 +
.../beam/runners/direct/CompletionCallback.java | 36 +
.../direct/ConsumerTrackingPipelineVisitor.java | 173 +++
.../runners/direct/EmptyTransformEvaluator.java | 50 +
.../direct/EncodabilityEnforcementFactory.java | 70 +
.../beam/runners/direct/EvaluatorKey.java | 55 +
.../runners/direct/ExecutorServiceFactory.java | 33 +
.../direct/ExecutorServiceParallelExecutor.java | 478 +++++++
.../runners/direct/FlattenEvaluatorFactory.java | 85 ++
.../runners/direct/ForwardingPTransform.java | 62 +
.../direct/GroupByKeyEvaluatorFactory.java | 274 ++++
.../ImmutabilityCheckingBundleFactory.java | 131 ++
.../direct/ImmutabilityEnforcementFactory.java | 103 ++
.../direct/InMemoryWatermarkManager.java | 1327 ++++++++++++++++++
.../runners/direct/InProcessBundleFactory.java | 162 +++
.../direct/InProcessBundleOutputManager.java | 51 +
.../direct/InProcessEvaluationContext.java | 425 ++++++
.../direct/InProcessExecutionContext.java | 105 ++
.../beam/runners/direct/InProcessExecutor.java | 48 +
.../direct/InProcessPipelineOptions.java | 101 ++
.../runners/direct/InProcessPipelineRunner.java | 370 +++++
.../beam/runners/direct/InProcessRegistrar.java | 55 +
.../direct/InProcessSideInputContainer.java | 271 ++++
.../runners/direct/InProcessTimerInternals.java | 84 ++
.../direct/InProcessTransformResult.java | 77 +
.../direct/KeyedPValueTrackingVisitor.java | 96 ++
.../beam/runners/direct/ModelEnforcement.java | 63 +
.../runners/direct/ModelEnforcementFactory.java | 30 +
.../beam/runners/direct/NanosOffsetClock.java | 59 +
.../direct/PTransformOverrideFactory.java | 33 +
.../runners/direct/ParDoInProcessEvaluator.java | 173 +++
.../direct/ParDoMultiEvaluatorFactory.java | 64 +
.../direct/ParDoSingleEvaluatorFactory.java | 63 +
.../direct/PassthroughTransformEvaluator.java | 49 +
.../runners/direct/ShardControlledWrite.java | 81 ++
.../apache/beam/runners/direct/StepAndKey.java | 71 +
.../runners/direct/StepTransformResult.java | 165 +++
.../direct/TextIOShardedWriteFactory.java | 78 +
.../beam/runners/direct/TransformEvaluator.java | 46 +
.../direct/TransformEvaluatorFactory.java | 44 +
.../direct/TransformEvaluatorRegistry.java | 77 +
.../beam/runners/direct/TransformExecutor.java | 176 +++
.../direct/TransformExecutorService.java | 35 +
.../direct/TransformExecutorServices.java | 154 ++
.../direct/UnboundedReadEvaluatorFactory.java | 177 +++
.../runners/direct/ViewEvaluatorFactory.java | 145 ++
.../direct/WatermarkCallbackExecutor.java | 146 ++
.../runners/direct/WindowEvaluatorFactory.java | 131 ++
.../direct/AvroIOShardedWriteFactoryTest.java | 112 ++
.../direct/BoundedReadEvaluatorFactoryTest.java | 290 ++++
.../runners/direct/CommittedResultTest.java | 77 +
.../ConsumerTrackingPipelineVisitorTest.java | 272 ++++
.../EncodabilityEnforcementFactoryTest.java | 257 ++++
.../direct/FlattenEvaluatorFactoryTest.java | 141 ++
.../direct/ForwardingPTransformTest.java | 112 ++
.../direct/GroupByKeyEvaluatorFactoryTest.java | 183 +++
.../ImmutabilityCheckingBundleFactoryTest.java | 220 +++
.../ImmutabilityEnforcementFactoryTest.java | 128 ++
.../direct/InMemoryWatermarkManagerTest.java | 1168 +++++++++++++++
.../direct/InProcessBundleFactoryTest.java | 223 +++
.../direct/InProcessEvaluationContextTest.java | 526 +++++++
.../direct/InProcessPipelineRegistrarTest.java | 74 +
.../direct/InProcessPipelineRunnerTest.java | 78 +
.../direct/InProcessSideInputContainerTest.java | 496 +++++++
.../direct/InProcessTimerInternalsTest.java | 133 ++
.../direct/KeyedPValueTrackingVisitorTest.java | 192 +++
.../apache/beam/runners/direct/MockClock.java | 62 +
.../direct/ParDoMultiEvaluatorFactoryTest.java | 431 ++++++
.../direct/ParDoSingleEvaluatorFactoryTest.java | 324 +++++
.../direct/TextIOShardedWriteFactoryTest.java | 112 ++
.../direct/TransformExecutorServicesTest.java | 136 ++
.../runners/direct/TransformExecutorTest.java | 538 +++++++
.../UnboundedReadEvaluatorFactoryTest.java | 334 +++++
.../direct/ViewEvaluatorFactoryTest.java | 101 ++
.../direct/WatermarkCallbackExecutorTest.java | 128 ++
.../direct/WindowEvaluatorFactoryTest.java | 222 +++
runners/pom.xml | 1 +
.../inprocess/AbstractModelEnforcement.java | 38 -
.../inprocess/AvroIOShardedWriteFactory.java | 76 -
.../inprocess/BoundedReadEvaluatorFactory.java | 155 --
.../sdk/runners/inprocess/BundleFactory.java | 49 -
.../CachedThreadPoolExecutorServiceFactory.java | 44 -
.../beam/sdk/runners/inprocess/Clock.java | 30 -
.../sdk/runners/inprocess/CommittedResult.java | 46 -
.../runners/inprocess/CompletionCallback.java | 36 -
.../ConsumerTrackingPipelineVisitor.java | 173 ---
.../inprocess/EmptyTransformEvaluator.java | 50 -
.../EncodabilityEnforcementFactory.java | 70 -
.../sdk/runners/inprocess/EvaluatorKey.java | 55 -
.../inprocess/ExecutorServiceFactory.java | 33 -
.../ExecutorServiceParallelExecutor.java | 478 -------
.../inprocess/FlattenEvaluatorFactory.java | 85 --
.../runners/inprocess/ForwardingPTransform.java | 62 -
.../inprocess/GroupByKeyEvaluatorFactory.java | 274 ----
.../ImmutabilityCheckingBundleFactory.java | 131 --
.../ImmutabilityEnforcementFactory.java | 103 --
.../inprocess/InMemoryWatermarkManager.java | 1327 ------------------
.../inprocess/InProcessBundleFactory.java | 162 ---
.../inprocess/InProcessBundleOutputManager.java | 51 -
.../inprocess/InProcessEvaluationContext.java | 425 ------
.../inprocess/InProcessExecutionContext.java | 105 --
.../runners/inprocess/InProcessExecutor.java | 48 -
.../inprocess/InProcessPipelineOptions.java | 101 --
.../inprocess/InProcessPipelineRunner.java | 370 -----
.../runners/inprocess/InProcessRegistrar.java | 55 -
.../inprocess/InProcessSideInputContainer.java | 271 ----
.../inprocess/InProcessTimerInternals.java | 84 --
.../inprocess/InProcessTransformResult.java | 77 -
.../inprocess/KeyedPValueTrackingVisitor.java | 96 --
.../sdk/runners/inprocess/ModelEnforcement.java | 63 -
.../inprocess/ModelEnforcementFactory.java | 30 -
.../sdk/runners/inprocess/NanosOffsetClock.java | 59 -
.../inprocess/PTransformOverrideFactory.java | 33 -
.../inprocess/ParDoInProcessEvaluator.java | 173 ---
.../inprocess/ParDoMultiEvaluatorFactory.java | 63 -
.../inprocess/ParDoSingleEvaluatorFactory.java | 63 -
.../PassthroughTransformEvaluator.java | 49 -
.../runners/inprocess/ShardControlledWrite.java | 81 --
.../beam/sdk/runners/inprocess/StepAndKey.java | 71 -
.../runners/inprocess/StepTransformResult.java | 165 ---
.../inprocess/TextIOShardedWriteFactory.java | 78 -
.../runners/inprocess/TransformEvaluator.java | 46 -
.../inprocess/TransformEvaluatorFactory.java | 44 -
.../inprocess/TransformEvaluatorRegistry.java | 77 -
.../runners/inprocess/TransformExecutor.java | 176 ---
.../inprocess/TransformExecutorService.java | 35 -
.../inprocess/TransformExecutorServices.java | 154 --
.../UnboundedReadEvaluatorFactory.java | 177 ---
.../runners/inprocess/ViewEvaluatorFactory.java | 145 --
.../inprocess/WatermarkCallbackExecutor.java | 146 --
.../inprocess/WindowEvaluatorFactory.java | 131 --
.../AvroIOShardedWriteFactoryTest.java | 112 --
.../BoundedReadEvaluatorFactoryTest.java | 290 ----
.../runners/inprocess/CommittedResultTest.java | 77 -
.../ConsumerTrackingPipelineVisitorTest.java | 272 ----
.../EncodabilityEnforcementFactoryTest.java | 257 ----
.../inprocess/FlattenEvaluatorFactoryTest.java | 141 --
.../inprocess/ForwardingPTransformTest.java | 112 --
.../GroupByKeyEvaluatorFactoryTest.java | 183 ---
.../ImmutabilityCheckingBundleFactoryTest.java | 220 ---
.../ImmutabilityEnforcementFactoryTest.java | 128 --
.../inprocess/InMemoryWatermarkManagerTest.java | 1168 ---------------
.../inprocess/InProcessBundleFactoryTest.java | 223 ---
.../InProcessEvaluationContextTest.java | 526 -------
.../InProcessPipelineRegistrarTest.java | 74 -
.../inprocess/InProcessPipelineRunnerTest.java | 78 -
.../InProcessSideInputContainerTest.java | 496 -------
.../inprocess/InProcessTimerInternalsTest.java | 133 --
.../KeyedPValueTrackingVisitorTest.java | 192 ---
.../beam/sdk/runners/inprocess/MockClock.java | 62 -
.../ParDoMultiEvaluatorFactoryTest.java | 431 ------
.../ParDoSingleEvaluatorFactoryTest.java | 324 -----
.../TextIOShardedWriteFactoryTest.java | 112 --
.../TransformExecutorServicesTest.java | 136 --
.../inprocess/TransformExecutorTest.java | 538 -------
.../UnboundedReadEvaluatorFactoryTest.java | 334 -----
.../inprocess/ViewEvaluatorFactoryTest.java | 101 --
.../WatermarkCallbackExecutorTest.java | 128 --
.../inprocess/WindowEvaluatorFactoryTest.java | 222 ---
sdks/java/pom.xml | 1 -
168 files changed, 14692 insertions(+), 14290 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index f1d9e3b..8aad36e 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -25,6 +25,7 @@ matrix:
env: CUSTOM_JDK="openjdk7" MAVEN_OVERRIDE="-DforkCount=0"
before_install:
+ - echo "MAVEN_OPTS='-Xmx2048m -XX:MaxPermSize=512m'" > ~/.mavenrc
- if [ "$TRAVIS_OS_NAME" == "osx" ]; then export JAVA_HOME=$(/usr/libexec/java_home); fi
- if [ "$TRAVIS_OS_NAME" == "linux" ]; then jdk_switcher use "$CUSTOM_JDK"; fi
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml
new file mode 100644
index 0000000..12ba329
--- /dev/null
+++ b/runners/direct-java/pom.xml
@@ -0,0 +1,400 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <groupId>org.apache.beam</groupId>
+ <artifactId>runners-parent</artifactId>
+ <version>0.1.0-incubating-SNAPSHOT</version>
+ <relativePath>../pom.xml</relativePath>
+ </parent>
+
+ <artifactId>direct-runner</artifactId>
+
+ <name>Apache Beam :: Runners :: Direct</name>
+
+ <packaging>jar</packaging>
+
+ <build>
+ <resources>
+ <resource>
+ <directory>src/main/resources</directory>
+ <filtering>true</filtering>
+ </resource>
+ </resources>
+
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ </plugin>
+
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-dependency-plugin</artifactId>
+ <executions>
+ <execution>
+ <goals><goal>analyze-only</goal></goals>
+ <configuration>
+ <failOnWarning>true</failOnWarning>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-checkstyle-plugin</artifactId>
+ </plugin>
+
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-jar-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>default-jar</id>
+ <goals>
+ <goal>jar</goal>
+ </goals>
+ </execution>
+ <execution>
+ <id>default-test-jar</id>
+ <goals>
+ <goal>test-jar</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+
+ <!-- Source plugin for generating source and test-source JARs. -->
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-source-plugin</artifactId>
+ <version>2.4</version>
+ <executions>
+ <execution>
+ <id>attach-sources</id>
+ <phase>verify</phase>
+ <goals>
+ <goal>jar-no-fork</goal>
+ </goals>
+ </execution>
+ <execution>
+ <id>attach-test-sources</id>
+ <phase>verify</phase>
+ <goals>
+ <goal>test-jar-no-fork</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-surefire-plugin</artifactId>
+ <executions>
+ <!-- For now, disables integration tests from the SDK as the runner is not ready. -->
+ <execution>
+ <id>runnable-on-service-tests</id>
+ <configuration>
+ <skip>true</skip>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-javadoc-plugin</artifactId>
+ <configuration>
+ <windowtitle>Apache Beam Direct Runner ${project.version}</windowtitle>
+ <doctitle>Apache Beam Direct Runner, version ${project.version}</doctitle>
+
+ <subpackages>org.apache.beam.runners.direct</subpackages>
+ <use>false</use>
+ <quiet>true</quiet>
+ <bottom><![CDATA[<br>]]></bottom>
+
+ <offlineLinks>
+ <offlineLink>
+ <url>https://developers.google.com/api-client-library/java/google-api-java-client/reference/1.20.0/</url>
+ <location>${basedir}/../../sdks/java/javadoc/apiclient-docs</location>
+ </offlineLink>
+ <offlineLink>
+ <url>http://avro.apache.org/docs/1.7.7/api/java/</url>
+ <location>${basedir}/../../sdks/java/javadoc/avro-docs</location>
+ </offlineLink>
+ <offlineLink>
+ <url>https://developers.google.com/resources/api-libraries/documentation/bigquery/v2/java/latest/</url>
+ <location>${basedir}/../../sdks/java/javadoc/bq-docs</location>
+ </offlineLink>
+ <offlineLink>
+ <url>https://cloud.google.com/datastore/docs/apis/javadoc/</url>
+ <location>${basedir}/../../sdks/java/javadoc/datastore-docs</location>
+ </offlineLink>
+ <offlineLink>
+ <url>http://docs.guava-libraries.googlecode.com/git-history/release19/javadoc/</url>
+ <location>${basedir}/../../sdks/java/javadoc/guava-docs</location>
+ </offlineLink>
+ <offlineLink>
+ <url>http://hamcrest.org/JavaHamcrest/javadoc/1.3/</url>
+ <location>${basedir}/../../sdks/java/javadoc/hamcrest-docs</location>
+ </offlineLink>
+ <offlineLink>
+ <url>http://fasterxml.github.io/jackson-annotations/javadoc/2.7/</url>
+ <location>${basedir}/../../sdks/java/javadoc/jackson-annotations-docs</location>
+ </offlineLink>
+ <offlineLink>
+ <url>http://fasterxml.github.io/jackson-databind/javadoc/2.7/</url>
+ <location>${basedir}/../../sdks/java/javadoc/jackson-databind-docs</location>
+ </offlineLink>
+ <offlineLink>
+ <url>http://www.joda.org/joda-time/apidocs</url>
+ <location>${basedir}/../../sdks/java/javadoc/joda-docs</location>
+ </offlineLink>
+ <offlineLink>
+ <url>http://junit.sourceforge.net/javadoc/</url>
+ <location>${basedir}/../../sdks/java/javadoc/junit-docs</location>
+ </offlineLink>
+ <offlineLink>
+ <url>https://developers.google.com/api-client-library/java/google-oauth-java-client/reference/1.20.0/</url>
+ <location>${basedir}/../../sdks/java/javadoc/oauth-docs</location>
+ </offlineLink>
+ </offlineLinks>
+ </configuration>
+ <executions>
+ <execution>
+ <goals>
+ <goal>jar</goal>
+ </goals>
+ <phase>package</phase>
+ </execution>
+ </executions>
+ </plugin>
+
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-shade-plugin</artifactId>
+ <version>2.4.1</version>
+ <executions>
+ <!-- In the first phase, we pick dependencies and relocate them. -->
+ <execution>
+ <id>bundle-and-repackage</id>
+ <phase>package</phase>
+ <goals>
+ <goal>shade</goal>
+ </goals>
+ <configuration>
+ <shadeTestJar>true</shadeTestJar>
+ <artifactSet>
+ <includes>
+ <include>com.google.guava:guava</include>
+ </includes>
+ </artifactSet>
+ <filters>
+ <filter>
+ <artifact>*:*</artifact>
+ <excludes>
+ <exclude>META-INF/*.SF</exclude>
+ <exclude>META-INF/*.DSA</exclude>
+ <exclude>META-INF/*.RSA</exclude>
+ </excludes>
+ </filter>
+ </filters>
+ <relocations>
+ <!-- TODO: Once ready, change the following pattern to 'com'
+ only, exclude 'org.apache.beam.**', and remove
+ the second relocation. -->
+ <relocation>
+ <pattern>com.google.common</pattern>
+ <shadedPattern>org.apache.beam.runners.direct.repackaged.com.google.common</shadedPattern>
+ </relocation>
+ </relocations>
+ </configuration>
+ </execution>
+
+ <!-- In the second phase, we pick remaining dependencies and bundle
+ them without repackaging. -->
+ <execution>
+ <id>bundle-rest-without-repackaging</id>
+ <phase>package</phase>
+ <goals>
+ <goal>shade</goal>
+ </goals>
+ <configuration>
+ <shadeTestJar>true</shadeTestJar>
+ <finalName>${project.artifactId}-bundled-${project.version}</finalName>
+ <artifactSet>
+ <excludes>
+ <exclude>com.google.guava:guava</exclude>
+ </excludes>
+ </artifactSet>
+ <filters>
+ <filter>
+ <artifact>*:*</artifact>
+ <excludes>
+ <exclude>META-INF/*.SF</exclude>
+ <exclude>META-INF/*.DSA</exclude>
+ <exclude>META-INF/*.RSA</exclude>
+ </excludes>
+ </filter>
+ </filters>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+
+ <!-- Coverage analysis for unit tests. -->
+ <plugin>
+ <groupId>org.jacoco</groupId>
+ <artifactId>jacoco-maven-plugin</artifactId>
+ </plugin>
+ </plugins>
+ </build>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.beam</groupId>
+ <artifactId>java-sdk-all</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+
+ <dependency>
+ <groupId>com.google.http-client</groupId>
+ <artifactId>google-http-client</artifactId>
+ <version>${google-clients.version}</version>
+ <exclusions>
+ <!-- Exclude an old version of guava that is being pulled
+ in by a transitive dependency of google-api-client -->
+ <exclusion>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava-jdk5</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+
+ <dependency>
+ <groupId>com.google.http-client</groupId>
+ <artifactId>google-http-client-protobuf</artifactId>
+ <version>${google-clients.version}</version>
+ <exclusions>
+ <!-- Exclude an old version of guava that is being pulled
+ in by a transitive dependency of google-api-client -->
+ <exclusion>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava-jdk5</artifactId>
+ </exclusion>
+ </exclusions>
+ <scope>runtime</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava</artifactId>
+ <!-- If updating version, please update the javadoc offlineLink -->
+ <version>${guava.version}</version>
+ </dependency>
+
+ <dependency>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava-testlib</artifactId>
+ <version>${guava.version}</version>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>joda-time</groupId>
+ <artifactId>joda-time</artifactId>
+ <version>${joda.version}</version>
+ </dependency>
+
+ <dependency>
+ <groupId>com.google.code.findbugs</groupId>
+ <artifactId>jsr305</artifactId>
+ <version>${jsr305.version}</version>
+ </dependency>
+
+ <dependency>
+ <groupId>com.fasterxml.jackson.core</groupId>
+ <artifactId>jackson-annotations</artifactId>
+ <version>${jackson.version}</version>
+ </dependency>
+
+ <dependency>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ <version>${slf4j.version}</version>
+ </dependency>
+
+ <!-- build dependencies -->
+ <dependency>
+ <groupId>com.google.auto.service</groupId>
+ <artifactId>auto-service</artifactId>
+ <version>1.0-rc2</version>
+ <optional>true</optional>
+ </dependency>
+
+ <dependency>
+ <groupId>com.google.auto.value</groupId>
+ <artifactId>auto-value</artifactId>
+ <version>1.1</version>
+ <scope>provided</scope>
+ </dependency>
+
+ <!-- test dependencies -->
+ <dependency>
+ <groupId>org.hamcrest</groupId>
+ <artifactId>hamcrest-all</artifactId>
+ <version>${hamcrest.version}</version>
+ <scope>provided</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <version>${junit.version}</version>
+ <scope>provided</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-jdk14</artifactId>
+ <version>${slf4j.version}</version>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <version>1.10.19</version>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.beam</groupId>
+ <artifactId>java-sdk-all</artifactId>
+ <version>${project.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+</project>
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AbstractModelEnforcement.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AbstractModelEnforcement.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AbstractModelEnforcement.java
new file mode 100644
index 0000000..948beb6
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AbstractModelEnforcement.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.util.WindowedValue;
+
+/**
+ * An abstract {@link ModelEnforcement} that provides default empty implementations for each method.
+ */
+abstract class AbstractModelEnforcement<T> implements ModelEnforcement<T> {
+ @Override
+ public void beforeElement(WindowedValue<T> element) {}
+
+ @Override
+ public void afterElement(WindowedValue<T> element) {}
+
+ @Override
+ public void afterFinish(
+ CommittedBundle<T> input,
+ InProcessTransformResult result,
+ Iterable<? extends CommittedBundle<?>> outputs) {}
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactory.java
new file mode 100644
index 0000000..7422f27
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactory.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.sdk.io.AvroIO;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.IOChannelUtils;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
+class AvroIOShardedWriteFactory implements PTransformOverrideFactory {
+ @Override
+ public <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
+ PTransform<InputT, OutputT> transform) {
+ if (transform instanceof AvroIO.Write.Bound) {
+ @SuppressWarnings("unchecked")
+ AvroIO.Write.Bound<InputT> originalWrite = (AvroIO.Write.Bound<InputT>) transform;
+ if (originalWrite.getNumShards() > 1
+ || (originalWrite.getNumShards() == 1
+ && !"".equals(originalWrite.getShardNameTemplate()))) {
+ @SuppressWarnings("unchecked")
+ PTransform<InputT, OutputT> override =
+ (PTransform<InputT, OutputT>) new AvroIOShardedWrite<InputT>(originalWrite);
+ return override;
+ }
+ }
+ return transform;
+ }
+
+ private class AvroIOShardedWrite<InputT> extends ShardControlledWrite<InputT> {
+ private final AvroIO.Write.Bound<InputT> initial;
+
+ private AvroIOShardedWrite(AvroIO.Write.Bound<InputT> initial) {
+ this.initial = initial;
+ }
+
+ @Override
+ int getNumShards() {
+ return initial.getNumShards();
+ }
+
+ @Override
+ PTransform<? super PCollection<InputT>, PDone> getSingleShardTransform(int shardNum) {
+ String shardName =
+ IOChannelUtils.constructName(
+ initial.getFilenamePrefix(),
+ initial.getShardNameTemplate(),
+ initial.getFilenameSuffix(),
+ shardNum,
+ getNumShards());
+ return initial.withoutSharding().to(shardName).withSuffix("");
+ }
+
+ @Override
+ protected PTransform<PCollection<InputT>, PDone> delegate() {
+ return initial;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java
new file mode 100644
index 0000000..3822d3b
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java
@@ -0,0 +1,155 @@
+/*
+ * 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.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.BoundedSource.BoundedReader;
+import org.apache.beam.sdk.io.Read.Bounded;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+import java.io.IOException;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ConcurrentMap;
+
+import javax.annotation.Nullable;
+
+/**
+ * A {@link TransformEvaluatorFactory} that produces {@link TransformEvaluator TransformEvaluators}
+ * for the {@link Bounded Read.Bounded} primitive {@link PTransform}.
+ */
+final class BoundedReadEvaluatorFactory implements TransformEvaluatorFactory {
+ /*
+ * An evaluator for a Source is stateful, to ensure data is not read multiple times.
+ * Evaluators are cached here to ensure that the reader is not restarted if the evaluator is
+ * retriggered.
+ */
+ private final ConcurrentMap<EvaluatorKey, Queue<? extends BoundedReadEvaluator<?>>>
+ sourceEvaluators = new ConcurrentHashMap<>();
+
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ @Override
+ public <InputT> TransformEvaluator<InputT> forApplication(
+ AppliedPTransform<?, ?, ?> application,
+ @Nullable CommittedBundle<?> inputBundle,
+ InProcessEvaluationContext evaluationContext)
+ throws IOException {
+ return getTransformEvaluator((AppliedPTransform) application, evaluationContext);
+ }
+
+ private <OutputT> TransformEvaluator<?> getTransformEvaluator(
+ final AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform,
+ final InProcessEvaluationContext evaluationContext) {
+ BoundedReadEvaluator<?> evaluator =
+ getTransformEvaluatorQueue(transform, evaluationContext).poll();
+ if (evaluator == null) {
+ return EmptyTransformEvaluator.create(transform);
+ }
+ return evaluator;
+ }
+
+ /**
+ * Get the queue of {@link TransformEvaluator TransformEvaluators} that produce elements for the
+ * provided application of {@link Bounded Read.Bounded}, initializing it if required.
+ *
+ * <p>This method is thread-safe, and will only produce new evaluators if no other invocation has
+ * already done so.
+ */
+ @SuppressWarnings("unchecked")
+ private <OutputT> Queue<BoundedReadEvaluator<OutputT>> getTransformEvaluatorQueue(
+ final AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform,
+ final InProcessEvaluationContext evaluationContext) {
+ // Key by the application and the context the evaluation is occurring in (which call to
+ // Pipeline#run).
+ EvaluatorKey key = new EvaluatorKey(transform, evaluationContext);
+ Queue<BoundedReadEvaluator<OutputT>> evaluatorQueue =
+ (Queue<BoundedReadEvaluator<OutputT>>) sourceEvaluators.get(key);
+ if (evaluatorQueue == null) {
+ evaluatorQueue = new ConcurrentLinkedQueue<>();
+ if (sourceEvaluators.putIfAbsent(key, evaluatorQueue) == null) {
+ // If no queue existed in the evaluators, add an evaluator to initialize the evaluator
+ // factory for this transform
+ BoundedSource<OutputT> source = transform.getTransform().getSource();
+ BoundedReadEvaluator<OutputT> evaluator =
+ new BoundedReadEvaluator<OutputT>(transform, evaluationContext, source);
+ evaluatorQueue.offer(evaluator);
+ } else {
+ // otherwise return the existing Queue that arrived before us
+ evaluatorQueue = (Queue<BoundedReadEvaluator<OutputT>>) sourceEvaluators.get(key);
+ }
+ }
+ return evaluatorQueue;
+ }
+
+ /**
+ * A {@link BoundedReadEvaluator} produces elements from an underlying {@link BoundedSource},
+ * discarding all input elements. Within the call to {@link #finishBundle()}, the evaluator
+ * creates the {@link BoundedReader} and consumes all available input.
+ *
+ * <p>A {@link BoundedReadEvaluator} should only be created once per {@link BoundedSource}, and
+ * each evaluator should only be called once per evaluation of the pipeline. Otherwise, the source
+ * may produce duplicate elements.
+ */
+ private static class BoundedReadEvaluator<OutputT> implements TransformEvaluator<Object> {
+ private final AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform;
+ private final InProcessEvaluationContext evaluationContext;
+ /**
+ * The source being read from by this {@link BoundedReadEvaluator}. This may not be the same
+ * as the source derived from {@link #transform} due to splitting.
+ */
+ private BoundedSource<OutputT> source;
+
+ public BoundedReadEvaluator(
+ AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform,
+ InProcessEvaluationContext evaluationContext,
+ BoundedSource<OutputT> source) {
+ this.transform = transform;
+ this.evaluationContext = evaluationContext;
+ this.source = source;
+ }
+
+ @Override
+ public void processElement(WindowedValue<Object> element) {}
+
+ @Override
+ public InProcessTransformResult finishBundle() throws IOException {
+ try (final BoundedReader<OutputT> reader =
+ source.createReader(evaluationContext.getPipelineOptions());) {
+ boolean contentsRemaining = reader.start();
+ UncommittedBundle<OutputT> output =
+ evaluationContext.createRootBundle(transform.getOutput());
+ while (contentsRemaining) {
+ output.add(
+ WindowedValue.timestampedValueInGlobalWindow(
+ reader.getCurrent(), reader.getCurrentTimestamp()));
+ contentsRemaining = reader.advance();
+ }
+ return StepTransformResult.withHold(transform, BoundedWindow.TIMESTAMP_MAX_VALUE)
+ .addOutput(output)
+ .build();
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BundleFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BundleFactory.java
new file mode 100644
index 0000000..34529e7
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BundleFactory.java
@@ -0,0 +1,49 @@
+/*
+ * 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.direct.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * A factory that creates {@link UncommittedBundle UncommittedBundles}.
+ */
+public interface BundleFactory {
+ /**
+ * Create an {@link UncommittedBundle} from an empty input. Elements added to the bundle belong to
+ * the {@code output} {@link PCollection}.
+ */
+ public <T> UncommittedBundle<T> createRootBundle(PCollection<T> output);
+
+ /**
+ * Create an {@link UncommittedBundle} from the specified input. Elements added to the bundle
+ * belong to the {@code output} {@link PCollection}.
+ */
+ public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output);
+
+ /**
+ * Create an {@link UncommittedBundle} with the specified keys at the specified step. For use by
+ * {@link InProcessGroupByKeyOnly} {@link PTransform PTransforms}. Elements added to the bundle
+ * belong to the {@code output} {@link PCollection}.
+ */
+ public <T> UncommittedBundle<T> createKeyedBundle(
+ CommittedBundle<?> input, Object key, PCollection<T> output);
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CachedThreadPoolExecutorServiceFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CachedThreadPoolExecutorServiceFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CachedThreadPoolExecutorServiceFactory.java
new file mode 100644
index 0000000..5b8e5fc
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CachedThreadPoolExecutorServiceFactory.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * A {@link ExecutorServiceFactory} that produces cached thread pools via
+ * {@link Executors#newCachedThreadPool()}.
+ */
+class CachedThreadPoolExecutorServiceFactory
+ implements DefaultValueFactory<ExecutorServiceFactory>, ExecutorServiceFactory {
+ private static final CachedThreadPoolExecutorServiceFactory INSTANCE =
+ new CachedThreadPoolExecutorServiceFactory();
+
+ @Override
+ public ExecutorServiceFactory create(PipelineOptions options) {
+ return INSTANCE;
+ }
+
+ @Override
+ public ExecutorService create() {
+ return Executors.newCachedThreadPool();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/Clock.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/Clock.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/Clock.java
new file mode 100644
index 0000000..88f8aab
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/Clock.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.joda.time.Instant;
+
+/**
+ * Access to the current time.
+ */
+public interface Clock {
+ /**
+ * Returns the current time as an {@link Instant}.
+ */
+ Instant now();
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java
new file mode 100644
index 0000000..d15e012
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.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 org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+
+import com.google.auto.value.AutoValue;
+
+/**
+ * A {@link InProcessTransformResult} that has been committed.
+ */
+@AutoValue
+abstract class CommittedResult {
+ /**
+ * Returns the {@link AppliedPTransform} that produced this result.
+ */
+ public abstract AppliedPTransform<?, ?, ?> getTransform();
+
+ /**
+ * Returns the outputs produced by the transform.
+ */
+ public abstract Iterable<? extends CommittedBundle<?>> getOutputs();
+
+ public static CommittedResult create(
+ InProcessTransformResult original, Iterable<? extends CommittedBundle<?>> outputs) {
+ return new AutoValue_CommittedResult(original.getTransform(),
+ outputs);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CompletionCallback.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CompletionCallback.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CompletionCallback.java
new file mode 100644
index 0000000..7c2c068
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CompletionCallback.java
@@ -0,0 +1,36 @@
+/*
+ * 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.direct.InProcessPipelineRunner.CommittedBundle;
+
+/**
+ * A callback for completing a bundle of input.
+ */
+interface CompletionCallback {
+ /**
+ * Handle a successful result, returning the committed outputs of the result.
+ */
+ CommittedResult handleResult(
+ CommittedBundle<?> inputBundle, InProcessTransformResult result);
+
+ /**
+ * Handle a result that terminated abnormally due to the provided {@link Throwable}.
+ */
+ void handleThrowable(CommittedBundle<?> inputBundle, Throwable t);
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java
new file mode 100644
index 0000000..c790463
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java
@@ -0,0 +1,173 @@
+/*
+ * 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 com.google.common.base.Preconditions.checkState;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.PValue;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Tracks the {@link AppliedPTransform AppliedPTransforms} that consume each {@link PValue} in the
+ * {@link Pipeline}. This is used to schedule consuming {@link PTransform PTransforms} to consume
+ * input after the upstream transform has produced and committed output.
+ */
+public class ConsumerTrackingPipelineVisitor implements PipelineVisitor {
+ private Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers = new HashMap<>();
+ private Collection<AppliedPTransform<?, ?, ?>> rootTransforms = new ArrayList<>();
+ private Collection<PCollectionView<?>> views = new ArrayList<>();
+ private Map<AppliedPTransform<?, ?, ?>, String> stepNames = new HashMap<>();
+ private Set<PValue> toFinalize = new HashSet<>();
+ private int numTransforms = 0;
+ private boolean finalized = false;
+
+ @Override
+ public void enterCompositeTransform(TransformTreeNode node) {
+ checkState(
+ !finalized,
+ "Attempting to traverse a pipeline (node %s) with a %s "
+ + "which has already visited a Pipeline and is finalized",
+ node.getFullName(),
+ ConsumerTrackingPipelineVisitor.class.getSimpleName());
+ }
+
+ @Override
+ public void leaveCompositeTransform(TransformTreeNode node) {
+ checkState(
+ !finalized,
+ "Attempting to traverse a pipeline (node %s) with a %s which is already finalized",
+ node.getFullName(),
+ ConsumerTrackingPipelineVisitor.class.getSimpleName());
+ if (node.isRootNode()) {
+ finalized = true;
+ }
+ }
+
+ @Override
+ public void visitTransform(TransformTreeNode node) {
+ toFinalize.removeAll(node.getInput().expand());
+ AppliedPTransform<?, ?, ?> appliedTransform = getAppliedTransform(node);
+ stepNames.put(appliedTransform, genStepName());
+ if (node.getInput().expand().isEmpty()) {
+ rootTransforms.add(appliedTransform);
+ } else {
+ for (PValue value : node.getInput().expand()) {
+ valueToConsumers.get(value).add(appliedTransform);
+ }
+ }
+ }
+
+ private AppliedPTransform<?, ?, ?> getAppliedTransform(TransformTreeNode node) {
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ AppliedPTransform<?, ?, ?> application = AppliedPTransform.of(
+ node.getFullName(), node.getInput(), node.getOutput(), (PTransform) node.getTransform());
+ return application;
+ }
+
+ @Override
+ public void visitValue(PValue value, TransformTreeNode producer) {
+ toFinalize.add(value);
+ for (PValue expandedValue : value.expand()) {
+ valueToConsumers.put(expandedValue, new ArrayList<AppliedPTransform<?, ?, ?>>());
+ if (expandedValue instanceof PCollectionView) {
+ views.add((PCollectionView<?>) expandedValue);
+ }
+ expandedValue.recordAsOutput(getAppliedTransform(producer));
+ }
+ value.recordAsOutput(getAppliedTransform(producer));
+ }
+
+ private String genStepName() {
+ return String.format("s%s", numTransforms++);
+ }
+
+
+ /**
+ * Returns a mapping of each fully-expanded {@link PValue} to each
+ * {@link AppliedPTransform} that consumes it. For each AppliedPTransform in the collection
+ * returned from {@code getValueToCustomers().get(PValue)},
+ * {@code AppliedPTransform#getInput().expand()} will contain the argument {@link PValue}.
+ */
+ public Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> getValueToConsumers() {
+ checkState(
+ finalized,
+ "Can't call getValueToConsumers before the Pipeline has been completely traversed");
+
+ return valueToConsumers;
+ }
+
+ /**
+ * Returns the mapping for each {@link AppliedPTransform} in the {@link Pipeline} to a unique step
+ * name.
+ */
+ public Map<AppliedPTransform<?, ?, ?>, String> getStepNames() {
+ checkState(
+ finalized, "Can't call getStepNames before the Pipeline has been completely traversed");
+
+ return stepNames;
+ }
+
+ /**
+ * Returns the root transforms of the {@link Pipeline}. A root {@link AppliedPTransform} consumes
+ * a {@link PInput} where the {@link PInput#expand()} returns an empty collection.
+ */
+ public Collection<AppliedPTransform<?, ?, ?>> getRootTransforms() {
+ checkState(
+ finalized,
+ "Can't call getRootTransforms before the Pipeline has been completely traversed");
+
+ return rootTransforms;
+ }
+
+ /**
+ * Returns all of the {@link PCollectionView PCollectionViews} contained in the visited
+ * {@link Pipeline}.
+ */
+ public Collection<PCollectionView<?>> getViews() {
+ checkState(finalized, "Can't call getViews before the Pipeline has been completely traversed");
+
+ return views;
+ }
+
+ /**
+ * Returns all of the {@link PValue PValues} that have been produced but not consumed. These
+ * {@link PValue PValues} should be finalized by the {@link PipelineRunner} before the
+ * {@link Pipeline} is executed.
+ */
+ public Set<PValue> getUnfinalizedPValues() {
+ checkState(
+ finalized,
+ "Can't call getUnfinalizedPValues before the Pipeline has been completely traversed");
+
+ return toFinalize;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EmptyTransformEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EmptyTransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EmptyTransformEvaluator.java
new file mode 100644
index 0000000..5379038
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EmptyTransformEvaluator.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+
+/**
+ * A {@link TransformEvaluator} that ignores all input and produces no output. The result of
+ * invoking {@link #finishBundle()} on this evaluator is to return an
+ * {@link InProcessTransformResult} with no elements and a timestamp hold equal to
+ * {@link BoundedWindow#TIMESTAMP_MIN_VALUE}. Because the result contains no elements, this hold
+ * will not affect the watermark.
+ */
+final class EmptyTransformEvaluator<T> implements TransformEvaluator<T> {
+ public static <T> TransformEvaluator<T> create(AppliedPTransform<?, ?, ?> transform) {
+ return new EmptyTransformEvaluator<T>(transform);
+ }
+
+ private final AppliedPTransform<?, ?, ?> transform;
+
+ private EmptyTransformEvaluator(AppliedPTransform<?, ?, ?> transform) {
+ this.transform = transform;
+ }
+
+ @Override
+ public void processElement(WindowedValue<T> element) throws Exception {}
+
+ @Override
+ public InProcessTransformResult finishBundle() throws Exception {
+ return StepTransformResult.withHold(transform, BoundedWindow.TIMESTAMP_MIN_VALUE)
+ .build();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactory.java
new file mode 100644
index 0000000..ccf4c2b
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactory.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 static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * Enforces that all elements in a {@link PCollection} can be encoded using that
+ * {@link PCollection PCollection's} {@link Coder}.
+ */
+class EncodabilityEnforcementFactory implements ModelEnforcementFactory {
+ public static EncodabilityEnforcementFactory create() {
+ return new EncodabilityEnforcementFactory();
+ }
+
+ @Override
+ public <T> ModelEnforcement<T> forBundle(
+ CommittedBundle<T> input, AppliedPTransform<?, ?, ?> consumer) {
+ return new EncodabilityEnforcement<>(input);
+ }
+
+ private static class EncodabilityEnforcement<T> extends AbstractModelEnforcement<T> {
+ private Coder<T> coder;
+
+ public EncodabilityEnforcement(CommittedBundle<T> input) {
+ coder = input.getPCollection().getCoder();
+ }
+
+ @Override
+ public void beforeElement(WindowedValue<T> element) {
+ try {
+ T clone = CoderUtils.clone(coder, element.getValue());
+ if (coder.consistentWithEquals()) {
+ checkArgument(
+ coder.structuralValue(element.getValue()).equals(coder.structuralValue(clone)),
+ "Coder %s of class %s does not maintain structural value equality"
+ + " on input element %s",
+ coder,
+ coder.getClass().getSimpleName(),
+ element.getValue());
+ }
+ } catch (Exception e) {
+ throw UserCodeException.wrap(e);
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluatorKey.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluatorKey.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluatorKey.java
new file mode 100644
index 0000000..1c36751
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluatorKey.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+
+import java.util.Objects;
+
+/**
+ * A (Transform, Pipeline Execution) key for stateful evaluators.
+ *
+ * Source evaluators are stateful to ensure data is not read multiple times. Evaluators are cached
+ * to ensure that the reader is not restarted if the evaluator is retriggered. An
+ * {@link EvaluatorKey} is used to ensure that multiple Pipelines can be executed without sharing
+ * the same evaluators.
+ */
+final class EvaluatorKey {
+ private final AppliedPTransform<?, ?, ?> transform;
+ private final InProcessEvaluationContext context;
+
+ public EvaluatorKey(AppliedPTransform<?, ?, ?> transform, InProcessEvaluationContext context) {
+ this.transform = transform;
+ this.context = context;
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(transform, context);
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (other == null || !(other instanceof EvaluatorKey)) {
+ return false;
+ }
+ EvaluatorKey that = (EvaluatorKey) other;
+ return Objects.equals(this.transform, that.transform)
+ && Objects.equals(this.context, that.context);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceFactory.java
new file mode 100644
index 0000000..91dc258
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceFactory.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import java.util.concurrent.ExecutorService;
+
+/**
+ * A factory that creates {@link ExecutorService ExecutorServices}.
+ * {@link ExecutorService ExecutorServices} created by this factory should be independent of one
+ * another (e.g., if any executor is shut down the remaining executors should continue to process
+ * work).
+ */
+public interface ExecutorServiceFactory {
+ /**
+ * Create a new {@link ExecutorService}.
+ */
+ ExecutorService create();
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java
new file mode 100644
index 0000000..18af363
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java
@@ -0,0 +1,478 @@
+/*
+ * 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.direct.InMemoryWatermarkManager.FiredTimers;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.KeyedWorkItem;
+import org.apache.beam.sdk.util.KeyedWorkItems;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PValue;
+
+import com.google.common.base.MoreObjects;
+import com.google.common.base.Optional;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.collect.ImmutableList;
+
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+
+import javax.annotation.Nullable;
+
+/**
+ * An {@link InProcessExecutor} that uses an underlying {@link ExecutorService} and
+ * {@link InProcessEvaluationContext} to execute a {@link Pipeline}.
+ */
+final class ExecutorServiceParallelExecutor implements InProcessExecutor {
+ private static final Logger LOG = LoggerFactory.getLogger(ExecutorServiceParallelExecutor.class);
+
+ private final ExecutorService executorService;
+
+ private final Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers;
+ private final Set<PValue> keyedPValues;
+ private final TransformEvaluatorRegistry registry;
+ @SuppressWarnings("rawtypes")
+ private final Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>>
+ transformEnforcements;
+
+ private final InProcessEvaluationContext evaluationContext;
+
+ private final LoadingCache<StepAndKey, TransformExecutorService> executorServices;
+ private final ConcurrentMap<TransformExecutor<?>, Boolean> scheduledExecutors;
+
+ private final Queue<ExecutorUpdate> allUpdates;
+ private final BlockingQueue<VisibleExecutorUpdate> visibleUpdates;
+
+ private final TransformExecutorService parallelExecutorService;
+ private final CompletionCallback defaultCompletionCallback;
+
+ private Collection<AppliedPTransform<?, ?, ?>> rootNodes;
+
+ public static ExecutorServiceParallelExecutor create(
+ ExecutorService executorService,
+ Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
+ Set<PValue> keyedPValues,
+ TransformEvaluatorRegistry registry,
+ @SuppressWarnings("rawtypes")
+ Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>> transformEnforcements,
+ InProcessEvaluationContext context) {
+ return new ExecutorServiceParallelExecutor(
+ executorService, valueToConsumers, keyedPValues, registry, transformEnforcements, context);
+ }
+
+ private ExecutorServiceParallelExecutor(
+ ExecutorService executorService,
+ Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
+ Set<PValue> keyedPValues,
+ TransformEvaluatorRegistry registry,
+ @SuppressWarnings("rawtypes")
+ Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>> transformEnforcements,
+ InProcessEvaluationContext context) {
+ this.executorService = executorService;
+ this.valueToConsumers = valueToConsumers;
+ this.keyedPValues = keyedPValues;
+ this.registry = registry;
+ this.transformEnforcements = transformEnforcements;
+ this.evaluationContext = context;
+
+ scheduledExecutors = new ConcurrentHashMap<>();
+ // Weak Values allows TransformExecutorServices that are no longer in use to be reclaimed.
+ // Executing TransformExecutorServices have a strong reference to their TransformExecutorService
+ // which stops the TransformExecutorServices from being prematurely garbage collected
+ executorServices =
+ CacheBuilder.newBuilder().weakValues().build(serialTransformExecutorServiceCacheLoader());
+
+ this.allUpdates = new ConcurrentLinkedQueue<>();
+ this.visibleUpdates = new ArrayBlockingQueue<>(20);
+
+ parallelExecutorService =
+ TransformExecutorServices.parallel(executorService, scheduledExecutors);
+ defaultCompletionCallback = new DefaultCompletionCallback();
+ }
+
+ private CacheLoader<StepAndKey, TransformExecutorService>
+ serialTransformExecutorServiceCacheLoader() {
+ return new CacheLoader<StepAndKey, TransformExecutorService>() {
+ @Override
+ public TransformExecutorService load(StepAndKey stepAndKey) throws Exception {
+ return TransformExecutorServices.serial(executorService, scheduledExecutors);
+ }
+ };
+ }
+
+ @Override
+ public void start(Collection<AppliedPTransform<?, ?, ?>> roots) {
+ rootNodes = ImmutableList.copyOf(roots);
+ Runnable monitorRunnable = new MonitorRunnable();
+ executorService.submit(monitorRunnable);
+ }
+
+ @SuppressWarnings("unchecked")
+ public void scheduleConsumption(
+ AppliedPTransform<?, ?, ?> consumer,
+ @Nullable CommittedBundle<?> bundle,
+ CompletionCallback onComplete) {
+ evaluateBundle(consumer, bundle, onComplete);
+ }
+
+ private <T> void evaluateBundle(
+ final AppliedPTransform<?, ?, ?> transform,
+ @Nullable final CommittedBundle<T> bundle,
+ final CompletionCallback onComplete) {
+ TransformExecutorService transformExecutor;
+
+ if (bundle != null && isKeyed(bundle.getPCollection())) {
+ final StepAndKey stepAndKey =
+ StepAndKey.of(transform, bundle == null ? null : bundle.getKey());
+ // This executor will remain reachable until it has executed all scheduled transforms.
+ // The TransformExecutors keep a strong reference to the Executor, the ExecutorService keeps
+ // a reference to the scheduled TransformExecutor callable. Follow-up TransformExecutors
+ // (scheduled due to the completion of another TransformExecutor) are provided to the
+ // ExecutorService before the Earlier TransformExecutor callable completes.
+ transformExecutor = executorServices.getUnchecked(stepAndKey);
+ } else {
+ transformExecutor = parallelExecutorService;
+ }
+
+ Collection<ModelEnforcementFactory> enforcements =
+ MoreObjects.firstNonNull(
+ transformEnforcements.get(transform.getTransform().getClass()),
+ Collections.<ModelEnforcementFactory>emptyList());
+
+ TransformExecutor<T> callable =
+ TransformExecutor.create(
+ registry,
+ enforcements,
+ evaluationContext,
+ bundle,
+ transform,
+ onComplete,
+ transformExecutor);
+ transformExecutor.schedule(callable);
+ }
+
+ private boolean isKeyed(PValue pvalue) {
+ return keyedPValues.contains(pvalue);
+ }
+
+ private void scheduleConsumers(CommittedBundle<?> bundle) {
+ for (AppliedPTransform<?, ?, ?> consumer : valueToConsumers.get(bundle.getPCollection())) {
+ scheduleConsumption(consumer, bundle, defaultCompletionCallback);
+ }
+ }
+
+ @Override
+ public void awaitCompletion() throws Throwable {
+ VisibleExecutorUpdate update;
+ do {
+ update = visibleUpdates.take();
+ if (update.throwable.isPresent()) {
+ throw update.throwable.get();
+ }
+ } while (!update.isDone());
+ executorService.shutdown();
+ }
+
+ /**
+ * The default {@link CompletionCallback}. The default completion callback is used to complete
+ * transform evaluations that are triggered due to the arrival of elements from an upstream
+ * transform, or for a source transform.
+ */
+ private class DefaultCompletionCallback implements CompletionCallback {
+ @Override
+ public CommittedResult handleResult(
+ CommittedBundle<?> inputBundle, InProcessTransformResult result) {
+ CommittedResult committedResult =
+ evaluationContext.handleResult(inputBundle, Collections.<TimerData>emptyList(), result);
+ for (CommittedBundle<?> outputBundle : committedResult.getOutputs()) {
+ allUpdates.offer(ExecutorUpdate.fromBundle(outputBundle));
+ }
+ return committedResult;
+ }
+
+ @Override
+ public void handleThrowable(CommittedBundle<?> inputBundle, Throwable t) {
+ allUpdates.offer(ExecutorUpdate.fromThrowable(t));
+ }
+ }
+
+ /**
+ * A {@link CompletionCallback} where the completed bundle was produced to deliver some collection
+ * of {@link TimerData timers}. When the evaluator completes successfully, reports all of the
+ * timers used to create the input to the {@link InProcessEvaluationContext evaluation context}
+ * as part of the result.
+ */
+ private class TimerCompletionCallback implements CompletionCallback {
+ private final Iterable<TimerData> timers;
+
+ private TimerCompletionCallback(Iterable<TimerData> timers) {
+ this.timers = timers;
+ }
+
+ @Override
+ public CommittedResult handleResult(
+ CommittedBundle<?> inputBundle, InProcessTransformResult result) {
+ CommittedResult committedResult =
+ evaluationContext.handleResult(inputBundle, timers, result);
+ for (CommittedBundle<?> outputBundle : committedResult.getOutputs()) {
+ allUpdates.offer(ExecutorUpdate.fromBundle(outputBundle));
+ }
+ return committedResult;
+ }
+
+ @Override
+ public void handleThrowable(CommittedBundle<?> inputBundle, Throwable t) {
+ allUpdates.offer(ExecutorUpdate.fromThrowable(t));
+ }
+ }
+
+ /**
+ * An internal status update on the state of the executor.
+ *
+ * Used to signal when the executor should be shut down (due to an exception).
+ */
+ private static class ExecutorUpdate {
+ private final Optional<? extends CommittedBundle<?>> bundle;
+ private final Optional<? extends Throwable> throwable;
+
+ public static ExecutorUpdate fromBundle(CommittedBundle<?> bundle) {
+ return new ExecutorUpdate(bundle, null);
+ }
+
+ public static ExecutorUpdate fromThrowable(Throwable t) {
+ return new ExecutorUpdate(null, t);
+ }
+
+ private ExecutorUpdate(CommittedBundle<?> producedBundle, Throwable throwable) {
+ this.bundle = Optional.fromNullable(producedBundle);
+ this.throwable = Optional.fromNullable(throwable);
+ }
+
+ public Optional<? extends CommittedBundle<?>> getBundle() {
+ return bundle;
+ }
+
+ public Optional<? extends Throwable> getException() {
+ return throwable;
+ }
+
+ @Override
+ public String toString() {
+ return MoreObjects.toStringHelper(ExecutorUpdate.class)
+ .add("bundle", bundle)
+ .add("exception", throwable)
+ .toString();
+ }
+ }
+
+ /**
+ * An update of interest to the user. Used in {@link #awaitCompletion} to decide whether to
+ * return normally or throw an exception.
+ */
+ private static class VisibleExecutorUpdate {
+ private final Optional<? extends Throwable> throwable;
+ private final boolean done;
+
+ public static VisibleExecutorUpdate fromThrowable(Throwable e) {
+ return new VisibleExecutorUpdate(false, e);
+ }
+
+ public static VisibleExecutorUpdate finished() {
+ return new VisibleExecutorUpdate(true, null);
+ }
+
+ private VisibleExecutorUpdate(boolean done, @Nullable Throwable exception) {
+ this.throwable = Optional.fromNullable(exception);
+ this.done = done;
+ }
+
+ public boolean isDone() {
+ return done;
+ }
+ }
+
+ private class MonitorRunnable implements Runnable {
+ private final String runnableName =
+ String.format(
+ "%s$%s-monitor",
+ evaluationContext.getPipelineOptions().getAppName(),
+ ExecutorServiceParallelExecutor.class.getSimpleName());
+
+ @Override
+ public void run() {
+ String oldName = Thread.currentThread().getName();
+ Thread.currentThread().setName(runnableName);
+ try {
+ ExecutorUpdate update = allUpdates.poll();
+ // pull all of the pending work off of the queue
+ while (update != null) {
+ LOG.debug("Executor Update: {}", update);
+ if (update.getBundle().isPresent()) {
+ scheduleConsumers(update.getBundle().get());
+ } else if (update.getException().isPresent()) {
+ visibleUpdates.offer(VisibleExecutorUpdate.fromThrowable(update.getException().get()));
+ }
+ update = allUpdates.poll();
+ }
+ boolean timersFired = fireTimers();
+ addWorkIfNecessary(timersFired);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ LOG.error("Monitor died due to being interrupted");
+ while (!visibleUpdates.offer(VisibleExecutorUpdate.fromThrowable(e))) {
+ visibleUpdates.poll();
+ }
+ } catch (Throwable t) {
+ LOG.error("Monitor thread died due to throwable", t);
+ while (!visibleUpdates.offer(VisibleExecutorUpdate.fromThrowable(t))) {
+ visibleUpdates.poll();
+ }
+ } finally {
+ if (!shouldShutdown()) {
+ // The monitor thread should always be scheduled; but we only need to be scheduled once
+ executorService.submit(this);
+ }
+ Thread.currentThread().setName(oldName);
+ }
+ }
+
+ /**
+ * Fires any available timers. Returns true if at least one timer was fired.
+ */
+ private boolean fireTimers() throws Exception {
+ try {
+ boolean firedTimers = false;
+ for (Map.Entry<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> transformTimers :
+ evaluationContext.extractFiredTimers().entrySet()) {
+ AppliedPTransform<?, ?, ?> transform = transformTimers.getKey();
+ for (Map.Entry<Object, FiredTimers> keyTimers : transformTimers.getValue().entrySet()) {
+ for (TimeDomain domain : TimeDomain.values()) {
+ Collection<TimerData> delivery = keyTimers.getValue().getTimers(domain);
+ if (delivery.isEmpty()) {
+ continue;
+ }
+ KeyedWorkItem<Object, Object> work =
+ KeyedWorkItems.timersWorkItem(keyTimers.getKey(), delivery);
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ CommittedBundle<?> bundle =
+ evaluationContext
+ .createKeyedBundle(
+ null, keyTimers.getKey(), (PCollection) transform.getInput())
+ .add(WindowedValue.valueInEmptyWindows(work))
+ .commit(Instant.now());
+ scheduleConsumption(transform, bundle, new TimerCompletionCallback(delivery));
+ firedTimers = true;
+ }
+ }
+ }
+ return firedTimers;
+ } catch (Exception e) {
+ LOG.error("Internal Error while delivering timers", e);
+ throw e;
+ }
+ }
+
+ private boolean shouldShutdown() {
+ if (evaluationContext.isDone()) {
+ LOG.debug("Pipeline is finished. Shutting down. {}");
+ while (!visibleUpdates.offer(VisibleExecutorUpdate.finished())) {
+ visibleUpdates.poll();
+ }
+ executorService.shutdown();
+ return true;
+ }
+ return false;
+ }
+
+ /**
+ * If all active {@link TransformExecutor TransformExecutors} are in a blocked state,
+ * add more work from root nodes that may have additional work. This ensures that if a pipeline
+ * has elements available from the root nodes it will add those elements when necessary.
+ */
+ private void addWorkIfNecessary(boolean firedTimers) {
+ // If any timers have fired, they will add more work; We don't need to add more
+ if (firedTimers) {
+ return;
+ }
+ for (TransformExecutor<?> executor : scheduledExecutors.keySet()) {
+ if (!isExecutorBlocked(executor)) {
+ // We have at least one executor that can proceed without adding additional work
+ return;
+ }
+ }
+ // All current TransformExecutors are blocked; add more work from the roots.
+ for (AppliedPTransform<?, ?, ?> root : rootNodes) {
+ if (!evaluationContext.isDone(root)) {
+ scheduleConsumption(root, null, defaultCompletionCallback);
+ }
+ }
+ }
+
+ /**
+ * Return true if the provided executor might make more progress if no action is taken.
+ *
+ * <p>May return false even if all executor threads are currently blocked or cleaning up, as
+ * these can cause more work to be scheduled. If this does not occur, after these calls
+ * terminate, future calls will return true if all executors are waiting.
+ */
+ private boolean isExecutorBlocked(TransformExecutor<?> executor) {
+ Thread thread = executor.getThread();
+ if (thread == null) {
+ return false;
+ }
+ switch (thread.getState()) {
+ case TERMINATED:
+ throw new IllegalStateException(String.format(
+ "Unexpectedly encountered a Terminated TransformExecutor %s", executor));
+ case WAITING:
+ case TIMED_WAITING:
+ // The thread is waiting for some external input. Adding more work may cause the thread
+ // to stop waiting (e.g. the thread is waiting on an unbounded side input)
+ return true;
+ case BLOCKED:
+ // The executor is blocked on acquisition of a java monitor. This usually means it is
+ // making a call to the EvaluationContext, but not a model-blocking call - and will
+ // eventually complete, at which point we may reevaluate.
+ default:
+ // NEW and RUNNABLE threads can make progress
+ return false;
+ }
+ }
+ }
+}
[12/17] incubator-beam git commit: Move InProcessRunner to its own
module
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java
new file mode 100644
index 0000000..491363a
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java
@@ -0,0 +1,290 @@
+/*
+ * 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.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.emptyIterable;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.coders.BigEndianLongCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.BoundedSource.BoundedReader;
+import org.apache.beam.sdk.io.CountingSource;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.Read.Bounded;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.common.collect.ImmutableList;
+
+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;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+/**
+ * Tests for {@link BoundedReadEvaluatorFactory}.
+ */
+@RunWith(JUnit4.class)
+public class BoundedReadEvaluatorFactoryTest {
+ private BoundedSource<Long> source;
+ private PCollection<Long> longs;
+ private TransformEvaluatorFactory factory;
+ @Mock private InProcessEvaluationContext context;
+ private BundleFactory bundleFactory;
+
+ @Before
+ public void setup() {
+ MockitoAnnotations.initMocks(this);
+ source = CountingSource.upTo(10L);
+ TestPipeline p = TestPipeline.create();
+ longs = p.apply(Read.from(source));
+
+ factory = new BoundedReadEvaluatorFactory();
+ bundleFactory = InProcessBundleFactory.create();
+ }
+
+ @Test
+ public void boundedSourceInMemoryTransformEvaluatorProducesElements() throws Exception {
+ UncommittedBundle<Long> output = bundleFactory.createRootBundle(longs);
+ when(context.createRootBundle(longs)).thenReturn(output);
+
+ TransformEvaluator<?> evaluator =
+ factory.forApplication(longs.getProducingTransformInternal(), null, context);
+ InProcessTransformResult result = evaluator.finishBundle();
+ assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
+ assertThat(
+ output.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements(),
+ containsInAnyOrder(
+ gw(1L), gw(2L), gw(4L), gw(8L), gw(9L), gw(7L), gw(6L), gw(5L), gw(3L), gw(0L)));
+ }
+
+ /**
+ * Demonstrate that acquiring multiple {@link TransformEvaluator TransformEvaluators} for the same
+ * {@link Bounded Read.Bounded} application with the same evaluation context only produces the
+ * elements once.
+ */
+ @Test
+ public void boundedSourceInMemoryTransformEvaluatorAfterFinishIsEmpty() throws Exception {
+ UncommittedBundle<Long> output = bundleFactory.createRootBundle(longs);
+ when(context.createRootBundle(longs)).thenReturn(output);
+
+ TransformEvaluator<?> evaluator =
+ factory.forApplication(longs.getProducingTransformInternal(), null, context);
+ InProcessTransformResult result = evaluator.finishBundle();
+ assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
+ Iterable<? extends WindowedValue<Long>> outputElements =
+ output.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements();
+ assertThat(
+ outputElements,
+ containsInAnyOrder(
+ gw(1L), gw(2L), gw(4L), gw(8L), gw(9L), gw(7L), gw(6L), gw(5L), gw(3L), gw(0L)));
+
+ UncommittedBundle<Long> secondOutput = bundleFactory.createRootBundle(longs);
+ when(context.createRootBundle(longs)).thenReturn(secondOutput);
+ TransformEvaluator<?> secondEvaluator =
+ factory.forApplication(longs.getProducingTransformInternal(), null, context);
+ InProcessTransformResult secondResult = secondEvaluator.finishBundle();
+ assertThat(secondResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
+ assertThat(secondResult.getOutputBundles(), emptyIterable());
+ assertThat(
+ secondOutput.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements(), emptyIterable());
+ assertThat(
+ outputElements,
+ containsInAnyOrder(
+ gw(1L), gw(2L), gw(4L), gw(8L), gw(9L), gw(7L), gw(6L), gw(5L), gw(3L), gw(0L)));
+ }
+
+ /**
+ * Demonstrates that acquiring multiple evaluators from the factory are independent, but
+ * the elements in the source are only produced once.
+ */
+ @Test
+ public void boundedSourceEvaluatorSimultaneousEvaluations() throws Exception {
+ UncommittedBundle<Long> output = bundleFactory.createRootBundle(longs);
+ UncommittedBundle<Long> secondOutput = bundleFactory.createRootBundle(longs);
+ when(context.createRootBundle(longs)).thenReturn(output).thenReturn(secondOutput);
+
+ // create both evaluators before finishing either.
+ TransformEvaluator<?> evaluator =
+ factory.forApplication(longs.getProducingTransformInternal(), null, context);
+ TransformEvaluator<?> secondEvaluator =
+ factory.forApplication(longs.getProducingTransformInternal(), null, context);
+
+ InProcessTransformResult secondResult = secondEvaluator.finishBundle();
+
+ InProcessTransformResult result = evaluator.finishBundle();
+ assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
+ Iterable<? extends WindowedValue<Long>> outputElements =
+ output.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements();
+
+ assertThat(
+ outputElements,
+ containsInAnyOrder(
+ gw(1L), gw(2L), gw(4L), gw(8L), gw(9L), gw(7L), gw(6L), gw(5L), gw(3L), gw(0L)));
+ assertThat(secondResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
+ assertThat(secondResult.getOutputBundles(), emptyIterable());
+ assertThat(
+ secondOutput.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements(), emptyIterable());
+ assertThat(
+ outputElements,
+ containsInAnyOrder(
+ gw(1L), gw(2L), gw(4L), gw(8L), gw(9L), gw(7L), gw(6L), gw(5L), gw(3L), gw(0L)));
+ }
+
+ @Test
+ public void boundedSourceEvaluatorClosesReader() throws Exception {
+ TestSource<Long> source = new TestSource<>(BigEndianLongCoder.of(), 1L, 2L, 3L);
+
+ TestPipeline p = TestPipeline.create();
+ PCollection<Long> pcollection = p.apply(Read.from(source));
+ AppliedPTransform<?, ?, ?> sourceTransform = pcollection.getProducingTransformInternal();
+
+ UncommittedBundle<Long> output = bundleFactory.createRootBundle(pcollection);
+ when(context.createRootBundle(pcollection)).thenReturn(output);
+
+ TransformEvaluator<?> evaluator = factory.forApplication(sourceTransform, null, context);
+ evaluator.finishBundle();
+ CommittedBundle<Long> committed = output.commit(Instant.now());
+ assertThat(committed.getElements(), containsInAnyOrder(gw(2L), gw(3L), gw(1L)));
+ assertThat(TestSource.readerClosed, is(true));
+ }
+
+ @Test
+ public void boundedSourceEvaluatorNoElementsClosesReader() throws Exception {
+ TestSource<Long> source = new TestSource<>(BigEndianLongCoder.of());
+
+ TestPipeline p = TestPipeline.create();
+ PCollection<Long> pcollection = p.apply(Read.from(source));
+ AppliedPTransform<?, ?, ?> sourceTransform = pcollection.getProducingTransformInternal();
+
+ UncommittedBundle<Long> output = bundleFactory.createRootBundle(pcollection);
+ when(context.createRootBundle(pcollection)).thenReturn(output);
+
+ TransformEvaluator<?> evaluator = factory.forApplication(sourceTransform, null, context);
+ evaluator.finishBundle();
+ CommittedBundle<Long> committed = output.commit(Instant.now());
+ assertThat(committed.getElements(), emptyIterable());
+ assertThat(TestSource.readerClosed, is(true));
+ }
+
+ private static class TestSource<T> extends BoundedSource<T> {
+ private static boolean readerClosed;
+ private final Coder<T> coder;
+ private final T[] elems;
+
+ public TestSource(Coder<T> coder, T... elems) {
+ this.elems = elems;
+ this.coder = coder;
+ readerClosed = false;
+ }
+
+ @Override
+ public List<? extends BoundedSource<T>> splitIntoBundles(
+ long desiredBundleSizeBytes, PipelineOptions options) throws Exception {
+ return ImmutableList.of(this);
+ }
+
+ @Override
+ public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
+ return 0;
+ }
+
+ @Override
+ public boolean producesSortedKeys(PipelineOptions options) throws Exception {
+ return false;
+ }
+
+ @Override
+ public BoundedSource.BoundedReader<T> createReader(PipelineOptions options) throws IOException {
+ return new TestReader<>(this, elems);
+ }
+
+ @Override
+ public void validate() {
+ }
+
+ @Override
+ public Coder<T> getDefaultOutputCoder() {
+ return coder;
+ }
+ }
+
+ private static class TestReader<T> extends BoundedReader<T> {
+ private final BoundedSource<T> source;
+ private final List<T> elems;
+ private int index;
+
+ public TestReader(BoundedSource<T> source, T... elems) {
+ this.source = source;
+ this.elems = Arrays.asList(elems);
+ this.index = -1;
+ }
+
+ @Override
+ public BoundedSource<T> getCurrentSource() {
+ return source;
+ }
+
+ @Override
+ public boolean start() throws IOException {
+ return advance();
+ }
+
+ @Override
+ public boolean advance() throws IOException {
+ if (elems.size() > index + 1) {
+ index++;
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public T getCurrent() throws NoSuchElementException {
+ return elems.get(index);
+ }
+
+ @Override
+ public void close() throws IOException {
+ TestSource.readerClosed = true;
+ }
+ }
+
+ private static WindowedValue<Long> gw(Long elem) {
+ return WindowedValue.valueInGlobalWindow(elem);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java
new file mode 100644
index 0000000..b30e005
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.direct;
+
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+
+import com.google.common.collect.ImmutableList;
+
+import org.hamcrest.Matchers;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Tests for {@link CommittedResult}.
+ */
+@RunWith(JUnit4.class)
+public class CommittedResultTest implements Serializable {
+ private transient TestPipeline p = TestPipeline.create();
+ private transient AppliedPTransform<?, ?, ?> transform =
+ AppliedPTransform.of("foo", p.begin(), PDone.in(p), new PTransform<PBegin, PDone>() {
+ });
+ private transient BundleFactory bundleFactory = InProcessBundleFactory.create();
+
+ @Test
+ public void getTransformExtractsFromResult() {
+ CommittedResult result =
+ CommittedResult.create(StepTransformResult.withoutHold(transform).build(),
+ Collections.<InProcessPipelineRunner.CommittedBundle<?>>emptyList());
+
+ assertThat(result.getTransform(), Matchers.<AppliedPTransform<?, ?, ?>>equalTo(transform));
+ }
+
+ @Test
+ public void getOutputsEqualInput() {
+ List<? extends InProcessPipelineRunner.CommittedBundle<?>> outputs =
+ ImmutableList.of(bundleFactory.createRootBundle(PCollection.createPrimitiveOutputInternal(p,
+ WindowingStrategy.globalDefault(),
+ PCollection.IsBounded.BOUNDED)).commit(Instant.now()),
+ bundleFactory.createRootBundle(PCollection.createPrimitiveOutputInternal(p,
+ WindowingStrategy.globalDefault(),
+ PCollection.IsBounded.UNBOUNDED)).commit(Instant.now()));
+ CommittedResult result =
+ CommittedResult.create(StepTransformResult.withoutHold(transform).build(), outputs);
+
+ assertThat(result.getOutputs(), Matchers.containsInAnyOrder(outputs.toArray()));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java
new file mode 100644
index 0000000..353eef6
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.emptyIterable;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.sdk.io.CountingInput;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+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.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.PValue;
+
+import org.hamcrest.Matchers;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.Serializable;
+import java.util.List;
+
+/**
+ * Tests for {@link ConsumerTrackingPipelineVisitor}.
+ */
+@RunWith(JUnit4.class)
+public class ConsumerTrackingPipelineVisitorTest implements Serializable {
+ @Rule public transient ExpectedException thrown = ExpectedException.none();
+
+ private transient TestPipeline p = TestPipeline.create();
+ private transient ConsumerTrackingPipelineVisitor visitor = new ConsumerTrackingPipelineVisitor();
+
+ @Test
+ public void getViewsReturnsViews() {
+ PCollectionView<List<String>> listView =
+ p.apply("listCreate", Create.of("foo", "bar"))
+ .apply(
+ ParDo.of(
+ new DoFn<String, String>() {
+ @Override
+ public void processElement(DoFn<String, String>.ProcessContext c)
+ throws Exception {
+ c.output(Integer.toString(c.element().length()));
+ }
+ }))
+ .apply(View.<String>asList());
+ PCollectionView<Object> singletonView =
+ p.apply("singletonCreate", Create.<Object>of(1, 2, 3)).apply(View.<Object>asSingleton());
+ p.traverseTopologically(visitor);
+ assertThat(
+ visitor.getViews(),
+ Matchers.<PCollectionView<?>>containsInAnyOrder(listView, singletonView));
+ }
+
+ @Test
+ public void getRootTransformsContainsPBegins() {
+ PCollection<String> created = p.apply(Create.of("foo", "bar"));
+ PCollection<Long> counted = p.apply(CountingInput.upTo(1234L));
+ PCollection<Long> unCounted = p.apply(CountingInput.unbounded());
+ p.traverseTopologically(visitor);
+ assertThat(
+ visitor.getRootTransforms(),
+ Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
+ created.getProducingTransformInternal(),
+ counted.getProducingTransformInternal(),
+ unCounted.getProducingTransformInternal()));
+ }
+
+ @Test
+ public void getRootTransformsContainsEmptyFlatten() {
+ PCollection<String> empty =
+ PCollectionList.<String>empty(p).apply(Flatten.<String>pCollections());
+ p.traverseTopologically(visitor);
+ assertThat(
+ visitor.getRootTransforms(),
+ Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
+ empty.getProducingTransformInternal()));
+ }
+
+ @Test
+ public void getValueToConsumersSucceeds() {
+ PCollection<String> created = p.apply(Create.of("1", "2", "3"));
+ PCollection<String> transformed =
+ created.apply(
+ ParDo.of(
+ new DoFn<String, String>() {
+ @Override
+ public void processElement(DoFn<String, String>.ProcessContext c)
+ throws Exception {
+ c.output(Integer.toString(c.element().length()));
+ }
+ }));
+
+ PCollection<String> flattened =
+ PCollectionList.of(created).and(transformed).apply(Flatten.<String>pCollections());
+
+ p.traverseTopologically(visitor);
+
+ assertThat(
+ visitor.getValueToConsumers().get(created),
+ Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
+ transformed.getProducingTransformInternal(),
+ flattened.getProducingTransformInternal()));
+ assertThat(
+ visitor.getValueToConsumers().get(transformed),
+ Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
+ flattened.getProducingTransformInternal()));
+ assertThat(visitor.getValueToConsumers().get(flattened), emptyIterable());
+ }
+
+ @Test
+ public void getUnfinalizedPValuesContainsDanglingOutputs() {
+ PCollection<String> created = p.apply(Create.of("1", "2", "3"));
+ PCollection<String> transformed =
+ created.apply(
+ ParDo.of(
+ new DoFn<String, String>() {
+ @Override
+ public void processElement(DoFn<String, String>.ProcessContext c)
+ throws Exception {
+ c.output(Integer.toString(c.element().length()));
+ }
+ }));
+
+ p.traverseTopologically(visitor);
+ assertThat(visitor.getUnfinalizedPValues(), Matchers.<PValue>contains(transformed));
+ }
+
+ @Test
+ public void getUnfinalizedPValuesEmpty() {
+ p.apply(Create.of("1", "2", "3"))
+ .apply(
+ ParDo.of(
+ new DoFn<String, String>() {
+ @Override
+ public void processElement(DoFn<String, String>.ProcessContext c)
+ throws Exception {
+ c.output(Integer.toString(c.element().length()));
+ }
+ }))
+ .apply(
+ new PTransform<PInput, PDone>() {
+ @Override
+ public PDone apply(PInput input) {
+ return PDone.in(input.getPipeline());
+ }
+ });
+
+ p.traverseTopologically(visitor);
+ assertThat(visitor.getUnfinalizedPValues(), emptyIterable());
+ }
+
+ @Test
+ public void getStepNamesContainsAllTransforms() {
+ PCollection<String> created = p.apply(Create.of("1", "2", "3"));
+ PCollection<String> transformed =
+ created.apply(
+ ParDo.of(
+ new DoFn<String, String>() {
+ @Override
+ public void processElement(DoFn<String, String>.ProcessContext c)
+ throws Exception {
+ c.output(Integer.toString(c.element().length()));
+ }
+ }));
+ PDone finished =
+ transformed.apply(
+ new PTransform<PInput, PDone>() {
+ @Override
+ public PDone apply(PInput input) {
+ return PDone.in(input.getPipeline());
+ }
+ });
+
+ p.traverseTopologically(visitor);
+ assertThat(
+ visitor.getStepNames(),
+ Matchers.<AppliedPTransform<?, ?, ?>, String>hasEntry(
+ created.getProducingTransformInternal(), "s0"));
+ assertThat(
+ visitor.getStepNames(),
+ Matchers.<AppliedPTransform<?, ?, ?>, String>hasEntry(
+ transformed.getProducingTransformInternal(), "s1"));
+ assertThat(
+ visitor.getStepNames(),
+ Matchers.<AppliedPTransform<?, ?, ?>, String>hasEntry(
+ finished.getProducingTransformInternal(), "s2"));
+ }
+
+ @Test
+ public void traverseMultipleTimesThrows() {
+ p.apply(Create.of(1, 2, 3));
+
+ p.traverseTopologically(visitor);
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage(ConsumerTrackingPipelineVisitor.class.getSimpleName());
+ thrown.expectMessage("is finalized");
+ p.traverseTopologically(visitor);
+ }
+
+ @Test
+ public void traverseIndependentPathsSucceeds() {
+ p.apply("left", Create.of(1, 2, 3));
+ p.apply("right", Create.of("foo", "bar", "baz"));
+
+ p.traverseTopologically(visitor);
+ }
+
+ @Test
+ public void getRootTransformsWithoutVisitingThrows() {
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage("completely traversed");
+ thrown.expectMessage("getRootTransforms");
+ visitor.getRootTransforms();
+ }
+ @Test
+ public void getStepNamesWithoutVisitingThrows() {
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage("completely traversed");
+ thrown.expectMessage("getStepNames");
+ visitor.getStepNames();
+ }
+ @Test
+ public void getUnfinalizedPValuesWithoutVisitingThrows() {
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage("completely traversed");
+ thrown.expectMessage("getUnfinalizedPValues");
+ visitor.getUnfinalizedPValues();
+ }
+
+ @Test
+ public void getValueToConsumersWithoutVisitingThrows() {
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage("completely traversed");
+ thrown.expectMessage("getValueToConsumers");
+ visitor.getValueToConsumers();
+ }
+
+ @Test
+ public void getViewsWithoutVisitingThrows() {
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage("completely traversed");
+ thrown.expectMessage("getViews");
+ visitor.getViews();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
new file mode 100644
index 0000000..9a358dd
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
@@ -0,0 +1,257 @@
+/*
+ * 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.hamcrest.Matchers.isA;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Collections;
+
+/**
+ * Tests for {@link EncodabilityEnforcementFactory}.
+ */
+@RunWith(JUnit4.class)
+public class EncodabilityEnforcementFactoryTest {
+ @Rule public ExpectedException thrown = ExpectedException.none();
+ private EncodabilityEnforcementFactory factory = EncodabilityEnforcementFactory.create();
+ private BundleFactory bundleFactory = InProcessBundleFactory.create();
+
+ @Test
+ public void encodeFailsThrows() {
+ WindowedValue<Record> record = WindowedValue.valueInGlobalWindow(new Record());
+
+ ModelEnforcement<Record> enforcement = createEnforcement(new RecordNoEncodeCoder(), record);
+
+ thrown.expect(UserCodeException.class);
+ thrown.expectCause(isA(CoderException.class));
+ thrown.expectMessage("Encode not allowed");
+ enforcement.beforeElement(record);
+ }
+
+ @Test
+ public void decodeFailsThrows() {
+ WindowedValue<Record> record = WindowedValue.valueInGlobalWindow(new Record());
+
+ ModelEnforcement<Record> enforcement = createEnforcement(new RecordNoDecodeCoder(), record);
+
+ thrown.expect(UserCodeException.class);
+ thrown.expectCause(isA(CoderException.class));
+ thrown.expectMessage("Decode not allowed");
+ enforcement.beforeElement(record);
+ }
+
+ @Test
+ public void consistentWithEqualsStructuralValueNotEqualThrows() {
+ WindowedValue<Record> record =
+ WindowedValue.<Record>valueInGlobalWindow(
+ new Record() {
+ @Override
+ public String toString() {
+ return "OriginalRecord";
+ }
+ });
+
+ ModelEnforcement<Record> enforcement =
+ createEnforcement(new RecordStructuralValueCoder(), record);
+
+ thrown.expect(UserCodeException.class);
+ thrown.expectCause(isA(IllegalArgumentException.class));
+ thrown.expectMessage("does not maintain structural value equality");
+ thrown.expectMessage(RecordStructuralValueCoder.class.getSimpleName());
+ thrown.expectMessage("OriginalRecord");
+ enforcement.beforeElement(record);
+ }
+
+ @Test
+ public void notConsistentWithEqualsStructuralValueNotEqualSucceeds() {
+ TestPipeline p = TestPipeline.create();
+ PCollection<Record> unencodable =
+ p.apply(
+ Create.of(new Record())
+ .withCoder(new RecordNotConsistentWithEqualsStructuralValueCoder()));
+ AppliedPTransform<?, ?, ?> consumer =
+ unencodable.apply(Count.<Record>globally()).getProducingTransformInternal();
+
+ WindowedValue<Record> record = WindowedValue.<Record>valueInGlobalWindow(new Record());
+
+ CommittedBundle<Record> input =
+ bundleFactory.createRootBundle(unencodable).add(record).commit(Instant.now());
+ ModelEnforcement<Record> enforcement = factory.forBundle(input, consumer);
+
+ enforcement.beforeElement(record);
+ enforcement.afterElement(record);
+ enforcement.afterFinish(
+ input,
+ StepTransformResult.withoutHold(consumer).build(),
+ Collections.<CommittedBundle<?>>emptyList());
+ }
+
+ private <T> ModelEnforcement<T> createEnforcement(Coder<T> coder, WindowedValue<T> record) {
+ TestPipeline p = TestPipeline.create();
+ PCollection<T> unencodable = p.apply(Create.<T>of().withCoder(coder));
+ AppliedPTransform<?, ?, ?> consumer =
+ unencodable.apply(Count.<T>globally()).getProducingTransformInternal();
+ CommittedBundle<T> input =
+ bundleFactory.createRootBundle(unencodable).add(record).commit(Instant.now());
+ ModelEnforcement<T> enforcement = factory.forBundle(input, consumer);
+ return enforcement;
+ }
+
+ @Test
+ public void structurallyEqualResultsSucceeds() {
+ TestPipeline p = TestPipeline.create();
+ PCollection<Integer> unencodable = p.apply(Create.of(1).withCoder(VarIntCoder.of()));
+ AppliedPTransform<?, ?, ?> consumer =
+ unencodable.apply(Count.<Integer>globally()).getProducingTransformInternal();
+
+ WindowedValue<Integer> value = WindowedValue.valueInGlobalWindow(1);
+
+ CommittedBundle<Integer> input =
+ bundleFactory.createRootBundle(unencodable).add(value).commit(Instant.now());
+ ModelEnforcement<Integer> enforcement = factory.forBundle(input, consumer);
+
+ enforcement.beforeElement(value);
+ enforcement.afterElement(value);
+ enforcement.afterFinish(
+ input,
+ StepTransformResult.withoutHold(consumer).build(),
+ Collections.<CommittedBundle<?>>emptyList());
+ }
+
+ private static class Record {}
+ private static class RecordNoEncodeCoder extends AtomicCoder<Record> {
+
+ @Override
+ public void encode(
+ Record value,
+ OutputStream outStream,
+ org.apache.beam.sdk.coders.Coder.Context context)
+ throws CoderException, IOException {
+ throw new CoderException("Encode not allowed");
+ }
+
+ @Override
+ public Record decode(
+ InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
+ throws CoderException, IOException {
+ return null;
+ }
+ }
+
+ private static class RecordNoDecodeCoder extends AtomicCoder<Record> {
+ @Override
+ public void encode(
+ Record value,
+ OutputStream outStream,
+ org.apache.beam.sdk.coders.Coder.Context context)
+ throws CoderException, IOException {}
+
+ @Override
+ public Record decode(
+ InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
+ throws CoderException, IOException {
+ throw new CoderException("Decode not allowed");
+ }
+ }
+
+ private static class RecordStructuralValueCoder extends AtomicCoder<Record> {
+ @Override
+ public void encode(
+ Record value,
+ OutputStream outStream,
+ org.apache.beam.sdk.coders.Coder.Context context)
+ throws CoderException, IOException {}
+
+ @Override
+ public Record decode(
+ InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
+ throws CoderException, IOException {
+ return new Record() {
+ @Override
+ public String toString() {
+ return "DecodedRecord";
+ }
+ };
+ }
+
+ @Override
+ public boolean consistentWithEquals() {
+ return true;
+ }
+
+ @Override
+ public Object structuralValue(Record value) {
+ return value;
+ }
+ }
+
+ private static class RecordNotConsistentWithEqualsStructuralValueCoder
+ extends AtomicCoder<Record> {
+ @Override
+ public void encode(
+ Record value,
+ OutputStream outStream,
+ org.apache.beam.sdk.coders.Coder.Context context)
+ throws CoderException, IOException {}
+
+ @Override
+ public Record decode(
+ InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
+ throws CoderException, IOException {
+ return new Record() {
+ @Override
+ public String toString() {
+ return "DecodedRecord";
+ }
+ };
+ }
+
+ @Override
+ public boolean consistentWithEquals() {
+ return false;
+ }
+
+ @Override
+ public Object structuralValue(Record value) {
+ return value;
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java
new file mode 100644
index 0000000..66a5106
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java
@@ -0,0 +1,141 @@
+/*
+ * 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.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.emptyIterable;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+
+import org.hamcrest.Matchers;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link FlattenEvaluatorFactory}.
+ */
+@RunWith(JUnit4.class)
+public class FlattenEvaluatorFactoryTest {
+ private BundleFactory bundleFactory = InProcessBundleFactory.create();
+ @Test
+ public void testFlattenInMemoryEvaluator() throws Exception {
+ TestPipeline p = TestPipeline.create();
+ PCollection<Integer> left = p.apply("left", Create.of(1, 2, 4));
+ PCollection<Integer> right = p.apply("right", Create.of(-1, 2, -4));
+ PCollectionList<Integer> list = PCollectionList.of(left).and(right);
+
+ PCollection<Integer> flattened = list.apply(Flatten.<Integer>pCollections());
+
+ CommittedBundle<Integer> leftBundle =
+ bundleFactory.createRootBundle(left).commit(Instant.now());
+ CommittedBundle<Integer> rightBundle =
+ bundleFactory.createRootBundle(right).commit(Instant.now());
+
+ InProcessEvaluationContext context = mock(InProcessEvaluationContext.class);
+
+ UncommittedBundle<Integer> flattenedLeftBundle = bundleFactory.createRootBundle(flattened);
+ UncommittedBundle<Integer> flattenedRightBundle = bundleFactory.createRootBundle(flattened);
+
+ when(context.createBundle(leftBundle, flattened)).thenReturn(flattenedLeftBundle);
+ when(context.createBundle(rightBundle, flattened)).thenReturn(flattenedRightBundle);
+
+ FlattenEvaluatorFactory factory = new FlattenEvaluatorFactory();
+ TransformEvaluator<Integer> leftSideEvaluator =
+ factory.forApplication(flattened.getProducingTransformInternal(), leftBundle, context);
+ TransformEvaluator<Integer> rightSideEvaluator =
+ factory.forApplication(
+ flattened.getProducingTransformInternal(),
+ rightBundle,
+ context);
+
+ leftSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(1));
+ rightSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(-1));
+ leftSideEvaluator.processElement(
+ WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1024)));
+ leftSideEvaluator.processElement(WindowedValue.valueInEmptyWindows(4, PaneInfo.NO_FIRING));
+ rightSideEvaluator.processElement(
+ WindowedValue.valueInEmptyWindows(2, PaneInfo.ON_TIME_AND_ONLY_FIRING));
+ rightSideEvaluator.processElement(
+ WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096)));
+
+ InProcessTransformResult rightSideResult = rightSideEvaluator.finishBundle();
+ InProcessTransformResult leftSideResult = leftSideEvaluator.finishBundle();
+
+ assertThat(
+ rightSideResult.getOutputBundles(),
+ Matchers.<UncommittedBundle<?>>contains(flattenedRightBundle));
+ assertThat(
+ rightSideResult.getTransform(),
+ Matchers.<AppliedPTransform<?, ?, ?>>equalTo(flattened.getProducingTransformInternal()));
+ assertThat(
+ leftSideResult.getOutputBundles(),
+ Matchers.<UncommittedBundle<?>>contains(flattenedLeftBundle));
+ assertThat(
+ leftSideResult.getTransform(),
+ Matchers.<AppliedPTransform<?, ?, ?>>equalTo(flattened.getProducingTransformInternal()));
+
+ assertThat(
+ flattenedLeftBundle.commit(Instant.now()).getElements(),
+ containsInAnyOrder(
+ WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1024)),
+ WindowedValue.valueInEmptyWindows(4, PaneInfo.NO_FIRING),
+ WindowedValue.valueInGlobalWindow(1)));
+ assertThat(
+ flattenedRightBundle.commit(Instant.now()).getElements(),
+ containsInAnyOrder(
+ WindowedValue.valueInEmptyWindows(2, PaneInfo.ON_TIME_AND_ONLY_FIRING),
+ WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096)),
+ WindowedValue.valueInGlobalWindow(-1)));
+ }
+
+ @Test
+ public void testFlattenInMemoryEvaluatorWithEmptyPCollectionList() throws Exception {
+ TestPipeline p = TestPipeline.create();
+ PCollectionList<Integer> list = PCollectionList.empty(p);
+
+ PCollection<Integer> flattened = list.apply(Flatten.<Integer>pCollections());
+
+ InProcessEvaluationContext context = mock(InProcessEvaluationContext.class);
+
+ FlattenEvaluatorFactory factory = new FlattenEvaluatorFactory();
+ TransformEvaluator<Integer> emptyEvaluator =
+ factory.forApplication(flattened.getProducingTransformInternal(), null, context);
+
+ InProcessTransformResult leftSideResult = emptyEvaluator.finishBundle();
+
+ assertThat(leftSideResult.getOutputBundles(), emptyIterable());
+ assertThat(
+ leftSideResult.getTransform(),
+ Matchers.<AppliedPTransform<?, ?, ?>>equalTo(flattened.getProducingTransformInternal()));
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ForwardingPTransformTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ForwardingPTransformTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ForwardingPTransformTest.java
new file mode 100644
index 0000000..9ea71d7
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ForwardingPTransformTest.java
@@ -0,0 +1,112 @@
+/*
+ * 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.hamcrest.Matchers.equalTo;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.values.PCollection;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+/**
+ * Tests for {@link ForwardingPTransform}.
+ */
+@RunWith(JUnit4.class)
+public class ForwardingPTransformTest {
+ @Rule public ExpectedException thrown = ExpectedException.none();
+
+ @Mock private PTransform<PCollection<Integer>, PCollection<String>> delegate;
+
+ private ForwardingPTransform<PCollection<Integer>, PCollection<String>> forwarding;
+
+ @Before
+ public void setup() {
+ MockitoAnnotations.initMocks(this);
+ forwarding =
+ new ForwardingPTransform<PCollection<Integer>, PCollection<String>>() {
+ @Override
+ protected PTransform<PCollection<Integer>, PCollection<String>> delegate() {
+ return delegate;
+ }
+ };
+ }
+
+ @Test
+ public void applyDelegates() {
+ @SuppressWarnings("unchecked")
+ PCollection<Integer> collection = mock(PCollection.class);
+ @SuppressWarnings("unchecked")
+ PCollection<String> output = mock(PCollection.class);
+ when(delegate.apply(collection)).thenReturn(output);
+ PCollection<String> result = forwarding.apply(collection);
+ assertThat(result, equalTo(output));
+ }
+
+ @Test
+ public void getNameDelegates() {
+ String name = "My_forwardingptransform-name;for!thisTest";
+ when(delegate.getName()).thenReturn(name);
+ assertThat(forwarding.getName(), equalTo(name));
+ }
+
+ @Test
+ public void validateDelegates() {
+ @SuppressWarnings("unchecked")
+ PCollection<Integer> input = mock(PCollection.class);
+ doThrow(RuntimeException.class).when(delegate).validate(input);
+
+ thrown.expect(RuntimeException.class);
+ forwarding.validate(input);
+ }
+
+ @Test
+ public void getDefaultOutputCoderDelegates() throws Exception {
+ @SuppressWarnings("unchecked")
+ PCollection<Integer> input = mock(PCollection.class);
+ @SuppressWarnings("unchecked")
+ PCollection<String> output = mock(PCollection.class);
+ @SuppressWarnings("unchecked")
+ Coder<String> outputCoder = mock(Coder.class);
+
+ when(delegate.getDefaultOutputCoder(input, output)).thenReturn(outputCoder);
+ assertThat(forwarding.getDefaultOutputCoder(input, output), equalTo(outputCoder));
+ }
+
+ @Test
+ public void populateDisplayDataDelegates() {
+ DisplayData.Builder builder = mock(DisplayData.Builder.class);
+ doThrow(RuntimeException.class).when(delegate).populateDisplayData(builder);
+
+ thrown.expect(RuntimeException.class);
+ forwarding.populateDisplayData(builder);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java
new file mode 100644
index 0000000..267266d
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java
@@ -0,0 +1,183 @@
+/*
+ * 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.hamcrest.Matchers.contains;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.ReifyTimestampsAndWindows;
+import org.apache.beam.sdk.util.KeyedWorkItem;
+import org.apache.beam.sdk.util.KeyedWorkItems;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Multiset;
+
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link GroupByKeyEvaluatorFactory}.
+ */
+@RunWith(JUnit4.class)
+public class GroupByKeyEvaluatorFactoryTest {
+ private BundleFactory bundleFactory = InProcessBundleFactory.create();
+
+ @Test
+ public void testInMemoryEvaluator() throws Exception {
+ TestPipeline p = TestPipeline.create();
+ KV<String, Integer> firstFoo = KV.of("foo", -1);
+ KV<String, Integer> secondFoo = KV.of("foo", 1);
+ KV<String, Integer> thirdFoo = KV.of("foo", 3);
+ KV<String, Integer> firstBar = KV.of("bar", 22);
+ KV<String, Integer> secondBar = KV.of("bar", 12);
+ KV<String, Integer> firstBaz = KV.of("baz", Integer.MAX_VALUE);
+ PCollection<KV<String, Integer>> values =
+ p.apply(Create.of(firstFoo, firstBar, secondFoo, firstBaz, secondBar, thirdFoo));
+ PCollection<KV<String, WindowedValue<Integer>>> kvs =
+ values.apply(new ReifyTimestampsAndWindows<String, Integer>());
+ PCollection<KeyedWorkItem<String, Integer>> groupedKvs =
+ kvs.apply(new GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly<String, Integer>());
+
+ CommittedBundle<KV<String, WindowedValue<Integer>>> inputBundle =
+ bundleFactory.createRootBundle(kvs).commit(Instant.now());
+ InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+
+ UncommittedBundle<KeyedWorkItem<String, Integer>> fooBundle =
+ bundleFactory.createKeyedBundle(null, "foo", groupedKvs);
+ UncommittedBundle<KeyedWorkItem<String, Integer>> barBundle =
+ bundleFactory.createKeyedBundle(null, "bar", groupedKvs);
+ UncommittedBundle<KeyedWorkItem<String, Integer>> bazBundle =
+ bundleFactory.createKeyedBundle(null, "baz", groupedKvs);
+
+ when(evaluationContext.createKeyedBundle(inputBundle, "foo", groupedKvs)).thenReturn(fooBundle);
+ when(evaluationContext.createKeyedBundle(inputBundle, "bar", groupedKvs)).thenReturn(barBundle);
+ when(evaluationContext.createKeyedBundle(inputBundle, "baz", groupedKvs)).thenReturn(bazBundle);
+
+ // The input to a GroupByKey is assumed to be a KvCoder
+ @SuppressWarnings("unchecked")
+ Coder<String> keyCoder =
+ ((KvCoder<String, WindowedValue<Integer>>) kvs.getCoder()).getKeyCoder();
+ TransformEvaluator<KV<String, WindowedValue<Integer>>> evaluator =
+ new GroupByKeyEvaluatorFactory()
+ .forApplication(
+ groupedKvs.getProducingTransformInternal(), inputBundle, evaluationContext);
+
+ evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(firstFoo)));
+ evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(secondFoo)));
+ evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(thirdFoo)));
+ evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(firstBar)));
+ evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(secondBar)));
+ evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(firstBaz)));
+
+ evaluator.finishBundle();
+
+ assertThat(
+ fooBundle.commit(Instant.now()).getElements(),
+ contains(
+ new KeyedWorkItemMatcher<String, Integer>(
+ KeyedWorkItems.elementsWorkItem(
+ "foo",
+ ImmutableSet.of(
+ WindowedValue.valueInGlobalWindow(-1),
+ WindowedValue.valueInGlobalWindow(1),
+ WindowedValue.valueInGlobalWindow(3))),
+ keyCoder)));
+ assertThat(
+ barBundle.commit(Instant.now()).getElements(),
+ contains(
+ new KeyedWorkItemMatcher<String, Integer>(
+ KeyedWorkItems.elementsWorkItem(
+ "bar",
+ ImmutableSet.of(
+ WindowedValue.valueInGlobalWindow(12),
+ WindowedValue.valueInGlobalWindow(22))),
+ keyCoder)));
+ assertThat(
+ bazBundle.commit(Instant.now()).getElements(),
+ contains(
+ new KeyedWorkItemMatcher<String, Integer>(
+ KeyedWorkItems.elementsWorkItem(
+ "baz",
+ ImmutableSet.of(WindowedValue.valueInGlobalWindow(Integer.MAX_VALUE))),
+ keyCoder)));
+ }
+
+ private <K, V> KV<K, WindowedValue<V>> gwValue(KV<K, V> kv) {
+ return KV.of(kv.getKey(), WindowedValue.valueInGlobalWindow(kv.getValue()));
+ }
+
+ private static class KeyedWorkItemMatcher<K, V>
+ extends BaseMatcher<WindowedValue<KeyedWorkItem<K, V>>> {
+ private final KeyedWorkItem<K, V> myWorkItem;
+ private final Coder<K> keyCoder;
+
+ public KeyedWorkItemMatcher(KeyedWorkItem<K, V> myWorkItem, Coder<K> keyCoder) {
+ this.myWorkItem = myWorkItem;
+ this.keyCoder = keyCoder;
+ }
+
+ @Override
+ public boolean matches(Object item) {
+ if (item == null || !(item instanceof WindowedValue)) {
+ return false;
+ }
+ WindowedValue<KeyedWorkItem<K, V>> that = (WindowedValue<KeyedWorkItem<K, V>>) item;
+ Multiset<WindowedValue<V>> myValues = HashMultiset.create();
+ Multiset<WindowedValue<V>> thatValues = HashMultiset.create();
+ for (WindowedValue<V> value : myWorkItem.elementsIterable()) {
+ myValues.add(value);
+ }
+ for (WindowedValue<V> value : that.getValue().elementsIterable()) {
+ thatValues.add(value);
+ }
+ try {
+ return myValues.equals(thatValues)
+ && keyCoder
+ .structuralValue(myWorkItem.key())
+ .equals(keyCoder.structuralValue(that.getValue().key()));
+ } catch (Exception e) {
+ return false;
+ }
+ }
+
+ @Override
+ public void describeTo(Description description) {
+ description
+ .appendText("KeyedWorkItem<K, V> containing key ")
+ .appendValue(myWorkItem.key())
+ .appendText(" and values ")
+ .appendValueList("[", ", ", "]", myWorkItem.elementsIterable());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
new file mode 100644
index 0000000..557ebff
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
@@ -0,0 +1,220 @@
+/*
+ * 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.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.isA;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+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.ParDo;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.IllegalMutationException;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link ImmutabilityCheckingBundleFactory}.
+ */
+@RunWith(JUnit4.class)
+public class ImmutabilityCheckingBundleFactoryTest {
+ @Rule public ExpectedException thrown = ExpectedException.none();
+ private ImmutabilityCheckingBundleFactory factory;
+ private PCollection<byte[]> created;
+ private PCollection<byte[]> transformed;
+
+ @Before
+ public void setup() {
+ TestPipeline p = TestPipeline.create();
+ created = p.apply(Create.<byte[]>of().withCoder(ByteArrayCoder.of()));
+ transformed = created.apply(ParDo.of(new IdentityDoFn<byte[]>()));
+ factory = ImmutabilityCheckingBundleFactory.create(InProcessBundleFactory.create());
+ }
+
+ @Test
+ public void noMutationRootBundleSucceeds() {
+ UncommittedBundle<byte[]> root = factory.createRootBundle(created);
+ byte[] array = new byte[] {0, 1, 2};
+ root.add(WindowedValue.valueInGlobalWindow(array));
+ CommittedBundle<byte[]> committed = root.commit(Instant.now());
+
+ assertThat(
+ committed.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(array)));
+ }
+
+ @Test
+ public void noMutationKeyedBundleSucceeds() {
+ CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
+ UncommittedBundle<byte[]> keyed = factory.createKeyedBundle(root, "mykey", transformed);
+
+ WindowedValue<byte[]> windowedArray =
+ WindowedValue.of(
+ new byte[] {4, 8, 12},
+ new Instant(891L),
+ new IntervalWindow(new Instant(0), new Instant(1000)),
+ PaneInfo.ON_TIME_AND_ONLY_FIRING);
+ keyed.add(windowedArray);
+
+ CommittedBundle<byte[]> committed = keyed.commit(Instant.now());
+ assertThat(committed.getElements(), containsInAnyOrder(windowedArray));
+ }
+
+ @Test
+ public void noMutationCreateBundleSucceeds() {
+ CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
+ UncommittedBundle<byte[]> intermediate = factory.createBundle(root, transformed);
+
+ WindowedValue<byte[]> windowedArray =
+ WindowedValue.of(
+ new byte[] {4, 8, 12},
+ new Instant(891L),
+ new IntervalWindow(new Instant(0), new Instant(1000)),
+ PaneInfo.ON_TIME_AND_ONLY_FIRING);
+ intermediate.add(windowedArray);
+
+ CommittedBundle<byte[]> committed = intermediate.commit(Instant.now());
+ assertThat(committed.getElements(), containsInAnyOrder(windowedArray));
+ }
+
+ @Test
+ public void mutationBeforeAddRootBundleSucceeds() {
+ UncommittedBundle<byte[]> root = factory.createRootBundle(created);
+ byte[] array = new byte[] {0, 1, 2};
+ array[1] = 2;
+ root.add(WindowedValue.valueInGlobalWindow(array));
+ CommittedBundle<byte[]> committed = root.commit(Instant.now());
+
+ assertThat(
+ committed.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(array)));
+ }
+
+ @Test
+ public void mutationBeforeAddKeyedBundleSucceeds() {
+ CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
+ UncommittedBundle<byte[]> keyed = factory.createKeyedBundle(root, "mykey", transformed);
+
+ byte[] array = new byte[] {4, 8, 12};
+ array[0] = Byte.MAX_VALUE;
+ WindowedValue<byte[]> windowedArray =
+ WindowedValue.of(
+ array,
+ new Instant(891L),
+ new IntervalWindow(new Instant(0), new Instant(1000)),
+ PaneInfo.ON_TIME_AND_ONLY_FIRING);
+ keyed.add(windowedArray);
+
+ CommittedBundle<byte[]> committed = keyed.commit(Instant.now());
+ assertThat(committed.getElements(), containsInAnyOrder(windowedArray));
+ }
+
+ @Test
+ public void mutationBeforeAddCreateBundleSucceeds() {
+ CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
+ UncommittedBundle<byte[]> intermediate = factory.createBundle(root, transformed);
+
+ byte[] array = new byte[] {4, 8, 12};
+ WindowedValue<byte[]> windowedArray =
+ WindowedValue.of(
+ array,
+ new Instant(891L),
+ new IntervalWindow(new Instant(0), new Instant(1000)),
+ PaneInfo.ON_TIME_AND_ONLY_FIRING);
+ array[2] = -3;
+ intermediate.add(windowedArray);
+
+ CommittedBundle<byte[]> committed = intermediate.commit(Instant.now());
+ assertThat(committed.getElements(), containsInAnyOrder(windowedArray));
+ }
+
+ @Test
+ public void mutationAfterAddRootBundleThrows() {
+ UncommittedBundle<byte[]> root = factory.createRootBundle(created);
+ byte[] array = new byte[] {0, 1, 2};
+ root.add(WindowedValue.valueInGlobalWindow(array));
+
+ array[1] = 2;
+ thrown.expect(UserCodeException.class);
+ thrown.expectCause(isA(IllegalMutationException.class));
+ thrown.expectMessage("Values must not be mutated in any way after being output");
+ CommittedBundle<byte[]> committed = root.commit(Instant.now());
+ }
+
+ @Test
+ public void mutationAfterAddKeyedBundleThrows() {
+ CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
+ UncommittedBundle<byte[]> keyed = factory.createKeyedBundle(root, "mykey", transformed);
+
+ byte[] array = new byte[] {4, 8, 12};
+ WindowedValue<byte[]> windowedArray =
+ WindowedValue.of(
+ array,
+ new Instant(891L),
+ new IntervalWindow(new Instant(0), new Instant(1000)),
+ PaneInfo.ON_TIME_AND_ONLY_FIRING);
+ keyed.add(windowedArray);
+
+ array[0] = Byte.MAX_VALUE;
+ thrown.expect(UserCodeException.class);
+ thrown.expectCause(isA(IllegalMutationException.class));
+ thrown.expectMessage("Values must not be mutated in any way after being output");
+ CommittedBundle<byte[]> committed = keyed.commit(Instant.now());
+ }
+
+ @Test
+ public void mutationAfterAddCreateBundleThrows() {
+ CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
+ UncommittedBundle<byte[]> intermediate = factory.createBundle(root, transformed);
+
+ byte[] array = new byte[] {4, 8, 12};
+ WindowedValue<byte[]> windowedArray =
+ WindowedValue.of(
+ array,
+ new Instant(891L),
+ new IntervalWindow(new Instant(0), new Instant(1000)),
+ PaneInfo.ON_TIME_AND_ONLY_FIRING);
+ intermediate.add(windowedArray);
+
+ array[2] = -3;
+ thrown.expect(UserCodeException.class);
+ thrown.expectCause(isA(IllegalMutationException.class));
+ thrown.expectMessage("Values must not be mutated in any way after being output");
+ CommittedBundle<byte[]> committed = intermediate.commit(Instant.now());
+ }
+
+ private static class IdentityDoFn<T> extends DoFn<T, T> {
+ @Override
+ public void processElement(DoFn<T, T>.ProcessContext c) throws Exception {
+ c.output(c.element());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
new file mode 100644
index 0000000..6cef60d
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
@@ -0,0 +1,128 @@
+/*
+ * 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.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.util.IllegalMutationException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.Serializable;
+import java.util.Collections;
+
+/**
+ * Tests for {@link ImmutabilityEnforcementFactory}.
+ */
+@RunWith(JUnit4.class)
+public class ImmutabilityEnforcementFactoryTest implements Serializable {
+ @Rule public transient ExpectedException thrown = ExpectedException.none();
+ private transient ImmutabilityEnforcementFactory factory;
+ private transient BundleFactory bundleFactory;
+ private transient PCollection<byte[]> pcollection;
+ private transient AppliedPTransform<?, ?, ?> consumer;
+
+ @Before
+ public void setup() {
+ factory = new ImmutabilityEnforcementFactory();
+ bundleFactory = InProcessBundleFactory.create();
+ TestPipeline p = TestPipeline.create();
+ pcollection =
+ p.apply(Create.of("foo".getBytes(), "spamhameggs".getBytes()))
+ .apply(
+ ParDo.of(
+ new DoFn<byte[], byte[]>() {
+ @Override
+ public void processElement(DoFn<byte[], byte[]>.ProcessContext c)
+ throws Exception {
+ c.element()[0] = 'b';
+ }
+ }));
+ consumer = pcollection.apply(Count.<byte[]>globally()).getProducingTransformInternal();
+ }
+
+ @Test
+ public void unchangedSucceeds() {
+ WindowedValue<byte[]> element = WindowedValue.valueInGlobalWindow("bar".getBytes());
+ CommittedBundle<byte[]> elements =
+ bundleFactory.createRootBundle(pcollection).add(element).commit(Instant.now());
+
+ ModelEnforcement<byte[]> enforcement = factory.forBundle(elements, consumer);
+ enforcement.beforeElement(element);
+ enforcement.afterElement(element);
+ enforcement.afterFinish(
+ elements,
+ StepTransformResult.withoutHold(consumer).build(),
+ Collections.<CommittedBundle<?>>emptyList());
+ }
+
+ @Test
+ public void mutatedDuringProcessElementThrows() {
+ WindowedValue<byte[]> element = WindowedValue.valueInGlobalWindow("bar".getBytes());
+ CommittedBundle<byte[]> elements =
+ bundleFactory.createRootBundle(pcollection).add(element).commit(Instant.now());
+
+ ModelEnforcement<byte[]> enforcement = factory.forBundle(elements, consumer);
+ enforcement.beforeElement(element);
+ element.getValue()[0] = 'f';
+ thrown.expect(IllegalMutationException.class);
+ thrown.expectMessage(consumer.getFullName());
+ thrown.expectMessage("illegaly mutated");
+ thrown.expectMessage("Input values must not be mutated");
+ enforcement.afterElement(element);
+ enforcement.afterFinish(
+ elements,
+ StepTransformResult.withoutHold(consumer).build(),
+ Collections.<CommittedBundle<?>>emptyList());
+ }
+
+ @Test
+ public void mutatedAfterProcessElementFails() {
+
+ WindowedValue<byte[]> element = WindowedValue.valueInGlobalWindow("bar".getBytes());
+ CommittedBundle<byte[]> elements =
+ bundleFactory.createRootBundle(pcollection).add(element).commit(Instant.now());
+
+ ModelEnforcement<byte[]> enforcement = factory.forBundle(elements, consumer);
+ enforcement.beforeElement(element);
+ enforcement.afterElement(element);
+
+ element.getValue()[0] = 'f';
+ thrown.expect(IllegalMutationException.class);
+ thrown.expectMessage(consumer.getFullName());
+ thrown.expectMessage("illegaly mutated");
+ thrown.expectMessage("Input values must not be mutated");
+ enforcement.afterFinish(
+ elements,
+ StepTransformResult.withoutHold(consumer).build(),
+ Collections.<CommittedBundle<?>>emptyList());
+ }
+}
[08/17] incubator-beam git commit: Move InProcessRunner to its own
module
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactory.java
deleted file mode 100644
index a394090..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactory.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.io.BoundedSource;
-import org.apache.beam.sdk.io.BoundedSource.BoundedReader;
-import org.apache.beam.sdk.io.Read.Bounded;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-import java.io.IOException;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ConcurrentMap;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link TransformEvaluatorFactory} that produces {@link TransformEvaluator TransformEvaluators}
- * for the {@link Bounded Read.Bounded} primitive {@link PTransform}.
- */
-final class BoundedReadEvaluatorFactory implements TransformEvaluatorFactory {
- /*
- * An evaluator for a Source is stateful, to ensure data is not read multiple times.
- * Evaluators are cached here to ensure that the reader is not restarted if the evaluator is
- * retriggered.
- */
- private final ConcurrentMap<EvaluatorKey, Queue<? extends BoundedReadEvaluator<?>>>
- sourceEvaluators = new ConcurrentHashMap<>();
-
- @SuppressWarnings({"unchecked", "rawtypes"})
- @Override
- public <InputT> TransformEvaluator<InputT> forApplication(
- AppliedPTransform<?, ?, ?> application,
- @Nullable CommittedBundle<?> inputBundle,
- InProcessEvaluationContext evaluationContext)
- throws IOException {
- return getTransformEvaluator((AppliedPTransform) application, evaluationContext);
- }
-
- private <OutputT> TransformEvaluator<?> getTransformEvaluator(
- final AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform,
- final InProcessEvaluationContext evaluationContext) {
- BoundedReadEvaluator<?> evaluator =
- getTransformEvaluatorQueue(transform, evaluationContext).poll();
- if (evaluator == null) {
- return EmptyTransformEvaluator.create(transform);
- }
- return evaluator;
- }
-
- /**
- * Get the queue of {@link TransformEvaluator TransformEvaluators} that produce elements for the
- * provided application of {@link Bounded Read.Bounded}, initializing it if required.
- *
- * <p>This method is thread-safe, and will only produce new evaluators if no other invocation has
- * already done so.
- */
- @SuppressWarnings("unchecked")
- private <OutputT> Queue<BoundedReadEvaluator<OutputT>> getTransformEvaluatorQueue(
- final AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform,
- final InProcessEvaluationContext evaluationContext) {
- // Key by the application and the context the evaluation is occurring in (which call to
- // Pipeline#run).
- EvaluatorKey key = new EvaluatorKey(transform, evaluationContext);
- Queue<BoundedReadEvaluator<OutputT>> evaluatorQueue =
- (Queue<BoundedReadEvaluator<OutputT>>) sourceEvaluators.get(key);
- if (evaluatorQueue == null) {
- evaluatorQueue = new ConcurrentLinkedQueue<>();
- if (sourceEvaluators.putIfAbsent(key, evaluatorQueue) == null) {
- // If no queue existed in the evaluators, add an evaluator to initialize the evaluator
- // factory for this transform
- BoundedSource<OutputT> source = transform.getTransform().getSource();
- BoundedReadEvaluator<OutputT> evaluator =
- new BoundedReadEvaluator<OutputT>(transform, evaluationContext, source);
- evaluatorQueue.offer(evaluator);
- } else {
- // otherwise return the existing Queue that arrived before us
- evaluatorQueue = (Queue<BoundedReadEvaluator<OutputT>>) sourceEvaluators.get(key);
- }
- }
- return evaluatorQueue;
- }
-
- /**
- * A {@link BoundedReadEvaluator} produces elements from an underlying {@link BoundedSource},
- * discarding all input elements. Within the call to {@link #finishBundle()}, the evaluator
- * creates the {@link BoundedReader} and consumes all available input.
- *
- * <p>A {@link BoundedReadEvaluator} should only be created once per {@link BoundedSource}, and
- * each evaluator should only be called once per evaluation of the pipeline. Otherwise, the source
- * may produce duplicate elements.
- */
- private static class BoundedReadEvaluator<OutputT> implements TransformEvaluator<Object> {
- private final AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform;
- private final InProcessEvaluationContext evaluationContext;
- /**
- * The source being read from by this {@link BoundedReadEvaluator}. This may not be the same
- * as the source derived from {@link #transform} due to splitting.
- */
- private BoundedSource<OutputT> source;
-
- public BoundedReadEvaluator(
- AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform,
- InProcessEvaluationContext evaluationContext,
- BoundedSource<OutputT> source) {
- this.transform = transform;
- this.evaluationContext = evaluationContext;
- this.source = source;
- }
-
- @Override
- public void processElement(WindowedValue<Object> element) {}
-
- @Override
- public InProcessTransformResult finishBundle() throws IOException {
- try (final BoundedReader<OutputT> reader =
- source.createReader(evaluationContext.getPipelineOptions());) {
- boolean contentsRemaining = reader.start();
- UncommittedBundle<OutputT> output =
- evaluationContext.createRootBundle(transform.getOutput());
- while (contentsRemaining) {
- output.add(
- WindowedValue.timestampedValueInGlobalWindow(
- reader.getCurrent(), reader.getCurrentTimestamp()));
- contentsRemaining = reader.advance();
- }
- return StepTransformResult.withHold(transform, BoundedWindow.TIMESTAMP_MAX_VALUE)
- .addOutput(output)
- .build();
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BundleFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BundleFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BundleFactory.java
deleted file mode 100644
index 5479b00..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BundleFactory.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * A factory that creates {@link UncommittedBundle UncommittedBundles}.
- */
-public interface BundleFactory {
- /**
- * Create an {@link UncommittedBundle} from an empty input. Elements added to the bundle belong to
- * the {@code output} {@link PCollection}.
- */
- public <T> UncommittedBundle<T> createRootBundle(PCollection<T> output);
-
- /**
- * Create an {@link UncommittedBundle} from the specified input. Elements added to the bundle
- * belong to the {@code output} {@link PCollection}.
- */
- public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output);
-
- /**
- * Create an {@link UncommittedBundle} with the specified keys at the specified step. For use by
- * {@link InProcessGroupByKeyOnly} {@link PTransform PTransforms}. Elements added to the bundle
- * belong to the {@code output} {@link PCollection}.
- */
- public <T> UncommittedBundle<T> createKeyedBundle(
- CommittedBundle<?> input, Object key, PCollection<T> output);
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CachedThreadPoolExecutorServiceFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CachedThreadPoolExecutorServiceFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CachedThreadPoolExecutorServiceFactory.java
deleted file mode 100644
index 12427d9..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CachedThreadPoolExecutorServiceFactory.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.options.DefaultValueFactory;
-import org.apache.beam.sdk.options.PipelineOptions;
-
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-/**
- * A {@link ExecutorServiceFactory} that produces cached thread pools via
- * {@link Executors#newCachedThreadPool()}.
- */
-class CachedThreadPoolExecutorServiceFactory
- implements DefaultValueFactory<ExecutorServiceFactory>, ExecutorServiceFactory {
- private static final CachedThreadPoolExecutorServiceFactory INSTANCE =
- new CachedThreadPoolExecutorServiceFactory();
-
- @Override
- public ExecutorServiceFactory create(PipelineOptions options) {
- return INSTANCE;
- }
-
- @Override
- public ExecutorService create() {
- return Executors.newCachedThreadPool();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/Clock.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/Clock.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/Clock.java
deleted file mode 100644
index 7a51251..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/Clock.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.joda.time.Instant;
-
-/**
- * Access to the current time.
- */
-public interface Clock {
- /**
- * Returns the current time as an {@link Instant}.
- */
- Instant now();
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CommittedResult.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CommittedResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CommittedResult.java
deleted file mode 100644
index 10e9697..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CommittedResult.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.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-
-import com.google.auto.value.AutoValue;
-
-/**
- * A {@link InProcessTransformResult} that has been committed.
- */
-@AutoValue
-abstract class CommittedResult {
- /**
- * Returns the {@link AppliedPTransform} that produced this result.
- */
- public abstract AppliedPTransform<?, ?, ?> getTransform();
-
- /**
- * Returns the outputs produced by the transform.
- */
- public abstract Iterable<? extends CommittedBundle<?>> getOutputs();
-
- public static CommittedResult create(
- InProcessTransformResult original, Iterable<? extends CommittedBundle<?>> outputs) {
- return new AutoValue_CommittedResult(original.getTransform(),
- outputs);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CompletionCallback.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CompletionCallback.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CompletionCallback.java
deleted file mode 100644
index 30a2b92..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CompletionCallback.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-
-/**
- * A callback for completing a bundle of input.
- */
-interface CompletionCallback {
- /**
- * Handle a successful result, returning the committed outputs of the result.
- */
- CommittedResult handleResult(
- CommittedBundle<?> inputBundle, InProcessTransformResult result);
-
- /**
- * Handle a result that terminated abnormally due to the provided {@link Throwable}.
- */
- void handleThrowable(CommittedBundle<?> inputBundle, Throwable t);
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitor.java
deleted file mode 100644
index f6ea4af..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitor.java
+++ /dev/null
@@ -1,173 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.Pipeline.PipelineVisitor;
-import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.runners.TransformTreeNode;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.PValue;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Tracks the {@link AppliedPTransform AppliedPTransforms} that consume each {@link PValue} in the
- * {@link Pipeline}. This is used to schedule consuming {@link PTransform PTransforms} to consume
- * input after the upstream transform has produced and committed output.
- */
-public class ConsumerTrackingPipelineVisitor implements PipelineVisitor {
- private Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers = new HashMap<>();
- private Collection<AppliedPTransform<?, ?, ?>> rootTransforms = new ArrayList<>();
- private Collection<PCollectionView<?>> views = new ArrayList<>();
- private Map<AppliedPTransform<?, ?, ?>, String> stepNames = new HashMap<>();
- private Set<PValue> toFinalize = new HashSet<>();
- private int numTransforms = 0;
- private boolean finalized = false;
-
- @Override
- public void enterCompositeTransform(TransformTreeNode node) {
- checkState(
- !finalized,
- "Attempting to traverse a pipeline (node %s) with a %s "
- + "which has already visited a Pipeline and is finalized",
- node.getFullName(),
- ConsumerTrackingPipelineVisitor.class.getSimpleName());
- }
-
- @Override
- public void leaveCompositeTransform(TransformTreeNode node) {
- checkState(
- !finalized,
- "Attempting to traverse a pipeline (node %s) with a %s which is already finalized",
- node.getFullName(),
- ConsumerTrackingPipelineVisitor.class.getSimpleName());
- if (node.isRootNode()) {
- finalized = true;
- }
- }
-
- @Override
- public void visitTransform(TransformTreeNode node) {
- toFinalize.removeAll(node.getInput().expand());
- AppliedPTransform<?, ?, ?> appliedTransform = getAppliedTransform(node);
- stepNames.put(appliedTransform, genStepName());
- if (node.getInput().expand().isEmpty()) {
- rootTransforms.add(appliedTransform);
- } else {
- for (PValue value : node.getInput().expand()) {
- valueToConsumers.get(value).add(appliedTransform);
- }
- }
- }
-
- private AppliedPTransform<?, ?, ?> getAppliedTransform(TransformTreeNode node) {
- @SuppressWarnings({"rawtypes", "unchecked"})
- AppliedPTransform<?, ?, ?> application = AppliedPTransform.of(
- node.getFullName(), node.getInput(), node.getOutput(), (PTransform) node.getTransform());
- return application;
- }
-
- @Override
- public void visitValue(PValue value, TransformTreeNode producer) {
- toFinalize.add(value);
- for (PValue expandedValue : value.expand()) {
- valueToConsumers.put(expandedValue, new ArrayList<AppliedPTransform<?, ?, ?>>());
- if (expandedValue instanceof PCollectionView) {
- views.add((PCollectionView<?>) expandedValue);
- }
- expandedValue.recordAsOutput(getAppliedTransform(producer));
- }
- value.recordAsOutput(getAppliedTransform(producer));
- }
-
- private String genStepName() {
- return String.format("s%s", numTransforms++);
- }
-
-
- /**
- * Returns a mapping of each fully-expanded {@link PValue} to each
- * {@link AppliedPTransform} that consumes it. For each AppliedPTransform in the collection
- * returned from {@code getValueToCustomers().get(PValue)},
- * {@code AppliedPTransform#getInput().expand()} will contain the argument {@link PValue}.
- */
- public Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> getValueToConsumers() {
- checkState(
- finalized,
- "Can't call getValueToConsumers before the Pipeline has been completely traversed");
-
- return valueToConsumers;
- }
-
- /**
- * Returns the mapping for each {@link AppliedPTransform} in the {@link Pipeline} to a unique step
- * name.
- */
- public Map<AppliedPTransform<?, ?, ?>, String> getStepNames() {
- checkState(
- finalized, "Can't call getStepNames before the Pipeline has been completely traversed");
-
- return stepNames;
- }
-
- /**
- * Returns the root transforms of the {@link Pipeline}. A root {@link AppliedPTransform} consumes
- * a {@link PInput} where the {@link PInput#expand()} returns an empty collection.
- */
- public Collection<AppliedPTransform<?, ?, ?>> getRootTransforms() {
- checkState(
- finalized,
- "Can't call getRootTransforms before the Pipeline has been completely traversed");
-
- return rootTransforms;
- }
-
- /**
- * Returns all of the {@link PCollectionView PCollectionViews} contained in the visited
- * {@link Pipeline}.
- */
- public Collection<PCollectionView<?>> getViews() {
- checkState(finalized, "Can't call getViews before the Pipeline has been completely traversed");
-
- return views;
- }
-
- /**
- * Returns all of the {@link PValue PValues} that have been produced but not consumed. These
- * {@link PValue PValues} should be finalized by the {@link PipelineRunner} before the
- * {@link Pipeline} is executed.
- */
- public Set<PValue> getUnfinalizedPValues() {
- checkState(
- finalized,
- "Can't call getUnfinalizedPValues before the Pipeline has been completely traversed");
-
- return toFinalize;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EmptyTransformEvaluator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EmptyTransformEvaluator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EmptyTransformEvaluator.java
deleted file mode 100644
index d198903..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EmptyTransformEvaluator.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-
-/**
- * A {@link TransformEvaluator} that ignores all input and produces no output. The result of
- * invoking {@link #finishBundle()} on this evaluator is to return an
- * {@link InProcessTransformResult} with no elements and a timestamp hold equal to
- * {@link BoundedWindow#TIMESTAMP_MIN_VALUE}. Because the result contains no elements, this hold
- * will not affect the watermark.
- */
-final class EmptyTransformEvaluator<T> implements TransformEvaluator<T> {
- public static <T> TransformEvaluator<T> create(AppliedPTransform<?, ?, ?> transform) {
- return new EmptyTransformEvaluator<T>(transform);
- }
-
- private final AppliedPTransform<?, ?, ?> transform;
-
- private EmptyTransformEvaluator(AppliedPTransform<?, ?, ?> transform) {
- this.transform = transform;
- }
-
- @Override
- public void processElement(WindowedValue<T> element) throws Exception {}
-
- @Override
- public InProcessTransformResult finishBundle() throws Exception {
- return StepTransformResult.withHold(transform, BoundedWindow.TIMESTAMP_MIN_VALUE)
- .build();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactory.java
deleted file mode 100644
index d234d4f..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactory.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.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.util.UserCodeException;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * Enforces that all elements in a {@link PCollection} can be encoded using that
- * {@link PCollection PCollection's} {@link Coder}.
- */
-class EncodabilityEnforcementFactory implements ModelEnforcementFactory {
- public static EncodabilityEnforcementFactory create() {
- return new EncodabilityEnforcementFactory();
- }
-
- @Override
- public <T> ModelEnforcement<T> forBundle(
- CommittedBundle<T> input, AppliedPTransform<?, ?, ?> consumer) {
- return new EncodabilityEnforcement<>(input);
- }
-
- private static class EncodabilityEnforcement<T> extends AbstractModelEnforcement<T> {
- private Coder<T> coder;
-
- public EncodabilityEnforcement(CommittedBundle<T> input) {
- coder = input.getPCollection().getCoder();
- }
-
- @Override
- public void beforeElement(WindowedValue<T> element) {
- try {
- T clone = CoderUtils.clone(coder, element.getValue());
- if (coder.consistentWithEquals()) {
- checkArgument(
- coder.structuralValue(element.getValue()).equals(coder.structuralValue(clone)),
- "Coder %s of class %s does not maintain structural value equality"
- + " on input element %s",
- coder,
- coder.getClass().getSimpleName(),
- element.getValue());
- }
- } catch (Exception e) {
- throw UserCodeException.wrap(e);
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EvaluatorKey.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EvaluatorKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EvaluatorKey.java
deleted file mode 100644
index 9d8fc43..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EvaluatorKey.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-
-import java.util.Objects;
-
-/**
- * A (Transform, Pipeline Execution) key for stateful evaluators.
- *
- * Source evaluators are stateful to ensure data is not read multiple times. Evaluators are cached
- * to ensure that the reader is not restarted if the evaluator is retriggered. An
- * {@link EvaluatorKey} is used to ensure that multiple Pipelines can be executed without sharing
- * the same evaluators.
- */
-final class EvaluatorKey {
- private final AppliedPTransform<?, ?, ?> transform;
- private final InProcessEvaluationContext context;
-
- public EvaluatorKey(AppliedPTransform<?, ?, ?> transform, InProcessEvaluationContext context) {
- this.transform = transform;
- this.context = context;
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(transform, context);
- }
-
- @Override
- public boolean equals(Object other) {
- if (other == null || !(other instanceof EvaluatorKey)) {
- return false;
- }
- EvaluatorKey that = (EvaluatorKey) other;
- return Objects.equals(this.transform, that.transform)
- && Objects.equals(this.context, that.context);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceFactory.java
deleted file mode 100644
index cfbf7b4..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceFactory.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import java.util.concurrent.ExecutorService;
-
-/**
- * A factory that creates {@link ExecutorService ExecutorServices}.
- * {@link ExecutorService ExecutorServices} created by this factory should be independent of one
- * another (e.g., if any executor is shut down the remaining executors should continue to process
- * work).
- */
-public interface ExecutorServiceFactory {
- /**
- * Create a new {@link ExecutorService}.
- */
- ExecutorService create();
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceParallelExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceParallelExecutor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceParallelExecutor.java
deleted file mode 100644
index 19bf35d..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceParallelExecutor.java
+++ /dev/null
@@ -1,478 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.FiredTimers;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.KeyedWorkItem;
-import org.apache.beam.sdk.util.KeyedWorkItems;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PValue;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Optional;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import com.google.common.collect.ImmutableList;
-
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.Queue;
-import java.util.Set;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutorService;
-
-import javax.annotation.Nullable;
-
-/**
- * An {@link InProcessExecutor} that uses an underlying {@link ExecutorService} and
- * {@link InProcessEvaluationContext} to execute a {@link Pipeline}.
- */
-final class ExecutorServiceParallelExecutor implements InProcessExecutor {
- private static final Logger LOG = LoggerFactory.getLogger(ExecutorServiceParallelExecutor.class);
-
- private final ExecutorService executorService;
-
- private final Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers;
- private final Set<PValue> keyedPValues;
- private final TransformEvaluatorRegistry registry;
- @SuppressWarnings("rawtypes")
- private final Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>>
- transformEnforcements;
-
- private final InProcessEvaluationContext evaluationContext;
-
- private final LoadingCache<StepAndKey, TransformExecutorService> executorServices;
- private final ConcurrentMap<TransformExecutor<?>, Boolean> scheduledExecutors;
-
- private final Queue<ExecutorUpdate> allUpdates;
- private final BlockingQueue<VisibleExecutorUpdate> visibleUpdates;
-
- private final TransformExecutorService parallelExecutorService;
- private final CompletionCallback defaultCompletionCallback;
-
- private Collection<AppliedPTransform<?, ?, ?>> rootNodes;
-
- public static ExecutorServiceParallelExecutor create(
- ExecutorService executorService,
- Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
- Set<PValue> keyedPValues,
- TransformEvaluatorRegistry registry,
- @SuppressWarnings("rawtypes")
- Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>> transformEnforcements,
- InProcessEvaluationContext context) {
- return new ExecutorServiceParallelExecutor(
- executorService, valueToConsumers, keyedPValues, registry, transformEnforcements, context);
- }
-
- private ExecutorServiceParallelExecutor(
- ExecutorService executorService,
- Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
- Set<PValue> keyedPValues,
- TransformEvaluatorRegistry registry,
- @SuppressWarnings("rawtypes")
- Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>> transformEnforcements,
- InProcessEvaluationContext context) {
- this.executorService = executorService;
- this.valueToConsumers = valueToConsumers;
- this.keyedPValues = keyedPValues;
- this.registry = registry;
- this.transformEnforcements = transformEnforcements;
- this.evaluationContext = context;
-
- scheduledExecutors = new ConcurrentHashMap<>();
- // Weak Values allows TransformExecutorServices that are no longer in use to be reclaimed.
- // Executing TransformExecutorServices have a strong reference to their TransformExecutorService
- // which stops the TransformExecutorServices from being prematurely garbage collected
- executorServices =
- CacheBuilder.newBuilder().weakValues().build(serialTransformExecutorServiceCacheLoader());
-
- this.allUpdates = new ConcurrentLinkedQueue<>();
- this.visibleUpdates = new ArrayBlockingQueue<>(20);
-
- parallelExecutorService =
- TransformExecutorServices.parallel(executorService, scheduledExecutors);
- defaultCompletionCallback = new DefaultCompletionCallback();
- }
-
- private CacheLoader<StepAndKey, TransformExecutorService>
- serialTransformExecutorServiceCacheLoader() {
- return new CacheLoader<StepAndKey, TransformExecutorService>() {
- @Override
- public TransformExecutorService load(StepAndKey stepAndKey) throws Exception {
- return TransformExecutorServices.serial(executorService, scheduledExecutors);
- }
- };
- }
-
- @Override
- public void start(Collection<AppliedPTransform<?, ?, ?>> roots) {
- rootNodes = ImmutableList.copyOf(roots);
- Runnable monitorRunnable = new MonitorRunnable();
- executorService.submit(monitorRunnable);
- }
-
- @SuppressWarnings("unchecked")
- public void scheduleConsumption(
- AppliedPTransform<?, ?, ?> consumer,
- @Nullable CommittedBundle<?> bundle,
- CompletionCallback onComplete) {
- evaluateBundle(consumer, bundle, onComplete);
- }
-
- private <T> void evaluateBundle(
- final AppliedPTransform<?, ?, ?> transform,
- @Nullable final CommittedBundle<T> bundle,
- final CompletionCallback onComplete) {
- TransformExecutorService transformExecutor;
-
- if (bundle != null && isKeyed(bundle.getPCollection())) {
- final StepAndKey stepAndKey =
- StepAndKey.of(transform, bundle == null ? null : bundle.getKey());
- // This executor will remain reachable until it has executed all scheduled transforms.
- // The TransformExecutors keep a strong reference to the Executor, the ExecutorService keeps
- // a reference to the scheduled TransformExecutor callable. Follow-up TransformExecutors
- // (scheduled due to the completion of another TransformExecutor) are provided to the
- // ExecutorService before the Earlier TransformExecutor callable completes.
- transformExecutor = executorServices.getUnchecked(stepAndKey);
- } else {
- transformExecutor = parallelExecutorService;
- }
-
- Collection<ModelEnforcementFactory> enforcements =
- MoreObjects.firstNonNull(
- transformEnforcements.get(transform.getTransform().getClass()),
- Collections.<ModelEnforcementFactory>emptyList());
-
- TransformExecutor<T> callable =
- TransformExecutor.create(
- registry,
- enforcements,
- evaluationContext,
- bundle,
- transform,
- onComplete,
- transformExecutor);
- transformExecutor.schedule(callable);
- }
-
- private boolean isKeyed(PValue pvalue) {
- return keyedPValues.contains(pvalue);
- }
-
- private void scheduleConsumers(CommittedBundle<?> bundle) {
- for (AppliedPTransform<?, ?, ?> consumer : valueToConsumers.get(bundle.getPCollection())) {
- scheduleConsumption(consumer, bundle, defaultCompletionCallback);
- }
- }
-
- @Override
- public void awaitCompletion() throws Throwable {
- VisibleExecutorUpdate update;
- do {
- update = visibleUpdates.take();
- if (update.throwable.isPresent()) {
- throw update.throwable.get();
- }
- } while (!update.isDone());
- executorService.shutdown();
- }
-
- /**
- * The default {@link CompletionCallback}. The default completion callback is used to complete
- * transform evaluations that are triggered due to the arrival of elements from an upstream
- * transform, or for a source transform.
- */
- private class DefaultCompletionCallback implements CompletionCallback {
- @Override
- public CommittedResult handleResult(
- CommittedBundle<?> inputBundle, InProcessTransformResult result) {
- CommittedResult committedResult =
- evaluationContext.handleResult(inputBundle, Collections.<TimerData>emptyList(), result);
- for (CommittedBundle<?> outputBundle : committedResult.getOutputs()) {
- allUpdates.offer(ExecutorUpdate.fromBundle(outputBundle));
- }
- return committedResult;
- }
-
- @Override
- public void handleThrowable(CommittedBundle<?> inputBundle, Throwable t) {
- allUpdates.offer(ExecutorUpdate.fromThrowable(t));
- }
- }
-
- /**
- * A {@link CompletionCallback} where the completed bundle was produced to deliver some collection
- * of {@link TimerData timers}. When the evaluator completes successfully, reports all of the
- * timers used to create the input to the {@link InProcessEvaluationContext evaluation context}
- * as part of the result.
- */
- private class TimerCompletionCallback implements CompletionCallback {
- private final Iterable<TimerData> timers;
-
- private TimerCompletionCallback(Iterable<TimerData> timers) {
- this.timers = timers;
- }
-
- @Override
- public CommittedResult handleResult(
- CommittedBundle<?> inputBundle, InProcessTransformResult result) {
- CommittedResult committedResult =
- evaluationContext.handleResult(inputBundle, timers, result);
- for (CommittedBundle<?> outputBundle : committedResult.getOutputs()) {
- allUpdates.offer(ExecutorUpdate.fromBundle(outputBundle));
- }
- return committedResult;
- }
-
- @Override
- public void handleThrowable(CommittedBundle<?> inputBundle, Throwable t) {
- allUpdates.offer(ExecutorUpdate.fromThrowable(t));
- }
- }
-
- /**
- * An internal status update on the state of the executor.
- *
- * Used to signal when the executor should be shut down (due to an exception).
- */
- private static class ExecutorUpdate {
- private final Optional<? extends CommittedBundle<?>> bundle;
- private final Optional<? extends Throwable> throwable;
-
- public static ExecutorUpdate fromBundle(CommittedBundle<?> bundle) {
- return new ExecutorUpdate(bundle, null);
- }
-
- public static ExecutorUpdate fromThrowable(Throwable t) {
- return new ExecutorUpdate(null, t);
- }
-
- private ExecutorUpdate(CommittedBundle<?> producedBundle, Throwable throwable) {
- this.bundle = Optional.fromNullable(producedBundle);
- this.throwable = Optional.fromNullable(throwable);
- }
-
- public Optional<? extends CommittedBundle<?>> getBundle() {
- return bundle;
- }
-
- public Optional<? extends Throwable> getException() {
- return throwable;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(ExecutorUpdate.class)
- .add("bundle", bundle)
- .add("exception", throwable)
- .toString();
- }
- }
-
- /**
- * An update of interest to the user. Used in {@link #awaitCompletion} to decide whether to
- * return normally or throw an exception.
- */
- private static class VisibleExecutorUpdate {
- private final Optional<? extends Throwable> throwable;
- private final boolean done;
-
- public static VisibleExecutorUpdate fromThrowable(Throwable e) {
- return new VisibleExecutorUpdate(false, e);
- }
-
- public static VisibleExecutorUpdate finished() {
- return new VisibleExecutorUpdate(true, null);
- }
-
- private VisibleExecutorUpdate(boolean done, @Nullable Throwable exception) {
- this.throwable = Optional.fromNullable(exception);
- this.done = done;
- }
-
- public boolean isDone() {
- return done;
- }
- }
-
- private class MonitorRunnable implements Runnable {
- private final String runnableName =
- String.format(
- "%s$%s-monitor",
- evaluationContext.getPipelineOptions().getAppName(),
- ExecutorServiceParallelExecutor.class.getSimpleName());
-
- @Override
- public void run() {
- String oldName = Thread.currentThread().getName();
- Thread.currentThread().setName(runnableName);
- try {
- ExecutorUpdate update = allUpdates.poll();
- // pull all of the pending work off of the queue
- while (update != null) {
- LOG.debug("Executor Update: {}", update);
- if (update.getBundle().isPresent()) {
- scheduleConsumers(update.getBundle().get());
- } else if (update.getException().isPresent()) {
- visibleUpdates.offer(VisibleExecutorUpdate.fromThrowable(update.getException().get()));
- }
- update = allUpdates.poll();
- }
- boolean timersFired = fireTimers();
- addWorkIfNecessary(timersFired);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- LOG.error("Monitor died due to being interrupted");
- while (!visibleUpdates.offer(VisibleExecutorUpdate.fromThrowable(e))) {
- visibleUpdates.poll();
- }
- } catch (Throwable t) {
- LOG.error("Monitor thread died due to throwable", t);
- while (!visibleUpdates.offer(VisibleExecutorUpdate.fromThrowable(t))) {
- visibleUpdates.poll();
- }
- } finally {
- if (!shouldShutdown()) {
- // The monitor thread should always be scheduled; but we only need to be scheduled once
- executorService.submit(this);
- }
- Thread.currentThread().setName(oldName);
- }
- }
-
- /**
- * Fires any available timers. Returns true if at least one timer was fired.
- */
- private boolean fireTimers() throws Exception {
- try {
- boolean firedTimers = false;
- for (Map.Entry<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> transformTimers :
- evaluationContext.extractFiredTimers().entrySet()) {
- AppliedPTransform<?, ?, ?> transform = transformTimers.getKey();
- for (Map.Entry<Object, FiredTimers> keyTimers : transformTimers.getValue().entrySet()) {
- for (TimeDomain domain : TimeDomain.values()) {
- Collection<TimerData> delivery = keyTimers.getValue().getTimers(domain);
- if (delivery.isEmpty()) {
- continue;
- }
- KeyedWorkItem<Object, Object> work =
- KeyedWorkItems.timersWorkItem(keyTimers.getKey(), delivery);
- @SuppressWarnings({"unchecked", "rawtypes"})
- CommittedBundle<?> bundle =
- evaluationContext
- .createKeyedBundle(
- null, keyTimers.getKey(), (PCollection) transform.getInput())
- .add(WindowedValue.valueInEmptyWindows(work))
- .commit(Instant.now());
- scheduleConsumption(transform, bundle, new TimerCompletionCallback(delivery));
- firedTimers = true;
- }
- }
- }
- return firedTimers;
- } catch (Exception e) {
- LOG.error("Internal Error while delivering timers", e);
- throw e;
- }
- }
-
- private boolean shouldShutdown() {
- if (evaluationContext.isDone()) {
- LOG.debug("Pipeline is finished. Shutting down. {}");
- while (!visibleUpdates.offer(VisibleExecutorUpdate.finished())) {
- visibleUpdates.poll();
- }
- executorService.shutdown();
- return true;
- }
- return false;
- }
-
- /**
- * If all active {@link TransformExecutor TransformExecutors} are in a blocked state,
- * add more work from root nodes that may have additional work. This ensures that if a pipeline
- * has elements available from the root nodes it will add those elements when necessary.
- */
- private void addWorkIfNecessary(boolean firedTimers) {
- // If any timers have fired, they will add more work; We don't need to add more
- if (firedTimers) {
- return;
- }
- for (TransformExecutor<?> executor : scheduledExecutors.keySet()) {
- if (!isExecutorBlocked(executor)) {
- // We have at least one executor that can proceed without adding additional work
- return;
- }
- }
- // All current TransformExecutors are blocked; add more work from the roots.
- for (AppliedPTransform<?, ?, ?> root : rootNodes) {
- if (!evaluationContext.isDone(root)) {
- scheduleConsumption(root, null, defaultCompletionCallback);
- }
- }
- }
-
- /**
- * Return true if the provided executor might make more progress if no action is taken.
- *
- * <p>May return false even if all executor threads are currently blocked or cleaning up, as
- * these can cause more work to be scheduled. If this does not occur, after these calls
- * terminate, future calls will return true if all executors are waiting.
- */
- private boolean isExecutorBlocked(TransformExecutor<?> executor) {
- Thread thread = executor.getThread();
- if (thread == null) {
- return false;
- }
- switch (thread.getState()) {
- case TERMINATED:
- throw new IllegalStateException(String.format(
- "Unexpectedly encountered a Terminated TransformExecutor %s", executor));
- case WAITING:
- case TIMED_WAITING:
- // The thread is waiting for some external input. Adding more work may cause the thread
- // to stop waiting (e.g. the thread is waiting on an unbounded side input)
- return true;
- case BLOCKED:
- // The executor is blocked on acquisition of a java monitor. This usually means it is
- // making a call to the EvaluationContext, but not a model-blocking call - and will
- // eventually complete, at which point we may reevaluate.
- default:
- // NEW and RUNNABLE threads can make progress
- return false;
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactory.java
deleted file mode 100644
index 4e23dde..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactory.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.sdk.transforms.Flatten.FlattenPCollectionList;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionList;
-
-/**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the {@link Flatten}
- * {@link PTransform}.
- */
-class FlattenEvaluatorFactory implements TransformEvaluatorFactory {
- @Override
- public <InputT> TransformEvaluator<InputT> forApplication(
- AppliedPTransform<?, ?, ?> application,
- CommittedBundle<?> inputBundle,
- InProcessEvaluationContext evaluationContext) {
- @SuppressWarnings({"cast", "unchecked", "rawtypes"})
- TransformEvaluator<InputT> evaluator = (TransformEvaluator<InputT>) createInMemoryEvaluator(
- (AppliedPTransform) application, inputBundle, evaluationContext);
- return evaluator;
- }
-
- private <InputT> TransformEvaluator<InputT> createInMemoryEvaluator(
- final AppliedPTransform<
- PCollectionList<InputT>, PCollection<InputT>, FlattenPCollectionList<InputT>>
- application,
- final CommittedBundle<InputT> inputBundle,
- final InProcessEvaluationContext evaluationContext) {
- if (inputBundle == null) {
- // it is impossible to call processElement on a flatten with no input bundle. A Flatten with
- // no input bundle occurs as an output of Flatten.pcollections(PCollectionList.empty())
- return new FlattenEvaluator<>(
- null, StepTransformResult.withoutHold(application).build());
- }
- final UncommittedBundle<InputT> outputBundle =
- evaluationContext.createBundle(inputBundle, application.getOutput());
- final InProcessTransformResult result =
- StepTransformResult.withoutHold(application).addOutput(outputBundle).build();
- return new FlattenEvaluator<>(outputBundle, result);
- }
-
- private static class FlattenEvaluator<InputT> implements TransformEvaluator<InputT> {
- private final UncommittedBundle<InputT> outputBundle;
- private final InProcessTransformResult result;
-
- public FlattenEvaluator(
- UncommittedBundle<InputT> outputBundle, InProcessTransformResult result) {
- this.outputBundle = outputBundle;
- this.result = result;
- }
-
- @Override
- public void processElement(WindowedValue<InputT> element) {
- outputBundle.add(element);
- }
-
- @Override
- public InProcessTransformResult finishBundle() {
- return result;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransform.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransform.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransform.java
deleted file mode 100644
index 85aa1c4..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransform.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.coders.CannotProvideCoderException;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-import org.apache.beam.sdk.values.TypedPValue;
-
-/**
- * A base class for implementing {@link PTransform} overrides, which behave identically to the
- * delegate transform but with overridden methods. Implementors are required to implement
- * {@link #delegate()}, which returns the object to forward calls to, and {@link #apply(PInput)}.
- */
-public abstract class ForwardingPTransform<InputT extends PInput, OutputT extends POutput>
- extends PTransform<InputT, OutputT> {
- protected abstract PTransform<InputT, OutputT> delegate();
-
- @Override
- public OutputT apply(InputT input) {
- return delegate().apply(input);
- }
-
- @Override
- public void validate(InputT input) {
- delegate().validate(input);
- }
-
- @Override
- public String getName() {
- return delegate().getName();
- }
-
- @Override
- public <T> Coder<T> getDefaultOutputCoder(InputT input, @SuppressWarnings("unused")
- TypedPValue<T> output) throws CannotProvideCoderException {
- return delegate().getDefaultOutputCoder(input, output);
- }
-
- @Override
- public void populateDisplayData(DisplayData.Builder builder) {
- delegate().populateDisplayData(builder);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactory.java
deleted file mode 100644
index 4cec841..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactory.java
+++ /dev/null
@@ -1,274 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.IterableCoder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.StepTransformResult.Builder;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.GroupAlsoByWindowViaWindowSetDoFn;
-import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.ReifyTimestampsAndWindows;
-import org.apache.beam.sdk.util.KeyedWorkItem;
-import org.apache.beam.sdk.util.KeyedWorkItemCoder;
-import org.apache.beam.sdk.util.KeyedWorkItems;
-import org.apache.beam.sdk.util.SystemReduceFn;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-
-import com.google.common.annotations.VisibleForTesting;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the {@link GroupByKey}
- * {@link PTransform}.
- */
-class GroupByKeyEvaluatorFactory implements TransformEvaluatorFactory {
- @Override
- public <InputT> TransformEvaluator<InputT> forApplication(
- AppliedPTransform<?, ?, ?> application,
- CommittedBundle<?> inputBundle,
- InProcessEvaluationContext evaluationContext) {
- @SuppressWarnings({"cast", "unchecked", "rawtypes"})
- TransformEvaluator<InputT> evaluator = createEvaluator(
- (AppliedPTransform) application, (CommittedBundle) inputBundle, evaluationContext);
- return evaluator;
- }
-
- private <K, V> TransformEvaluator<KV<K, WindowedValue<V>>> createEvaluator(
- final AppliedPTransform<
- PCollection<KV<K, WindowedValue<V>>>, PCollection<KeyedWorkItem<K, V>>,
- InProcessGroupByKeyOnly<K, V>>
- application,
- final CommittedBundle<KV<K, V>> inputBundle,
- final InProcessEvaluationContext evaluationContext) {
- return new GroupByKeyEvaluator<K, V>(evaluationContext, inputBundle, application);
- }
-
- private static class GroupByKeyEvaluator<K, V>
- implements TransformEvaluator<KV<K, WindowedValue<V>>> {
- private final InProcessEvaluationContext evaluationContext;
-
- private final CommittedBundle<KV<K, V>> inputBundle;
- private final AppliedPTransform<
- PCollection<KV<K, WindowedValue<V>>>, PCollection<KeyedWorkItem<K, V>>,
- InProcessGroupByKeyOnly<K, V>>
- application;
- private final Coder<K> keyCoder;
- private Map<GroupingKey<K>, List<WindowedValue<V>>> groupingMap;
-
- public GroupByKeyEvaluator(
- InProcessEvaluationContext evaluationContext,
- CommittedBundle<KV<K, V>> inputBundle,
- AppliedPTransform<
- PCollection<KV<K, WindowedValue<V>>>, PCollection<KeyedWorkItem<K, V>>,
- InProcessGroupByKeyOnly<K, V>>
- application) {
- this.evaluationContext = evaluationContext;
- this.inputBundle = inputBundle;
- this.application = application;
-
- PCollection<KV<K, WindowedValue<V>>> input = application.getInput();
- keyCoder = getKeyCoder(input.getCoder());
- groupingMap = new HashMap<>();
- }
-
- private Coder<K> getKeyCoder(Coder<KV<K, WindowedValue<V>>> coder) {
- if (!(coder instanceof KvCoder)) {
- throw new IllegalStateException();
- }
- @SuppressWarnings("unchecked")
- Coder<K> keyCoder = ((KvCoder<K, WindowedValue<V>>) coder).getKeyCoder();
- return keyCoder;
- }
-
- @Override
- public void processElement(WindowedValue<KV<K, WindowedValue<V>>> element) {
- KV<K, WindowedValue<V>> kv = element.getValue();
- K key = kv.getKey();
- byte[] encodedKey;
- try {
- encodedKey = encodeToByteArray(keyCoder, key);
- } catch (CoderException exn) {
- // TODO: Put in better element printing:
- // truncate if too long.
- throw new IllegalArgumentException(
- String.format("unable to encode key %s of input to %s using %s", key, this, keyCoder),
- exn);
- }
- GroupingKey<K> groupingKey = new GroupingKey<>(key, encodedKey);
- List<WindowedValue<V>> values = groupingMap.get(groupingKey);
- if (values == null) {
- values = new ArrayList<WindowedValue<V>>();
- groupingMap.put(groupingKey, values);
- }
- values.add(kv.getValue());
- }
-
- @Override
- public InProcessTransformResult finishBundle() {
- Builder resultBuilder = StepTransformResult.withoutHold(application);
- for (Map.Entry<GroupingKey<K>, List<WindowedValue<V>>> groupedEntry :
- groupingMap.entrySet()) {
- K key = groupedEntry.getKey().key;
- KeyedWorkItem<K, V> groupedKv =
- KeyedWorkItems.elementsWorkItem(key, groupedEntry.getValue());
- UncommittedBundle<KeyedWorkItem<K, V>> bundle =
- evaluationContext.createKeyedBundle(inputBundle, key, application.getOutput());
- bundle.add(WindowedValue.valueInGlobalWindow(groupedKv));
- resultBuilder.addOutput(bundle);
- }
- return resultBuilder.build();
- }
-
- private static class GroupingKey<K> {
- private K key;
- private byte[] encodedKey;
-
- public GroupingKey(K key, byte[] encodedKey) {
- this.key = key;
- this.encodedKey = encodedKey;
- }
-
- @Override
- public boolean equals(Object o) {
- if (o instanceof GroupingKey) {
- GroupingKey<?> that = (GroupingKey<?>) o;
- return Arrays.equals(this.encodedKey, that.encodedKey);
- } else {
- return false;
- }
- }
-
- @Override
- public int hashCode() {
- return Arrays.hashCode(encodedKey);
- }
- }
- }
-
- /**
- * A {@link PTransformOverrideFactory} for {@link GroupByKey} PTransforms.
- */
- public static final class InProcessGroupByKeyOverrideFactory
- implements PTransformOverrideFactory {
- @Override
- public <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
- PTransform<InputT, OutputT> transform) {
- if (transform instanceof GroupByKey) {
- @SuppressWarnings({"rawtypes", "unchecked"})
- PTransform<InputT, OutputT> override = new InProcessGroupByKey((GroupByKey) transform);
- return override;
- }
- return transform;
- }
- }
-
- /**
- * An in-memory implementation of the {@link GroupByKey} primitive as a composite
- * {@link PTransform}.
- */
- private static final class InProcessGroupByKey<K, V>
- extends ForwardingPTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> {
- private final GroupByKey<K, V> original;
-
- private InProcessGroupByKey(GroupByKey<K, V> from) {
- this.original = from;
- }
-
- @Override
- public PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> delegate() {
- return original;
- }
-
- @Override
- public PCollection<KV<K, Iterable<V>>> apply(PCollection<KV<K, V>> input) {
- KvCoder<K, V> inputCoder = (KvCoder<K, V>) input.getCoder();
-
- // This operation groups by the combination of key and window,
- // merging windows as needed, using the windows assigned to the
- // key/value input elements and the window merge operation of the
- // window function associated with the input PCollection.
- WindowingStrategy<?, ?> windowingStrategy = input.getWindowingStrategy();
-
- // Use the default GroupAlsoByWindow implementation
- DoFn<KeyedWorkItem<K, V>, KV<K, Iterable<V>>> groupAlsoByWindow =
- groupAlsoByWindow(windowingStrategy, inputCoder.getValueCoder());
-
- // By default, implement GroupByKey via a series of lower-level operations.
- return input
- // Make each input element's timestamp and assigned windows
- // explicit, in the value part.
- .apply(new ReifyTimestampsAndWindows<K, V>())
-
- .apply(new InProcessGroupByKeyOnly<K, V>())
- .setCoder(KeyedWorkItemCoder.of(inputCoder.getKeyCoder(),
- inputCoder.getValueCoder(), input.getWindowingStrategy().getWindowFn().windowCoder()))
-
- // Group each key's values by window, merging windows as needed.
- .apply("GroupAlsoByWindow", ParDo.of(groupAlsoByWindow))
-
- // And update the windowing strategy as appropriate.
- .setWindowingStrategyInternal(original.updateWindowingStrategy(windowingStrategy))
- .setCoder(
- KvCoder.of(inputCoder.getKeyCoder(), IterableCoder.of(inputCoder.getValueCoder())));
- }
-
- private <W extends BoundedWindow>
- DoFn<KeyedWorkItem<K, V>, KV<K, Iterable<V>>> groupAlsoByWindow(
- final WindowingStrategy<?, W> windowingStrategy, final Coder<V> inputCoder) {
- return GroupAlsoByWindowViaWindowSetDoFn.create(
- windowingStrategy, SystemReduceFn.<K, V, W>buffering(inputCoder));
- }
- }
-
- /**
- * An implementation primitive to use in the evaluation of a {@link GroupByKey}
- * {@link PTransform}.
- */
- public static final class InProcessGroupByKeyOnly<K, V>
- extends PTransform<PCollection<KV<K, WindowedValue<V>>>, PCollection<KeyedWorkItem<K, V>>> {
- @Override
- public PCollection<KeyedWorkItem<K, V>> apply(PCollection<KV<K, WindowedValue<V>>> input) {
- return PCollection.<KeyedWorkItem<K, V>>createPrimitiveOutputInternal(
- input.getPipeline(), input.getWindowingStrategy(), input.isBounded());
- }
-
- @VisibleForTesting
- InProcessGroupByKeyOnly() {}
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactory.java
deleted file mode 100644
index 04ece1c..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactory.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.util.IllegalMutationException;
-import org.apache.beam.sdk.util.MutationDetector;
-import org.apache.beam.sdk.util.MutationDetectors;
-import org.apache.beam.sdk.util.UserCodeException;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.api.client.util.Throwables;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.SetMultimap;
-
-import org.joda.time.Instant;
-
-/**
- * A {@link BundleFactory} that ensures that elements added to it are not mutated after being
- * output. Immutability checks are enforced at the time {@link UncommittedBundle#commit(Instant)} is
- * called, checking the value at that time against the value at the time the element was added. All
- * elements added to the bundle will be encoded by the {@link Coder} of the underlying
- * {@link PCollection}.
- *
- * <p>This catches errors during the execution of a {@link DoFn} caused by modifying an element
- * after it is added to an output {@link PCollection}.
- */
-class ImmutabilityCheckingBundleFactory implements BundleFactory {
- /**
- * Create a new {@link ImmutabilityCheckingBundleFactory} that uses the underlying
- * {@link BundleFactory} to create the output bundle.
- */
- public static ImmutabilityCheckingBundleFactory create(BundleFactory underlying) {
- return new ImmutabilityCheckingBundleFactory(underlying);
- }
-
- private final BundleFactory underlying;
-
- private ImmutabilityCheckingBundleFactory(BundleFactory underlying) {
- this.underlying = checkNotNull(underlying);
- }
-
- @Override
- public <T> UncommittedBundle<T> createRootBundle(PCollection<T> output) {
- return new ImmutabilityEnforcingBundle<>(underlying.createRootBundle(output));
- }
-
- @Override
- public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output) {
- return new ImmutabilityEnforcingBundle<>(underlying.createBundle(input, output));
- }
-
- @Override
- public <T> UncommittedBundle<T> createKeyedBundle(
- CommittedBundle<?> input, Object key, PCollection<T> output) {
- return new ImmutabilityEnforcingBundle<>(underlying.createKeyedBundle(input, key, output));
- }
-
- private static class ImmutabilityEnforcingBundle<T> implements UncommittedBundle<T> {
- private final UncommittedBundle<T> underlying;
- private final SetMultimap<WindowedValue<T>, MutationDetector> mutationDetectors;
- private Coder<T> coder;
-
- public ImmutabilityEnforcingBundle(UncommittedBundle<T> underlying) {
- this.underlying = underlying;
- mutationDetectors = HashMultimap.create();
- coder = getPCollection().getCoder();
- }
-
- @Override
- public PCollection<T> getPCollection() {
- return underlying.getPCollection();
- }
-
- @Override
- public UncommittedBundle<T> add(WindowedValue<T> element) {
- try {
- mutationDetectors.put(
- element, MutationDetectors.forValueWithCoder(element.getValue(), coder));
- } catch (CoderException e) {
- throw Throwables.propagate(e);
- }
- underlying.add(element);
- return this;
- }
-
- @Override
- public CommittedBundle<T> commit(Instant synchronizedProcessingTime) {
- for (MutationDetector detector : mutationDetectors.values()) {
- try {
- detector.verifyUnmodified();
- } catch (IllegalMutationException exn) {
- throw UserCodeException.wrap(
- new IllegalMutationException(
- String.format(
- "PTransform %s mutated value %s after it was output (new value was %s)."
- + " Values must not be mutated in any way after being output.",
- underlying.getPCollection().getProducingTransformInternal().getFullName(),
- exn.getSavedValue(),
- exn.getNewValue()),
- exn.getSavedValue(),
- exn.getNewValue(),
- exn));
- }
- }
- return underlying.commit(synchronizedProcessingTime);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactory.java
deleted file mode 100644
index 2f21032..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactory.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.util.IllegalMutationException;
-import org.apache.beam.sdk.util.MutationDetector;
-import org.apache.beam.sdk.util.MutationDetectors;
-import org.apache.beam.sdk.util.UserCodeException;
-import org.apache.beam.sdk.util.WindowedValue;
-
-import java.util.IdentityHashMap;
-import java.util.Map;
-
-/**
- * {@link ModelEnforcement} that enforces elements are not modified over the course of processing
- * an element.
- *
- * <p>Implies {@link EncodabilityEnforcment}.
- */
-class ImmutabilityEnforcementFactory implements ModelEnforcementFactory {
- public static ModelEnforcementFactory create() {
- return new ImmutabilityEnforcementFactory();
- }
-
- @Override
- public <T> ModelEnforcement<T> forBundle(
- CommittedBundle<T> input, AppliedPTransform<?, ?, ?> consumer) {
- return new ImmutabilityCheckingEnforcement<T>(input, consumer);
- }
-
- private static class ImmutabilityCheckingEnforcement<T> extends AbstractModelEnforcement<T> {
- private final AppliedPTransform<?, ?, ?> transform;
- private final Map<WindowedValue<T>, MutationDetector> mutationElements;
- private final Coder<T> coder;
-
- private ImmutabilityCheckingEnforcement(
- CommittedBundle<T> input, AppliedPTransform<?, ?, ?> transform) {
- this.transform = transform;
- coder = input.getPCollection().getCoder();
- mutationElements = new IdentityHashMap<>();
- }
-
- @Override
- public void beforeElement(WindowedValue<T> element) {
- try {
- mutationElements.put(
- element, MutationDetectors.forValueWithCoder(element.getValue(), coder));
- } catch (CoderException e) {
- throw UserCodeException.wrap(e);
- }
- }
-
- @Override
- public void afterElement(WindowedValue<T> element) {
- verifyUnmodified(mutationElements.get(element));
- }
-
- @Override
- public void afterFinish(
- CommittedBundle<T> input,
- InProcessTransformResult result,
- Iterable<? extends CommittedBundle<?>> outputs) {
- for (MutationDetector detector : mutationElements.values()) {
- verifyUnmodified(detector);
- }
- }
-
- private void verifyUnmodified(MutationDetector detector) {
- try {
- detector.verifyUnmodified();
- } catch (IllegalMutationException e) {
- throw new IllegalMutationException(
- String.format(
- "PTransform %s illegaly mutated value %s of class %s."
- + " Input values must not be mutated in any way.",
- transform.getFullName(),
- e.getSavedValue(),
- e.getSavedValue().getClass()),
- e.getSavedValue(),
- e.getNewValue());
- }
- }
- }
-}
[17/17] incubator-beam git commit: This closes #256
Posted by ke...@apache.org.
This closes #256
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/b9116ac4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/b9116ac4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/b9116ac4
Branch: refs/heads/master
Commit: b9116ac426f989af882e6df5dafc5da6c9f203d8
Parents: bba4c64 e13cacb
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Apr 29 14:49:11 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Fri Apr 29 14:49:11 2016 -0700
----------------------------------------------------------------------
.travis.yml | 1 +
runners/direct-java/pom.xml | 400 ++++++
.../direct/AbstractModelEnforcement.java | 38 +
.../direct/AvroIOShardedWriteFactory.java | 76 +
.../direct/BoundedReadEvaluatorFactory.java | 155 ++
.../beam/runners/direct/BundleFactory.java | 49 +
.../CachedThreadPoolExecutorServiceFactory.java | 44 +
.../org/apache/beam/runners/direct/Clock.java | 30 +
.../beam/runners/direct/CommittedResult.java | 46 +
.../beam/runners/direct/CompletionCallback.java | 36 +
.../direct/ConsumerTrackingPipelineVisitor.java | 173 +++
.../runners/direct/EmptyTransformEvaluator.java | 50 +
.../direct/EncodabilityEnforcementFactory.java | 70 +
.../beam/runners/direct/EvaluatorKey.java | 55 +
.../runners/direct/ExecutorServiceFactory.java | 33 +
.../direct/ExecutorServiceParallelExecutor.java | 478 +++++++
.../runners/direct/FlattenEvaluatorFactory.java | 85 ++
.../runners/direct/ForwardingPTransform.java | 62 +
.../direct/GroupByKeyEvaluatorFactory.java | 274 ++++
.../ImmutabilityCheckingBundleFactory.java | 131 ++
.../direct/ImmutabilityEnforcementFactory.java | 103 ++
.../direct/InMemoryWatermarkManager.java | 1327 ++++++++++++++++++
.../runners/direct/InProcessBundleFactory.java | 162 +++
.../direct/InProcessBundleOutputManager.java | 51 +
.../direct/InProcessEvaluationContext.java | 425 ++++++
.../direct/InProcessExecutionContext.java | 105 ++
.../beam/runners/direct/InProcessExecutor.java | 48 +
.../direct/InProcessPipelineOptions.java | 101 ++
.../runners/direct/InProcessPipelineRunner.java | 370 +++++
.../beam/runners/direct/InProcessRegistrar.java | 55 +
.../direct/InProcessSideInputContainer.java | 271 ++++
.../runners/direct/InProcessTimerInternals.java | 84 ++
.../direct/InProcessTransformResult.java | 77 +
.../direct/KeyedPValueTrackingVisitor.java | 96 ++
.../beam/runners/direct/ModelEnforcement.java | 63 +
.../runners/direct/ModelEnforcementFactory.java | 30 +
.../beam/runners/direct/NanosOffsetClock.java | 59 +
.../direct/PTransformOverrideFactory.java | 33 +
.../runners/direct/ParDoInProcessEvaluator.java | 173 +++
.../direct/ParDoMultiEvaluatorFactory.java | 64 +
.../direct/ParDoSingleEvaluatorFactory.java | 63 +
.../direct/PassthroughTransformEvaluator.java | 49 +
.../runners/direct/ShardControlledWrite.java | 81 ++
.../apache/beam/runners/direct/StepAndKey.java | 71 +
.../runners/direct/StepTransformResult.java | 165 +++
.../direct/TextIOShardedWriteFactory.java | 78 +
.../beam/runners/direct/TransformEvaluator.java | 46 +
.../direct/TransformEvaluatorFactory.java | 44 +
.../direct/TransformEvaluatorRegistry.java | 77 +
.../beam/runners/direct/TransformExecutor.java | 176 +++
.../direct/TransformExecutorService.java | 35 +
.../direct/TransformExecutorServices.java | 154 ++
.../direct/UnboundedReadEvaluatorFactory.java | 177 +++
.../runners/direct/ViewEvaluatorFactory.java | 145 ++
.../direct/WatermarkCallbackExecutor.java | 146 ++
.../runners/direct/WindowEvaluatorFactory.java | 131 ++
.../direct/AvroIOShardedWriteFactoryTest.java | 112 ++
.../direct/BoundedReadEvaluatorFactoryTest.java | 290 ++++
.../runners/direct/CommittedResultTest.java | 77 +
.../ConsumerTrackingPipelineVisitorTest.java | 272 ++++
.../EncodabilityEnforcementFactoryTest.java | 257 ++++
.../direct/FlattenEvaluatorFactoryTest.java | 141 ++
.../direct/ForwardingPTransformTest.java | 112 ++
.../direct/GroupByKeyEvaluatorFactoryTest.java | 183 +++
.../ImmutabilityCheckingBundleFactoryTest.java | 220 +++
.../ImmutabilityEnforcementFactoryTest.java | 128 ++
.../direct/InMemoryWatermarkManagerTest.java | 1168 +++++++++++++++
.../direct/InProcessBundleFactoryTest.java | 223 +++
.../direct/InProcessEvaluationContextTest.java | 526 +++++++
.../direct/InProcessPipelineRegistrarTest.java | 74 +
.../direct/InProcessPipelineRunnerTest.java | 78 +
.../direct/InProcessSideInputContainerTest.java | 496 +++++++
.../direct/InProcessTimerInternalsTest.java | 133 ++
.../direct/KeyedPValueTrackingVisitorTest.java | 192 +++
.../apache/beam/runners/direct/MockClock.java | 62 +
.../direct/ParDoMultiEvaluatorFactoryTest.java | 431 ++++++
.../direct/ParDoSingleEvaluatorFactoryTest.java | 324 +++++
.../direct/TextIOShardedWriteFactoryTest.java | 112 ++
.../direct/TransformExecutorServicesTest.java | 136 ++
.../runners/direct/TransformExecutorTest.java | 538 +++++++
.../UnboundedReadEvaluatorFactoryTest.java | 334 +++++
.../direct/ViewEvaluatorFactoryTest.java | 101 ++
.../direct/WatermarkCallbackExecutorTest.java | 128 ++
.../direct/WindowEvaluatorFactoryTest.java | 222 +++
runners/pom.xml | 1 +
.../inprocess/AbstractModelEnforcement.java | 38 -
.../inprocess/AvroIOShardedWriteFactory.java | 76 -
.../inprocess/BoundedReadEvaluatorFactory.java | 155 --
.../sdk/runners/inprocess/BundleFactory.java | 49 -
.../CachedThreadPoolExecutorServiceFactory.java | 44 -
.../beam/sdk/runners/inprocess/Clock.java | 30 -
.../sdk/runners/inprocess/CommittedResult.java | 46 -
.../runners/inprocess/CompletionCallback.java | 36 -
.../ConsumerTrackingPipelineVisitor.java | 173 ---
.../inprocess/EmptyTransformEvaluator.java | 50 -
.../EncodabilityEnforcementFactory.java | 70 -
.../sdk/runners/inprocess/EvaluatorKey.java | 55 -
.../inprocess/ExecutorServiceFactory.java | 33 -
.../ExecutorServiceParallelExecutor.java | 478 -------
.../inprocess/FlattenEvaluatorFactory.java | 85 --
.../runners/inprocess/ForwardingPTransform.java | 62 -
.../inprocess/GroupByKeyEvaluatorFactory.java | 274 ----
.../ImmutabilityCheckingBundleFactory.java | 131 --
.../ImmutabilityEnforcementFactory.java | 103 --
.../inprocess/InMemoryWatermarkManager.java | 1327 ------------------
.../inprocess/InProcessBundleFactory.java | 162 ---
.../inprocess/InProcessBundleOutputManager.java | 51 -
.../inprocess/InProcessEvaluationContext.java | 425 ------
.../inprocess/InProcessExecutionContext.java | 105 --
.../runners/inprocess/InProcessExecutor.java | 48 -
.../inprocess/InProcessPipelineOptions.java | 101 --
.../inprocess/InProcessPipelineRunner.java | 370 -----
.../runners/inprocess/InProcessRegistrar.java | 55 -
.../inprocess/InProcessSideInputContainer.java | 271 ----
.../inprocess/InProcessTimerInternals.java | 84 --
.../inprocess/InProcessTransformResult.java | 77 -
.../inprocess/KeyedPValueTrackingVisitor.java | 96 --
.../sdk/runners/inprocess/ModelEnforcement.java | 63 -
.../inprocess/ModelEnforcementFactory.java | 30 -
.../sdk/runners/inprocess/NanosOffsetClock.java | 59 -
.../inprocess/PTransformOverrideFactory.java | 33 -
.../inprocess/ParDoInProcessEvaluator.java | 173 ---
.../inprocess/ParDoMultiEvaluatorFactory.java | 63 -
.../inprocess/ParDoSingleEvaluatorFactory.java | 63 -
.../PassthroughTransformEvaluator.java | 49 -
.../runners/inprocess/ShardControlledWrite.java | 81 --
.../beam/sdk/runners/inprocess/StepAndKey.java | 71 -
.../runners/inprocess/StepTransformResult.java | 165 ---
.../inprocess/TextIOShardedWriteFactory.java | 78 -
.../runners/inprocess/TransformEvaluator.java | 46 -
.../inprocess/TransformEvaluatorFactory.java | 44 -
.../inprocess/TransformEvaluatorRegistry.java | 77 -
.../runners/inprocess/TransformExecutor.java | 176 ---
.../inprocess/TransformExecutorService.java | 35 -
.../inprocess/TransformExecutorServices.java | 154 --
.../UnboundedReadEvaluatorFactory.java | 177 ---
.../runners/inprocess/ViewEvaluatorFactory.java | 145 --
.../inprocess/WatermarkCallbackExecutor.java | 146 --
.../inprocess/WindowEvaluatorFactory.java | 131 --
.../AvroIOShardedWriteFactoryTest.java | 112 --
.../BoundedReadEvaluatorFactoryTest.java | 290 ----
.../runners/inprocess/CommittedResultTest.java | 77 -
.../ConsumerTrackingPipelineVisitorTest.java | 272 ----
.../EncodabilityEnforcementFactoryTest.java | 257 ----
.../inprocess/FlattenEvaluatorFactoryTest.java | 141 --
.../inprocess/ForwardingPTransformTest.java | 112 --
.../GroupByKeyEvaluatorFactoryTest.java | 183 ---
.../ImmutabilityCheckingBundleFactoryTest.java | 220 ---
.../ImmutabilityEnforcementFactoryTest.java | 128 --
.../inprocess/InMemoryWatermarkManagerTest.java | 1168 ---------------
.../inprocess/InProcessBundleFactoryTest.java | 223 ---
.../InProcessEvaluationContextTest.java | 526 -------
.../InProcessPipelineRegistrarTest.java | 74 -
.../inprocess/InProcessPipelineRunnerTest.java | 78 -
.../InProcessSideInputContainerTest.java | 496 -------
.../inprocess/InProcessTimerInternalsTest.java | 133 --
.../KeyedPValueTrackingVisitorTest.java | 192 ---
.../beam/sdk/runners/inprocess/MockClock.java | 62 -
.../ParDoMultiEvaluatorFactoryTest.java | 431 ------
.../ParDoSingleEvaluatorFactoryTest.java | 324 -----
.../TextIOShardedWriteFactoryTest.java | 112 --
.../TransformExecutorServicesTest.java | 136 --
.../inprocess/TransformExecutorTest.java | 538 -------
.../UnboundedReadEvaluatorFactoryTest.java | 334 -----
.../inprocess/ViewEvaluatorFactoryTest.java | 101 --
.../WatermarkCallbackExecutorTest.java | 128 --
.../inprocess/WindowEvaluatorFactoryTest.java | 222 ---
sdks/java/pom.xml | 1 -
168 files changed, 14692 insertions(+), 14290 deletions(-)
----------------------------------------------------------------------
[06/17] incubator-beam git commit: Move InProcessRunner to its own
module
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineOptions.java
deleted file mode 100644
index bdc525a..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineOptions.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.options.ApplicationNameOptions;
-import org.apache.beam.sdk.options.Default;
-import org.apache.beam.sdk.options.Description;
-import org.apache.beam.sdk.options.Hidden;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.Validation.Required;
-import org.apache.beam.sdk.transforms.PTransform;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-/**
- * Options that can be used to configure the {@link InProcessPipelineRunner}.
- */
-public interface InProcessPipelineOptions extends PipelineOptions, ApplicationNameOptions {
- /**
- * Gets the {@link ExecutorServiceFactory} to use to create instances of {@link ExecutorService}
- * to execute {@link PTransform PTransforms}.
- *
- * <p>Note that {@link ExecutorService ExecutorServices} returned by the factory must ensure that
- * it cannot enter a state in which it will not schedule additional pending work unless currently
- * scheduled work completes, as this may cause the {@link Pipeline} to cease processing.
- *
- * <p>Defaults to a {@link CachedThreadPoolExecutorServiceFactory}, which produces instances of
- * {@link Executors#newCachedThreadPool()}.
- */
- @JsonIgnore
- @Required
- @Hidden
- @Default.InstanceFactory(CachedThreadPoolExecutorServiceFactory.class)
- ExecutorServiceFactory getExecutorServiceFactory();
-
- void setExecutorServiceFactory(ExecutorServiceFactory executorService);
-
- /**
- * Gets the {@link Clock} used by this pipeline. The clock is used in place of accessing the
- * system time when time values are required by the evaluator.
- */
- @Default.InstanceFactory(NanosOffsetClock.Factory.class)
- @JsonIgnore
- @Required
- @Hidden
- @Description(
- "The processing time source used by the pipeline. When the current time is "
- + "needed by the evaluator, the result of clock#now() is used.")
- Clock getClock();
-
- void setClock(Clock clock);
-
- @Default.Boolean(false)
- @Description(
- "If the pipeline should shut down producers which have reached the maximum "
- + "representable watermark. If this is set to true, a pipeline in which all PTransforms "
- + "have reached the maximum watermark will be shut down, even if there are unbounded "
- + "sources that could produce additional (late) data. By default, if the pipeline "
- + "contains any unbounded PCollections, it will run until explicitly shut down.")
- boolean isShutdownUnboundedProducersWithMaxWatermark();
-
- void setShutdownUnboundedProducersWithMaxWatermark(boolean shutdown);
-
- @Default.Boolean(true)
- @Description(
- "If the pipeline should block awaiting completion of the pipeline. If set to true, "
- + "a call to Pipeline#run() will block until all PTransforms are complete. Otherwise, "
- + "the Pipeline will execute asynchronously. If set to false, the completion of the "
- + "pipeline can be awaited on by use of InProcessPipelineResult#awaitCompletion().")
- boolean isBlockOnRun();
-
- void setBlockOnRun(boolean b);
-
- @Default.Boolean(true)
- @Description(
- "Controls whether the runner should ensure that all of the elements of every "
- + "PCollection are not mutated. PTransforms are not permitted to mutate input elements "
- + "at any point, or output elements after they are output.")
- boolean isTestImmutability();
-
- void setTestImmutability(boolean test);
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunner.java
deleted file mode 100644
index 7897f2e..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunner.java
+++ /dev/null
@@ -1,370 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
-import org.apache.beam.sdk.PipelineResult;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.io.AvroIO;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.AggregatorPipelineExtractor;
-import org.apache.beam.sdk.runners.AggregatorRetrievalException;
-import org.apache.beam.sdk.runners.AggregatorValues;
-import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
-import org.apache.beam.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOverrideFactory;
-import org.apache.beam.sdk.runners.inprocess.ViewEvaluatorFactory.InProcessViewOverrideFactory;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.View.CreatePCollectionView;
-import org.apache.beam.sdk.util.MapAggregatorValues;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.UserCodeException;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.common.Counter;
-import org.apache.beam.sdk.util.common.CounterSet;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollection.IsBounded;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-import org.apache.beam.sdk.values.PValue;
-
-import com.google.common.base.Throwables;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-
-import org.joda.time.Instant;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
-
-import javax.annotation.Nullable;
-
-/**
- * An In-Memory implementation of the Dataflow Programming Model. Supports Unbounded
- * {@link PCollection PCollections}.
- */
-@Experimental
-public class InProcessPipelineRunner
- extends PipelineRunner<InProcessPipelineRunner.InProcessPipelineResult> {
- /**
- * The default set of transform overrides to use in the {@link InProcessPipelineRunner}.
- *
- * <p>A transform override must have a single-argument constructor that takes an instance of the
- * type of transform it is overriding.
- */
- @SuppressWarnings("rawtypes")
- private static Map<Class<? extends PTransform>, PTransformOverrideFactory>
- defaultTransformOverrides =
- ImmutableMap.<Class<? extends PTransform>, PTransformOverrideFactory>builder()
- .put(GroupByKey.class, new InProcessGroupByKeyOverrideFactory())
- .put(CreatePCollectionView.class, new InProcessViewOverrideFactory())
- .put(AvroIO.Write.Bound.class, new AvroIOShardedWriteFactory())
- .put(TextIO.Write.Bound.class, new TextIOShardedWriteFactory())
- .build();
-
- /**
- * Part of a {@link PCollection}. Elements are output to a bundle, which will cause them to be
- * executed by {@link PTransform PTransforms} that consume the {@link PCollection} this bundle is
- * a part of at a later point. This is an uncommitted bundle and can have elements added to it.
- *
- * @param <T> the type of elements that can be added to this bundle
- */
- public static interface UncommittedBundle<T> {
- /**
- * Returns the PCollection that the elements of this {@link UncommittedBundle} belong to.
- */
- PCollection<T> getPCollection();
-
- /**
- * Outputs an element to this bundle.
- *
- * @param element the element to add to this bundle
- * @return this bundle
- */
- UncommittedBundle<T> add(WindowedValue<T> element);
-
- /**
- * Commits this {@link UncommittedBundle}, returning an immutable {@link CommittedBundle}
- * containing all of the elements that were added to it. The {@link #add(WindowedValue)} method
- * will throw an {@link IllegalStateException} if called after a call to commit.
- * @param synchronizedProcessingTime the synchronized processing time at which this bundle was
- * committed
- */
- CommittedBundle<T> commit(Instant synchronizedProcessingTime);
- }
-
- /**
- * Part of a {@link PCollection}. Elements are output to an {@link UncommittedBundle}, which will
- * eventually committed. Committed elements are executed by the {@link PTransform PTransforms}
- * that consume the {@link PCollection} this bundle is
- * a part of at a later point.
- * @param <T> the type of elements contained within this bundle
- */
- public static interface CommittedBundle<T> {
- /**
- * Returns the PCollection that the elements of this bundle belong to.
- */
- PCollection<T> getPCollection();
-
- /**
- * Returns the (possibly null) key that was output in the most recent {@link GroupByKey} in the
- * execution of this bundle.
- */
- @Nullable
- Object getKey();
-
- /**
- * Returns an {@link Iterable} containing all of the elements that have been added to this
- * {@link CommittedBundle}.
- */
- Iterable<WindowedValue<T>> getElements();
-
- /**
- * Returns the processing time output watermark at the time the producing {@link PTransform}
- * committed this bundle. Downstream synchronized processing time watermarks cannot progress
- * past this point before consuming this bundle.
- *
- * <p>This value is no greater than the earliest incomplete processing time or synchronized
- * processing time {@link TimerData timer} at the time this bundle was committed, including any
- * timers that fired to produce this bundle.
- */
- Instant getSynchronizedProcessingOutputWatermark();
-
- /**
- * Return a new {@link CommittedBundle} that is like this one, except calls to
- * {@link #getElements()} will return the provided elements. This bundle is unchanged.
- *
- * <p>
- * The value of the {@link #getSynchronizedProcessingOutputWatermark() synchronized processing
- * output watermark} of the returned {@link CommittedBundle} is equal to the value returned from
- * the current bundle. This is used to ensure a {@link PTransform} that could not complete
- * processing on input elements properly holds the synchronized processing time to the
- * appropriate value.
- */
- CommittedBundle<T> withElements(Iterable<WindowedValue<T>> elements);
- }
-
- /**
- * A {@link PCollectionViewWriter} is responsible for writing contents of a {@link PCollection} to
- * a storage mechanism that can be read from while constructing a {@link PCollectionView}.
- * @param <ElemT> the type of elements the input {@link PCollection} contains.
- * @param <ViewT> the type of the PCollectionView this writer writes to.
- */
- public static interface PCollectionViewWriter<ElemT, ViewT> {
- void add(Iterable<WindowedValue<ElemT>> values);
- }
-
- ////////////////////////////////////////////////////////////////////////////////////////////////
- private final InProcessPipelineOptions options;
-
- public static InProcessPipelineRunner fromOptions(PipelineOptions options) {
- return new InProcessPipelineRunner(options.as(InProcessPipelineOptions.class));
- }
-
- private InProcessPipelineRunner(InProcessPipelineOptions options) {
- this.options = options;
- }
-
- /**
- * Returns the {@link PipelineOptions} used to create this {@link InProcessPipelineRunner}.
- */
- public InProcessPipelineOptions getPipelineOptions() {
- return options;
- }
-
- @Override
- public <OutputT extends POutput, InputT extends PInput> OutputT apply(
- PTransform<InputT, OutputT> transform, InputT input) {
- PTransformOverrideFactory overrideFactory = defaultTransformOverrides.get(transform.getClass());
- if (overrideFactory != null) {
- PTransform<InputT, OutputT> customTransform = overrideFactory.override(transform);
-
- return super.apply(customTransform, input);
- }
- // If there is no override, or we should not apply the override, apply the original transform
- return super.apply(transform, input);
- }
-
- @Override
- public InProcessPipelineResult run(Pipeline pipeline) {
- ConsumerTrackingPipelineVisitor consumerTrackingVisitor = new ConsumerTrackingPipelineVisitor();
- pipeline.traverseTopologically(consumerTrackingVisitor);
- for (PValue unfinalized : consumerTrackingVisitor.getUnfinalizedPValues()) {
- unfinalized.finishSpecifying();
- }
- @SuppressWarnings("rawtypes")
- KeyedPValueTrackingVisitor keyedPValueVisitor =
- KeyedPValueTrackingVisitor.create(
- ImmutableSet.<Class<? extends PTransform>>of(
- GroupByKey.class, InProcessGroupByKeyOnly.class));
- pipeline.traverseTopologically(keyedPValueVisitor);
-
- InProcessEvaluationContext context =
- InProcessEvaluationContext.create(
- getPipelineOptions(),
- createBundleFactory(getPipelineOptions()),
- consumerTrackingVisitor.getRootTransforms(),
- consumerTrackingVisitor.getValueToConsumers(),
- consumerTrackingVisitor.getStepNames(),
- consumerTrackingVisitor.getViews());
-
- // independent executor service for each run
- ExecutorService executorService =
- context.getPipelineOptions().getExecutorServiceFactory().create();
- InProcessExecutor executor =
- ExecutorServiceParallelExecutor.create(
- executorService,
- consumerTrackingVisitor.getValueToConsumers(),
- keyedPValueVisitor.getKeyedPValues(),
- TransformEvaluatorRegistry.defaultRegistry(),
- defaultModelEnforcements(options),
- context);
- executor.start(consumerTrackingVisitor.getRootTransforms());
-
- Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps =
- new AggregatorPipelineExtractor(pipeline).getAggregatorSteps();
- InProcessPipelineResult result =
- new InProcessPipelineResult(executor, context, aggregatorSteps);
- if (options.isBlockOnRun()) {
- try {
- result.awaitCompletion();
- } catch (UserCodeException userException) {
- throw new PipelineExecutionException(userException.getCause());
- } catch (Throwable t) {
- Throwables.propagate(t);
- }
- }
- return result;
- }
-
- private Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>>
- defaultModelEnforcements(InProcessPipelineOptions options) {
- ImmutableMap.Builder<Class<? extends PTransform>, Collection<ModelEnforcementFactory>>
- enforcements = ImmutableMap.builder();
- Collection<ModelEnforcementFactory> parDoEnforcements = createParDoEnforcements(options);
- enforcements.put(ParDo.Bound.class, parDoEnforcements);
- enforcements.put(ParDo.BoundMulti.class, parDoEnforcements);
- return enforcements.build();
- }
-
- private Collection<ModelEnforcementFactory> createParDoEnforcements(
- InProcessPipelineOptions options) {
- ImmutableList.Builder<ModelEnforcementFactory> enforcements = ImmutableList.builder();
- if (options.isTestImmutability()) {
- enforcements.add(ImmutabilityEnforcementFactory.create());
- }
- return enforcements.build();
- }
-
- private BundleFactory createBundleFactory(InProcessPipelineOptions pipelineOptions) {
- BundleFactory bundleFactory = InProcessBundleFactory.create();
- if (pipelineOptions.isTestImmutability()) {
- bundleFactory = ImmutabilityCheckingBundleFactory.create(bundleFactory);
- }
- return bundleFactory;
- }
-
- /**
- * The result of running a {@link Pipeline} with the {@link InProcessPipelineRunner}.
- *
- * Throws {@link UnsupportedOperationException} for all methods.
- */
- public static class InProcessPipelineResult implements PipelineResult {
- private final InProcessExecutor executor;
- private final InProcessEvaluationContext evaluationContext;
- private final Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps;
- private State state;
-
- private InProcessPipelineResult(
- InProcessExecutor executor,
- InProcessEvaluationContext evaluationContext,
- Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps) {
- this.executor = executor;
- this.evaluationContext = evaluationContext;
- this.aggregatorSteps = aggregatorSteps;
- // Only ever constructed after the executor has started.
- this.state = State.RUNNING;
- }
-
- @Override
- public State getState() {
- return state;
- }
-
- @Override
- public <T> AggregatorValues<T> getAggregatorValues(Aggregator<?, T> aggregator)
- throws AggregatorRetrievalException {
- CounterSet counters = evaluationContext.getCounters();
- Collection<PTransform<?, ?>> steps = aggregatorSteps.get(aggregator);
- Map<String, T> stepValues = new HashMap<>();
- for (AppliedPTransform<?, ?, ?> transform : evaluationContext.getSteps()) {
- if (steps.contains(transform.getTransform())) {
- String stepName =
- String.format(
- "user-%s-%s", evaluationContext.getStepName(transform), aggregator.getName());
- Counter<T> counter = (Counter<T>) counters.getExistingCounter(stepName);
- if (counter != null) {
- stepValues.put(transform.getFullName(), counter.getAggregate());
- }
- }
- }
- return new MapAggregatorValues<>(stepValues);
- }
-
- /**
- * Blocks until the {@link Pipeline} execution represented by this
- * {@link InProcessPipelineResult} is complete, returning the terminal state.
- *
- * <p>If the pipeline terminates abnormally by throwing an exception, this will rethrow the
- * exception. Future calls to {@link #getState()} will return
- * {@link org.apache.beam.sdk.PipelineResult.State#FAILED}.
- *
- * <p>NOTE: if the {@link Pipeline} contains an {@link IsBounded#UNBOUNDED unbounded}
- * {@link PCollection}, and the {@link PipelineRunner} was created with
- * {@link InProcessPipelineOptions#isShutdownUnboundedProducersWithMaxWatermark()} set to false,
- * this method will never return.
- *
- * See also {@link InProcessExecutor#awaitCompletion()}.
- */
- public State awaitCompletion() throws Throwable {
- if (!state.isTerminal()) {
- try {
- executor.awaitCompletion();
- state = State.DONE;
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw e;
- } catch (Throwable t) {
- state = State.FAILED;
- throw t;
- }
- }
- return state;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessRegistrar.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessRegistrar.java
deleted file mode 100644
index 8d29d01..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessRegistrar.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
-import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
-
-import com.google.auto.service.AutoService;
-import com.google.common.collect.ImmutableList;
-
-/**
- * Contains the {@link PipelineRunnerRegistrar} and {@link PipelineOptionsRegistrar} for the
- * {@link InProcessPipelineRunner}.
- */
-public class InProcessRegistrar {
- private InProcessRegistrar() {}
- /**
- * Registers the {@link InProcessPipelineRunner}.
- */
- @AutoService(PipelineRunnerRegistrar.class)
- public static class InProcessRunner implements PipelineRunnerRegistrar {
- @Override
- public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
- return ImmutableList.<Class<? extends PipelineRunner<?>>>of(InProcessPipelineRunner.class);
- }
- }
-
- /**
- * Registers the {@link InProcessPipelineOptions}.
- */
- @AutoService(PipelineOptionsRegistrar.class)
- public static class InProcessOptions implements PipelineOptionsRegistrar {
- @Override
- public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
- return ImmutableList.<Class<? extends PipelineOptions>>of(InProcessPipelineOptions.class);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainer.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainer.java
deleted file mode 100644
index fda78fc..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainer.java
+++ /dev/null
@@ -1,271 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessEvaluationContext.ReadyCheckingSideInputReader;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.PCollectionViewWindow;
-import org.apache.beam.sdk.util.SideInputReader;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.values.PCollectionView;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Throwables;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.SettableFuture;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-
-import javax.annotation.Nullable;
-
-/**
- * An in-process container for {@link PCollectionView PCollectionViews}, which provides methods for
- * constructing {@link SideInputReader SideInputReaders} which block until a side input is
- * available and writing to a {@link PCollectionView}.
- */
-class InProcessSideInputContainer {
- private final InProcessEvaluationContext evaluationContext;
- private final Collection<PCollectionView<?>> containedViews;
- private final LoadingCache<PCollectionViewWindow<?>,
- SettableFuture<Iterable<? extends WindowedValue<?>>>> viewByWindows;
-
- /**
- * Create a new {@link InProcessSideInputContainer} with the provided views and the provided
- * context.
- */
- public static InProcessSideInputContainer create(
- InProcessEvaluationContext context, Collection<PCollectionView<?>> containedViews) {
- CacheLoader<PCollectionViewWindow<?>, SettableFuture<Iterable<? extends WindowedValue<?>>>>
- loader = new CacheLoader<PCollectionViewWindow<?>,
- SettableFuture<Iterable<? extends WindowedValue<?>>>>() {
- @Override
- public SettableFuture<Iterable<? extends WindowedValue<?>>> load(
- PCollectionViewWindow<?> view) {
- return SettableFuture.create();
- }
- };
- LoadingCache<PCollectionViewWindow<?>, SettableFuture<Iterable<? extends WindowedValue<?>>>>
- viewByWindows = CacheBuilder.newBuilder().build(loader);
- return new InProcessSideInputContainer(context, containedViews, viewByWindows);
- }
-
- private InProcessSideInputContainer(InProcessEvaluationContext context,
- Collection<PCollectionView<?>> containedViews,
- LoadingCache<PCollectionViewWindow<?>, SettableFuture<Iterable<? extends WindowedValue<?>>>>
- viewByWindows) {
- this.evaluationContext = context;
- this.containedViews = ImmutableSet.copyOf(containedViews);
- this.viewByWindows = viewByWindows;
- }
-
- /**
- * Return a view of this {@link InProcessSideInputContainer} that contains only the views in the
- * provided argument. The returned {@link InProcessSideInputContainer} is unmodifiable without
- * casting, but will change as this {@link InProcessSideInputContainer} is modified.
- */
- public ReadyCheckingSideInputReader createReaderForViews(
- Collection<PCollectionView<?>> newContainedViews) {
- if (!containedViews.containsAll(newContainedViews)) {
- Set<PCollectionView<?>> currentlyContained = ImmutableSet.copyOf(containedViews);
- Set<PCollectionView<?>> newRequested = ImmutableSet.copyOf(newContainedViews);
- throw new IllegalArgumentException("Can't create a SideInputReader with unknown views "
- + Sets.difference(newRequested, currentlyContained));
- }
- return new SideInputContainerSideInputReader(newContainedViews);
- }
-
- /**
- * Write the provided values to the provided view.
- *
- * <p>The windowed values are first exploded, then for each window the pane is determined. For
- * each window, if the pane is later than the current pane stored within this container, write
- * all of the values to the container as the new values of the {@link PCollectionView}.
- *
- * <p>The provided iterable is expected to contain only a single window and pane.
- */
- public void write(PCollectionView<?> view, Iterable<? extends WindowedValue<?>> values) {
- Map<BoundedWindow, Collection<WindowedValue<?>>> valuesPerWindow =
- indexValuesByWindow(values);
- for (Map.Entry<BoundedWindow, Collection<WindowedValue<?>>> windowValues :
- valuesPerWindow.entrySet()) {
- updatePCollectionViewWindowValues(view, windowValues.getKey(), windowValues.getValue());
- }
- }
-
- /**
- * Index the provided values by all {@link BoundedWindow windows} in which they appear.
- */
- private Map<BoundedWindow, Collection<WindowedValue<?>>> indexValuesByWindow(
- Iterable<? extends WindowedValue<?>> values) {
- Map<BoundedWindow, Collection<WindowedValue<?>>> valuesPerWindow = new HashMap<>();
- for (WindowedValue<?> value : values) {
- for (BoundedWindow window : value.getWindows()) {
- Collection<WindowedValue<?>> windowValues = valuesPerWindow.get(window);
- if (windowValues == null) {
- windowValues = new ArrayList<>();
- valuesPerWindow.put(window, windowValues);
- }
- windowValues.add(value);
- }
- }
- return valuesPerWindow;
- }
-
- /**
- * Set the value of the {@link PCollectionView} in the {@link BoundedWindow} to be based on the
- * specified values, if the values are part of a later pane than currently exist within the
- * {@link PCollectionViewWindow}.
- */
- private void updatePCollectionViewWindowValues(
- PCollectionView<?> view, BoundedWindow window, Collection<WindowedValue<?>> windowValues) {
- PCollectionViewWindow<?> windowedView = PCollectionViewWindow.of(view, window);
- SettableFuture<Iterable<? extends WindowedValue<?>>> future = null;
- try {
- future = viewByWindows.get(windowedView);
- if (future.isDone()) {
- Iterator<? extends WindowedValue<?>> existingValues = future.get().iterator();
- PaneInfo newPane = windowValues.iterator().next().getPane();
- // The current value may have no elements, if no elements were produced for the window,
- // but we are recieving late data.
- if (!existingValues.hasNext()
- || newPane.getIndex() > existingValues.next().getPane().getIndex()) {
- viewByWindows.invalidate(windowedView);
- viewByWindows.get(windowedView).set(windowValues);
- }
- } else {
- future.set(windowValues);
- }
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- if (future != null && !future.isDone()) {
- future.set(Collections.<WindowedValue<?>>emptyList());
- }
- } catch (ExecutionException e) {
- Throwables.propagate(e.getCause());
- }
- }
-
- private final class SideInputContainerSideInputReader implements ReadyCheckingSideInputReader {
- private final Collection<PCollectionView<?>> readerViews;
-
- private SideInputContainerSideInputReader(Collection<PCollectionView<?>> readerViews) {
- this.readerViews = ImmutableSet.copyOf(readerViews);
- }
-
- @Override
- public boolean isReady(final PCollectionView<?> view, final BoundedWindow window) {
- checkArgument(
- readerViews.contains(view),
- "Tried to check if view %s was ready in a SideInputReader that does not contain it. "
- + "Contained views; %s",
- view,
- readerViews);
- return getViewFuture(view, window).isDone();
- }
-
- @Override
- @Nullable
- public <T> T get(final PCollectionView<T> view, final BoundedWindow window) {
- checkArgument(
- readerViews.contains(view), "calling get(PCollectionView) with unknown view: " + view);
- try {
- final Future<Iterable<? extends WindowedValue<?>>> future = getViewFuture(view, window);
- // Safe covariant cast
- @SuppressWarnings("unchecked")
- Iterable<WindowedValue<?>> values = (Iterable<WindowedValue<?>>) future.get();
- return view.fromIterableInternal(values);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- return null;
- } catch (ExecutionException e) {
- throw new RuntimeException(e);
- }
- }
-
- /**
- * Gets the future containing the contents of the provided {@link PCollectionView} in the
- * provided {@link BoundedWindow}, setting up a callback to populate the future with empty
- * contents if necessary.
- */
- private <T> Future<Iterable<? extends WindowedValue<?>>> getViewFuture(
- final PCollectionView<T> view, final BoundedWindow window) {
- PCollectionViewWindow<T> windowedView = PCollectionViewWindow.of(view, window);
- final SettableFuture<Iterable<? extends WindowedValue<?>>> future =
- viewByWindows.getUnchecked(windowedView);
-
- WindowingStrategy<?, ?> windowingStrategy = view.getWindowingStrategyInternal();
- evaluationContext.scheduleAfterOutputWouldBeProduced(
- view, window, windowingStrategy, new WriteEmptyViewContents(view, window, future));
- return future;
- }
-
- @Override
- public <T> boolean contains(PCollectionView<T> view) {
- return readerViews.contains(view);
- }
-
- @Override
- public boolean isEmpty() {
- return readerViews.isEmpty();
- }
- }
-
- private static class WriteEmptyViewContents implements Runnable {
- private final PCollectionView<?> view;
- private final BoundedWindow window;
- private final SettableFuture<Iterable<? extends WindowedValue<?>>> future;
-
- private WriteEmptyViewContents(PCollectionView<?> view, BoundedWindow window,
- SettableFuture<Iterable<? extends WindowedValue<?>>> future) {
- this.future = future;
- this.view = view;
- this.window = window;
- }
-
- @Override
- public void run() {
- // The requested window has closed without producing elements, so reflect that in
- // the PCollectionView. If set has already been called, will do nothing.
- future.set(Collections.<WindowedValue<?>>emptyList());
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(this)
- .add("view", view)
- .add("window", window)
- .toString();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternals.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternals.java
deleted file mode 100644
index 3422efd..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternals.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate.TimerUpdateBuilder;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
-import org.apache.beam.sdk.util.TimerInternals;
-
-import org.joda.time.Instant;
-
-import javax.annotation.Nullable;
-
-/**
- * An implementation of {@link TimerInternals} where all relevant data exists in memory.
- */
-public class InProcessTimerInternals implements TimerInternals {
- private final Clock processingTimeClock;
- private final TransformWatermarks watermarks;
- private final TimerUpdateBuilder timerUpdateBuilder;
-
- public static InProcessTimerInternals create(
- Clock clock, TransformWatermarks watermarks, TimerUpdateBuilder timerUpdateBuilder) {
- return new InProcessTimerInternals(clock, watermarks, timerUpdateBuilder);
- }
-
- private InProcessTimerInternals(
- Clock clock, TransformWatermarks watermarks, TimerUpdateBuilder timerUpdateBuilder) {
- this.processingTimeClock = clock;
- this.watermarks = watermarks;
- this.timerUpdateBuilder = timerUpdateBuilder;
- }
-
- @Override
- public void setTimer(TimerData timerKey) {
- timerUpdateBuilder.setTimer(timerKey);
- }
-
- @Override
- public void deleteTimer(TimerData timerKey) {
- timerUpdateBuilder.deletedTimer(timerKey);
- }
-
- public TimerUpdate getTimerUpdate() {
- return timerUpdateBuilder.build();
- }
-
- @Override
- public Instant currentProcessingTime() {
- return processingTimeClock.now();
- }
-
- @Override
- @Nullable
- public Instant currentSynchronizedProcessingTime() {
- return watermarks.getSynchronizedProcessingInputTime();
- }
-
- @Override
- public Instant currentInputWatermarkTime() {
- return watermarks.getInputWatermark();
- }
-
- @Override
- @Nullable
- public Instant currentOutputWatermarkTime() {
- return watermarks.getOutputWatermark();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTransformResult.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTransformResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTransformResult.java
deleted file mode 100644
index ed77f70..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTransformResult.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.common.CounterSet;
-import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-
-import org.joda.time.Instant;
-
-import javax.annotation.Nullable;
-
-/**
- * The result of evaluating an {@link AppliedPTransform} with a {@link TransformEvaluator}.
- */
-public interface InProcessTransformResult {
- /**
- * Returns the {@link AppliedPTransform} that produced this result.
- */
- AppliedPTransform<?, ?, ?> getTransform();
-
- /**
- * Returns the {@link UncommittedBundle (uncommitted) Bundles} output by this transform. These
- * will be committed by the evaluation context as part of completing this result.
- */
- Iterable<? extends UncommittedBundle<?>> getOutputBundles();
-
- /**
- * Returns the {@link CounterSet} used by this {@link PTransform}, or null if this transform did
- * not use a {@link CounterSet}.
- */
- @Nullable CounterSet getCounters();
-
- /**
- * Returns the Watermark Hold for the transform at the time this result was produced.
- *
- * If the transform does not set any watermark hold, returns
- * {@link BoundedWindow#TIMESTAMP_MAX_VALUE}.
- */
- Instant getWatermarkHold();
-
- /**
- * Returns the State used by the transform.
- *
- * If this evaluation did not access state, this may return null.
- */
- @Nullable
- CopyOnAccessInMemoryStateInternals<?> getState();
-
- /**
- * Returns a TimerUpdateBuilder that was produced as a result of this evaluation. If the
- * evaluation was triggered due to the delivery of one or more timers, those timers must be added
- * to the builder before it is complete.
- *
- * <p>If this evaluation did not add or remove any timers, returns an empty TimerUpdate.
- */
- TimerUpdate getTimerUpdate();
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitor.java
deleted file mode 100644
index 0e6b7e8..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitor.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import org.apache.beam.sdk.Pipeline.PipelineVisitor;
-import org.apache.beam.sdk.runners.TransformTreeNode;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PValue;
-
-import java.util.HashSet;
-import java.util.Set;
-
-/**
- * A pipeline visitor that tracks all keyed {@link PValue PValues}. A {@link PValue} is keyed if it
- * is the result of a {@link PTransform} that produces keyed outputs. A {@link PTransform} that
- * produces keyed outputs is assumed to colocate output elements that share a key.
- *
- * <p>All {@link GroupByKey} transforms, or their runner-specific implementation primitive, produce
- * keyed output.
- */
-// TODO: Handle Key-preserving transforms when appropriate and more aggressively make PTransforms
-// unkeyed
-class KeyedPValueTrackingVisitor implements PipelineVisitor {
- @SuppressWarnings("rawtypes")
- private final Set<Class<? extends PTransform>> producesKeyedOutputs;
- private final Set<PValue> keyedValues;
- private boolean finalized;
-
- public static KeyedPValueTrackingVisitor create(
- @SuppressWarnings("rawtypes") Set<Class<? extends PTransform>> producesKeyedOutputs) {
- return new KeyedPValueTrackingVisitor(producesKeyedOutputs);
- }
-
- private KeyedPValueTrackingVisitor(
- @SuppressWarnings("rawtypes") Set<Class<? extends PTransform>> producesKeyedOutputs) {
- this.producesKeyedOutputs = producesKeyedOutputs;
- this.keyedValues = new HashSet<>();
- }
-
- @Override
- public void enterCompositeTransform(TransformTreeNode node) {
- checkState(
- !finalized,
- "Attempted to use a %s that has already been finalized on a pipeline (visiting node %s)",
- KeyedPValueTrackingVisitor.class.getSimpleName(),
- node);
- }
-
- @Override
- public void leaveCompositeTransform(TransformTreeNode node) {
- checkState(
- !finalized,
- "Attempted to use a %s that has already been finalized on a pipeline (visiting node %s)",
- KeyedPValueTrackingVisitor.class.getSimpleName(),
- node);
- if (node.isRootNode()) {
- finalized = true;
- } else if (producesKeyedOutputs.contains(node.getTransform().getClass())) {
- keyedValues.addAll(node.getExpandedOutputs());
- }
- }
-
- @Override
- public void visitTransform(TransformTreeNode node) {}
-
- @Override
- public void visitValue(PValue value, TransformTreeNode producer) {
- if (producesKeyedOutputs.contains(producer.getTransform().getClass())) {
- keyedValues.addAll(value.expand());
- }
- }
-
- public Set<PValue> getKeyedPValues() {
- checkState(
- finalized, "can't call getKeyedPValues before a Pipeline has been completely traversed");
- return keyedValues;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcement.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcement.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcement.java
deleted file mode 100644
index 4a3d17a..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcement.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * Enforcement tools that verify that executing code conforms to the model.
- *
- * <p>ModelEnforcement is performed on a per-element and per-bundle basis. The
- * {@link ModelEnforcement} is provided with the input bundle as part of
- * {@link ModelEnforcementFactory#forBundle(CommittedBundle, AppliedPTransform)}, each element
- * before and after that element is provided to an underlying {@link TransformEvaluator}, and the
- * output {@link InProcessTransformResult} and committed output bundles after the
- * {@link TransformEvaluator} has completed.
- *
- * <p>Typically, {@link ModelEnforcement} will obtain required metadata (such as the {@link Coder}
- * of the input {@link PCollection} on construction, and then enforce per-element behavior
- * (such as the immutability of input elements). When the element is output or the bundle is
- * completed, the required conditions can be enforced across all elements.
- */
-public interface ModelEnforcement<T> {
- /**
- * Called before a call to {@link TransformEvaluator#processElement(WindowedValue)} on the
- * provided {@link WindowedValue}.
- */
- void beforeElement(WindowedValue<T> element);
-
- /**
- * Called after a call to {@link TransformEvaluator#processElement(WindowedValue)} on the
- * provided {@link WindowedValue}.
- */
- void afterElement(WindowedValue<T> element);
-
- /**
- * Called after a bundle has been completed and {@link TransformEvaluator#finishBundle()} has been
- * called, producing the provided {@link InProcessTransformResult} and
- * {@link CommittedBundle output bundles}.
- */
- void afterFinish(
- CommittedBundle<T> input,
- InProcessTransformResult result,
- Iterable<? extends CommittedBundle<?>> outputs);
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcementFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcementFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcementFactory.java
deleted file mode 100644
index 1fa36d6..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcementFactory.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-
-/**
- * Creates {@link ModelEnforcement} instances for an {@link AppliedPTransform} on an input
- * {@link CommittedBundle bundle}. {@link ModelEnforcement} instances are created before the
- * {@link TransformEvaluator} is created.
- */
-public interface ModelEnforcementFactory {
- <T> ModelEnforcement<T> forBundle(CommittedBundle<T> input, AppliedPTransform<?, ?, ?> consumer);
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/NanosOffsetClock.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/NanosOffsetClock.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/NanosOffsetClock.java
deleted file mode 100644
index 71039fa..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/NanosOffsetClock.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.options.DefaultValueFactory;
-import org.apache.beam.sdk.options.PipelineOptions;
-
-import org.joda.time.Instant;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * A {@link Clock} that uses {@link System#nanoTime()} to track the progress of time.
- */
-public class NanosOffsetClock implements Clock {
- private final long baseMillis;
- private final long nanosAtBaseMillis;
-
- public static NanosOffsetClock create() {
- return new NanosOffsetClock();
- }
-
- private NanosOffsetClock() {
- baseMillis = System.currentTimeMillis();
- nanosAtBaseMillis = System.nanoTime();
- }
-
- @Override
- public Instant now() {
- return new Instant(
- baseMillis + (TimeUnit.MILLISECONDS.convert(
- System.nanoTime() - nanosAtBaseMillis, TimeUnit.NANOSECONDS)));
- }
-
- /**
- * Creates instances of {@link NanosOffsetClock}.
- */
- public static class Factory implements DefaultValueFactory<Clock> {
- @Override
- public Clock create(PipelineOptions options) {
- return new NanosOffsetClock();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PTransformOverrideFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PTransformOverrideFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PTransformOverrideFactory.java
deleted file mode 100644
index 2b4bf09..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PTransformOverrideFactory.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-
-interface PTransformOverrideFactory {
- /**
- * Create a {@link PTransform} override for the provided {@link PTransform} if applicable.
- * Otherwise, return the input {@link PTransform}.
- *
- * <p>The returned PTransform must be semantically equivalent to the input {@link PTransform}.
- */
- <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
- PTransform<InputT, OutputT> transform);
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoInProcessEvaluator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoInProcessEvaluator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoInProcessEvaluator.java
deleted file mode 100644
index 35639bd..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoInProcessEvaluator.java
+++ /dev/null
@@ -1,173 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessExecutionContext.InProcessStepContext;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.util.DoFnRunner;
-import org.apache.beam.sdk.util.DoFnRunners;
-import org.apache.beam.sdk.util.DoFnRunners.OutputManager;
-import org.apache.beam.sdk.util.SerializableUtils;
-import org.apache.beam.sdk.util.UserCodeException;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.common.CounterSet;
-import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.TupleTag;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-class ParDoInProcessEvaluator<T> implements TransformEvaluator<T> {
- public static <InputT, OutputT> ParDoInProcessEvaluator<InputT> create(
- InProcessEvaluationContext evaluationContext,
- CommittedBundle<InputT> inputBundle,
- AppliedPTransform<PCollection<InputT>, ?, ?> application,
- DoFn<InputT, OutputT> fn,
- List<PCollectionView<?>> sideInputs,
- TupleTag<OutputT> mainOutputTag,
- List<TupleTag<?>> sideOutputTags,
- Map<TupleTag<?>, PCollection<?>> outputs) {
- InProcessExecutionContext executionContext =
- evaluationContext.getExecutionContext(application, inputBundle.getKey());
- String stepName = evaluationContext.getStepName(application);
- InProcessStepContext stepContext =
- executionContext.getOrCreateStepContext(stepName, stepName);
-
- CounterSet counters = evaluationContext.createCounterSet();
-
- Map<TupleTag<?>, UncommittedBundle<?>> outputBundles = new HashMap<>();
- for (Map.Entry<TupleTag<?>, PCollection<?>> outputEntry : outputs.entrySet()) {
- outputBundles.put(
- outputEntry.getKey(),
- evaluationContext.createBundle(inputBundle, outputEntry.getValue()));
- }
-
- DoFnRunner<InputT, OutputT> runner =
- DoFnRunners.createDefault(
- evaluationContext.getPipelineOptions(),
- SerializableUtils.clone(fn),
- evaluationContext.createSideInputReader(sideInputs),
- BundleOutputManager.create(outputBundles),
- mainOutputTag,
- sideOutputTags,
- stepContext,
- counters.getAddCounterMutator(),
- application.getInput().getWindowingStrategy());
-
- try {
- runner.startBundle();
- } catch (Exception e) {
- throw UserCodeException.wrap(e);
- }
-
- return new ParDoInProcessEvaluator<>(
- runner, application, counters, outputBundles.values(), stepContext);
- }
-
- ////////////////////////////////////////////////////////////////////////////////////////////////
-
- private final DoFnRunner<T, ?> fnRunner;
- private final AppliedPTransform<PCollection<T>, ?, ?> transform;
- private final CounterSet counters;
- private final Collection<UncommittedBundle<?>> outputBundles;
- private final InProcessStepContext stepContext;
-
- private ParDoInProcessEvaluator(
- DoFnRunner<T, ?> fnRunner,
- AppliedPTransform<PCollection<T>, ?, ?> transform,
- CounterSet counters,
- Collection<UncommittedBundle<?>> outputBundles,
- InProcessStepContext stepContext) {
- this.fnRunner = fnRunner;
- this.transform = transform;
- this.counters = counters;
- this.outputBundles = outputBundles;
- this.stepContext = stepContext;
- }
-
- @Override
- public void processElement(WindowedValue<T> element) {
- try {
- fnRunner.processElement(element);
- } catch (Exception e) {
- throw UserCodeException.wrap(e);
- }
- }
-
- @Override
- public InProcessTransformResult finishBundle() {
- try {
- fnRunner.finishBundle();
- } catch (Exception e) {
- throw UserCodeException.wrap(e);
- }
- StepTransformResult.Builder resultBuilder;
- CopyOnAccessInMemoryStateInternals<?> state = stepContext.commitState();
- if (state != null) {
- resultBuilder =
- StepTransformResult.withHold(transform, state.getEarliestWatermarkHold())
- .withState(state);
- } else {
- resultBuilder = StepTransformResult.withoutHold(transform);
- }
- return resultBuilder
- .addOutput(outputBundles)
- .withTimerUpdate(stepContext.getTimerUpdate())
- .withCounters(counters)
- .build();
- }
-
- static class BundleOutputManager implements OutputManager {
- private final Map<TupleTag<?>, UncommittedBundle<?>> bundles;
- private final Map<TupleTag<?>, List<?>> undeclaredOutputs;
-
- public static BundleOutputManager create(Map<TupleTag<?>, UncommittedBundle<?>> outputBundles) {
- return new BundleOutputManager(outputBundles);
- }
-
- private BundleOutputManager(Map<TupleTag<?>, UncommittedBundle<?>> bundles) {
- this.bundles = bundles;
- undeclaredOutputs = new HashMap<>();
- }
-
- @SuppressWarnings("unchecked")
- @Override
- public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
- @SuppressWarnings("rawtypes")
- UncommittedBundle bundle = bundles.get(tag);
- if (bundle == null) {
- List undeclaredContents = undeclaredOutputs.get(tag);
- if (undeclaredContents == null) {
- undeclaredContents = new ArrayList<T>();
- undeclaredOutputs.put(tag, undeclaredContents);
- }
- undeclaredContents.add(output);
- } else {
- bundle.add(output);
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactory.java
deleted file mode 100644
index 299d3a8..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactory.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo.BoundMulti;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-
-import java.util.Map;
-
-/**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
- * {@link BoundMulti} primitive {@link PTransform}.
- */
-class ParDoMultiEvaluatorFactory implements TransformEvaluatorFactory {
- @Override
- public <T> TransformEvaluator<T> forApplication(
- AppliedPTransform<?, ?, ?> application,
- CommittedBundle<?> inputBundle,
- InProcessEvaluationContext evaluationContext) {
- @SuppressWarnings({"unchecked", "rawtypes"})
- TransformEvaluator<T> evaluator =
- createMultiEvaluator((AppliedPTransform) application, inputBundle, evaluationContext);
- return evaluator;
- }
-
- private static <InT, OuT> ParDoInProcessEvaluator<InT> createMultiEvaluator(
- AppliedPTransform<PCollection<InT>, PCollectionTuple, BoundMulti<InT, OuT>> application,
- CommittedBundle<InT> inputBundle,
- InProcessEvaluationContext evaluationContext) {
- Map<TupleTag<?>, PCollection<?>> outputs = application.getOutput().getAll();
- DoFn<InT, OuT> fn = application.getTransform().getFn();
-
- return ParDoInProcessEvaluator.create(
- evaluationContext,
- inputBundle,
- application,
- fn,
- application.getTransform().getSideInputs(),
- application.getTransform().getMainOutputTag(),
- application.getTransform().getSideOutputTags().getAll(),
- outputs);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactory.java
deleted file mode 100644
index 4d38448..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactory.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo.Bound;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.TupleTag;
-
-import com.google.common.collect.ImmutableMap;
-
-import java.util.Collections;
-
-/**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
- * {@link Bound ParDo.Bound} primitive {@link PTransform}.
- */
-class ParDoSingleEvaluatorFactory implements TransformEvaluatorFactory {
- @Override
- public <T> TransformEvaluator<T> forApplication(
- final AppliedPTransform<?, ?, ?> application,
- CommittedBundle<?> inputBundle,
- InProcessEvaluationContext evaluationContext) {
- @SuppressWarnings({"unchecked", "rawtypes"})
- TransformEvaluator<T> evaluator =
- createSingleEvaluator((AppliedPTransform) application, inputBundle, evaluationContext);
- return evaluator;
- }
-
- private static <InputT, OutputT> ParDoInProcessEvaluator<InputT> createSingleEvaluator(
- @SuppressWarnings("rawtypes") AppliedPTransform<PCollection<InputT>, PCollection<OutputT>,
- Bound<InputT, OutputT>> application,
- CommittedBundle<InputT> inputBundle, InProcessEvaluationContext evaluationContext) {
- TupleTag<OutputT> mainOutputTag = new TupleTag<>("out");
-
- return ParDoInProcessEvaluator.create(
- evaluationContext,
- inputBundle,
- application,
- application.getTransform().getFn(),
- application.getTransform().getSideInputs(),
- mainOutputTag,
- Collections.<TupleTag<?>>emptyList(),
- ImmutableMap.<TupleTag<?>, PCollection<?>>of(mainOutputTag, application.getOutput()));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PassthroughTransformEvaluator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PassthroughTransformEvaluator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PassthroughTransformEvaluator.java
deleted file mode 100644
index a90cd7b..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PassthroughTransformEvaluator.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.util.WindowedValue;
-
-class PassthroughTransformEvaluator<InputT> implements TransformEvaluator<InputT> {
- public static <InputT> PassthroughTransformEvaluator<InputT> create(
- AppliedPTransform<?, ?, ?> transform, UncommittedBundle<InputT> output) {
- return new PassthroughTransformEvaluator<>(transform, output);
- }
-
- private final AppliedPTransform<?, ?, ?> transform;
- private final UncommittedBundle<InputT> output;
-
- private PassthroughTransformEvaluator(
- AppliedPTransform<?, ?, ?> transform, UncommittedBundle<InputT> output) {
- this.transform = transform;
- this.output = output;
- }
-
- @Override
- public void processElement(WindowedValue<InputT> element) throws Exception {
- output.add(element);
- }
-
- @Override
- public InProcessTransformResult finishBundle() throws Exception {
- return StepTransformResult.withoutHold(transform).addOutput(output).build();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ShardControlledWrite.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ShardControlledWrite.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ShardControlledWrite.java
deleted file mode 100644
index 88630ad..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ShardControlledWrite.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.Partition;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionList;
-import org.apache.beam.sdk.values.PDone;
-
-import java.util.concurrent.ThreadLocalRandom;
-
-/**
- * A write that explicitly controls its number of output shards.
- */
-abstract class ShardControlledWrite<InputT>
- extends ForwardingPTransform<PCollection<InputT>, PDone> {
- @Override
- public PDone apply(PCollection<InputT> input) {
- int numShards = getNumShards();
- checkArgument(
- numShards >= 1,
- "%s should only be applied if the output has a controlled number of shards (> 1); got %s",
- getClass().getSimpleName(),
- getNumShards());
- PCollectionList<InputT> shards =
- input.apply(
- "PartitionInto" + numShards + "Shards",
- Partition.of(getNumShards(), new RandomSeedPartitionFn<InputT>()));
- for (int i = 0; i < shards.size(); i++) {
- PCollection<InputT> shard = shards.get(i);
- PTransform<? super PCollection<InputT>, PDone> writeShard = getSingleShardTransform(i);
- shard.apply(String.format("%s(Shard:%s)", writeShard.getName(), i), writeShard);
- }
- return PDone.in(input.getPipeline());
- }
-
- /**
- * Returns the number of shards this {@link PTransform} should write to.
- */
- abstract int getNumShards();
-
- /**
- * Returns a {@link PTransform} that performs a write to the shard with the specified shard
- * number.
- *
- * <p>This method will be called n times, where n is the value of {@link #getNumShards()}, for
- * shard numbers {@code [0...n)}.
- */
- abstract PTransform<? super PCollection<InputT>, PDone> getSingleShardTransform(int shardNum);
-
- private static class RandomSeedPartitionFn<T> implements Partition.PartitionFn<T> {
- int nextPartition = -1;
- @Override
- public int partitionFor(T elem, int numPartitions) {
- if (nextPartition < 0) {
- nextPartition = ThreadLocalRandom.current().nextInt(numPartitions);
- }
- nextPartition++;
- nextPartition %= numPartitions;
- return nextPartition;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepAndKey.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepAndKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepAndKey.java
deleted file mode 100644
index 9c4d9aa..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepAndKey.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-
-import com.google.common.base.MoreObjects;
-
-import java.util.Objects;
-
-/**
- * A (Step, Key) pair. This is useful as a map key or cache key for things that are available
- * per-step in a keyed manner (e.g. State).
- */
-final class StepAndKey {
- private final AppliedPTransform<?, ?, ?> step;
- private final Object key;
-
- /**
- * Create a new {@link StepAndKey} with the provided step and key.
- */
- public static StepAndKey of(AppliedPTransform<?, ?, ?> step, Object key) {
- return new StepAndKey(step, key);
- }
-
- private StepAndKey(AppliedPTransform<?, ?, ?> step, Object key) {
- this.step = step;
- this.key = key;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(StepAndKey.class)
- .add("step", step.getFullName())
- .add("key", key)
- .toString();
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(step, key);
- }
-
- @Override
- public boolean equals(Object other) {
- if (other == this) {
- return true;
- } else if (!(other instanceof StepAndKey)) {
- return false;
- } else {
- StepAndKey that = (StepAndKey) other;
- return Objects.equals(this.step, that.step)
- && Objects.equals(this.key, that.key);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepTransformResult.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepTransformResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepTransformResult.java
deleted file mode 100644
index 8874eda..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepTransformResult.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.common.CounterSet;
-import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.ImmutableList;
-
-import org.joda.time.Instant;
-
-import java.util.Collection;
-
-import javax.annotation.Nullable;
-
-/**
- * An immutable {@link InProcessTransformResult}.
- */
-public class StepTransformResult implements InProcessTransformResult {
- private final AppliedPTransform<?, ?, ?> transform;
- private final Iterable<? extends UncommittedBundle<?>> bundles;
- @Nullable private final CopyOnAccessInMemoryStateInternals<?> state;
- private final TimerUpdate timerUpdate;
- @Nullable private final CounterSet counters;
- private final Instant watermarkHold;
-
- private StepTransformResult(
- AppliedPTransform<?, ?, ?> transform,
- Iterable<? extends UncommittedBundle<?>> outputBundles,
- CopyOnAccessInMemoryStateInternals<?> state,
- TimerUpdate timerUpdate,
- CounterSet counters,
- Instant watermarkHold) {
- this.transform = checkNotNull(transform);
- this.bundles = checkNotNull(outputBundles);
- this.state = state;
- this.timerUpdate = checkNotNull(timerUpdate);
- this.counters = counters;
- this.watermarkHold = checkNotNull(watermarkHold);
- }
-
- @Override
- public Iterable<? extends UncommittedBundle<?>> getOutputBundles() {
- return bundles;
- }
-
- @Override
- public CounterSet getCounters() {
- return counters;
- }
-
- @Override
- public AppliedPTransform<?, ?, ?> getTransform() {
- return transform;
- }
-
- @Override
- public Instant getWatermarkHold() {
- return watermarkHold;
- }
-
- @Nullable
- @Override
- public CopyOnAccessInMemoryStateInternals<?> getState() {
- return state;
- }
-
- @Override
- public TimerUpdate getTimerUpdate() {
- return timerUpdate;
- }
-
- public static Builder withHold(AppliedPTransform<?, ?, ?> transform, Instant watermarkHold) {
- return new Builder(transform, watermarkHold);
- }
-
- public static Builder withoutHold(AppliedPTransform<?, ?, ?> transform) {
- return new Builder(transform, BoundedWindow.TIMESTAMP_MAX_VALUE);
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(StepTransformResult.class)
- .add("transform", transform)
- .toString();
- }
-
- /**
- * A builder for creating instances of {@link StepTransformResult}.
- */
- public static class Builder {
- private final AppliedPTransform<?, ?, ?> transform;
- private final ImmutableList.Builder<UncommittedBundle<?>> bundlesBuilder;
- private CopyOnAccessInMemoryStateInternals<?> state;
- private TimerUpdate timerUpdate;
- private CounterSet counters;
- private final Instant watermarkHold;
-
- private Builder(AppliedPTransform<?, ?, ?> transform, Instant watermarkHold) {
- this.transform = transform;
- this.watermarkHold = watermarkHold;
- this.bundlesBuilder = ImmutableList.builder();
- this.timerUpdate = TimerUpdate.builder(null).build();
- }
-
- public StepTransformResult build() {
- return new StepTransformResult(
- transform,
- bundlesBuilder.build(),
- state,
- timerUpdate,
- counters,
- watermarkHold);
- }
-
- public Builder withCounters(CounterSet counters) {
- this.counters = counters;
- return this;
- }
-
- public Builder withState(CopyOnAccessInMemoryStateInternals<?> state) {
- this.state = state;
- return this;
- }
-
- public Builder withTimerUpdate(TimerUpdate timerUpdate) {
- this.timerUpdate = timerUpdate;
- return this;
- }
-
- public Builder addOutput(
- UncommittedBundle<?> outputBundle, UncommittedBundle<?>... outputBundles) {
- bundlesBuilder.add(outputBundle);
- bundlesBuilder.add(outputBundles);
- return this;
- }
-
- public Builder addOutput(Collection<UncommittedBundle<?>> outputBundles) {
- bundlesBuilder.addAll(outputBundles);
- return this;
- }
- }
-}
[15/17] incubator-beam git commit: Move InProcessRunner to its own
module
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java
new file mode 100644
index 0000000..2efaad3
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java
@@ -0,0 +1,85 @@
+/*
+ * 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.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.Flatten.FlattenPCollectionList;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+
+/**
+ * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the {@link Flatten}
+ * {@link PTransform}.
+ */
+class FlattenEvaluatorFactory implements TransformEvaluatorFactory {
+ @Override
+ public <InputT> TransformEvaluator<InputT> forApplication(
+ AppliedPTransform<?, ?, ?> application,
+ CommittedBundle<?> inputBundle,
+ InProcessEvaluationContext evaluationContext) {
+ @SuppressWarnings({"cast", "unchecked", "rawtypes"})
+ TransformEvaluator<InputT> evaluator = (TransformEvaluator<InputT>) createInMemoryEvaluator(
+ (AppliedPTransform) application, inputBundle, evaluationContext);
+ return evaluator;
+ }
+
+ private <InputT> TransformEvaluator<InputT> createInMemoryEvaluator(
+ final AppliedPTransform<
+ PCollectionList<InputT>, PCollection<InputT>, FlattenPCollectionList<InputT>>
+ application,
+ final CommittedBundle<InputT> inputBundle,
+ final InProcessEvaluationContext evaluationContext) {
+ if (inputBundle == null) {
+ // it is impossible to call processElement on a flatten with no input bundle. A Flatten with
+ // no input bundle occurs as an output of Flatten.pcollections(PCollectionList.empty())
+ return new FlattenEvaluator<>(
+ null, StepTransformResult.withoutHold(application).build());
+ }
+ final UncommittedBundle<InputT> outputBundle =
+ evaluationContext.createBundle(inputBundle, application.getOutput());
+ final InProcessTransformResult result =
+ StepTransformResult.withoutHold(application).addOutput(outputBundle).build();
+ return new FlattenEvaluator<>(outputBundle, result);
+ }
+
+ private static class FlattenEvaluator<InputT> implements TransformEvaluator<InputT> {
+ private final UncommittedBundle<InputT> outputBundle;
+ private final InProcessTransformResult result;
+
+ public FlattenEvaluator(
+ UncommittedBundle<InputT> outputBundle, InProcessTransformResult result) {
+ this.outputBundle = outputBundle;
+ this.result = result;
+ }
+
+ @Override
+ public void processElement(WindowedValue<InputT> element) {
+ outputBundle.add(element);
+ }
+
+ @Override
+ public InProcessTransformResult finishBundle() {
+ return result;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ForwardingPTransform.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ForwardingPTransform.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ForwardingPTransform.java
new file mode 100644
index 0000000..3160b58
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ForwardingPTransform.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.TypedPValue;
+
+/**
+ * A base class for implementing {@link PTransform} overrides, which behave identically to the
+ * delegate transform but with overridden methods. Implementors are required to implement
+ * {@link #delegate()}, which returns the object to forward calls to, and {@link #apply(PInput)}.
+ */
+public abstract class ForwardingPTransform<InputT extends PInput, OutputT extends POutput>
+ extends PTransform<InputT, OutputT> {
+ protected abstract PTransform<InputT, OutputT> delegate();
+
+ @Override
+ public OutputT apply(InputT input) {
+ return delegate().apply(input);
+ }
+
+ @Override
+ public void validate(InputT input) {
+ delegate().validate(input);
+ }
+
+ @Override
+ public String getName() {
+ return delegate().getName();
+ }
+
+ @Override
+ public <T> Coder<T> getDefaultOutputCoder(InputT input, @SuppressWarnings("unused")
+ TypedPValue<T> output) throws CannotProvideCoderException {
+ return delegate().getDefaultOutputCoder(input, output);
+ }
+
+ @Override
+ public void populateDisplayData(DisplayData.Builder builder) {
+ delegate().populateDisplayData(builder);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactory.java
new file mode 100644
index 0000000..874ec17
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactory.java
@@ -0,0 +1,274 @@
+/*
+ * 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.apache.beam.sdk.util.CoderUtils.encodeToByteArray;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.runners.direct.StepTransformResult.Builder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.GroupAlsoByWindowViaWindowSetDoFn;
+import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.ReifyTimestampsAndWindows;
+import org.apache.beam.sdk.util.KeyedWorkItem;
+import org.apache.beam.sdk.util.KeyedWorkItemCoder;
+import org.apache.beam.sdk.util.KeyedWorkItems;
+import org.apache.beam.sdk.util.SystemReduceFn;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the {@link GroupByKey}
+ * {@link PTransform}.
+ */
+class GroupByKeyEvaluatorFactory implements TransformEvaluatorFactory {
+ @Override
+ public <InputT> TransformEvaluator<InputT> forApplication(
+ AppliedPTransform<?, ?, ?> application,
+ CommittedBundle<?> inputBundle,
+ InProcessEvaluationContext evaluationContext) {
+ @SuppressWarnings({"cast", "unchecked", "rawtypes"})
+ TransformEvaluator<InputT> evaluator = createEvaluator(
+ (AppliedPTransform) application, (CommittedBundle) inputBundle, evaluationContext);
+ return evaluator;
+ }
+
+ private <K, V> TransformEvaluator<KV<K, WindowedValue<V>>> createEvaluator(
+ final AppliedPTransform<
+ PCollection<KV<K, WindowedValue<V>>>, PCollection<KeyedWorkItem<K, V>>,
+ InProcessGroupByKeyOnly<K, V>>
+ application,
+ final CommittedBundle<KV<K, V>> inputBundle,
+ final InProcessEvaluationContext evaluationContext) {
+ return new GroupByKeyEvaluator<K, V>(evaluationContext, inputBundle, application);
+ }
+
+ private static class GroupByKeyEvaluator<K, V>
+ implements TransformEvaluator<KV<K, WindowedValue<V>>> {
+ private final InProcessEvaluationContext evaluationContext;
+
+ private final CommittedBundle<KV<K, V>> inputBundle;
+ private final AppliedPTransform<
+ PCollection<KV<K, WindowedValue<V>>>, PCollection<KeyedWorkItem<K, V>>,
+ InProcessGroupByKeyOnly<K, V>>
+ application;
+ private final Coder<K> keyCoder;
+ private Map<GroupingKey<K>, List<WindowedValue<V>>> groupingMap;
+
+ public GroupByKeyEvaluator(
+ InProcessEvaluationContext evaluationContext,
+ CommittedBundle<KV<K, V>> inputBundle,
+ AppliedPTransform<
+ PCollection<KV<K, WindowedValue<V>>>, PCollection<KeyedWorkItem<K, V>>,
+ InProcessGroupByKeyOnly<K, V>>
+ application) {
+ this.evaluationContext = evaluationContext;
+ this.inputBundle = inputBundle;
+ this.application = application;
+
+ PCollection<KV<K, WindowedValue<V>>> input = application.getInput();
+ keyCoder = getKeyCoder(input.getCoder());
+ groupingMap = new HashMap<>();
+ }
+
+ private Coder<K> getKeyCoder(Coder<KV<K, WindowedValue<V>>> coder) {
+ if (!(coder instanceof KvCoder)) {
+ throw new IllegalStateException();
+ }
+ @SuppressWarnings("unchecked")
+ Coder<K> keyCoder = ((KvCoder<K, WindowedValue<V>>) coder).getKeyCoder();
+ return keyCoder;
+ }
+
+ @Override
+ public void processElement(WindowedValue<KV<K, WindowedValue<V>>> element) {
+ KV<K, WindowedValue<V>> kv = element.getValue();
+ K key = kv.getKey();
+ byte[] encodedKey;
+ try {
+ encodedKey = encodeToByteArray(keyCoder, key);
+ } catch (CoderException exn) {
+ // TODO: Put in better element printing:
+ // truncate if too long.
+ throw new IllegalArgumentException(
+ String.format("unable to encode key %s of input to %s using %s", key, this, keyCoder),
+ exn);
+ }
+ GroupingKey<K> groupingKey = new GroupingKey<>(key, encodedKey);
+ List<WindowedValue<V>> values = groupingMap.get(groupingKey);
+ if (values == null) {
+ values = new ArrayList<WindowedValue<V>>();
+ groupingMap.put(groupingKey, values);
+ }
+ values.add(kv.getValue());
+ }
+
+ @Override
+ public InProcessTransformResult finishBundle() {
+ Builder resultBuilder = StepTransformResult.withoutHold(application);
+ for (Map.Entry<GroupingKey<K>, List<WindowedValue<V>>> groupedEntry :
+ groupingMap.entrySet()) {
+ K key = groupedEntry.getKey().key;
+ KeyedWorkItem<K, V> groupedKv =
+ KeyedWorkItems.elementsWorkItem(key, groupedEntry.getValue());
+ UncommittedBundle<KeyedWorkItem<K, V>> bundle =
+ evaluationContext.createKeyedBundle(inputBundle, key, application.getOutput());
+ bundle.add(WindowedValue.valueInGlobalWindow(groupedKv));
+ resultBuilder.addOutput(bundle);
+ }
+ return resultBuilder.build();
+ }
+
+ private static class GroupingKey<K> {
+ private K key;
+ private byte[] encodedKey;
+
+ public GroupingKey(K key, byte[] encodedKey) {
+ this.key = key;
+ this.encodedKey = encodedKey;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (o instanceof GroupingKey) {
+ GroupingKey<?> that = (GroupingKey<?>) o;
+ return Arrays.equals(this.encodedKey, that.encodedKey);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return Arrays.hashCode(encodedKey);
+ }
+ }
+ }
+
+ /**
+ * A {@link PTransformOverrideFactory} for {@link GroupByKey} PTransforms.
+ */
+ public static final class InProcessGroupByKeyOverrideFactory
+ implements PTransformOverrideFactory {
+ @Override
+ public <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
+ PTransform<InputT, OutputT> transform) {
+ if (transform instanceof GroupByKey) {
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ PTransform<InputT, OutputT> override = new InProcessGroupByKey((GroupByKey) transform);
+ return override;
+ }
+ return transform;
+ }
+ }
+
+ /**
+ * An in-memory implementation of the {@link GroupByKey} primitive as a composite
+ * {@link PTransform}.
+ */
+ private static final class InProcessGroupByKey<K, V>
+ extends ForwardingPTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> {
+ private final GroupByKey<K, V> original;
+
+ private InProcessGroupByKey(GroupByKey<K, V> from) {
+ this.original = from;
+ }
+
+ @Override
+ public PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> delegate() {
+ return original;
+ }
+
+ @Override
+ public PCollection<KV<K, Iterable<V>>> apply(PCollection<KV<K, V>> input) {
+ KvCoder<K, V> inputCoder = (KvCoder<K, V>) input.getCoder();
+
+ // This operation groups by the combination of key and window,
+ // merging windows as needed, using the windows assigned to the
+ // key/value input elements and the window merge operation of the
+ // window function associated with the input PCollection.
+ WindowingStrategy<?, ?> windowingStrategy = input.getWindowingStrategy();
+
+ // Use the default GroupAlsoByWindow implementation
+ DoFn<KeyedWorkItem<K, V>, KV<K, Iterable<V>>> groupAlsoByWindow =
+ groupAlsoByWindow(windowingStrategy, inputCoder.getValueCoder());
+
+ // By default, implement GroupByKey via a series of lower-level operations.
+ return input
+ // Make each input element's timestamp and assigned windows
+ // explicit, in the value part.
+ .apply(new ReifyTimestampsAndWindows<K, V>())
+
+ .apply(new InProcessGroupByKeyOnly<K, V>())
+ .setCoder(KeyedWorkItemCoder.of(inputCoder.getKeyCoder(),
+ inputCoder.getValueCoder(), input.getWindowingStrategy().getWindowFn().windowCoder()))
+
+ // Group each key's values by window, merging windows as needed.
+ .apply("GroupAlsoByWindow", ParDo.of(groupAlsoByWindow))
+
+ // And update the windowing strategy as appropriate.
+ .setWindowingStrategyInternal(original.updateWindowingStrategy(windowingStrategy))
+ .setCoder(
+ KvCoder.of(inputCoder.getKeyCoder(), IterableCoder.of(inputCoder.getValueCoder())));
+ }
+
+ private <W extends BoundedWindow>
+ DoFn<KeyedWorkItem<K, V>, KV<K, Iterable<V>>> groupAlsoByWindow(
+ final WindowingStrategy<?, W> windowingStrategy, final Coder<V> inputCoder) {
+ return GroupAlsoByWindowViaWindowSetDoFn.create(
+ windowingStrategy, SystemReduceFn.<K, V, W>buffering(inputCoder));
+ }
+ }
+
+ /**
+ * An implementation primitive to use in the evaluation of a {@link GroupByKey}
+ * {@link PTransform}.
+ */
+ public static final class InProcessGroupByKeyOnly<K, V>
+ extends PTransform<PCollection<KV<K, WindowedValue<V>>>, PCollection<KeyedWorkItem<K, V>>> {
+ @Override
+ public PCollection<KeyedWorkItem<K, V>> apply(PCollection<KV<K, WindowedValue<V>>> input) {
+ return PCollection.<KeyedWorkItem<K, V>>createPrimitiveOutputInternal(
+ input.getPipeline(), input.getWindowingStrategy(), input.isBounded());
+ }
+
+ @VisibleForTesting
+ InProcessGroupByKeyOnly() {}
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java
new file mode 100644
index 0000000..2103ad3
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java
@@ -0,0 +1,131 @@
+/*
+ * 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 com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.util.IllegalMutationException;
+import org.apache.beam.sdk.util.MutationDetector;
+import org.apache.beam.sdk.util.MutationDetectors;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.api.client.util.Throwables;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.SetMultimap;
+
+import org.joda.time.Instant;
+
+/**
+ * A {@link BundleFactory} that ensures that elements added to it are not mutated after being
+ * output. Immutability checks are enforced at the time {@link UncommittedBundle#commit(Instant)} is
+ * called, checking the value at that time against the value at the time the element was added. All
+ * elements added to the bundle will be encoded by the {@link Coder} of the underlying
+ * {@link PCollection}.
+ *
+ * <p>This catches errors during the execution of a {@link DoFn} caused by modifying an element
+ * after it is added to an output {@link PCollection}.
+ */
+class ImmutabilityCheckingBundleFactory implements BundleFactory {
+ /**
+ * Create a new {@link ImmutabilityCheckingBundleFactory} that uses the underlying
+ * {@link BundleFactory} to create the output bundle.
+ */
+ public static ImmutabilityCheckingBundleFactory create(BundleFactory underlying) {
+ return new ImmutabilityCheckingBundleFactory(underlying);
+ }
+
+ private final BundleFactory underlying;
+
+ private ImmutabilityCheckingBundleFactory(BundleFactory underlying) {
+ this.underlying = checkNotNull(underlying);
+ }
+
+ @Override
+ public <T> UncommittedBundle<T> createRootBundle(PCollection<T> output) {
+ return new ImmutabilityEnforcingBundle<>(underlying.createRootBundle(output));
+ }
+
+ @Override
+ public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output) {
+ return new ImmutabilityEnforcingBundle<>(underlying.createBundle(input, output));
+ }
+
+ @Override
+ public <T> UncommittedBundle<T> createKeyedBundle(
+ CommittedBundle<?> input, Object key, PCollection<T> output) {
+ return new ImmutabilityEnforcingBundle<>(underlying.createKeyedBundle(input, key, output));
+ }
+
+ private static class ImmutabilityEnforcingBundle<T> implements UncommittedBundle<T> {
+ private final UncommittedBundle<T> underlying;
+ private final SetMultimap<WindowedValue<T>, MutationDetector> mutationDetectors;
+ private Coder<T> coder;
+
+ public ImmutabilityEnforcingBundle(UncommittedBundle<T> underlying) {
+ this.underlying = underlying;
+ mutationDetectors = HashMultimap.create();
+ coder = getPCollection().getCoder();
+ }
+
+ @Override
+ public PCollection<T> getPCollection() {
+ return underlying.getPCollection();
+ }
+
+ @Override
+ public UncommittedBundle<T> add(WindowedValue<T> element) {
+ try {
+ mutationDetectors.put(
+ element, MutationDetectors.forValueWithCoder(element.getValue(), coder));
+ } catch (CoderException e) {
+ throw Throwables.propagate(e);
+ }
+ underlying.add(element);
+ return this;
+ }
+
+ @Override
+ public CommittedBundle<T> commit(Instant synchronizedProcessingTime) {
+ for (MutationDetector detector : mutationDetectors.values()) {
+ try {
+ detector.verifyUnmodified();
+ } catch (IllegalMutationException exn) {
+ throw UserCodeException.wrap(
+ new IllegalMutationException(
+ String.format(
+ "PTransform %s mutated value %s after it was output (new value was %s)."
+ + " Values must not be mutated in any way after being output.",
+ underlying.getPCollection().getProducingTransformInternal().getFullName(),
+ exn.getSavedValue(),
+ exn.getNewValue()),
+ exn.getSavedValue(),
+ exn.getNewValue(),
+ exn));
+ }
+ }
+ return underlying.commit(synchronizedProcessingTime);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java
new file mode 100644
index 0000000..bfecc9d
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java
@@ -0,0 +1,103 @@
+/*
+ * 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.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.util.IllegalMutationException;
+import org.apache.beam.sdk.util.MutationDetector;
+import org.apache.beam.sdk.util.MutationDetectors;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+
+import java.util.IdentityHashMap;
+import java.util.Map;
+
+/**
+ * {@link ModelEnforcement} that enforces elements are not modified over the course of processing
+ * an element.
+ *
+ * <p>Implies {@link EncodabilityEnforcment}.
+ */
+class ImmutabilityEnforcementFactory implements ModelEnforcementFactory {
+ public static ModelEnforcementFactory create() {
+ return new ImmutabilityEnforcementFactory();
+ }
+
+ @Override
+ public <T> ModelEnforcement<T> forBundle(
+ CommittedBundle<T> input, AppliedPTransform<?, ?, ?> consumer) {
+ return new ImmutabilityCheckingEnforcement<T>(input, consumer);
+ }
+
+ private static class ImmutabilityCheckingEnforcement<T> extends AbstractModelEnforcement<T> {
+ private final AppliedPTransform<?, ?, ?> transform;
+ private final Map<WindowedValue<T>, MutationDetector> mutationElements;
+ private final Coder<T> coder;
+
+ private ImmutabilityCheckingEnforcement(
+ CommittedBundle<T> input, AppliedPTransform<?, ?, ?> transform) {
+ this.transform = transform;
+ coder = input.getPCollection().getCoder();
+ mutationElements = new IdentityHashMap<>();
+ }
+
+ @Override
+ public void beforeElement(WindowedValue<T> element) {
+ try {
+ mutationElements.put(
+ element, MutationDetectors.forValueWithCoder(element.getValue(), coder));
+ } catch (CoderException e) {
+ throw UserCodeException.wrap(e);
+ }
+ }
+
+ @Override
+ public void afterElement(WindowedValue<T> element) {
+ verifyUnmodified(mutationElements.get(element));
+ }
+
+ @Override
+ public void afterFinish(
+ CommittedBundle<T> input,
+ InProcessTransformResult result,
+ Iterable<? extends CommittedBundle<?>> outputs) {
+ for (MutationDetector detector : mutationElements.values()) {
+ verifyUnmodified(detector);
+ }
+ }
+
+ private void verifyUnmodified(MutationDetector detector) {
+ try {
+ detector.verifyUnmodified();
+ } catch (IllegalMutationException e) {
+ throw new IllegalMutationException(
+ String.format(
+ "PTransform %s illegaly mutated value %s of class %s."
+ + " Input values must not be mutated in any way.",
+ transform.getFullName(),
+ e.getSavedValue(),
+ e.getSavedValue().getClass()),
+ e.getSavedValue(),
+ e.getNewValue());
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InMemoryWatermarkManager.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InMemoryWatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InMemoryWatermarkManager.java
new file mode 100644
index 0000000..07b6bb4
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InMemoryWatermarkManager.java
@@ -0,0 +1,1327 @@
+/*
+ * 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.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PValue;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.base.MoreObjects;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ComparisonChain;
+import com.google.common.collect.FluentIterable;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Ordering;
+import com.google.common.collect.SortedMultiset;
+import com.google.common.collect.TreeMultiset;
+
+import org.joda.time.Instant;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Objects;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicReference;
+
+import javax.annotation.Nullable;
+
+/**
+ * Manages watermarks of {@link PCollection PCollections} and input and output watermarks of
+ * {@link AppliedPTransform AppliedPTransforms} to provide event-time and completion tracking for
+ * in-memory execution. {@link InMemoryWatermarkManager} is designed to update and return a
+ * consistent view of watermarks in the presence of concurrent updates.
+ *
+ * <p>An {@link InMemoryWatermarkManager} is provided with the collection of root
+ * {@link AppliedPTransform AppliedPTransforms} and a map of {@link PCollection PCollections} to
+ * all the {@link AppliedPTransform AppliedPTransforms} that consume them at construction time.
+ *
+ * <p>Whenever a root {@link AppliedPTransform transform} produces elements, the
+ * {@link InMemoryWatermarkManager} is provided with the produced elements and the output watermark
+ * of the producing {@link AppliedPTransform transform}. The
+ * {@link InMemoryWatermarkManager watermark manager} is responsible for computing the watermarks
+ * of all {@link AppliedPTransform transforms} that consume one or more
+ * {@link PCollection PCollections}.
+ *
+ * <p>Whenever a non-root {@link AppliedPTransform} finishes processing one or more in-flight
+ * elements (referred to as the input {@link CommittedBundle bundle}), the following occurs
+ * atomically:
+ * <ul>
+ * <li>All of the in-flight elements are removed from the collection of pending elements for the
+ * {@link AppliedPTransform}.</li>
+ * <li>All of the elements produced by the {@link AppliedPTransform} are added to the collection
+ * of pending elements for each {@link AppliedPTransform} that consumes them.</li>
+ * <li>The input watermark for the {@link AppliedPTransform} becomes the maximum value of
+ * <ul>
+ * <li>the previous input watermark</li>
+ * <li>the minimum of
+ * <ul>
+ * <li>the timestamps of all currently pending elements</li>
+ * <li>all input {@link PCollection} watermarks</li>
+ * </ul>
+ * </li>
+ * </ul>
+ * </li>
+ * <li>The output watermark for the {@link AppliedPTransform} becomes the maximum of
+ * <ul>
+ * <li>the previous output watermark</li>
+ * <li>the minimum of
+ * <ul>
+ * <li>the current input watermark</li>
+ * <li>the current watermark holds</li>
+ * </ul>
+ * </li>
+ * </ul>
+ * </li>
+ * <li>The watermark of the output {@link PCollection} can be advanced to the output watermark of
+ * the {@link AppliedPTransform}</li>
+ * <li>The watermark of all downstream {@link AppliedPTransform AppliedPTransforms} can be
+ * advanced.</li>
+ * </ul>
+ *
+ * <p>The watermark of a {@link PCollection} is equal to the output watermark of the
+ * {@link AppliedPTransform} that produces it.
+ *
+ * <p>The watermarks for a {@link PTransform} are updated as follows when output is committed:<pre>
+ * Watermark_In' = MAX(Watermark_In, MIN(U(TS_Pending), U(Watermark_InputPCollection)))
+ * Watermark_Out' = MAX(Watermark_Out, MIN(Watermark_In', U(StateHold)))
+ * Watermark_PCollection = Watermark_Out_ProducingPTransform
+ * </pre>
+ */
+public class InMemoryWatermarkManager {
+ /**
+ * The watermark of some {@link Pipeline} element, usually a {@link PTransform} or a
+ * {@link PCollection}.
+ *
+ * <p>A watermark is a monotonically increasing value, which represents the point up to which the
+ * system believes it has received all of the data. Data that arrives with a timestamp that is
+ * before the watermark is considered late. {@link BoundedWindow#TIMESTAMP_MAX_VALUE} is a special
+ * timestamp which indicates we have received all of the data and there will be no more on-time or
+ * late data. This value is represented by {@link InMemoryWatermarkManager#THE_END_OF_TIME}.
+ */
+ private static interface Watermark {
+ /**
+ * Returns the current value of this watermark.
+ */
+ Instant get();
+
+ /**
+ * Refreshes the value of this watermark from its input watermarks and watermark holds.
+ *
+ * @return true if the value of the watermark has changed (and thus dependent watermark must
+ * also be updated
+ */
+ WatermarkUpdate refresh();
+ }
+
+ /**
+ * The result of computing a {@link Watermark}.
+ */
+ private static enum WatermarkUpdate {
+ /** The watermark is later than the value at the previous time it was computed. */
+ ADVANCED(true),
+ /** The watermark is equal to the value at the previous time it was computed. */
+ NO_CHANGE(false);
+
+ private final boolean advanced;
+
+ private WatermarkUpdate(boolean advanced) {
+ this.advanced = advanced;
+ }
+
+ public boolean isAdvanced() {
+ return advanced;
+ }
+
+ /**
+ * Returns the {@link WatermarkUpdate} that is a result of combining the two watermark updates.
+ *
+ * If either of the input {@link WatermarkUpdate WatermarkUpdates} were advanced, the result
+ * {@link WatermarkUpdate} has been advanced.
+ */
+ public WatermarkUpdate union(WatermarkUpdate that) {
+ if (this.advanced) {
+ return this;
+ }
+ return that;
+ }
+
+ /**
+ * Returns the {@link WatermarkUpdate} based on the former and current
+ * {@link Instant timestamps}.
+ */
+ public static WatermarkUpdate fromTimestamps(Instant oldTime, Instant currentTime) {
+ if (currentTime.isAfter(oldTime)) {
+ return ADVANCED;
+ }
+ return NO_CHANGE;
+ }
+ }
+
+ /**
+ * The input {@link Watermark} of an {@link AppliedPTransform}.
+ *
+ * <p>At any point, the value of an {@link AppliedPTransformInputWatermark} is equal to the
+ * minimum watermark across all of its input {@link Watermark Watermarks}, and the minimum
+ * timestamp of all of the pending elements, restricted to be monotonically increasing.
+ *
+ * <p>See {@link #refresh()} for more information.
+ */
+ private static class AppliedPTransformInputWatermark implements Watermark {
+ private final Collection<? extends Watermark> inputWatermarks;
+ private final SortedMultiset<WindowedValue<?>> pendingElements;
+ private final Map<Object, NavigableSet<TimerData>> objectTimers;
+
+ private AtomicReference<Instant> currentWatermark;
+
+ public AppliedPTransformInputWatermark(Collection<? extends Watermark> inputWatermarks) {
+ this.inputWatermarks = inputWatermarks;
+ this.pendingElements = TreeMultiset.create(new WindowedValueByTimestampComparator());
+ this.objectTimers = new HashMap<>();
+ currentWatermark = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
+ }
+
+ @Override
+ public Instant get() {
+ return currentWatermark.get();
+ }
+
+ /**
+ * {@inheritDoc}.
+ *
+ * <p>When refresh is called, the value of the {@link AppliedPTransformInputWatermark} becomes
+ * equal to the maximum value of
+ * <ul>
+ * <li>the previous input watermark</li>
+ * <li>the minimum of
+ * <ul>
+ * <li>the timestamps of all currently pending elements</li>
+ * <li>all input {@link PCollection} watermarks</li>
+ * </ul>
+ * </li>
+ * </ul>
+ */
+ @Override
+ public synchronized WatermarkUpdate refresh() {
+ Instant oldWatermark = currentWatermark.get();
+ Instant minInputWatermark = BoundedWindow.TIMESTAMP_MAX_VALUE;
+ for (Watermark inputWatermark : inputWatermarks) {
+ minInputWatermark = INSTANT_ORDERING.min(minInputWatermark, inputWatermark.get());
+ }
+ if (!pendingElements.isEmpty()) {
+ minInputWatermark = INSTANT_ORDERING.min(
+ minInputWatermark, pendingElements.firstEntry().getElement().getTimestamp());
+ }
+ Instant newWatermark = INSTANT_ORDERING.max(oldWatermark, minInputWatermark);
+ currentWatermark.set(newWatermark);
+ return WatermarkUpdate.fromTimestamps(oldWatermark, newWatermark);
+ }
+
+ private synchronized void addPendingElements(Iterable<? extends WindowedValue<?>> newPending) {
+ for (WindowedValue<?> pendingElement : newPending) {
+ pendingElements.add(pendingElement);
+ }
+ }
+
+ private synchronized void removePendingElements(
+ Iterable<? extends WindowedValue<?>> finishedElements) {
+ for (WindowedValue<?> finishedElement : finishedElements) {
+ pendingElements.remove(finishedElement);
+ }
+ }
+
+ private synchronized void updateTimers(TimerUpdate update) {
+ NavigableSet<TimerData> keyTimers = objectTimers.get(update.key);
+ if (keyTimers == null) {
+ keyTimers = new TreeSet<>();
+ objectTimers.put(update.key, keyTimers);
+ }
+ for (TimerData timer : update.setTimers) {
+ if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) {
+ keyTimers.add(timer);
+ }
+ }
+ for (TimerData timer : update.deletedTimers) {
+ if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) {
+ keyTimers.remove(timer);
+ }
+ }
+ // We don't keep references to timers that have been fired and delivered via #getFiredTimers()
+ }
+
+ private synchronized Map<Object, List<TimerData>> extractFiredEventTimeTimers() {
+ return extractFiredTimers(currentWatermark.get(), objectTimers);
+ }
+
+ @Override
+ public synchronized String toString() {
+ return MoreObjects.toStringHelper(AppliedPTransformInputWatermark.class)
+ .add("pendingElements", pendingElements)
+ .add("currentWatermark", currentWatermark)
+ .toString();
+ }
+ }
+
+ /**
+ * The output {@link Watermark} of an {@link AppliedPTransform}.
+ *
+ * <p>The value of an {@link AppliedPTransformOutputWatermark} is equal to the minimum of the
+ * current watermark hold and the {@link AppliedPTransformInputWatermark} for the same
+ * {@link AppliedPTransform}, restricted to be monotonically increasing. See
+ * {@link #refresh()} for more information.
+ */
+ private static class AppliedPTransformOutputWatermark implements Watermark {
+ private final Watermark inputWatermark;
+ private final PerKeyHolds holds;
+ private AtomicReference<Instant> currentWatermark;
+
+ public AppliedPTransformOutputWatermark(AppliedPTransformInputWatermark inputWatermark) {
+ this.inputWatermark = inputWatermark;
+ holds = new PerKeyHolds();
+ currentWatermark = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
+ }
+
+ public synchronized void updateHold(Object key, Instant newHold) {
+ if (newHold == null) {
+ holds.removeHold(key);
+ } else {
+ holds.updateHold(key, newHold);
+ }
+ }
+
+ @Override
+ public Instant get() {
+ return currentWatermark.get();
+ }
+
+ /**
+ * {@inheritDoc}.
+ *
+ * <p>When refresh is called, the value of the {@link AppliedPTransformOutputWatermark} becomes
+ * equal to the maximum value of:
+ * <ul>
+ * <li>the previous output watermark</li>
+ * <li>the minimum of
+ * <ul>
+ * <li>the current input watermark</li>
+ * <li>the current watermark holds</li>
+ * </ul>
+ * </li>
+ * </ul>
+ */
+ @Override
+ public synchronized WatermarkUpdate refresh() {
+ Instant oldWatermark = currentWatermark.get();
+ Instant newWatermark = INSTANT_ORDERING.min(inputWatermark.get(), holds.getMinHold());
+ newWatermark = INSTANT_ORDERING.max(oldWatermark, newWatermark);
+ currentWatermark.set(newWatermark);
+ return WatermarkUpdate.fromTimestamps(oldWatermark, newWatermark);
+ }
+
+ @Override
+ public synchronized String toString() {
+ return MoreObjects.toStringHelper(AppliedPTransformOutputWatermark.class)
+ .add("holds", holds)
+ .add("currentWatermark", currentWatermark)
+ .toString();
+ }
+ }
+
+ /**
+ * The input {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} hold for an
+ * {@link AppliedPTransform}.
+ *
+ * <p>At any point, the hold value of an {@link SynchronizedProcessingTimeInputWatermark} is equal
+ * to the minimum across all pending bundles at the {@link AppliedPTransform} and all upstream
+ * {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} watermarks. The value of the input
+ * synchronized processing time at any step is equal to the maximum of:
+ * <ul>
+ * <li>The most recently returned synchronized processing input time
+ * <li>The minimum of
+ * <ul>
+ * <li>The current processing time
+ * <li>The current synchronized processing time input hold
+ * </ul>
+ * </ul>
+ */
+ private static class SynchronizedProcessingTimeInputWatermark implements Watermark {
+ private final Collection<? extends Watermark> inputWms;
+ private final Collection<CommittedBundle<?>> pendingBundles;
+ private final Map<Object, NavigableSet<TimerData>> processingTimers;
+ private final Map<Object, NavigableSet<TimerData>> synchronizedProcessingTimers;
+
+ private final PriorityQueue<TimerData> pendingTimers;
+
+ private AtomicReference<Instant> earliestHold;
+
+ public SynchronizedProcessingTimeInputWatermark(Collection<? extends Watermark> inputWms) {
+ this.inputWms = inputWms;
+ this.pendingBundles = new HashSet<>();
+ this.processingTimers = new HashMap<>();
+ this.synchronizedProcessingTimers = new HashMap<>();
+ this.pendingTimers = new PriorityQueue<>();
+ Instant initialHold = BoundedWindow.TIMESTAMP_MAX_VALUE;
+ for (Watermark wm : inputWms) {
+ initialHold = INSTANT_ORDERING.min(initialHold, wm.get());
+ }
+ earliestHold = new AtomicReference<>(initialHold);
+ }
+
+ @Override
+ public Instant get() {
+ return earliestHold.get();
+ }
+
+ /**
+ * {@inheritDoc}.
+ *
+ * <p>When refresh is called, the value of the {@link SynchronizedProcessingTimeInputWatermark}
+ * becomes equal to the minimum value of
+ * <ul>
+ * <li>the timestamps of all currently pending bundles</li>
+ * <li>all input {@link PCollection} synchronized processing time watermarks</li>
+ * </ul>
+ *
+ * <p>Note that this value is not monotonic, but the returned value for the synchronized
+ * processing time must be.
+ */
+ @Override
+ public synchronized WatermarkUpdate refresh() {
+ Instant oldHold = earliestHold.get();
+ Instant minTime = THE_END_OF_TIME.get();
+ for (Watermark input : inputWms) {
+ minTime = INSTANT_ORDERING.min(minTime, input.get());
+ }
+ for (CommittedBundle<?> bundle : pendingBundles) {
+ // TODO: Track elements in the bundle by the processing time they were output instead of
+ // entire bundles. Requried to support arbitrarily splitting and merging bundles between
+ // steps
+ minTime = INSTANT_ORDERING.min(minTime, bundle.getSynchronizedProcessingOutputWatermark());
+ }
+ earliestHold.set(minTime);
+ return WatermarkUpdate.fromTimestamps(oldHold, minTime);
+ }
+
+ public synchronized void addPending(CommittedBundle<?> bundle) {
+ pendingBundles.add(bundle);
+ }
+
+ public synchronized void removePending(CommittedBundle<?> bundle) {
+ pendingBundles.remove(bundle);
+ }
+
+ /**
+ * Return the earliest timestamp of the earliest timer that has not been completed. This is
+ * either the earliest timestamp across timers that have not been completed, or the earliest
+ * timestamp across timers that have been delivered but have not been completed.
+ */
+ public synchronized Instant getEarliestTimerTimestamp() {
+ Instant earliest = THE_END_OF_TIME.get();
+ for (NavigableSet<TimerData> timers : processingTimers.values()) {
+ if (!timers.isEmpty()) {
+ earliest = INSTANT_ORDERING.min(timers.first().getTimestamp(), earliest);
+ }
+ }
+ for (NavigableSet<TimerData> timers : synchronizedProcessingTimers.values()) {
+ if (!timers.isEmpty()) {
+ earliest = INSTANT_ORDERING.min(timers.first().getTimestamp(), earliest);
+ }
+ }
+ if (!pendingTimers.isEmpty()) {
+ earliest = INSTANT_ORDERING.min(pendingTimers.peek().getTimestamp(), earliest);
+ }
+ return earliest;
+ }
+
+ private synchronized void updateTimers(TimerUpdate update) {
+ for (TimerData completedTimer : update.completedTimers) {
+ pendingTimers.remove(completedTimer);
+ }
+ Map<TimeDomain, NavigableSet<TimerData>> timerMap = timerMap(update.key);
+ for (TimerData addedTimer : update.setTimers) {
+ NavigableSet<TimerData> timerQueue = timerMap.get(addedTimer.getDomain());
+ if (timerQueue != null) {
+ timerQueue.add(addedTimer);
+ }
+ }
+ for (TimerData deletedTimer : update.deletedTimers) {
+ NavigableSet<TimerData> timerQueue = timerMap.get(deletedTimer.getDomain());
+ if (timerQueue != null) {
+ timerQueue.remove(deletedTimer);
+ }
+ }
+ }
+
+ private synchronized Map<Object, List<TimerData>> extractFiredDomainTimers(
+ TimeDomain domain, Instant firingTime) {
+ Map<Object, List<TimerData>> firedTimers;
+ switch (domain) {
+ case PROCESSING_TIME:
+ firedTimers = extractFiredTimers(firingTime, processingTimers);
+ break;
+ case SYNCHRONIZED_PROCESSING_TIME:
+ firedTimers =
+ extractFiredTimers(
+ INSTANT_ORDERING.min(firingTime, earliestHold.get()),
+ synchronizedProcessingTimers);
+ break;
+ default:
+ throw new IllegalArgumentException(
+ "Called getFiredTimers on a Synchronized Processing Time watermark"
+ + " and gave a non-processing time domain "
+ + domain);
+ }
+ for (Map.Entry<Object, ? extends Collection<TimerData>> firedTimer : firedTimers.entrySet()) {
+ pendingTimers.addAll(firedTimer.getValue());
+ }
+ return firedTimers;
+ }
+
+ private Map<TimeDomain, NavigableSet<TimerData>> timerMap(Object key) {
+ NavigableSet<TimerData> processingQueue = processingTimers.get(key);
+ if (processingQueue == null) {
+ processingQueue = new TreeSet<>();
+ processingTimers.put(key, processingQueue);
+ }
+ NavigableSet<TimerData> synchronizedProcessingQueue =
+ synchronizedProcessingTimers.get(key);
+ if (synchronizedProcessingQueue == null) {
+ synchronizedProcessingQueue = new TreeSet<>();
+ synchronizedProcessingTimers.put(key, synchronizedProcessingQueue);
+ }
+ EnumMap<TimeDomain, NavigableSet<TimerData>> result = new EnumMap<>(TimeDomain.class);
+ result.put(TimeDomain.PROCESSING_TIME, processingQueue);
+ result.put(TimeDomain.SYNCHRONIZED_PROCESSING_TIME, synchronizedProcessingQueue);
+ return result;
+ }
+
+ @Override
+ public synchronized String toString() {
+ return MoreObjects.toStringHelper(SynchronizedProcessingTimeInputWatermark.class)
+ .add("earliestHold", earliestHold)
+ .toString();
+ }
+ }
+
+ /**
+ * The output {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} hold for an
+ * {@link AppliedPTransform}.
+ *
+ * <p>At any point, the hold value of an {@link SynchronizedProcessingTimeOutputWatermark} is
+ * equal to the minimum across all incomplete timers at the {@link AppliedPTransform} and all
+ * upstream {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} watermarks. The value of the output
+ * synchronized processing time at any step is equal to the maximum of:
+ * <ul>
+ * <li>The most recently returned synchronized processing output time
+ * <li>The minimum of
+ * <ul>
+ * <li>The current processing time
+ * <li>The current synchronized processing time output hold
+ * </ul>
+ * </ul>
+ */
+ private static class SynchronizedProcessingTimeOutputWatermark implements Watermark {
+ private final SynchronizedProcessingTimeInputWatermark inputWm;
+ private AtomicReference<Instant> latestRefresh;
+
+ public SynchronizedProcessingTimeOutputWatermark(
+ SynchronizedProcessingTimeInputWatermark inputWm) {
+ this.inputWm = inputWm;
+ this.latestRefresh = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
+ }
+
+ @Override
+ public Instant get() {
+ return latestRefresh.get();
+ }
+
+ /**
+ * {@inheritDoc}.
+ *
+ * <p>When refresh is called, the value of the {@link SynchronizedProcessingTimeOutputWatermark}
+ * becomes equal to the minimum value of:
+ * <ul>
+ * <li>the current input watermark.
+ * <li>all {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} timers that are based on the input
+ * watermark.
+ * <li>all {@link TimeDomain#PROCESSING_TIME} timers that are based on the input watermark.
+ * </ul>
+ *
+ * <p>Note that this value is not monotonic, but the returned value for the synchronized
+ * processing time must be.
+ */
+ @Override
+ public synchronized WatermarkUpdate refresh() {
+ // Hold the output synchronized processing time to the input watermark, which takes into
+ // account buffered bundles, and the earliest pending timer, which determines what to hold
+ // downstream timers to.
+ Instant oldRefresh = latestRefresh.get();
+ Instant newTimestamp =
+ INSTANT_ORDERING.min(inputWm.get(), inputWm.getEarliestTimerTimestamp());
+ latestRefresh.set(newTimestamp);
+ return WatermarkUpdate.fromTimestamps(oldRefresh, newTimestamp);
+ }
+
+ @Override
+ public synchronized String toString() {
+ return MoreObjects.toStringHelper(SynchronizedProcessingTimeOutputWatermark.class)
+ .add("latestRefresh", latestRefresh)
+ .toString();
+ }
+ }
+
+ /**
+ * The {@code Watermark} that is after the latest time it is possible to represent in the global
+ * window. This is a distinguished value representing a complete {@link PTransform}.
+ */
+ private static final Watermark THE_END_OF_TIME = new Watermark() {
+ @Override
+ public WatermarkUpdate refresh() {
+ // THE_END_OF_TIME is a distinguished value that cannot be advanced.
+ return WatermarkUpdate.NO_CHANGE;
+ }
+
+ @Override
+ public Instant get() {
+ return BoundedWindow.TIMESTAMP_MAX_VALUE;
+ }
+ };
+
+ private static final Ordering<Instant> INSTANT_ORDERING = Ordering.natural();
+
+ /**
+ * A function that takes a WindowedValue and returns the exploded representation of that
+ * {@link WindowedValue}.
+ */
+ private static final Function<WindowedValue<?>, ? extends Iterable<? extends WindowedValue<?>>>
+ EXPLODE_WINDOWS_FN =
+ new Function<WindowedValue<?>, Iterable<? extends WindowedValue<?>>>() {
+ @Override
+ public Iterable<? extends WindowedValue<?>> apply(WindowedValue<?> input) {
+ return input.explodeWindows();
+ }
+ };
+
+ /**
+ * For each (Object, PriorityQueue) pair in the provided map, remove each Timer that is before the
+ * latestTime argument and put in in the result with the same key, then remove all of the keys
+ * which have no more pending timers.
+ *
+ * The result collection retains ordering of timers (from earliest to latest).
+ */
+ private static Map<Object, List<TimerData>> extractFiredTimers(
+ Instant latestTime, Map<Object, NavigableSet<TimerData>> objectTimers) {
+ Map<Object, List<TimerData>> result = new HashMap<>();
+ Set<Object> emptyKeys = new HashSet<>();
+ for (Map.Entry<Object, NavigableSet<TimerData>> pendingTimers : objectTimers.entrySet()) {
+ NavigableSet<TimerData> timers = pendingTimers.getValue();
+ if (!timers.isEmpty() && timers.first().getTimestamp().isBefore(latestTime)) {
+ ArrayList<TimerData> keyFiredTimers = new ArrayList<>();
+ result.put(pendingTimers.getKey(), keyFiredTimers);
+ while (!timers.isEmpty() && timers.first().getTimestamp().isBefore(latestTime)) {
+ keyFiredTimers.add(timers.first());
+ timers.remove(timers.first());
+ }
+ }
+ if (timers.isEmpty()) {
+ emptyKeys.add(pendingTimers.getKey());
+ }
+ }
+ objectTimers.keySet().removeAll(emptyKeys);
+ return result;
+ }
+
+ ////////////////////////////////////////////////////////////////////////////////////////////////
+
+ /**
+ * The {@link Clock} providing the current time in the {@link TimeDomain#PROCESSING_TIME} domain.
+ */
+ private final Clock clock;
+
+ /**
+ * A map from each {@link PCollection} to all {@link AppliedPTransform PTransform applications}
+ * that consume that {@link PCollection}.
+ */
+ private final Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers;
+
+ /**
+ * The input and output watermark of each {@link AppliedPTransform}.
+ */
+ private final Map<AppliedPTransform<?, ?, ?>, TransformWatermarks> transformToWatermarks;
+
+ /**
+ * Creates a new {@link InMemoryWatermarkManager}. All watermarks within the newly created
+ * {@link InMemoryWatermarkManager} start at {@link BoundedWindow#TIMESTAMP_MIN_VALUE}, the
+ * minimum watermark, with no watermark holds or pending elements.
+ *
+ * @param rootTransforms the root-level transforms of the {@link Pipeline}
+ * @param consumers a mapping between each {@link PCollection} in the {@link Pipeline} to the
+ * transforms that consume it as a part of their input
+ */
+ public static InMemoryWatermarkManager create(
+ Clock clock,
+ Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
+ Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers) {
+ return new InMemoryWatermarkManager(clock, rootTransforms, consumers);
+ }
+
+ private InMemoryWatermarkManager(
+ Clock clock,
+ Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
+ Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers) {
+ this.clock = clock;
+ this.consumers = consumers;
+
+ transformToWatermarks = new HashMap<>();
+
+ for (AppliedPTransform<?, ?, ?> rootTransform : rootTransforms) {
+ getTransformWatermark(rootTransform);
+ }
+ for (Collection<AppliedPTransform<?, ?, ?>> intermediateTransforms : consumers.values()) {
+ for (AppliedPTransform<?, ?, ?> transform : intermediateTransforms) {
+ getTransformWatermark(transform);
+ }
+ }
+ }
+
+ private TransformWatermarks getTransformWatermark(AppliedPTransform<?, ?, ?> transform) {
+ TransformWatermarks wms = transformToWatermarks.get(transform);
+ if (wms == null) {
+ List<Watermark> inputCollectionWatermarks = getInputWatermarks(transform);
+ AppliedPTransformInputWatermark inputWatermark =
+ new AppliedPTransformInputWatermark(inputCollectionWatermarks);
+ AppliedPTransformOutputWatermark outputWatermark =
+ new AppliedPTransformOutputWatermark(inputWatermark);
+
+ SynchronizedProcessingTimeInputWatermark inputProcessingWatermark =
+ new SynchronizedProcessingTimeInputWatermark(getInputProcessingWatermarks(transform));
+ SynchronizedProcessingTimeOutputWatermark outputProcessingWatermark =
+ new SynchronizedProcessingTimeOutputWatermark(inputProcessingWatermark);
+
+ wms =
+ new TransformWatermarks(
+ inputWatermark, outputWatermark, inputProcessingWatermark, outputProcessingWatermark);
+ transformToWatermarks.put(transform, wms);
+ }
+ return wms;
+ }
+
+ private Collection<Watermark> getInputProcessingWatermarks(
+ AppliedPTransform<?, ?, ?> transform) {
+ ImmutableList.Builder<Watermark> inputWmsBuilder = ImmutableList.builder();
+ Collection<? extends PValue> inputs = transform.getInput().expand();
+ if (inputs.isEmpty()) {
+ inputWmsBuilder.add(THE_END_OF_TIME);
+ }
+ for (PValue pvalue : inputs) {
+ Watermark producerOutputWatermark =
+ getTransformWatermark(pvalue.getProducingTransformInternal())
+ .synchronizedProcessingOutputWatermark;
+ inputWmsBuilder.add(producerOutputWatermark);
+ }
+ return inputWmsBuilder.build();
+ }
+
+ private List<Watermark> getInputWatermarks(AppliedPTransform<?, ?, ?> transform) {
+ ImmutableList.Builder<Watermark> inputWatermarksBuilder = ImmutableList.builder();
+ Collection<? extends PValue> inputs = transform.getInput().expand();
+ if (inputs.isEmpty()) {
+ inputWatermarksBuilder.add(THE_END_OF_TIME);
+ }
+ for (PValue pvalue : inputs) {
+ Watermark producerOutputWatermark =
+ getTransformWatermark(pvalue.getProducingTransformInternal()).outputWatermark;
+ inputWatermarksBuilder.add(producerOutputWatermark);
+ }
+ List<Watermark> inputCollectionWatermarks = inputWatermarksBuilder.build();
+ return inputCollectionWatermarks;
+ }
+
+ ////////////////////////////////////////////////////////////////////////////////////////////////
+
+ /**
+ * Gets the input and output watermarks for an {@link AppliedPTransform}. If the
+ * {@link AppliedPTransform PTransform} has not processed any elements, return a watermark of
+ * {@link BoundedWindow#TIMESTAMP_MIN_VALUE}.
+ *
+ * @return a snapshot of the input watermark and output watermark for the provided transform
+ */
+ public TransformWatermarks getWatermarks(AppliedPTransform<?, ?, ?> transform) {
+ return transformToWatermarks.get(transform);
+ }
+
+ /**
+ * Updates the watermarks of a transform with one or more inputs.
+ *
+ * <p>Each transform has two monotonically increasing watermarks: the input watermark, which can,
+ * at any time, be updated to equal:
+ * <pre>
+ * MAX(CurrentInputWatermark, MIN(PendingElements, InputPCollectionWatermarks))
+ * </pre>
+ * and the output watermark, which can, at any time, be updated to equal:
+ * <pre>
+ * MAX(CurrentOutputWatermark, MIN(InputWatermark, WatermarkHolds))
+ * </pre>.
+ *
+ * @param completed the input that has completed
+ * @param transform the transform that has completed processing the input
+ * @param outputs the bundles the transform has output
+ * @param earliestHold the earliest watermark hold in the transform's state. {@code null} if there
+ * is no hold
+ */
+ public void updateWatermarks(
+ @Nullable CommittedBundle<?> completed,
+ AppliedPTransform<?, ?, ?> transform,
+ TimerUpdate timerUpdate,
+ Iterable<? extends CommittedBundle<?>> outputs,
+ @Nullable Instant earliestHold) {
+ updatePending(completed, transform, timerUpdate, outputs);
+ TransformWatermarks transformWms = transformToWatermarks.get(transform);
+ transformWms.setEventTimeHold(completed == null ? null : completed.getKey(), earliestHold);
+ refreshWatermarks(transform);
+ }
+
+ private void refreshWatermarks(AppliedPTransform<?, ?, ?> transform) {
+ TransformWatermarks myWatermarks = transformToWatermarks.get(transform);
+ WatermarkUpdate updateResult = myWatermarks.refresh();
+ if (updateResult.isAdvanced()) {
+ for (PValue outputPValue : transform.getOutput().expand()) {
+ Collection<AppliedPTransform<?, ?, ?>> downstreamTransforms = consumers.get(outputPValue);
+ if (downstreamTransforms != null) {
+ for (AppliedPTransform<?, ?, ?> downstreamTransform : downstreamTransforms) {
+ refreshWatermarks(downstreamTransform);
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * Removes all of the completed Timers from the collection of pending timers, adds all new timers,
+ * and removes all deleted timers. Removes all elements consumed by the input bundle from the
+ * {@link PTransform PTransforms} collection of pending elements, and adds all elements produced
+ * by the {@link PTransform} to the pending queue of each consumer.
+ */
+ private void updatePending(
+ CommittedBundle<?> input,
+ AppliedPTransform<?, ?, ?> transform,
+ TimerUpdate timerUpdate,
+ Iterable<? extends CommittedBundle<?>> outputs) {
+ TransformWatermarks completedTransform = transformToWatermarks.get(transform);
+ completedTransform.updateTimers(timerUpdate);
+ if (input != null) {
+ completedTransform.removePending(input);
+ }
+
+ for (CommittedBundle<?> bundle : outputs) {
+ for (AppliedPTransform<?, ?, ?> consumer : consumers.get(bundle.getPCollection())) {
+ TransformWatermarks watermarks = transformToWatermarks.get(consumer);
+ watermarks.addPending(bundle);
+ }
+ }
+ }
+
+ /**
+ * Returns a map of each {@link PTransform} that has pending timers to those timers. All of the
+ * pending timers will be removed from this {@link InMemoryWatermarkManager}.
+ */
+ public Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> extractFiredTimers() {
+ Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> allTimers = new HashMap<>();
+ for (Map.Entry<AppliedPTransform<?, ?, ?>, TransformWatermarks> watermarksEntry :
+ transformToWatermarks.entrySet()) {
+ Map<Object, FiredTimers> keyFiredTimers = watermarksEntry.getValue().extractFiredTimers();
+ if (!keyFiredTimers.isEmpty()) {
+ allTimers.put(watermarksEntry.getKey(), keyFiredTimers);
+ }
+ }
+ return allTimers;
+ }
+
+ /**
+ * A (key, Instant) pair that holds the watermark. Holds are per-key, but the watermark is global,
+ * and as such the watermark manager must track holds and the release of holds on a per-key basis.
+ *
+ * <p>The {@link #compareTo(KeyedHold)} method of {@link KeyedHold} is not consistent with equals,
+ * as the key is arbitrarily ordered via identity, rather than object equality.
+ */
+ private static final class KeyedHold implements Comparable<KeyedHold> {
+ private static final Ordering<Object> KEY_ORDERING = Ordering.arbitrary().nullsLast();
+
+ private final Object key;
+ private final Instant timestamp;
+
+ /**
+ * Create a new KeyedHold with the specified key and timestamp.
+ */
+ public static KeyedHold of(Object key, Instant timestamp) {
+ return new KeyedHold(key, MoreObjects.firstNonNull(timestamp, THE_END_OF_TIME.get()));
+ }
+
+ private KeyedHold(Object key, Instant timestamp) {
+ this.key = key;
+ this.timestamp = timestamp;
+ }
+
+ @Override
+ public int compareTo(KeyedHold that) {
+ return ComparisonChain.start()
+ .compare(this.timestamp, that.timestamp)
+ .compare(this.key, that.key, KEY_ORDERING)
+ .result();
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(timestamp, key);
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (other == null || !(other instanceof KeyedHold)) {
+ return false;
+ }
+ KeyedHold that = (KeyedHold) other;
+ return Objects.equals(this.timestamp, that.timestamp) && Objects.equals(this.key, that.key);
+ }
+
+ /**
+ * Get the value of this {@link KeyedHold}.
+ */
+ public Instant getTimestamp() {
+ return timestamp;
+ }
+
+ @Override
+ public String toString() {
+ return MoreObjects.toStringHelper(KeyedHold.class)
+ .add("key", key)
+ .add("hold", timestamp)
+ .toString();
+ }
+ }
+
+ private static class PerKeyHolds {
+ private final Map<Object, KeyedHold> keyedHolds;
+ private final PriorityQueue<KeyedHold> allHolds;
+
+ private PerKeyHolds() {
+ this.keyedHolds = new HashMap<>();
+ this.allHolds = new PriorityQueue<>();
+ }
+
+ /**
+ * Gets the minimum hold across all keys in this {@link PerKeyHolds}, or THE_END_OF_TIME if
+ * there are no holds within this {@link PerKeyHolds}.
+ */
+ public Instant getMinHold() {
+ return allHolds.isEmpty() ? THE_END_OF_TIME.get() : allHolds.peek().getTimestamp();
+ }
+
+ /**
+ * Updates the hold of the provided key to the provided value, removing any other holds for
+ * the same key.
+ */
+ public void updateHold(@Nullable Object key, Instant newHold) {
+ removeHold(key);
+ KeyedHold newKeyedHold = KeyedHold.of(key, newHold);
+ keyedHolds.put(key, newKeyedHold);
+ allHolds.offer(newKeyedHold);
+ }
+
+ /**
+ * Removes the hold of the provided key.
+ */
+ public void removeHold(Object key) {
+ KeyedHold oldHold = keyedHolds.get(key);
+ if (oldHold != null) {
+ allHolds.remove(oldHold);
+ }
+ }
+ }
+
+ /**
+ * A reference to the input and output watermarks of an {@link AppliedPTransform}.
+ */
+ public class TransformWatermarks {
+ private final AppliedPTransformInputWatermark inputWatermark;
+ private final AppliedPTransformOutputWatermark outputWatermark;
+
+ private final SynchronizedProcessingTimeInputWatermark synchronizedProcessingInputWatermark;
+ private final SynchronizedProcessingTimeOutputWatermark synchronizedProcessingOutputWatermark;
+
+ private Instant latestSynchronizedInputWm;
+ private Instant latestSynchronizedOutputWm;
+
+ private TransformWatermarks(
+ AppliedPTransformInputWatermark inputWatermark,
+ AppliedPTransformOutputWatermark outputWatermark,
+ SynchronizedProcessingTimeInputWatermark inputSynchProcessingWatermark,
+ SynchronizedProcessingTimeOutputWatermark outputSynchProcessingWatermark) {
+ this.inputWatermark = inputWatermark;
+ this.outputWatermark = outputWatermark;
+
+ this.synchronizedProcessingInputWatermark = inputSynchProcessingWatermark;
+ this.synchronizedProcessingOutputWatermark = outputSynchProcessingWatermark;
+ this.latestSynchronizedInputWm = BoundedWindow.TIMESTAMP_MIN_VALUE;
+ this.latestSynchronizedOutputWm = BoundedWindow.TIMESTAMP_MIN_VALUE;
+ }
+
+ /**
+ * Returns the input watermark of the {@link AppliedPTransform}.
+ */
+ public Instant getInputWatermark() {
+ return Preconditions.checkNotNull(inputWatermark.get());
+ }
+
+ /**
+ * Returns the output watermark of the {@link AppliedPTransform}.
+ */
+ public Instant getOutputWatermark() {
+ return outputWatermark.get();
+ }
+
+ /**
+ * Returns the synchronized processing input time of the {@link AppliedPTransform}.
+ *
+ * <p>The returned value is guaranteed to be monotonically increasing, and outside of the
+ * presence of holds, will increase as the system time progresses.
+ */
+ public synchronized Instant getSynchronizedProcessingInputTime() {
+ latestSynchronizedInputWm = INSTANT_ORDERING.max(
+ latestSynchronizedInputWm,
+ INSTANT_ORDERING.min(clock.now(), synchronizedProcessingInputWatermark.get()));
+ return latestSynchronizedInputWm;
+ }
+
+ /**
+ * Returns the synchronized processing output time of the {@link AppliedPTransform}.
+ *
+ * <p>The returned value is guaranteed to be monotonically increasing, and outside of the
+ * presence of holds, will increase as the system time progresses.
+ */
+ public synchronized Instant getSynchronizedProcessingOutputTime() {
+ latestSynchronizedOutputWm = INSTANT_ORDERING.max(
+ latestSynchronizedOutputWm,
+ INSTANT_ORDERING.min(clock.now(), synchronizedProcessingOutputWatermark.get()));
+ return latestSynchronizedOutputWm;
+ }
+
+ private WatermarkUpdate refresh() {
+ inputWatermark.refresh();
+ synchronizedProcessingInputWatermark.refresh();
+ WatermarkUpdate eventOutputUpdate = outputWatermark.refresh();
+ WatermarkUpdate syncOutputUpdate = synchronizedProcessingOutputWatermark.refresh();
+ return eventOutputUpdate.union(syncOutputUpdate);
+ }
+
+ private void setEventTimeHold(Object key, Instant newHold) {
+ outputWatermark.updateHold(key, newHold);
+ }
+
+ private void removePending(CommittedBundle<?> bundle) {
+ inputWatermark.removePendingElements(elementsFromBundle(bundle));
+ synchronizedProcessingInputWatermark.removePending(bundle);
+ }
+
+ private void addPending(CommittedBundle<?> bundle) {
+ inputWatermark.addPendingElements(elementsFromBundle(bundle));
+ synchronizedProcessingInputWatermark.addPending(bundle);
+ }
+
+ private Iterable<? extends WindowedValue<?>> elementsFromBundle(CommittedBundle<?> bundle) {
+ return FluentIterable.from(bundle.getElements()).transformAndConcat(EXPLODE_WINDOWS_FN);
+ }
+
+ private Map<Object, FiredTimers> extractFiredTimers() {
+ Map<Object, List<TimerData>> eventTimeTimers = inputWatermark.extractFiredEventTimeTimers();
+ Map<Object, List<TimerData>> processingTimers;
+ Map<Object, List<TimerData>> synchronizedTimers;
+ if (inputWatermark.get().equals(BoundedWindow.TIMESTAMP_MAX_VALUE)) {
+ processingTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers(
+ TimeDomain.PROCESSING_TIME, BoundedWindow.TIMESTAMP_MAX_VALUE);
+ synchronizedTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers(
+ TimeDomain.PROCESSING_TIME, BoundedWindow.TIMESTAMP_MAX_VALUE);
+ } else {
+ processingTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers(
+ TimeDomain.PROCESSING_TIME, clock.now());
+ synchronizedTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers(
+ TimeDomain.SYNCHRONIZED_PROCESSING_TIME, getSynchronizedProcessingInputTime());
+ }
+ Map<Object, Map<TimeDomain, List<TimerData>>> groupedTimers = new HashMap<>();
+ groupFiredTimers(groupedTimers, eventTimeTimers, processingTimers, synchronizedTimers);
+
+ Map<Object, FiredTimers> keyFiredTimers = new HashMap<>();
+ for (Map.Entry<Object, Map<TimeDomain, List<TimerData>>> firedTimers :
+ groupedTimers.entrySet()) {
+ keyFiredTimers.put(firedTimers.getKey(), new FiredTimers(firedTimers.getValue()));
+ }
+ return keyFiredTimers;
+ }
+
+ @SafeVarargs
+ private final void groupFiredTimers(
+ Map<Object, Map<TimeDomain, List<TimerData>>> groupedToMutate,
+ Map<Object, List<TimerData>>... timersToGroup) {
+ for (Map<Object, List<TimerData>> subGroup : timersToGroup) {
+ for (Map.Entry<Object, List<TimerData>> newTimers : subGroup.entrySet()) {
+ Map<TimeDomain, List<TimerData>> grouped = groupedToMutate.get(newTimers.getKey());
+ if (grouped == null) {
+ grouped = new HashMap<>();
+ groupedToMutate.put(newTimers.getKey(), grouped);
+ }
+ grouped.put(newTimers.getValue().get(0).getDomain(), newTimers.getValue());
+ }
+ }
+ }
+
+ private void updateTimers(TimerUpdate update) {
+ inputWatermark.updateTimers(update);
+ synchronizedProcessingInputWatermark.updateTimers(update);
+ }
+
+ @Override
+ public String toString() {
+ return MoreObjects.toStringHelper(TransformWatermarks.class)
+ .add("inputWatermark", inputWatermark)
+ .add("outputWatermark", outputWatermark)
+ .add("inputProcessingTime", synchronizedProcessingInputWatermark)
+ .add("outputProcessingTime", synchronizedProcessingOutputWatermark)
+ .toString();
+ }
+ }
+
+ /**
+ * A collection of newly set, deleted, and completed timers.
+ *
+ * <p>setTimers and deletedTimers are collections of {@link TimerData} that have been added to the
+ * {@link TimerInternals} of an executed step. completedTimers are timers that were delivered as
+ * the input to the executed step.
+ */
+ public static class TimerUpdate {
+ private final Object key;
+ private final Iterable<? extends TimerData> completedTimers;
+
+ private final Iterable<? extends TimerData> setTimers;
+ private final Iterable<? extends TimerData> deletedTimers;
+
+ /**
+ * Returns a TimerUpdate for a null key with no timers.
+ */
+ public static TimerUpdate empty() {
+ return new TimerUpdate(
+ null,
+ Collections.<TimerData>emptyList(),
+ Collections.<TimerData>emptyList(),
+ Collections.<TimerData>emptyList());
+ }
+
+ /**
+ * Creates a new {@link TimerUpdate} builder with the provided completed timers that needs the
+ * set and deleted timers to be added to it.
+ */
+ public static TimerUpdateBuilder builder(Object key) {
+ return new TimerUpdateBuilder(key);
+ }
+
+ /**
+ * A {@link TimerUpdate} builder that needs to be provided with set timers and deleted timers.
+ */
+ public static final class TimerUpdateBuilder {
+ private final Object key;
+ private final Collection<TimerData> completedTimers;
+ private final Collection<TimerData> setTimers;
+ private final Collection<TimerData> deletedTimers;
+
+ private TimerUpdateBuilder(Object key) {
+ this.key = key;
+ this.completedTimers = new HashSet<>();
+ this.setTimers = new HashSet<>();
+ this.deletedTimers = new HashSet<>();
+ }
+
+ /**
+ * Adds all of the provided timers to the collection of completed timers, and returns this
+ * {@link TimerUpdateBuilder}.
+ */
+ public TimerUpdateBuilder withCompletedTimers(Iterable<TimerData> completedTimers) {
+ Iterables.addAll(this.completedTimers, completedTimers);
+ return this;
+ }
+
+ /**
+ * Adds the provided timer to the collection of set timers, removing it from deleted timers if
+ * it has previously been deleted. Returns this {@link TimerUpdateBuilder}.
+ */
+ public TimerUpdateBuilder setTimer(TimerData setTimer) {
+ deletedTimers.remove(setTimer);
+ setTimers.add(setTimer);
+ return this;
+ }
+
+ /**
+ * Adds the provided timer to the collection of deleted timers, removing it from set timers if
+ * it has previously been set. Returns this {@link TimerUpdateBuilder}.
+ */
+ public TimerUpdateBuilder deletedTimer(TimerData deletedTimer) {
+ deletedTimers.add(deletedTimer);
+ setTimers.remove(deletedTimer);
+ return this;
+ }
+
+ /**
+ * Returns a new {@link TimerUpdate} with the most recently set completedTimers, setTimers,
+ * and deletedTimers.
+ */
+ public TimerUpdate build() {
+ return new TimerUpdate(
+ key,
+ ImmutableSet.copyOf(completedTimers),
+ ImmutableSet.copyOf(setTimers),
+ ImmutableSet.copyOf(deletedTimers));
+ }
+ }
+
+ private TimerUpdate(
+ Object key,
+ Iterable<? extends TimerData> completedTimers,
+ Iterable<? extends TimerData> setTimers,
+ Iterable<? extends TimerData> deletedTimers) {
+ this.key = key;
+ this.completedTimers = completedTimers;
+ this.setTimers = setTimers;
+ this.deletedTimers = deletedTimers;
+ }
+
+ @VisibleForTesting
+ Object getKey() {
+ return key;
+ }
+
+ @VisibleForTesting
+ Iterable<? extends TimerData> getCompletedTimers() {
+ return completedTimers;
+ }
+
+ @VisibleForTesting
+ Iterable<? extends TimerData> getSetTimers() {
+ return setTimers;
+ }
+
+ @VisibleForTesting
+ Iterable<? extends TimerData> getDeletedTimers() {
+ return deletedTimers;
+ }
+
+ /**
+ * Returns a {@link TimerUpdate} that is like this one, but with the specified completed timers.
+ */
+ public TimerUpdate withCompletedTimers(Iterable<TimerData> completedTimers) {
+ return new TimerUpdate(this.key, completedTimers, setTimers, deletedTimers);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(key, completedTimers, setTimers, deletedTimers);
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (other == null || !(other instanceof TimerUpdate)) {
+ return false;
+ }
+ TimerUpdate that = (TimerUpdate) other;
+ return Objects.equals(this.key, that.key)
+ && Objects.equals(this.completedTimers, that.completedTimers)
+ && Objects.equals(this.setTimers, that.setTimers)
+ && Objects.equals(this.deletedTimers, that.deletedTimers);
+ }
+ }
+
+ /**
+ * A pair of {@link TimerData} and key which can be delivered to the appropriate
+ * {@link AppliedPTransform}. A timer fires at the transform that set it with a specific key when
+ * the time domain in which it lives progresses past a specified time, as determined by the
+ * {@link InMemoryWatermarkManager}.
+ */
+ public static class FiredTimers {
+ private final Map<TimeDomain, ? extends Collection<TimerData>> timers;
+
+ private FiredTimers(Map<TimeDomain, ? extends Collection<TimerData>> timers) {
+ this.timers = timers;
+ }
+
+ /**
+ * Gets all of the timers that have fired within the provided {@link TimeDomain}. If no timers
+ * fired within the provided domain, return an empty collection.
+ *
+ * <p>Timers within a {@link TimeDomain} are guaranteed to be in order of increasing timestamp.
+ */
+ public Collection<TimerData> getTimers(TimeDomain domain) {
+ Collection<TimerData> domainTimers = timers.get(domain);
+ if (domainTimers == null) {
+ return Collections.emptyList();
+ }
+ return domainTimers;
+ }
+
+ @Override
+ public String toString() {
+ return MoreObjects.toStringHelper(FiredTimers.class).add("timers", timers).toString();
+ }
+ }
+
+ private static class WindowedValueByTimestampComparator extends Ordering<WindowedValue<?>> {
+ @Override
+ public int compare(WindowedValue<?> o1, WindowedValue<?> o2) {
+ return ComparisonChain.start()
+ .compare(o1.getTimestamp(), o2.getTimestamp())
+ .result();
+ }
+ }
+
+ public Set<AppliedPTransform<?, ?, ?>> getCompletedTransforms() {
+ Set<AppliedPTransform<?, ?, ?>> result = new HashSet<>();
+ for (Map.Entry<AppliedPTransform<?, ?, ?>, TransformWatermarks> wms :
+ transformToWatermarks.entrySet()) {
+ if (wms.getValue().getOutputWatermark().equals(THE_END_OF_TIME.get())) {
+ result.add(wms.getKey());
+ }
+ }
+ return result;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessBundleFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessBundleFactory.java
new file mode 100644
index 0000000..bc9b04c
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessBundleFactory.java
@@ -0,0 +1,162 @@
+/*
+ * 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 com.google.common.base.Preconditions.checkState;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.common.base.MoreObjects;
+import com.google.common.collect.ImmutableList;
+
+import org.joda.time.Instant;
+
+import javax.annotation.Nullable;
+
+/**
+ * A factory that produces bundles that perform no additional validation.
+ */
+class InProcessBundleFactory implements BundleFactory {
+ public static InProcessBundleFactory create() {
+ return new InProcessBundleFactory();
+ }
+
+ private InProcessBundleFactory() {}
+
+ @Override
+ public <T> UncommittedBundle<T> createRootBundle(PCollection<T> output) {
+ return InProcessBundle.create(output, null);
+ }
+
+ @Override
+ public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output) {
+ return InProcessBundle.create(output, input.getKey());
+ }
+
+ @Override
+ public <T> UncommittedBundle<T> createKeyedBundle(
+ CommittedBundle<?> input, @Nullable Object key, PCollection<T> output) {
+ return InProcessBundle.create(output, key);
+ }
+
+ /**
+ * A {@link UncommittedBundle} that buffers elements in memory.
+ */
+ private static final class InProcessBundle<T> implements UncommittedBundle<T> {
+ private final PCollection<T> pcollection;
+ @Nullable private final Object key;
+ private boolean committed = false;
+ private ImmutableList.Builder<WindowedValue<T>> elements;
+
+ /**
+ * Create a new {@link InProcessBundle} for the specified {@link PCollection}.
+ */
+ public static <T> InProcessBundle<T> create(PCollection<T> pcollection, @Nullable Object key) {
+ return new InProcessBundle<T>(pcollection, key);
+ }
+
+ private InProcessBundle(PCollection<T> pcollection, Object key) {
+ this.pcollection = pcollection;
+ this.key = key;
+ this.elements = ImmutableList.builder();
+ }
+
+ @Override
+ public PCollection<T> getPCollection() {
+ return pcollection;
+ }
+
+ @Override
+ public InProcessBundle<T> add(WindowedValue<T> element) {
+ checkState(
+ !committed,
+ "Can't add element %s to committed bundle in PCollection %s",
+ element,
+ pcollection);
+ elements.add(element);
+ return this;
+ }
+
+ @Override
+ public CommittedBundle<T> commit(final Instant synchronizedCompletionTime) {
+ checkState(!committed, "Can't commit already committed bundle %s", this);
+ committed = true;
+ final Iterable<WindowedValue<T>> committedElements = elements.build();
+ return new CommittedInProcessBundle<>(
+ pcollection, key, committedElements, synchronizedCompletionTime);
+ }
+ }
+
+ private static class CommittedInProcessBundle<T> implements CommittedBundle<T> {
+ public CommittedInProcessBundle(
+ PCollection<T> pcollection,
+ Object key,
+ Iterable<WindowedValue<T>> committedElements,
+ Instant synchronizedCompletionTime) {
+ this.pcollection = pcollection;
+ this.key = key;
+ this.committedElements = committedElements;
+ this.synchronizedCompletionTime = synchronizedCompletionTime;
+ }
+
+ private final PCollection<T> pcollection;
+ private final Object key;
+ private final Iterable<WindowedValue<T>> committedElements;
+ private final Instant synchronizedCompletionTime;
+
+ @Override
+ @Nullable
+ public Object getKey() {
+ return key;
+ }
+
+ @Override
+ public Iterable<WindowedValue<T>> getElements() {
+ return committedElements;
+ }
+
+ @Override
+ public PCollection<T> getPCollection() {
+ return pcollection;
+ }
+
+ @Override
+ public Instant getSynchronizedProcessingOutputWatermark() {
+ return synchronizedCompletionTime;
+ }
+
+ @Override
+ public String toString() {
+ return MoreObjects.toStringHelper(this)
+ .omitNullValues()
+ .add("pcollection", pcollection)
+ .add("key", key)
+ .add("elements", committedElements)
+ .toString();
+ }
+
+ @Override
+ public CommittedBundle<T> withElements(Iterable<WindowedValue<T>> elements) {
+ return new CommittedInProcessBundle<>(
+ pcollection, key, ImmutableList.copyOf(elements), synchronizedCompletionTime);
+ }
+ }
+}
[05/17] incubator-beam git commit: Move InProcessRunner to its own
module
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactory.java
deleted file mode 100644
index fac5a40..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactory.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.io.TextIO.Write.Bound;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.IOChannelUtils;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-
-class TextIOShardedWriteFactory implements PTransformOverrideFactory {
-
- @Override
- public <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
- PTransform<InputT, OutputT> transform) {
- if (transform instanceof TextIO.Write.Bound) {
- @SuppressWarnings("unchecked")
- TextIO.Write.Bound<InputT> originalWrite = (TextIO.Write.Bound<InputT>) transform;
- if (originalWrite.getNumShards() > 1
- || (originalWrite.getNumShards() == 1
- && !"".equals(originalWrite.getShardNameTemplate()))) {
- @SuppressWarnings("unchecked")
- PTransform<InputT, OutputT> override =
- (PTransform<InputT, OutputT>) new TextIOShardedWrite<InputT>(originalWrite);
- return override;
- }
- }
- return transform;
- }
-
- private static class TextIOShardedWrite<InputT> extends ShardControlledWrite<InputT> {
- private final TextIO.Write.Bound<InputT> initial;
-
- private TextIOShardedWrite(Bound<InputT> initial) {
- this.initial = initial;
- }
-
- @Override
- int getNumShards() {
- return initial.getNumShards();
- }
-
- @Override
- PTransform<PCollection<InputT>, PDone> getSingleShardTransform(int shardNum) {
- String shardName =
- IOChannelUtils.constructName(
- initial.getFilenamePrefix(),
- initial.getShardTemplate(),
- initial.getFilenameSuffix(),
- shardNum,
- getNumShards());
- return TextIO.Write.withCoder(initial.getCoder()).to(shardName).withoutSharding();
- }
-
- @Override
- protected PTransform<PCollection<InputT>, PDone> delegate() {
- return initial;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluator.java
deleted file mode 100644
index e002329..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluator.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.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.util.WindowedValue;
-
-/**
- * An evaluator of a specific application of a transform. Will be used for at least one
- * {@link CommittedBundle}.
- *
- * @param <InputT> the type of elements that will be passed to {@link #processElement}
- */
-public interface TransformEvaluator<InputT> {
- /**
- * Process an element in the input {@link CommittedBundle}.
- *
- * @param element the element to process
- */
- void processElement(WindowedValue<InputT> element) throws Exception;
-
- /**
- * Finish processing the bundle of this {@link TransformEvaluator}.
- *
- * After {@link #finishBundle()} is called, the {@link TransformEvaluator} will not be reused,
- * and no more elements will be processed.
- *
- * @return an {@link InProcessTransformResult} containing the results of this bundle evaluation.
- */
- InProcessTransformResult finishBundle() throws Exception;
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorFactory.java
deleted file mode 100644
index a9f6759..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorFactory.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-
-import javax.annotation.Nullable;
-
-/**
- * A factory for creating instances of {@link TransformEvaluator} for the application of a
- * {@link PTransform}.
- */
-public interface TransformEvaluatorFactory {
- /**
- * Create a new {@link TransformEvaluator} for the application of the {@link PTransform}.
- *
- * Any work that must be done before input elements are processed (such as calling
- * {@link DoFn#startBundle(DoFn.Context)}) must be done before the {@link TransformEvaluator} is
- * made available to the caller.
- *
- * @throws Exception whenever constructing the underlying evaluator throws an exception
- */
- <InputT> TransformEvaluator<InputT> forApplication(
- AppliedPTransform<?, ?, ?> application, @Nullable CommittedBundle<?> inputBundle,
- InProcessEvaluationContext evaluationContext) throws Exception;
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorRegistry.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorRegistry.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorRegistry.java
deleted file mode 100644
index f6542b8..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorRegistry.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Flatten.FlattenPCollectionList;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.Window;
-
-import com.google.common.collect.ImmutableMap;
-
-import java.util.Map;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link TransformEvaluatorFactory} that delegates to primitive {@link TransformEvaluatorFactory}
- * implementations based on the type of {@link PTransform} of the application.
- */
-class TransformEvaluatorRegistry implements TransformEvaluatorFactory {
- public static TransformEvaluatorRegistry defaultRegistry() {
- @SuppressWarnings("rawtypes")
- ImmutableMap<Class<? extends PTransform>, TransformEvaluatorFactory> primitives =
- ImmutableMap.<Class<? extends PTransform>, TransformEvaluatorFactory>builder()
- .put(Read.Bounded.class, new BoundedReadEvaluatorFactory())
- .put(Read.Unbounded.class, new UnboundedReadEvaluatorFactory())
- .put(ParDo.Bound.class, new ParDoSingleEvaluatorFactory())
- .put(ParDo.BoundMulti.class, new ParDoMultiEvaluatorFactory())
- .put(
- GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly.class,
- new GroupByKeyEvaluatorFactory())
- .put(FlattenPCollectionList.class, new FlattenEvaluatorFactory())
- .put(ViewEvaluatorFactory.WriteView.class, new ViewEvaluatorFactory())
- .put(Window.Bound.class, new WindowEvaluatorFactory())
- .build();
- return new TransformEvaluatorRegistry(primitives);
- }
-
- // the TransformEvaluatorFactories can construct instances of all generic types of transform,
- // so all instances of a primitive can be handled with the same evaluator factory.
- @SuppressWarnings("rawtypes")
- private final Map<Class<? extends PTransform>, TransformEvaluatorFactory> factories;
-
- private TransformEvaluatorRegistry(
- @SuppressWarnings("rawtypes")
- Map<Class<? extends PTransform>, TransformEvaluatorFactory> factories) {
- this.factories = factories;
- }
-
- @Override
- public <InputT> TransformEvaluator<InputT> forApplication(
- AppliedPTransform<?, ?, ?> application,
- @Nullable CommittedBundle<?> inputBundle,
- InProcessEvaluationContext evaluationContext)
- throws Exception {
- TransformEvaluatorFactory factory = factories.get(application.getTransform().getClass());
- return factory.forApplication(application, inputBundle, evaluationContext);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutor.java
deleted file mode 100644
index a93c7b2..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutor.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.util.WindowedValue;
-
-import com.google.common.base.Throwables;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.concurrent.Callable;
-import java.util.concurrent.atomic.AtomicReference;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link Callable} responsible for constructing a {@link TransformEvaluator} from a
- * {@link TransformEvaluatorFactory} and evaluating it on some bundle of input, and registering
- * the result using a registered {@link CompletionCallback}.
- *
- * <p>A {@link TransformExecutor} that is currently executing also provides access to the thread
- * that it is being executed on.
- */
-class TransformExecutor<T> implements Callable<InProcessTransformResult> {
- public static <T> TransformExecutor<T> create(
- TransformEvaluatorFactory factory,
- Iterable<? extends ModelEnforcementFactory> modelEnforcements,
- InProcessEvaluationContext evaluationContext,
- CommittedBundle<T> inputBundle,
- AppliedPTransform<?, ?, ?> transform,
- CompletionCallback completionCallback,
- TransformExecutorService transformEvaluationState) {
- return new TransformExecutor<>(
- factory,
- modelEnforcements,
- evaluationContext,
- inputBundle,
- transform,
- completionCallback,
- transformEvaluationState);
- }
-
- private final TransformEvaluatorFactory evaluatorFactory;
- private final Iterable<? extends ModelEnforcementFactory> modelEnforcements;
-
- private final InProcessEvaluationContext evaluationContext;
-
- /** The transform that will be evaluated. */
- private final AppliedPTransform<?, ?, ?> transform;
- /** The inputs this {@link TransformExecutor} will deliver to the transform. */
- private final CommittedBundle<T> inputBundle;
-
- private final CompletionCallback onComplete;
- private final TransformExecutorService transformEvaluationState;
-
- private final AtomicReference<Thread> thread;
-
- private TransformExecutor(
- TransformEvaluatorFactory factory,
- Iterable<? extends ModelEnforcementFactory> modelEnforcements,
- InProcessEvaluationContext evaluationContext,
- CommittedBundle<T> inputBundle,
- AppliedPTransform<?, ?, ?> transform,
- CompletionCallback completionCallback,
- TransformExecutorService transformEvaluationState) {
- this.evaluatorFactory = factory;
- this.modelEnforcements = modelEnforcements;
- this.evaluationContext = evaluationContext;
-
- this.inputBundle = inputBundle;
- this.transform = transform;
-
- this.onComplete = completionCallback;
-
- this.transformEvaluationState = transformEvaluationState;
- this.thread = new AtomicReference<>();
- }
-
- @Override
- public InProcessTransformResult call() {
- checkState(
- thread.compareAndSet(null, Thread.currentThread()),
- "Tried to execute %s for %s on thread %s, but is already executing on thread %s",
- TransformExecutor.class.getSimpleName(),
- transform.getFullName(),
- Thread.currentThread(),
- thread.get());
- try {
- Collection<ModelEnforcement<T>> enforcements = new ArrayList<>();
- for (ModelEnforcementFactory enforcementFactory : modelEnforcements) {
- ModelEnforcement<T> enforcement = enforcementFactory.forBundle(inputBundle, transform);
- enforcements.add(enforcement);
- }
- TransformEvaluator<T> evaluator =
- evaluatorFactory.forApplication(transform, inputBundle, evaluationContext);
-
- processElements(evaluator, enforcements);
-
- InProcessTransformResult result = finishBundle(evaluator, enforcements);
- return result;
- } catch (Throwable t) {
- onComplete.handleThrowable(inputBundle, t);
- throw Throwables.propagate(t);
- } finally {
- transformEvaluationState.complete(this);
- }
- }
-
- /**
- * Processes all the elements in the input bundle using the transform evaluator, applying any
- * necessary {@link ModelEnforcement ModelEnforcements}.
- */
- private void processElements(
- TransformEvaluator<T> evaluator, Collection<ModelEnforcement<T>> enforcements)
- throws Exception {
- if (inputBundle != null) {
- for (WindowedValue<T> value : inputBundle.getElements()) {
- for (ModelEnforcement<T> enforcement : enforcements) {
- enforcement.beforeElement(value);
- }
-
- evaluator.processElement(value);
-
- for (ModelEnforcement<T> enforcement : enforcements) {
- enforcement.afterElement(value);
- }
- }
- }
- }
-
- /**
- * Finishes processing the input bundle and commit the result using the
- * {@link CompletionCallback}, applying any {@link ModelEnforcement} if necessary.
- *
- * @return the {@link InProcessTransformResult} produced by
- * {@link TransformEvaluator#finishBundle()}
- */
- private InProcessTransformResult finishBundle(
- TransformEvaluator<T> evaluator, Collection<ModelEnforcement<T>> enforcements)
- throws Exception {
- InProcessTransformResult result = evaluator.finishBundle();
- CommittedResult outputs = onComplete.handleResult(inputBundle, result);
- for (ModelEnforcement<T> enforcement : enforcements) {
- enforcement.afterFinish(inputBundle, result, outputs.getOutputs());
- }
- return result;
- }
-
- /**
- * If this {@link TransformExecutor} is currently executing, return the thread it is executing in.
- * Otherwise, return null.
- */
- @Nullable
- public Thread getThread() {
- return thread.get();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorService.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorService.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorService.java
deleted file mode 100644
index 600102f..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorService.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-/**
- * Schedules and completes {@link TransformExecutor TransformExecutors}, controlling concurrency as
- * appropriate for the {@link StepAndKey} the executor exists for.
- */
-interface TransformExecutorService {
- /**
- * Schedule the provided work to be eventually executed.
- */
- void schedule(TransformExecutor<?> work);
-
- /**
- * Finish executing the provided work. This may cause additional
- * {@link TransformExecutor TransformExecutors} to be evaluated.
- */
- void complete(TransformExecutor<?> completed);
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServices.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServices.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServices.java
deleted file mode 100644
index 3194340..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServices.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import com.google.common.base.MoreObjects;
-
-import java.util.Map;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * Static factory methods for constructing instances of {@link TransformExecutorService}.
- */
-final class TransformExecutorServices {
- private TransformExecutorServices() {
- // Do not instantiate
- }
-
- /**
- * Returns an EvaluationState that evaluates {@link TransformExecutor TransformExecutors} in
- * parallel.
- */
- public static TransformExecutorService parallel(
- ExecutorService executor, Map<TransformExecutor<?>, Boolean> scheduled) {
- return new ParallelEvaluationState(executor, scheduled);
- }
-
- /**
- * Returns an EvaluationState that evaluates {@link TransformExecutor TransformExecutors} in
- * serial.
- */
- public static TransformExecutorService serial(
- ExecutorService executor, Map<TransformExecutor<?>, Boolean> scheduled) {
- return new SerialEvaluationState(executor, scheduled);
- }
-
- /**
- * A {@link TransformExecutorService} with unlimited parallelism. Any {@link TransformExecutor}
- * scheduled will be immediately submitted to the {@link ExecutorService}.
- *
- * <p>A principal use of this is for the evaluation of an unkeyed Step. Unkeyed computations are
- * processed in parallel.
- */
- private static class ParallelEvaluationState implements TransformExecutorService {
- private final ExecutorService executor;
- private final Map<TransformExecutor<?>, Boolean> scheduled;
-
- private ParallelEvaluationState(
- ExecutorService executor, Map<TransformExecutor<?>, Boolean> scheduled) {
- this.executor = executor;
- this.scheduled = scheduled;
- }
-
- @Override
- public void schedule(TransformExecutor<?> work) {
- executor.submit(work);
- scheduled.put(work, true);
- }
-
- @Override
- public void complete(TransformExecutor<?> completed) {
- scheduled.remove(completed);
- }
- }
-
- /**
- * A {@link TransformExecutorService} with a single work queue. Any {@link TransformExecutor}
- * scheduled will be placed on the work queue. Only one item of work will be submitted to the
- * {@link ExecutorService} at any time.
- *
- * <p>A principal use of this is for the serial evaluation of a (Step, Key) pair.
- * Keyed computations are processed serially per step.
- */
- private static class SerialEvaluationState implements TransformExecutorService {
- private final ExecutorService executor;
- private final Map<TransformExecutor<?>, Boolean> scheduled;
-
- private AtomicReference<TransformExecutor<?>> currentlyEvaluating;
- private final Queue<TransformExecutor<?>> workQueue;
-
- private SerialEvaluationState(
- ExecutorService executor, Map<TransformExecutor<?>, Boolean> scheduled) {
- this.scheduled = scheduled;
- this.executor = executor;
- this.currentlyEvaluating = new AtomicReference<>();
- this.workQueue = new ConcurrentLinkedQueue<>();
- }
-
- /**
- * Schedules the work, adding it to the work queue if there is a bundle currently being
- * evaluated and scheduling it immediately otherwise.
- */
- @Override
- public void schedule(TransformExecutor<?> work) {
- workQueue.offer(work);
- updateCurrentlyEvaluating();
- }
-
- @Override
- public void complete(TransformExecutor<?> completed) {
- if (!currentlyEvaluating.compareAndSet(completed, null)) {
- throw new IllegalStateException(
- "Finished work "
- + completed
- + " but could not complete due to unexpected currently executing "
- + currentlyEvaluating.get());
- }
- scheduled.remove(completed);
- updateCurrentlyEvaluating();
- }
-
- private void updateCurrentlyEvaluating() {
- if (currentlyEvaluating.get() == null) {
- // Only synchronize if we need to update what's currently evaluating
- synchronized (this) {
- TransformExecutor<?> newWork = workQueue.poll();
- if (newWork != null) {
- if (currentlyEvaluating.compareAndSet(null, newWork)) {
- scheduled.put(newWork, true);
- executor.submit(newWork);
- } else {
- workQueue.offer(newWork);
- }
- }
- }
- }
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(SerialEvaluationState.class)
- .add("currentlyEvaluating", currentlyEvaluating)
- .add("workQueue", workQueue)
- .toString();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactory.java
deleted file mode 100644
index 0cebf43..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactory.java
+++ /dev/null
@@ -1,177 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.io.Read.Unbounded;
-import org.apache.beam.sdk.io.UnboundedSource;
-import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
-import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-import java.io.IOException;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ConcurrentMap;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link TransformEvaluatorFactory} that produces {@link TransformEvaluator TransformEvaluators}
- * for the {@link Unbounded Read.Unbounded} primitive {@link PTransform}.
- */
-class UnboundedReadEvaluatorFactory implements TransformEvaluatorFactory {
- /*
- * An evaluator for a Source is stateful, to ensure the CheckpointMark is properly persisted.
- * Evaluators are cached here to ensure that the checkpoint mark is appropriately reused
- * and any splits are honored.
- */
- private final ConcurrentMap<EvaluatorKey, Queue<? extends UnboundedReadEvaluator<?>>>
- sourceEvaluators = new ConcurrentHashMap<>();
-
- @SuppressWarnings({"unchecked", "rawtypes"})
- @Override
- public <InputT> TransformEvaluator<InputT> forApplication(AppliedPTransform<?, ?, ?> application,
- @Nullable CommittedBundle<?> inputBundle, InProcessEvaluationContext evaluationContext) {
- return getTransformEvaluator((AppliedPTransform) application, evaluationContext);
- }
-
- private <OutputT> TransformEvaluator<?> getTransformEvaluator(
- final AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform,
- final InProcessEvaluationContext evaluationContext) {
- UnboundedReadEvaluator<?> currentEvaluator =
- getTransformEvaluatorQueue(transform, evaluationContext).poll();
- if (currentEvaluator == null) {
- return EmptyTransformEvaluator.create(transform);
- }
- return currentEvaluator;
- }
-
- /**
- * Get the queue of {@link TransformEvaluator TransformEvaluators} that produce elements for the
- * provided application of {@link Unbounded Read.Unbounded}, initializing it if required.
- *
- * <p>This method is thread-safe, and will only produce new evaluators if no other invocation has
- * already done so.
- */
- @SuppressWarnings("unchecked")
- private <OutputT> Queue<UnboundedReadEvaluator<OutputT>> getTransformEvaluatorQueue(
- final AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform,
- final InProcessEvaluationContext evaluationContext) {
- // Key by the application and the context the evaluation is occurring in (which call to
- // Pipeline#run).
- EvaluatorKey key = new EvaluatorKey(transform, evaluationContext);
- @SuppressWarnings("unchecked")
- Queue<UnboundedReadEvaluator<OutputT>> evaluatorQueue =
- (Queue<UnboundedReadEvaluator<OutputT>>) sourceEvaluators.get(key);
- if (evaluatorQueue == null) {
- evaluatorQueue = new ConcurrentLinkedQueue<>();
- if (sourceEvaluators.putIfAbsent(key, evaluatorQueue) == null) {
- // If no queue existed in the evaluators, add an evaluator to initialize the evaluator
- // factory for this transform
- UnboundedSource<OutputT, ?> source = transform.getTransform().getSource();
- UnboundedReadEvaluator<OutputT> evaluator =
- new UnboundedReadEvaluator<OutputT>(
- transform, evaluationContext, source, evaluatorQueue);
- evaluatorQueue.offer(evaluator);
- } else {
- // otherwise return the existing Queue that arrived before us
- evaluatorQueue = (Queue<UnboundedReadEvaluator<OutputT>>) sourceEvaluators.get(key);
- }
- }
- return evaluatorQueue;
- }
-
- /**
- * A {@link UnboundedReadEvaluator} produces elements from an underlying {@link UnboundedSource},
- * discarding all input elements. Within the call to {@link #finishBundle()}, the evaluator
- * creates the {@link UnboundedReader} and consumes some currently available input.
- *
- * <p>Calls to {@link UnboundedReadEvaluator} are not internally thread-safe, and should only be
- * used by a single thread at a time. Each {@link UnboundedReadEvaluator} maintains its own
- * checkpoint, and constructs its reader from the current checkpoint in each call to
- * {@link #finishBundle()}.
- */
- private static class UnboundedReadEvaluator<OutputT> implements TransformEvaluator<Object> {
- private static final int ARBITRARY_MAX_ELEMENTS = 10;
- private final AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform;
- private final InProcessEvaluationContext evaluationContext;
- private final Queue<UnboundedReadEvaluator<OutputT>> evaluatorQueue;
- /**
- * The source being read from by this {@link UnboundedReadEvaluator}. This may not be the same
- * source as derived from {@link #transform} due to splitting.
- */
- private final UnboundedSource<OutputT, ?> source;
- private CheckpointMark checkpointMark;
-
- public UnboundedReadEvaluator(
- AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform,
- InProcessEvaluationContext evaluationContext,
- UnboundedSource<OutputT, ?> source,
- Queue<UnboundedReadEvaluator<OutputT>> evaluatorQueue) {
- this.transform = transform;
- this.evaluationContext = evaluationContext;
- this.evaluatorQueue = evaluatorQueue;
- this.source = source;
- this.checkpointMark = null;
- }
-
- @Override
- public void processElement(WindowedValue<Object> element) {}
-
- @Override
- public InProcessTransformResult finishBundle() throws IOException {
- UncommittedBundle<OutputT> output = evaluationContext.createRootBundle(transform.getOutput());
- try (UnboundedReader<OutputT> reader =
- createReader(source, evaluationContext.getPipelineOptions());) {
- int numElements = 0;
- if (reader.start()) {
- do {
- output.add(
- WindowedValue.timestampedValueInGlobalWindow(
- reader.getCurrent(), reader.getCurrentTimestamp()));
- numElements++;
- } while (numElements < ARBITRARY_MAX_ELEMENTS && reader.advance());
- }
- checkpointMark = reader.getCheckpointMark();
- checkpointMark.finalizeCheckpoint();
- // TODO: When exercising create initial splits, make this the minimum watermark across all
- // existing readers
- StepTransformResult result =
- StepTransformResult.withHold(transform, reader.getWatermark())
- .addOutput(output)
- .build();
- evaluatorQueue.offer(this);
- return result;
- }
- }
-
- private <CheckpointMarkT extends CheckpointMark> UnboundedReader<OutputT> createReader(
- UnboundedSource<OutputT, CheckpointMarkT> source, PipelineOptions options) {
- @SuppressWarnings("unchecked")
- CheckpointMarkT mark = (CheckpointMarkT) checkpointMark;
- return source.createReader(options, mark);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactory.java
deleted file mode 100644
index 0b54ba8..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactory.java
+++ /dev/null
@@ -1,145 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.coders.VoidCoder;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.PCollectionViewWriter;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.Values;
-import org.apache.beam.sdk.transforms.View.CreatePCollectionView;
-import org.apache.beam.sdk.transforms.WithKeys;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
- * {@link CreatePCollectionView} primitive {@link PTransform}.
- *
- * <p>The {@link ViewEvaluatorFactory} produces {@link TransformEvaluator TransformEvaluators} for
- * the {@link WriteView} {@link PTransform}, which is part of the
- * {@link InProcessCreatePCollectionView} composite transform. This transform is an override for the
- * {@link CreatePCollectionView} transform that applies windowing and triggers before the view is
- * written.
- */
-class ViewEvaluatorFactory implements TransformEvaluatorFactory {
- @Override
- public <T> TransformEvaluator<T> forApplication(
- AppliedPTransform<?, ?, ?> application,
- InProcessPipelineRunner.CommittedBundle<?> inputBundle,
- InProcessEvaluationContext evaluationContext) {
- @SuppressWarnings({"cast", "unchecked", "rawtypes"})
- TransformEvaluator<T> evaluator = createEvaluator(
- (AppliedPTransform) application, evaluationContext);
- return evaluator;
- }
-
- private <InT, OuT> TransformEvaluator<Iterable<InT>> createEvaluator(
- final AppliedPTransform<PCollection<Iterable<InT>>, PCollectionView<OuT>, WriteView<InT, OuT>>
- application,
- InProcessEvaluationContext context) {
- PCollection<Iterable<InT>> input = application.getInput();
- final PCollectionViewWriter<InT, OuT> writer =
- context.createPCollectionViewWriter(input, application.getOutput());
- return new TransformEvaluator<Iterable<InT>>() {
- private final List<WindowedValue<InT>> elements = new ArrayList<>();
-
- @Override
- public void processElement(WindowedValue<Iterable<InT>> element) {
- for (InT input : element.getValue()) {
- elements.add(element.withValue(input));
- }
- }
-
- @Override
- public InProcessTransformResult finishBundle() {
- writer.add(elements);
- return StepTransformResult.withoutHold(application).build();
- }
- };
- }
-
- public static class InProcessViewOverrideFactory implements PTransformOverrideFactory {
- @Override
- public <InputT extends PInput, OutputT extends POutput>
- PTransform<InputT, OutputT> override(PTransform<InputT, OutputT> transform) {
- if (transform instanceof CreatePCollectionView) {
-
- }
- @SuppressWarnings({"rawtypes", "unchecked"})
- PTransform<InputT, OutputT> createView =
- (PTransform<InputT, OutputT>)
- new InProcessCreatePCollectionView<>((CreatePCollectionView) transform);
- return createView;
- }
- }
-
- /**
- * An in-process override for {@link CreatePCollectionView}.
- */
- private static class InProcessCreatePCollectionView<ElemT, ViewT>
- extends ForwardingPTransform<PCollection<ElemT>, PCollectionView<ViewT>> {
- private final CreatePCollectionView<ElemT, ViewT> og;
-
- private InProcessCreatePCollectionView(CreatePCollectionView<ElemT, ViewT> og) {
- this.og = og;
- }
-
- @Override
- public PCollectionView<ViewT> apply(PCollection<ElemT> input) {
- return input.apply(WithKeys.<Void, ElemT>of((Void) null))
- .setCoder(KvCoder.of(VoidCoder.of(), input.getCoder()))
- .apply(GroupByKey.<Void, ElemT>create())
- .apply(Values.<Iterable<ElemT>>create())
- .apply(new WriteView<ElemT, ViewT>(og));
- }
-
- @Override
- protected PTransform<PCollection<ElemT>, PCollectionView<ViewT>> delegate() {
- return og;
- }
- }
-
- /**
- * An in-process implementation of the {@link CreatePCollectionView} primitive.
- *
- * This implementation requires the input {@link PCollection} to be an iterable, which is provided
- * to {@link PCollectionView#fromIterableInternal(Iterable)}.
- */
- public static final class WriteView<ElemT, ViewT>
- extends PTransform<PCollection<Iterable<ElemT>>, PCollectionView<ViewT>> {
- private final CreatePCollectionView<ElemT, ViewT> og;
-
- WriteView(CreatePCollectionView<ElemT, ViewT> og) {
- this.og = og;
- }
-
- @Override
- public PCollectionView<ViewT> apply(PCollection<Iterable<ElemT>> input) {
- return og.getView();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutor.java
deleted file mode 100644
index 3e4aca6..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutor.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowingStrategy;
-
-import com.google.common.collect.ComparisonChain;
-import com.google.common.collect.Ordering;
-
-import org.joda.time.Instant;
-
-import java.util.PriorityQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-/**
- * Executes callbacks that occur based on the progression of the watermark per-step.
- *
- * <p>Callbacks are registered by calls to
- * {@link #callOnGuaranteedFiring(AppliedPTransform, BoundedWindow, WindowingStrategy, Runnable)},
- * and are executed after a call to {@link #fireForWatermark(AppliedPTransform, Instant)} with the
- * same {@link AppliedPTransform} and a watermark sufficient to ensure that the trigger for the
- * windowing strategy would have been produced.
- *
- * <p>NOTE: {@link WatermarkCallbackExecutor} does not track the latest observed watermark for any
- * {@link AppliedPTransform} - any call to
- * {@link #callOnGuaranteedFiring(AppliedPTransform, BoundedWindow, WindowingStrategy, Runnable)}
- * that could have potentially already fired should be followed by a call to
- * {@link #fireForWatermark(AppliedPTransform, Instant)} for the same transform with the current
- * value of the watermark.
- */
-class WatermarkCallbackExecutor {
- /**
- * Create a new {@link WatermarkCallbackExecutor}.
- */
- public static WatermarkCallbackExecutor create() {
- return new WatermarkCallbackExecutor();
- }
-
- private final ConcurrentMap<AppliedPTransform<?, ?, ?>, PriorityQueue<WatermarkCallback>>
- callbacks;
- private final ExecutorService executor;
-
- private WatermarkCallbackExecutor() {
- this.callbacks = new ConcurrentHashMap<>();
- this.executor = Executors.newSingleThreadExecutor();
- }
-
- /**
- * Execute the provided {@link Runnable} after the next call to
- * {@link #fireForWatermark(AppliedPTransform, Instant)} where the window is guaranteed to have
- * produced output.
- */
- public void callOnGuaranteedFiring(
- AppliedPTransform<?, ?, ?> step,
- BoundedWindow window,
- WindowingStrategy<?, ?> windowingStrategy,
- Runnable runnable) {
- WatermarkCallback callback =
- WatermarkCallback.onGuaranteedFiring(window, windowingStrategy, runnable);
-
- PriorityQueue<WatermarkCallback> callbackQueue = callbacks.get(step);
- if (callbackQueue == null) {
- callbackQueue = new PriorityQueue<>(11, new CallbackOrdering());
- if (callbacks.putIfAbsent(step, callbackQueue) != null) {
- callbackQueue = callbacks.get(step);
- }
- }
-
- synchronized (callbackQueue) {
- callbackQueue.offer(callback);
- }
- }
-
- /**
- * Schedule all pending callbacks that must have produced output by the time of the provided
- * watermark.
- */
- public void fireForWatermark(AppliedPTransform<?, ?, ?> step, Instant watermark) {
- PriorityQueue<WatermarkCallback> callbackQueue = callbacks.get(step);
- if (callbackQueue == null) {
- return;
- }
- synchronized (callbackQueue) {
- while (!callbackQueue.isEmpty() && callbackQueue.peek().shouldFire(watermark)) {
- executor.submit(callbackQueue.poll().getCallback());
- }
- }
- }
-
- private static class WatermarkCallback {
- public static <W extends BoundedWindow> WatermarkCallback onGuaranteedFiring(
- BoundedWindow window, WindowingStrategy<?, W> strategy, Runnable callback) {
- @SuppressWarnings("unchecked")
- Instant firingAfter =
- strategy.getTrigger().getSpec().getWatermarkThatGuaranteesFiring((W) window);
- return new WatermarkCallback(firingAfter, callback);
- }
-
- private final Instant fireAfter;
- private final Runnable callback;
-
- private WatermarkCallback(Instant fireAfter, Runnable callback) {
- this.fireAfter = fireAfter;
- this.callback = callback;
- }
-
- public boolean shouldFire(Instant currentWatermark) {
- return currentWatermark.isAfter(fireAfter)
- || currentWatermark.equals(BoundedWindow.TIMESTAMP_MAX_VALUE);
- }
-
- public Runnable getCallback() {
- return callback;
- }
- }
-
- private static class CallbackOrdering extends Ordering<WatermarkCallback> {
- @Override
- public int compare(WatermarkCallback left, WatermarkCallback right) {
- return ComparisonChain.start()
- .compare(left.fireAfter, right.fireAfter)
- .compare(left.callback, right.callback, Ordering.arbitrary())
- .result();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactory.java
deleted file mode 100644
index 4cdacec..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactory.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-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;
-
-import org.joda.time.Instant;
-
-import java.util.Collection;
-
-import javax.annotation.Nullable;
-
-/**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
- * {@link Bound Window.Bound} primitive {@link PTransform}.
- */
-class WindowEvaluatorFactory implements TransformEvaluatorFactory {
-
- @Override
- public <InputT> TransformEvaluator<InputT> forApplication(
- AppliedPTransform<?, ?, ?> application,
- @Nullable CommittedBundle<?> inputBundle,
- InProcessEvaluationContext evaluationContext)
- throws Exception {
- return createTransformEvaluator(
- (AppliedPTransform) application, inputBundle, evaluationContext);
- }
-
- private <InputT> TransformEvaluator<InputT> createTransformEvaluator(
- AppliedPTransform<PCollection<InputT>, PCollection<InputT>, Window.Bound<InputT>> transform,
- CommittedBundle<?> inputBundle,
- InProcessEvaluationContext evaluationContext) {
- WindowFn<? super InputT, ?> fn = transform.getTransform().getWindowFn();
- UncommittedBundle<InputT> outputBundle =
- evaluationContext.createBundle(inputBundle, transform.getOutput());
- if (fn == null) {
- return PassthroughTransformEvaluator.create(transform, outputBundle);
- }
- return new WindowIntoEvaluator<>(transform, fn, outputBundle);
- }
-
- private static class WindowIntoEvaluator<InputT> implements TransformEvaluator<InputT> {
- private final AppliedPTransform<PCollection<InputT>, PCollection<InputT>, Window.Bound<InputT>>
- transform;
- private final WindowFn<InputT, ?> windowFn;
- private final UncommittedBundle<InputT> outputBundle;
-
- @SuppressWarnings("unchecked")
- public WindowIntoEvaluator(
- AppliedPTransform<PCollection<InputT>, PCollection<InputT>, Window.Bound<InputT>> transform,
- WindowFn<? super InputT, ?> windowFn,
- UncommittedBundle<InputT> outputBundle) {
- this.outputBundle = outputBundle;
- this.transform = transform;
- // Safe contravariant cast
- this.windowFn = (WindowFn<InputT, ?>) windowFn;
- }
-
- @Override
- public void processElement(WindowedValue<InputT> element) throws Exception {
- Collection<? extends BoundedWindow> windows = assignWindows(windowFn, element);
- outputBundle.add(
- WindowedValue.<InputT>of(
- element.getValue(), element.getTimestamp(), windows, PaneInfo.NO_FIRING));
- }
-
- private <W extends BoundedWindow> Collection<? extends BoundedWindow> assignWindows(
- WindowFn<InputT, W> windowFn, WindowedValue<InputT> element) throws Exception {
- WindowFn<InputT, W>.AssignContext assignContext =
- new InProcessAssignContext<>(windowFn, element);
- Collection<? extends BoundedWindow> windows = windowFn.assignWindows(assignContext);
- return windows;
- }
-
- @Override
- public InProcessTransformResult finishBundle() throws Exception {
- return StepTransformResult.withoutHold(transform).addOutput(outputBundle).build();
- }
- }
-
- private static class InProcessAssignContext<InputT, W extends BoundedWindow>
- extends WindowFn<InputT, W>.AssignContext {
- private final WindowedValue<InputT> value;
-
- public InProcessAssignContext(WindowFn<InputT, W> fn, WindowedValue<InputT> value) {
- fn.super();
- this.value = value;
- }
-
- @Override
- public InputT element() {
- return value.getValue();
- }
-
- @Override
- public Instant timestamp() {
- return value.getTimestamp();
- }
-
- @Override
- public Collection<? extends BoundedWindow> windows() {
- return value.getWindows();
- }
-
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactoryTest.java
deleted file mode 100644
index 43367dd..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactoryTest.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.theInstance;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.io.AvroIO;
-import org.apache.beam.sdk.io.AvroIOTest;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-
-import org.hamcrest.Matchers;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.File;
-
-/**
- * Tests for {@link AvroIOShardedWriteFactory}.
- */
-@RunWith(JUnit4.class)
-public class AvroIOShardedWriteFactoryTest {
-
- @Rule public TemporaryFolder tmp = new TemporaryFolder();
- private AvroIOShardedWriteFactory factory;
-
- @Before
- public void setup() {
- factory = new AvroIOShardedWriteFactory();
- }
-
- @Test
- public void originalWithoutShardingReturnsOriginal() throws Exception {
- File file = tmp.newFile("foo");
- PTransform<PCollection<String>, PDone> original =
- AvroIO.Write.withSchema(String.class).to(file.getAbsolutePath()).withoutSharding();
- PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
- assertThat(overridden, theInstance(original));
- }
-
- @Test
- public void originalShardingNotSpecifiedReturnsOriginal() throws Exception {
- File file = tmp.newFile("foo");
- PTransform<PCollection<String>, PDone> original =
- AvroIO.Write.withSchema(String.class).to(file.getAbsolutePath());
- PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
- assertThat(overridden, theInstance(original));
- }
-
- @Test
- public void originalShardedToOneReturnsExplicitlySharded() throws Exception {
- File file = tmp.newFile("foo");
- AvroIO.Write.Bound<String> original =
- AvroIO.Write.withSchema(String.class).to(file.getAbsolutePath()).withNumShards(1);
- PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
- assertThat(overridden, not(Matchers.<PTransform<PCollection<String>, PDone>>equalTo(original)));
-
- TestPipeline p = TestPipeline.create();
- String[] elems = new String[] {"foo", "bar", "baz"};
- p.apply(Create.<String>of(elems)).apply(overridden);
-
- file.delete();
-
- p.run();
- AvroIOTest.assertTestOutputs(elems, 1, file.getAbsolutePath(), original.getShardNameTemplate());
- }
-
- @Test
- public void originalShardedToManyReturnsExplicitlySharded() throws Exception {
- File file = tmp.newFile("foo");
- AvroIO.Write.Bound<String> original =
- AvroIO.Write.withSchema(String.class).to(file.getAbsolutePath()).withNumShards(3);
- PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
- assertThat(overridden, not(Matchers.<PTransform<PCollection<String>, PDone>>equalTo(original)));
-
- TestPipeline p = TestPipeline.create();
- String[] elems = new String[] {"foo", "bar", "baz", "spam", "ham", "eggs"};
- p.apply(Create.<String>of(elems)).apply(overridden);
-
- file.delete();
- p.run();
- AvroIOTest.assertTestOutputs(elems, 3, file.getAbsolutePath(), original.getShardNameTemplate());
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactoryTest.java
deleted file mode 100644
index 146dd98..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactoryTest.java
+++ /dev/null
@@ -1,290 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.is;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.coders.BigEndianLongCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.io.BoundedSource;
-import org.apache.beam.sdk.io.BoundedSource.BoundedReader;
-import org.apache.beam.sdk.io.CountingSource;
-import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.io.Read.Bounded;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.common.collect.ImmutableList;
-
-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;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-/**
- * Tests for {@link BoundedReadEvaluatorFactory}.
- */
-@RunWith(JUnit4.class)
-public class BoundedReadEvaluatorFactoryTest {
- private BoundedSource<Long> source;
- private PCollection<Long> longs;
- private TransformEvaluatorFactory factory;
- @Mock private InProcessEvaluationContext context;
- private BundleFactory bundleFactory;
-
- @Before
- public void setup() {
- MockitoAnnotations.initMocks(this);
- source = CountingSource.upTo(10L);
- TestPipeline p = TestPipeline.create();
- longs = p.apply(Read.from(source));
-
- factory = new BoundedReadEvaluatorFactory();
- bundleFactory = InProcessBundleFactory.create();
- }
-
- @Test
- public void boundedSourceInMemoryTransformEvaluatorProducesElements() throws Exception {
- UncommittedBundle<Long> output = bundleFactory.createRootBundle(longs);
- when(context.createRootBundle(longs)).thenReturn(output);
-
- TransformEvaluator<?> evaluator =
- factory.forApplication(longs.getProducingTransformInternal(), null, context);
- InProcessTransformResult result = evaluator.finishBundle();
- assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
- assertThat(
- output.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements(),
- containsInAnyOrder(
- gw(1L), gw(2L), gw(4L), gw(8L), gw(9L), gw(7L), gw(6L), gw(5L), gw(3L), gw(0L)));
- }
-
- /**
- * Demonstrate that acquiring multiple {@link TransformEvaluator TransformEvaluators} for the same
- * {@link Bounded Read.Bounded} application with the same evaluation context only produces the
- * elements once.
- */
- @Test
- public void boundedSourceInMemoryTransformEvaluatorAfterFinishIsEmpty() throws Exception {
- UncommittedBundle<Long> output = bundleFactory.createRootBundle(longs);
- when(context.createRootBundle(longs)).thenReturn(output);
-
- TransformEvaluator<?> evaluator =
- factory.forApplication(longs.getProducingTransformInternal(), null, context);
- InProcessTransformResult result = evaluator.finishBundle();
- assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
- Iterable<? extends WindowedValue<Long>> outputElements =
- output.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements();
- assertThat(
- outputElements,
- containsInAnyOrder(
- gw(1L), gw(2L), gw(4L), gw(8L), gw(9L), gw(7L), gw(6L), gw(5L), gw(3L), gw(0L)));
-
- UncommittedBundle<Long> secondOutput = bundleFactory.createRootBundle(longs);
- when(context.createRootBundle(longs)).thenReturn(secondOutput);
- TransformEvaluator<?> secondEvaluator =
- factory.forApplication(longs.getProducingTransformInternal(), null, context);
- InProcessTransformResult secondResult = secondEvaluator.finishBundle();
- assertThat(secondResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
- assertThat(secondResult.getOutputBundles(), emptyIterable());
- assertThat(
- secondOutput.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements(), emptyIterable());
- assertThat(
- outputElements,
- containsInAnyOrder(
- gw(1L), gw(2L), gw(4L), gw(8L), gw(9L), gw(7L), gw(6L), gw(5L), gw(3L), gw(0L)));
- }
-
- /**
- * Demonstrates that acquiring multiple evaluators from the factory are independent, but
- * the elements in the source are only produced once.
- */
- @Test
- public void boundedSourceEvaluatorSimultaneousEvaluations() throws Exception {
- UncommittedBundle<Long> output = bundleFactory.createRootBundle(longs);
- UncommittedBundle<Long> secondOutput = bundleFactory.createRootBundle(longs);
- when(context.createRootBundle(longs)).thenReturn(output).thenReturn(secondOutput);
-
- // create both evaluators before finishing either.
- TransformEvaluator<?> evaluator =
- factory.forApplication(longs.getProducingTransformInternal(), null, context);
- TransformEvaluator<?> secondEvaluator =
- factory.forApplication(longs.getProducingTransformInternal(), null, context);
-
- InProcessTransformResult secondResult = secondEvaluator.finishBundle();
-
- InProcessTransformResult result = evaluator.finishBundle();
- assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
- Iterable<? extends WindowedValue<Long>> outputElements =
- output.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements();
-
- assertThat(
- outputElements,
- containsInAnyOrder(
- gw(1L), gw(2L), gw(4L), gw(8L), gw(9L), gw(7L), gw(6L), gw(5L), gw(3L), gw(0L)));
- assertThat(secondResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
- assertThat(secondResult.getOutputBundles(), emptyIterable());
- assertThat(
- secondOutput.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements(), emptyIterable());
- assertThat(
- outputElements,
- containsInAnyOrder(
- gw(1L), gw(2L), gw(4L), gw(8L), gw(9L), gw(7L), gw(6L), gw(5L), gw(3L), gw(0L)));
- }
-
- @Test
- public void boundedSourceEvaluatorClosesReader() throws Exception {
- TestSource<Long> source = new TestSource<>(BigEndianLongCoder.of(), 1L, 2L, 3L);
-
- TestPipeline p = TestPipeline.create();
- PCollection<Long> pcollection = p.apply(Read.from(source));
- AppliedPTransform<?, ?, ?> sourceTransform = pcollection.getProducingTransformInternal();
-
- UncommittedBundle<Long> output = bundleFactory.createRootBundle(pcollection);
- when(context.createRootBundle(pcollection)).thenReturn(output);
-
- TransformEvaluator<?> evaluator = factory.forApplication(sourceTransform, null, context);
- evaluator.finishBundle();
- CommittedBundle<Long> committed = output.commit(Instant.now());
- assertThat(committed.getElements(), containsInAnyOrder(gw(2L), gw(3L), gw(1L)));
- assertThat(TestSource.readerClosed, is(true));
- }
-
- @Test
- public void boundedSourceEvaluatorNoElementsClosesReader() throws Exception {
- TestSource<Long> source = new TestSource<>(BigEndianLongCoder.of());
-
- TestPipeline p = TestPipeline.create();
- PCollection<Long> pcollection = p.apply(Read.from(source));
- AppliedPTransform<?, ?, ?> sourceTransform = pcollection.getProducingTransformInternal();
-
- UncommittedBundle<Long> output = bundleFactory.createRootBundle(pcollection);
- when(context.createRootBundle(pcollection)).thenReturn(output);
-
- TransformEvaluator<?> evaluator = factory.forApplication(sourceTransform, null, context);
- evaluator.finishBundle();
- CommittedBundle<Long> committed = output.commit(Instant.now());
- assertThat(committed.getElements(), emptyIterable());
- assertThat(TestSource.readerClosed, is(true));
- }
-
- private static class TestSource<T> extends BoundedSource<T> {
- private static boolean readerClosed;
- private final Coder<T> coder;
- private final T[] elems;
-
- public TestSource(Coder<T> coder, T... elems) {
- this.elems = elems;
- this.coder = coder;
- readerClosed = false;
- }
-
- @Override
- public List<? extends BoundedSource<T>> splitIntoBundles(
- long desiredBundleSizeBytes, PipelineOptions options) throws Exception {
- return ImmutableList.of(this);
- }
-
- @Override
- public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
- return 0;
- }
-
- @Override
- public boolean producesSortedKeys(PipelineOptions options) throws Exception {
- return false;
- }
-
- @Override
- public BoundedSource.BoundedReader<T> createReader(PipelineOptions options) throws IOException {
- return new TestReader<>(this, elems);
- }
-
- @Override
- public void validate() {
- }
-
- @Override
- public Coder<T> getDefaultOutputCoder() {
- return coder;
- }
- }
-
- private static class TestReader<T> extends BoundedReader<T> {
- private final BoundedSource<T> source;
- private final List<T> elems;
- private int index;
-
- public TestReader(BoundedSource<T> source, T... elems) {
- this.source = source;
- this.elems = Arrays.asList(elems);
- this.index = -1;
- }
-
- @Override
- public BoundedSource<T> getCurrentSource() {
- return source;
- }
-
- @Override
- public boolean start() throws IOException {
- return advance();
- }
-
- @Override
- public boolean advance() throws IOException {
- if (elems.size() > index + 1) {
- index++;
- return true;
- }
- return false;
- }
-
- @Override
- public T getCurrent() throws NoSuchElementException {
- return elems.get(index);
- }
-
- @Override
- public void close() throws IOException {
- TestSource.readerClosed = true;
- }
- }
-
- private static WindowedValue<Long> gw(Long elem) {
- return WindowedValue.valueInGlobalWindow(elem);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/CommittedResultTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/CommittedResultTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/CommittedResultTest.java
deleted file mode 100644
index c888a65..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/CommittedResultTest.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.values.PBegin;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-
-import com.google.common.collect.ImmutableList;
-
-import org.hamcrest.Matchers;
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.Serializable;
-import java.util.Collections;
-import java.util.List;
-
-/**
- * Tests for {@link CommittedResult}.
- */
-@RunWith(JUnit4.class)
-public class CommittedResultTest implements Serializable {
- private transient TestPipeline p = TestPipeline.create();
- private transient AppliedPTransform<?, ?, ?> transform =
- AppliedPTransform.of("foo", p.begin(), PDone.in(p), new PTransform<PBegin, PDone>() {
- });
- private transient BundleFactory bundleFactory = InProcessBundleFactory.create();
-
- @Test
- public void getTransformExtractsFromResult() {
- CommittedResult result =
- CommittedResult.create(StepTransformResult.withoutHold(transform).build(),
- Collections.<InProcessPipelineRunner.CommittedBundle<?>>emptyList());
-
- assertThat(result.getTransform(), Matchers.<AppliedPTransform<?, ?, ?>>equalTo(transform));
- }
-
- @Test
- public void getOutputsEqualInput() {
- List<? extends InProcessPipelineRunner.CommittedBundle<?>> outputs =
- ImmutableList.of(bundleFactory.createRootBundle(PCollection.createPrimitiveOutputInternal(p,
- WindowingStrategy.globalDefault(),
- PCollection.IsBounded.BOUNDED)).commit(Instant.now()),
- bundleFactory.createRootBundle(PCollection.createPrimitiveOutputInternal(p,
- WindowingStrategy.globalDefault(),
- PCollection.IsBounded.UNBOUNDED)).commit(Instant.now()));
- CommittedResult result =
- CommittedResult.create(StepTransformResult.withoutHold(transform).build(), outputs);
-
- assertThat(result.getOutputs(), Matchers.containsInAnyOrder(outputs.toArray()));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitorTest.java
deleted file mode 100644
index aef4845..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitorTest.java
+++ /dev/null
@@ -1,272 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.io.CountingInput;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.Flatten;
-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.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionList;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PDone;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.PValue;
-
-import org.hamcrest.Matchers;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.Serializable;
-import java.util.List;
-
-/**
- * Tests for {@link ConsumerTrackingPipelineVisitor}.
- */
-@RunWith(JUnit4.class)
-public class ConsumerTrackingPipelineVisitorTest implements Serializable {
- @Rule public transient ExpectedException thrown = ExpectedException.none();
-
- private transient TestPipeline p = TestPipeline.create();
- private transient ConsumerTrackingPipelineVisitor visitor = new ConsumerTrackingPipelineVisitor();
-
- @Test
- public void getViewsReturnsViews() {
- PCollectionView<List<String>> listView =
- p.apply("listCreate", Create.of("foo", "bar"))
- .apply(
- ParDo.of(
- new DoFn<String, String>() {
- @Override
- public void processElement(DoFn<String, String>.ProcessContext c)
- throws Exception {
- c.output(Integer.toString(c.element().length()));
- }
- }))
- .apply(View.<String>asList());
- PCollectionView<Object> singletonView =
- p.apply("singletonCreate", Create.<Object>of(1, 2, 3)).apply(View.<Object>asSingleton());
- p.traverseTopologically(visitor);
- assertThat(
- visitor.getViews(),
- Matchers.<PCollectionView<?>>containsInAnyOrder(listView, singletonView));
- }
-
- @Test
- public void getRootTransformsContainsPBegins() {
- PCollection<String> created = p.apply(Create.of("foo", "bar"));
- PCollection<Long> counted = p.apply(CountingInput.upTo(1234L));
- PCollection<Long> unCounted = p.apply(CountingInput.unbounded());
- p.traverseTopologically(visitor);
- assertThat(
- visitor.getRootTransforms(),
- Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
- created.getProducingTransformInternal(),
- counted.getProducingTransformInternal(),
- unCounted.getProducingTransformInternal()));
- }
-
- @Test
- public void getRootTransformsContainsEmptyFlatten() {
- PCollection<String> empty =
- PCollectionList.<String>empty(p).apply(Flatten.<String>pCollections());
- p.traverseTopologically(visitor);
- assertThat(
- visitor.getRootTransforms(),
- Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
- empty.getProducingTransformInternal()));
- }
-
- @Test
- public void getValueToConsumersSucceeds() {
- PCollection<String> created = p.apply(Create.of("1", "2", "3"));
- PCollection<String> transformed =
- created.apply(
- ParDo.of(
- new DoFn<String, String>() {
- @Override
- public void processElement(DoFn<String, String>.ProcessContext c)
- throws Exception {
- c.output(Integer.toString(c.element().length()));
- }
- }));
-
- PCollection<String> flattened =
- PCollectionList.of(created).and(transformed).apply(Flatten.<String>pCollections());
-
- p.traverseTopologically(visitor);
-
- assertThat(
- visitor.getValueToConsumers().get(created),
- Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
- transformed.getProducingTransformInternal(),
- flattened.getProducingTransformInternal()));
- assertThat(
- visitor.getValueToConsumers().get(transformed),
- Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
- flattened.getProducingTransformInternal()));
- assertThat(visitor.getValueToConsumers().get(flattened), emptyIterable());
- }
-
- @Test
- public void getUnfinalizedPValuesContainsDanglingOutputs() {
- PCollection<String> created = p.apply(Create.of("1", "2", "3"));
- PCollection<String> transformed =
- created.apply(
- ParDo.of(
- new DoFn<String, String>() {
- @Override
- public void processElement(DoFn<String, String>.ProcessContext c)
- throws Exception {
- c.output(Integer.toString(c.element().length()));
- }
- }));
-
- p.traverseTopologically(visitor);
- assertThat(visitor.getUnfinalizedPValues(), Matchers.<PValue>contains(transformed));
- }
-
- @Test
- public void getUnfinalizedPValuesEmpty() {
- p.apply(Create.of("1", "2", "3"))
- .apply(
- ParDo.of(
- new DoFn<String, String>() {
- @Override
- public void processElement(DoFn<String, String>.ProcessContext c)
- throws Exception {
- c.output(Integer.toString(c.element().length()));
- }
- }))
- .apply(
- new PTransform<PInput, PDone>() {
- @Override
- public PDone apply(PInput input) {
- return PDone.in(input.getPipeline());
- }
- });
-
- p.traverseTopologically(visitor);
- assertThat(visitor.getUnfinalizedPValues(), emptyIterable());
- }
-
- @Test
- public void getStepNamesContainsAllTransforms() {
- PCollection<String> created = p.apply(Create.of("1", "2", "3"));
- PCollection<String> transformed =
- created.apply(
- ParDo.of(
- new DoFn<String, String>() {
- @Override
- public void processElement(DoFn<String, String>.ProcessContext c)
- throws Exception {
- c.output(Integer.toString(c.element().length()));
- }
- }));
- PDone finished =
- transformed.apply(
- new PTransform<PInput, PDone>() {
- @Override
- public PDone apply(PInput input) {
- return PDone.in(input.getPipeline());
- }
- });
-
- p.traverseTopologically(visitor);
- assertThat(
- visitor.getStepNames(),
- Matchers.<AppliedPTransform<?, ?, ?>, String>hasEntry(
- created.getProducingTransformInternal(), "s0"));
- assertThat(
- visitor.getStepNames(),
- Matchers.<AppliedPTransform<?, ?, ?>, String>hasEntry(
- transformed.getProducingTransformInternal(), "s1"));
- assertThat(
- visitor.getStepNames(),
- Matchers.<AppliedPTransform<?, ?, ?>, String>hasEntry(
- finished.getProducingTransformInternal(), "s2"));
- }
-
- @Test
- public void traverseMultipleTimesThrows() {
- p.apply(Create.of(1, 2, 3));
-
- p.traverseTopologically(visitor);
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage(ConsumerTrackingPipelineVisitor.class.getSimpleName());
- thrown.expectMessage("is finalized");
- p.traverseTopologically(visitor);
- }
-
- @Test
- public void traverseIndependentPathsSucceeds() {
- p.apply("left", Create.of(1, 2, 3));
- p.apply("right", Create.of("foo", "bar", "baz"));
-
- p.traverseTopologically(visitor);
- }
-
- @Test
- public void getRootTransformsWithoutVisitingThrows() {
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage("completely traversed");
- thrown.expectMessage("getRootTransforms");
- visitor.getRootTransforms();
- }
- @Test
- public void getStepNamesWithoutVisitingThrows() {
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage("completely traversed");
- thrown.expectMessage("getStepNames");
- visitor.getStepNames();
- }
- @Test
- public void getUnfinalizedPValuesWithoutVisitingThrows() {
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage("completely traversed");
- thrown.expectMessage("getUnfinalizedPValues");
- visitor.getUnfinalizedPValues();
- }
-
- @Test
- public void getValueToConsumersWithoutVisitingThrows() {
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage("completely traversed");
- thrown.expectMessage("getValueToConsumers");
- visitor.getValueToConsumers();
- }
-
- @Test
- public void getViewsWithoutVisitingThrows() {
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage("completely traversed");
- thrown.expectMessage("getViews");
- visitor.getViews();
- }
-}
[09/17] incubator-beam git commit: Move InProcessRunner to its own
module
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
new file mode 100644
index 0000000..236ad17
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
@@ -0,0 +1,324 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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.ParDo;
+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.OutputTimeFns;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+import org.apache.beam.sdk.util.state.WatermarkHoldState;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TupleTag;
+
+import org.hamcrest.Matchers;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.Serializable;
+
+/**
+ * Tests for {@link ParDoSingleEvaluatorFactory}.
+ */
+@RunWith(JUnit4.class)
+public class ParDoSingleEvaluatorFactoryTest implements Serializable {
+ private transient BundleFactory bundleFactory = InProcessBundleFactory.create();
+
+ @Test
+ public void testParDoInMemoryTransformEvaluator() throws Exception {
+ TestPipeline p = TestPipeline.create();
+
+ PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
+ PCollection<Integer> collection =
+ input.apply(
+ ParDo.of(
+ new DoFn<String, Integer>() {
+ @Override
+ public void processElement(ProcessContext c) {
+ c.output(c.element().length());
+ }
+ }));
+ CommittedBundle<String> inputBundle =
+ bundleFactory.createRootBundle(input).commit(Instant.now());
+
+ InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+ UncommittedBundle<Integer> outputBundle = bundleFactory.createRootBundle(collection);
+ when(evaluationContext.createBundle(inputBundle, collection)).thenReturn(outputBundle);
+ InProcessExecutionContext executionContext =
+ new InProcessExecutionContext(null, null, null, null);
+ when(evaluationContext.getExecutionContext(collection.getProducingTransformInternal(), null))
+ .thenReturn(executionContext);
+ CounterSet counters = new CounterSet();
+ when(evaluationContext.createCounterSet()).thenReturn(counters);
+
+ org.apache.beam.runners.direct.TransformEvaluator<String> evaluator =
+ new ParDoSingleEvaluatorFactory()
+ .forApplication(
+ collection.getProducingTransformInternal(), inputBundle, evaluationContext);
+
+ evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
+ evaluator.processElement(
+ WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
+ evaluator.processElement(
+ WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
+
+ InProcessTransformResult result = evaluator.finishBundle();
+ assertThat(result.getOutputBundles(), Matchers.<UncommittedBundle<?>>contains(outputBundle));
+ assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
+ assertThat(result.getCounters(), equalTo(counters));
+
+ assertThat(
+ outputBundle.commit(Instant.now()).getElements(),
+ Matchers.<WindowedValue<Integer>>containsInAnyOrder(
+ WindowedValue.valueInGlobalWindow(3),
+ WindowedValue.timestampedValueInGlobalWindow(4, new Instant(1000)),
+ WindowedValue.valueInGlobalWindow(5, PaneInfo.ON_TIME_AND_ONLY_FIRING)));
+ }
+
+ @Test
+ public void testSideOutputToUndeclaredSideOutputSucceeds() throws Exception {
+ TestPipeline p = TestPipeline.create();
+
+ PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
+ final TupleTag<Integer> sideOutputTag = new TupleTag<Integer>() {};
+ PCollection<Integer> collection =
+ input.apply(
+ ParDo.of(
+ new DoFn<String, Integer>() {
+ @Override
+ public void processElement(ProcessContext c) {
+ c.sideOutput(sideOutputTag, c.element().length());
+ }
+ }));
+ CommittedBundle<String> inputBundle =
+ bundleFactory.createRootBundle(input).commit(Instant.now());
+
+ InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+ UncommittedBundle<Integer> outputBundle = bundleFactory.createRootBundle(collection);
+ when(evaluationContext.createBundle(inputBundle, collection)).thenReturn(outputBundle);
+ InProcessExecutionContext executionContext =
+ new InProcessExecutionContext(null, null, null, null);
+ when(evaluationContext.getExecutionContext(collection.getProducingTransformInternal(), null))
+ .thenReturn(executionContext);
+ CounterSet counters = new CounterSet();
+ when(evaluationContext.createCounterSet()).thenReturn(counters);
+
+ TransformEvaluator<String> evaluator =
+ new ParDoSingleEvaluatorFactory()
+ .forApplication(
+ collection.getProducingTransformInternal(), inputBundle, evaluationContext);
+
+ evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
+ evaluator.processElement(
+ WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
+ evaluator.processElement(
+ WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
+
+ InProcessTransformResult result = evaluator.finishBundle();
+ assertThat(
+ result.getOutputBundles(), Matchers.<UncommittedBundle<?>>containsInAnyOrder(outputBundle));
+ assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
+ assertThat(result.getCounters(), equalTo(counters));
+ }
+
+ @Test
+ public void finishBundleWithStatePutsStateInResult() throws Exception {
+ TestPipeline p = TestPipeline.create();
+
+ PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
+
+ final StateTag<Object, WatermarkHoldState<BoundedWindow>> watermarkTag =
+ StateTags.watermarkStateInternal("myId", OutputTimeFns.outputAtEarliestInputTimestamp());
+ final StateTag<Object, BagState<String>> bagTag = StateTags.bag("myBag", StringUtf8Coder.of());
+ final StateNamespace windowNs =
+ StateNamespaces.window(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE);
+ ParDo.Bound<String, KV<String, Integer>> pardo =
+ ParDo.of(
+ new DoFn<String, KV<String, Integer>>() {
+ @Override
+ public void processElement(ProcessContext c) {
+ c.windowingInternals()
+ .stateInternals()
+ .state(StateNamespaces.global(), watermarkTag)
+ .add(new Instant(124443L - c.element().length()));
+ c.windowingInternals()
+ .stateInternals()
+ .state(
+ StateNamespaces.window(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE),
+ bagTag)
+ .add(c.element());
+ }
+ });
+ PCollection<KV<String, Integer>> mainOutput = input.apply(pardo);
+
+ CommittedBundle<String> inputBundle =
+ bundleFactory.createRootBundle(input).commit(Instant.now());
+
+ InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+ UncommittedBundle<KV<String, Integer>> mainOutputBundle =
+ bundleFactory.createRootBundle(mainOutput);
+
+ when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
+
+ InProcessExecutionContext executionContext =
+ new InProcessExecutionContext(null, "myKey", null, null);
+ when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
+ .thenReturn(executionContext);
+ CounterSet counters = new CounterSet();
+ when(evaluationContext.createCounterSet()).thenReturn(counters);
+
+ org.apache.beam.runners.direct.TransformEvaluator<String> evaluator =
+ new ParDoSingleEvaluatorFactory()
+ .forApplication(
+ mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
+
+ evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
+ evaluator.processElement(
+ WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
+ evaluator.processElement(
+ WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
+
+ InProcessTransformResult result = evaluator.finishBundle();
+ assertThat(result.getWatermarkHold(), equalTo(new Instant(124438L)));
+ assertThat(result.getState(), not(nullValue()));
+ assertThat(
+ result.getState().state(StateNamespaces.global(), watermarkTag).read(),
+ equalTo(new Instant(124438L)));
+ assertThat(
+ result.getState().state(windowNs, bagTag).read(),
+ containsInAnyOrder("foo", "bara", "bazam"));
+ }
+
+ @Test
+ public void finishBundleWithStateAndTimersPutsTimersInResult() throws Exception {
+ TestPipeline p = TestPipeline.create();
+
+ PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
+
+ final TimerData addedTimer =
+ TimerData.of(
+ StateNamespaces.window(
+ IntervalWindow.getCoder(),
+ new IntervalWindow(
+ new Instant(0).plus(Duration.standardMinutes(5)),
+ new Instant(1)
+ .plus(Duration.standardMinutes(5))
+ .plus(Duration.standardHours(1)))),
+ new Instant(54541L),
+ TimeDomain.EVENT_TIME);
+ final TimerData deletedTimer =
+ TimerData.of(
+ StateNamespaces.window(
+ IntervalWindow.getCoder(),
+ new IntervalWindow(new Instant(0), new Instant(0).plus(Duration.standardHours(1)))),
+ new Instant(3400000),
+ TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+
+ ParDo.Bound<String, KV<String, Integer>> pardo =
+ ParDo.of(
+ new DoFn<String, KV<String, Integer>>() {
+ @Override
+ public void processElement(ProcessContext c) {
+ c.windowingInternals().stateInternals();
+ c.windowingInternals()
+ .timerInternals()
+ .setTimer(
+ TimerData.of(
+ StateNamespaces.window(
+ IntervalWindow.getCoder(),
+ new IntervalWindow(
+ new Instant(0).plus(Duration.standardMinutes(5)),
+ new Instant(1)
+ .plus(Duration.standardMinutes(5))
+ .plus(Duration.standardHours(1)))),
+ new Instant(54541L),
+ TimeDomain.EVENT_TIME));
+ c.windowingInternals()
+ .timerInternals()
+ .deleteTimer(
+ TimerData.of(
+ StateNamespaces.window(
+ IntervalWindow.getCoder(),
+ new IntervalWindow(
+ new Instant(0),
+ new Instant(0).plus(Duration.standardHours(1)))),
+ new Instant(3400000),
+ TimeDomain.SYNCHRONIZED_PROCESSING_TIME));
+ }
+ });
+ PCollection<KV<String, Integer>> mainOutput = input.apply(pardo);
+
+ CommittedBundle<String> inputBundle =
+ bundleFactory.createRootBundle(input).commit(Instant.now());
+
+ InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+ UncommittedBundle<KV<String, Integer>> mainOutputBundle =
+ bundleFactory.createRootBundle(mainOutput);
+
+ when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
+
+ InProcessExecutionContext executionContext =
+ new InProcessExecutionContext(null, "myKey", null, null);
+ when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
+ .thenReturn(executionContext);
+ CounterSet counters = new CounterSet();
+ when(evaluationContext.createCounterSet()).thenReturn(counters);
+
+ TransformEvaluator<String> evaluator =
+ new ParDoSingleEvaluatorFactory()
+ .forApplication(
+ mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
+
+ evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
+
+ InProcessTransformResult result = evaluator.finishBundle();
+ assertThat(
+ result.getTimerUpdate(),
+ equalTo(
+ TimerUpdate.builder("myKey").setTimer(addedTimer).deletedTimer(deletedTimer).build()));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TextIOShardedWriteFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TextIOShardedWriteFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TextIOShardedWriteFactoryTest.java
new file mode 100644
index 0000000..49c9061
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TextIOShardedWriteFactoryTest.java
@@ -0,0 +1,112 @@
+/*
+ * 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.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.theInstance;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.TextIOTest;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+
+import org.hamcrest.Matchers;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.File;
+
+/**
+ * Tests for {@link TextIOShardedWriteFactory}.
+ */
+@RunWith(JUnit4.class)
+public class TextIOShardedWriteFactoryTest {
+ @Rule public TemporaryFolder tmp = new TemporaryFolder();
+ private TextIOShardedWriteFactory factory;
+
+ @Before
+ public void setup() {
+ factory = new TextIOShardedWriteFactory();
+ }
+
+ @Test
+ public void originalWithoutShardingReturnsOriginal() throws Exception {
+ File file = tmp.newFile("foo");
+ PTransform<PCollection<String>, PDone> original =
+ TextIO.Write.to(file.getAbsolutePath()).withoutSharding();
+ PTransform<PCollection<String>, PDone> overridden = factory.override(original);
+
+ assertThat(overridden, theInstance(original));
+ }
+
+ @Test
+ public void originalShardingNotSpecifiedReturnsOriginal() throws Exception {
+ File file = tmp.newFile("foo");
+ PTransform<PCollection<String>, PDone> original = TextIO.Write.to(file.getAbsolutePath());
+ PTransform<PCollection<String>, PDone> overridden = factory.override(original);
+
+ assertThat(overridden, theInstance(original));
+ }
+
+ @Test
+ public void originalShardedToOneReturnsExplicitlySharded() throws Exception {
+ File file = tmp.newFile("foo");
+ TextIO.Write.Bound<String> original =
+ TextIO.Write.to(file.getAbsolutePath()).withNumShards(1);
+ PTransform<PCollection<String>, PDone> overridden = factory.override(original);
+
+ assertThat(overridden, not(Matchers.<PTransform<PCollection<String>, PDone>>equalTo(original)));
+
+ TestPipeline p = TestPipeline.create();
+ String[] elems = new String[] {"foo", "bar", "baz"};
+ p.apply(Create.<String>of(elems)).apply(overridden);
+
+ file.delete();
+
+ p.run();
+ TextIOTest.assertOutputFiles(
+ elems, StringUtf8Coder.of(), 1, tmp, "foo", original.getShardNameTemplate());
+ }
+
+ @Test
+ public void originalShardedToManyReturnsExplicitlySharded() throws Exception {
+ File file = tmp.newFile("foo");
+ TextIO.Write.Bound<String> original = TextIO.Write.to(file.getAbsolutePath()).withNumShards(3);
+ PTransform<PCollection<String>, PDone> overridden = factory.override(original);
+
+ assertThat(overridden, not(Matchers.<PTransform<PCollection<String>, PDone>>equalTo(original)));
+
+ TestPipeline p = TestPipeline.create();
+ String[] elems = new String[] {"foo", "bar", "baz", "spam", "ham", "eggs"};
+ p.apply(Create.<String>of(elems)).apply(overridden);
+
+ file.delete();
+ p.run();
+ TextIOTest.assertOutputFiles(
+ elems, StringUtf8Coder.of(), 3, tmp, "foo", original.getShardNameTemplate());
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorServicesTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorServicesTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorServicesTest.java
new file mode 100644
index 0000000..7dd5830
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorServicesTest.java
@@ -0,0 +1,136 @@
+/*
+ * 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.hamcrest.Matchers.any;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+
+import com.google.common.util.concurrent.MoreExecutors;
+
+import org.hamcrest.Matchers;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+
+/**
+ * Tests for {@link TransformExecutorServices}.
+ */
+@RunWith(JUnit4.class)
+public class TransformExecutorServicesTest {
+ @Rule public ExpectedException thrown = ExpectedException.none();
+
+ private ExecutorService executorService;
+ private Map<TransformExecutor<?>, Boolean> scheduled;
+
+ @Before
+ public void setup() {
+ executorService = MoreExecutors.newDirectExecutorService();
+ scheduled = new ConcurrentHashMap<>();
+ }
+
+ @Test
+ public void parallelScheduleMultipleSchedulesBothImmediately() {
+ @SuppressWarnings("unchecked")
+ TransformExecutor<Object> first = mock(TransformExecutor.class);
+ @SuppressWarnings("unchecked")
+ TransformExecutor<Object> second = mock(TransformExecutor.class);
+
+ TransformExecutorService parallel =
+ TransformExecutorServices.parallel(executorService, scheduled);
+ parallel.schedule(first);
+ parallel.schedule(second);
+
+ verify(first).call();
+ verify(second).call();
+ assertThat(
+ scheduled,
+ Matchers.allOf(
+ Matchers.<TransformExecutor<?>, Boolean>hasEntry(first, true),
+ Matchers.<TransformExecutor<?>, Boolean>hasEntry(second, true)));
+
+ parallel.complete(first);
+ assertThat(scheduled, Matchers.<TransformExecutor<?>, Boolean>hasEntry(second, true));
+ assertThat(
+ scheduled,
+ not(
+ Matchers.<TransformExecutor<?>, Boolean>hasEntry(
+ Matchers.<TransformExecutor<?>>equalTo(first), any(Boolean.class))));
+ parallel.complete(second);
+ assertThat(scheduled.isEmpty(), is(true));
+ }
+
+ @Test
+ public void serialScheduleTwoWaitsForFirstToComplete() {
+ @SuppressWarnings("unchecked")
+ TransformExecutor<Object> first = mock(TransformExecutor.class);
+ @SuppressWarnings("unchecked")
+ TransformExecutor<Object> second = mock(TransformExecutor.class);
+
+ TransformExecutorService serial = TransformExecutorServices.serial(executorService, scheduled);
+ serial.schedule(first);
+ verify(first).call();
+
+ serial.schedule(second);
+ verify(second, never()).call();
+
+ assertThat(scheduled, Matchers.<TransformExecutor<?>, Boolean>hasEntry(first, true));
+ assertThat(
+ scheduled,
+ not(
+ Matchers.<TransformExecutor<?>, Boolean>hasEntry(
+ Matchers.<TransformExecutor<?>>equalTo(second), any(Boolean.class))));
+
+ serial.complete(first);
+ verify(second).call();
+ assertThat(scheduled, Matchers.<TransformExecutor<?>, Boolean>hasEntry(second, true));
+ assertThat(
+ scheduled,
+ not(
+ Matchers.<TransformExecutor<?>, Boolean>hasEntry(
+ Matchers.<TransformExecutor<?>>equalTo(first), any(Boolean.class))));
+
+ serial.complete(second);
+ }
+
+ @Test
+ public void serialCompleteNotExecutingTaskThrows() {
+ @SuppressWarnings("unchecked")
+ TransformExecutor<Object> first = mock(TransformExecutor.class);
+ @SuppressWarnings("unchecked")
+ TransformExecutor<Object> second = mock(TransformExecutor.class);
+
+ TransformExecutorService serial = TransformExecutorServices.serial(executorService, scheduled);
+ serial.schedule(first);
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage("unexpected currently executing");
+
+ serial.complete(second);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java
new file mode 100644
index 0000000..959e9d3
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java
@@ -0,0 +1,538 @@
+/*
+ * 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.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.isA;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+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.WithKeys;
+import org.apache.beam.sdk.util.IllegalMutationException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.common.util.concurrent.MoreExecutors;
+
+import org.hamcrest.Matchers;
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * Tests for {@link TransformExecutor}.
+ */
+@RunWith(JUnit4.class)
+public class TransformExecutorTest {
+ @Rule public ExpectedException thrown = ExpectedException.none();
+ private PCollection<String> created;
+ private PCollection<KV<Integer, String>> downstream;
+
+ private CountDownLatch evaluatorCompleted;
+
+ private RegisteringCompletionCallback completionCallback;
+ private TransformExecutorService transformEvaluationState;
+ private BundleFactory bundleFactory;
+ @Mock private InProcessEvaluationContext evaluationContext;
+ @Mock private TransformEvaluatorRegistry registry;
+ private Map<TransformExecutor<?>, Boolean> scheduled;
+
+ @Before
+ public void setup() {
+ MockitoAnnotations.initMocks(this);
+
+ bundleFactory = InProcessBundleFactory.create();
+
+ scheduled = new HashMap<>();
+ transformEvaluationState =
+ TransformExecutorServices.parallel(MoreExecutors.newDirectExecutorService(), scheduled);
+
+ evaluatorCompleted = new CountDownLatch(1);
+ completionCallback = new RegisteringCompletionCallback(evaluatorCompleted);
+
+ TestPipeline p = TestPipeline.create();
+ created = p.apply(Create.of("foo", "spam", "third"));
+ downstream = created.apply(WithKeys.<Integer, String>of(3));
+ }
+
+ @Test
+ public void callWithNullInputBundleFinishesBundleAndCompletes() throws Exception {
+ final InProcessTransformResult result =
+ StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
+ final AtomicBoolean finishCalled = new AtomicBoolean(false);
+ TransformEvaluator<Object> evaluator =
+ new TransformEvaluator<Object>() {
+ @Override
+ public void processElement(WindowedValue<Object> element) throws Exception {
+ throw new IllegalArgumentException("Shouldn't be called");
+ }
+
+ @Override
+ public InProcessTransformResult finishBundle() throws Exception {
+ finishCalled.set(true);
+ return result;
+ }
+ };
+
+ when(registry.forApplication(created.getProducingTransformInternal(), null, evaluationContext))
+ .thenReturn(evaluator);
+
+ TransformExecutor<Object> executor =
+ TransformExecutor.create(
+ registry,
+ Collections.<ModelEnforcementFactory>emptyList(),
+ evaluationContext,
+ null,
+ created.getProducingTransformInternal(),
+ completionCallback,
+ transformEvaluationState);
+ executor.call();
+
+ assertThat(finishCalled.get(), is(true));
+ assertThat(completionCallback.handledResult, equalTo(result));
+ assertThat(completionCallback.handledThrowable, is(nullValue()));
+ assertThat(scheduled, not(Matchers.<TransformExecutor<?>>hasKey(executor)));
+ }
+
+ @Test
+ public void inputBundleProcessesEachElementFinishesAndCompletes() throws Exception {
+ final InProcessTransformResult result =
+ StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build();
+ final Collection<WindowedValue<String>> elementsProcessed = new ArrayList<>();
+ TransformEvaluator<String> evaluator =
+ new TransformEvaluator<String>() {
+ @Override
+ public void processElement(WindowedValue<String> element) throws Exception {
+ elementsProcessed.add(element);
+ return;
+ }
+
+ @Override
+ public InProcessTransformResult finishBundle() throws Exception {
+ return result;
+ }
+ };
+
+ WindowedValue<String> foo = WindowedValue.valueInGlobalWindow("foo");
+ WindowedValue<String> spam = WindowedValue.valueInGlobalWindow("spam");
+ WindowedValue<String> third = WindowedValue.valueInGlobalWindow("third");
+ CommittedBundle<String> inputBundle =
+ bundleFactory.createRootBundle(created).add(foo).add(spam).add(third).commit(Instant.now());
+ when(
+ registry.<String>forApplication(
+ downstream.getProducingTransformInternal(), inputBundle, evaluationContext))
+ .thenReturn(evaluator);
+
+ TransformExecutor<String> executor =
+ TransformExecutor.create(
+ registry,
+ Collections.<ModelEnforcementFactory>emptyList(),
+ evaluationContext,
+ inputBundle,
+ downstream.getProducingTransformInternal(),
+ completionCallback,
+ transformEvaluationState);
+
+ Executors.newSingleThreadExecutor().submit(executor);
+
+ evaluatorCompleted.await();
+
+ assertThat(elementsProcessed, containsInAnyOrder(spam, third, foo));
+ assertThat(completionCallback.handledResult, equalTo(result));
+ assertThat(completionCallback.handledThrowable, is(nullValue()));
+ assertThat(scheduled, not(Matchers.<TransformExecutor<?>>hasKey(executor)));
+ }
+
+ @Test
+ public void processElementThrowsExceptionCallsback() throws Exception {
+ final InProcessTransformResult result =
+ StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build();
+ final Exception exception = new Exception();
+ TransformEvaluator<String> evaluator =
+ new TransformEvaluator<String>() {
+ @Override
+ public void processElement(WindowedValue<String> element) throws Exception {
+ throw exception;
+ }
+
+ @Override
+ public InProcessTransformResult finishBundle() throws Exception {
+ return result;
+ }
+ };
+
+ WindowedValue<String> foo = WindowedValue.valueInGlobalWindow("foo");
+ CommittedBundle<String> inputBundle =
+ bundleFactory.createRootBundle(created).add(foo).commit(Instant.now());
+ when(
+ registry.<String>forApplication(
+ downstream.getProducingTransformInternal(), inputBundle, evaluationContext))
+ .thenReturn(evaluator);
+
+ TransformExecutor<String> executor =
+ TransformExecutor.create(
+ registry,
+ Collections.<ModelEnforcementFactory>emptyList(),
+ evaluationContext,
+ inputBundle,
+ downstream.getProducingTransformInternal(),
+ completionCallback,
+ transformEvaluationState);
+ Executors.newSingleThreadExecutor().submit(executor);
+
+ evaluatorCompleted.await();
+
+ assertThat(completionCallback.handledResult, is(nullValue()));
+ assertThat(completionCallback.handledThrowable, Matchers.<Throwable>equalTo(exception));
+ assertThat(scheduled, not(Matchers.<TransformExecutor<?>>hasKey(executor)));
+ }
+
+ @Test
+ public void finishBundleThrowsExceptionCallsback() throws Exception {
+ final Exception exception = new Exception();
+ TransformEvaluator<String> evaluator =
+ new TransformEvaluator<String>() {
+ @Override
+ public void processElement(WindowedValue<String> element) throws Exception {}
+
+ @Override
+ public InProcessTransformResult finishBundle() throws Exception {
+ throw exception;
+ }
+ };
+
+ CommittedBundle<String> inputBundle =
+ bundleFactory.createRootBundle(created).commit(Instant.now());
+ when(
+ registry.<String>forApplication(
+ downstream.getProducingTransformInternal(), inputBundle, evaluationContext))
+ .thenReturn(evaluator);
+
+ TransformExecutor<String> executor =
+ TransformExecutor.create(
+ registry,
+ Collections.<ModelEnforcementFactory>emptyList(),
+ evaluationContext,
+ inputBundle,
+ downstream.getProducingTransformInternal(),
+ completionCallback,
+ transformEvaluationState);
+ Executors.newSingleThreadExecutor().submit(executor);
+
+ evaluatorCompleted.await();
+
+ assertThat(completionCallback.handledResult, is(nullValue()));
+ assertThat(completionCallback.handledThrowable, Matchers.<Throwable>equalTo(exception));
+ assertThat(scheduled, not(Matchers.<TransformExecutor<?>>hasKey(executor)));
+ }
+
+ @Test
+ public void duringCallGetThreadIsNonNull() throws Exception {
+ final InProcessTransformResult result =
+ StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build();
+ final CountDownLatch testLatch = new CountDownLatch(1);
+ final CountDownLatch evaluatorLatch = new CountDownLatch(1);
+ TransformEvaluator<Object> evaluator =
+ new TransformEvaluator<Object>() {
+ @Override
+ public void processElement(WindowedValue<Object> element) throws Exception {
+ throw new IllegalArgumentException("Shouldn't be called");
+ }
+
+ @Override
+ public InProcessTransformResult finishBundle() throws Exception {
+ testLatch.countDown();
+ evaluatorLatch.await();
+ return result;
+ }
+ };
+
+ when(registry.forApplication(created.getProducingTransformInternal(), null, evaluationContext))
+ .thenReturn(evaluator);
+
+ TransformExecutor<String> executor =
+ TransformExecutor.create(
+ registry,
+ Collections.<ModelEnforcementFactory>emptyList(),
+ evaluationContext,
+ null,
+ created.getProducingTransformInternal(),
+ completionCallback,
+ transformEvaluationState);
+
+ Executors.newSingleThreadExecutor().submit(executor);
+ testLatch.await();
+ assertThat(executor.getThread(), not(nullValue()));
+
+ // Finish the execution so everything can get closed down cleanly.
+ evaluatorLatch.countDown();
+ }
+
+ @Test
+ public void callWithEnforcementAppliesEnforcement() throws Exception {
+ final InProcessTransformResult result =
+ StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build();
+
+ TransformEvaluator<Object> evaluator =
+ new TransformEvaluator<Object>() {
+ @Override
+ public void processElement(WindowedValue<Object> element) throws Exception {
+ }
+
+ @Override
+ public InProcessTransformResult finishBundle() throws Exception {
+ return result;
+ }
+ };
+
+ WindowedValue<String> fooElem = WindowedValue.valueInGlobalWindow("foo");
+ WindowedValue<String> barElem = WindowedValue.valueInGlobalWindow("bar");
+ CommittedBundle<String> inputBundle =
+ bundleFactory.createRootBundle(created).add(fooElem).add(barElem).commit(Instant.now());
+ when(
+ registry.forApplication(
+ downstream.getProducingTransformInternal(), inputBundle, evaluationContext))
+ .thenReturn(evaluator);
+
+ TestEnforcementFactory enforcement = new TestEnforcementFactory();
+ TransformExecutor<String> executor =
+ TransformExecutor.create(
+ registry,
+ Collections.<ModelEnforcementFactory>singleton(enforcement),
+ evaluationContext,
+ inputBundle,
+ downstream.getProducingTransformInternal(),
+ completionCallback,
+ transformEvaluationState);
+
+ executor.call();
+ TestEnforcement<?> testEnforcement = enforcement.instance;
+ assertThat(
+ testEnforcement.beforeElements,
+ Matchers.<WindowedValue<?>>containsInAnyOrder(barElem, fooElem));
+ assertThat(
+ testEnforcement.afterElements,
+ Matchers.<WindowedValue<?>>containsInAnyOrder(barElem, fooElem));
+ assertThat(testEnforcement.finishedBundles, contains(result));
+ }
+
+ @Test
+ public void callWithEnforcementThrowsOnFinishPropagates() throws Exception {
+ PCollection<byte[]> pcBytes =
+ created.apply(
+ new PTransform<PCollection<String>, PCollection<byte[]>>() {
+ @Override
+ public PCollection<byte[]> apply(PCollection<String> input) {
+ return PCollection.<byte[]>createPrimitiveOutputInternal(
+ input.getPipeline(), input.getWindowingStrategy(), input.isBounded())
+ .setCoder(ByteArrayCoder.of());
+ }
+ });
+
+ final InProcessTransformResult result =
+ StepTransformResult.withoutHold(pcBytes.getProducingTransformInternal()).build();
+ final CountDownLatch testLatch = new CountDownLatch(1);
+ final CountDownLatch evaluatorLatch = new CountDownLatch(1);
+
+ TransformEvaluator<Object> evaluator =
+ new TransformEvaluator<Object>() {
+ @Override
+ public void processElement(WindowedValue<Object> element) throws Exception {}
+
+ @Override
+ public InProcessTransformResult finishBundle() throws Exception {
+ testLatch.countDown();
+ evaluatorLatch.await();
+ return result;
+ }
+ };
+
+ WindowedValue<byte[]> fooBytes = WindowedValue.valueInGlobalWindow("foo".getBytes());
+ CommittedBundle<byte[]> inputBundle =
+ bundleFactory.createRootBundle(pcBytes).add(fooBytes).commit(Instant.now());
+ when(
+ registry.forApplication(
+ pcBytes.getProducingTransformInternal(), inputBundle, evaluationContext))
+ .thenReturn(evaluator);
+
+ TransformExecutor<byte[]> executor =
+ TransformExecutor.create(
+ registry,
+ Collections.<ModelEnforcementFactory>singleton(ImmutabilityEnforcementFactory.create()),
+ evaluationContext,
+ inputBundle,
+ pcBytes.getProducingTransformInternal(),
+ completionCallback,
+ transformEvaluationState);
+
+ Future<InProcessTransformResult> task = Executors.newSingleThreadExecutor().submit(executor);
+ testLatch.await();
+ fooBytes.getValue()[0] = 'b';
+ evaluatorLatch.countDown();
+
+ thrown.expectCause(isA(IllegalMutationException.class));
+ task.get();
+ }
+
+ @Test
+ public void callWithEnforcementThrowsOnElementPropagates() throws Exception {
+ PCollection<byte[]> pcBytes =
+ created.apply(
+ new PTransform<PCollection<String>, PCollection<byte[]>>() {
+ @Override
+ public PCollection<byte[]> apply(PCollection<String> input) {
+ return PCollection.<byte[]>createPrimitiveOutputInternal(
+ input.getPipeline(), input.getWindowingStrategy(), input.isBounded())
+ .setCoder(ByteArrayCoder.of());
+ }
+ });
+
+ final InProcessTransformResult result =
+ StepTransformResult.withoutHold(pcBytes.getProducingTransformInternal()).build();
+ final CountDownLatch testLatch = new CountDownLatch(1);
+ final CountDownLatch evaluatorLatch = new CountDownLatch(1);
+
+ TransformEvaluator<Object> evaluator =
+ new TransformEvaluator<Object>() {
+ @Override
+ public void processElement(WindowedValue<Object> element) throws Exception {
+ testLatch.countDown();
+ evaluatorLatch.await();
+ }
+
+ @Override
+ public InProcessTransformResult finishBundle() throws Exception {
+ return result;
+ }
+ };
+
+ WindowedValue<byte[]> fooBytes = WindowedValue.valueInGlobalWindow("foo".getBytes());
+ CommittedBundle<byte[]> inputBundle =
+ bundleFactory.createRootBundle(pcBytes).add(fooBytes).commit(Instant.now());
+ when(
+ registry.forApplication(
+ pcBytes.getProducingTransformInternal(), inputBundle, evaluationContext))
+ .thenReturn(evaluator);
+
+ TransformExecutor<byte[]> executor =
+ TransformExecutor.create(
+ registry,
+ Collections.<ModelEnforcementFactory>singleton(ImmutabilityEnforcementFactory.create()),
+ evaluationContext,
+ inputBundle,
+ pcBytes.getProducingTransformInternal(),
+ completionCallback,
+ transformEvaluationState);
+
+ Future<InProcessTransformResult> task = Executors.newSingleThreadExecutor().submit(executor);
+ testLatch.await();
+ fooBytes.getValue()[0] = 'b';
+ evaluatorLatch.countDown();
+
+ thrown.expectCause(isA(IllegalMutationException.class));
+ task.get();
+ }
+
+ private static class RegisteringCompletionCallback implements CompletionCallback {
+ private InProcessTransformResult handledResult = null;
+ private Throwable handledThrowable = null;
+ private final CountDownLatch onMethod;
+
+ private RegisteringCompletionCallback(CountDownLatch onMethod) {
+ this.onMethod = onMethod;
+ }
+
+ @Override
+ public CommittedResult handleResult(
+ CommittedBundle<?> inputBundle, InProcessTransformResult result) {
+ handledResult = result;
+ onMethod.countDown();
+ return CommittedResult.create(result, Collections.<CommittedBundle<?>>emptyList());
+ }
+
+ @Override
+ public void handleThrowable(CommittedBundle<?> inputBundle, Throwable t) {
+ handledThrowable = t;
+ onMethod.countDown();
+ }
+ }
+
+ private static class TestEnforcementFactory implements ModelEnforcementFactory {
+ private TestEnforcement<?> instance;
+ @Override
+ public <T> TestEnforcement<T> forBundle(
+ CommittedBundle<T> input, AppliedPTransform<?, ?, ?> consumer) {
+ TestEnforcement<T> newEnforcement = new TestEnforcement<>();
+ instance = newEnforcement;
+ return newEnforcement;
+ }
+ }
+
+ private static class TestEnforcement<T> implements ModelEnforcement<T> {
+ private final List<WindowedValue<T>> beforeElements = new ArrayList<>();
+ private final List<WindowedValue<T>> afterElements = new ArrayList<>();
+ private final List<InProcessTransformResult> finishedBundles = new ArrayList<>();
+
+ @Override
+ public void beforeElement(WindowedValue<T> element) {
+ beforeElements.add(element);
+ }
+
+ @Override
+ public void afterElement(WindowedValue<T> element) {
+ afterElements.add(element);
+ }
+
+ @Override
+ public void afterFinish(
+ CommittedBundle<T> input,
+ InProcessTransformResult result,
+ Iterable<? extends CommittedBundle<?>> outputs) {
+ finishedBundles.add(result);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java
new file mode 100644
index 0000000..9f3909e
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java
@@ -0,0 +1,334 @@
+/*
+ * 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.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.emptyIterable;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.BigEndianLongCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.io.CountingSource;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.common.collect.ImmutableList;
+
+import org.hamcrest.Matchers;
+import org.joda.time.DateTime;
+import org.joda.time.Instant;
+import org.joda.time.ReadableInstant;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import javax.annotation.Nullable;
+/**
+ * Tests for {@link UnboundedReadEvaluatorFactory}.
+ */
+@RunWith(JUnit4.class)
+public class UnboundedReadEvaluatorFactoryTest {
+ private PCollection<Long> longs;
+ private TransformEvaluatorFactory factory;
+ private InProcessEvaluationContext context;
+ private UncommittedBundle<Long> output;
+
+ private BundleFactory bundleFactory = InProcessBundleFactory.create();
+
+ @Before
+ public void setup() {
+ UnboundedSource<Long, ?> source =
+ CountingSource.unboundedWithTimestampFn(new LongToInstantFn());
+ TestPipeline p = TestPipeline.create();
+ longs = p.apply(Read.from(source));
+
+ factory = new UnboundedReadEvaluatorFactory();
+ context = mock(InProcessEvaluationContext.class);
+ output = bundleFactory.createRootBundle(longs);
+ when(context.createRootBundle(longs)).thenReturn(output);
+ }
+
+ @Test
+ public void unboundedSourceInMemoryTransformEvaluatorProducesElements() throws Exception {
+ TransformEvaluator<?> evaluator =
+ factory.forApplication(longs.getProducingTransformInternal(), null, context);
+
+ InProcessTransformResult result = evaluator.finishBundle();
+ assertThat(
+ result.getWatermarkHold(), Matchers.<ReadableInstant>lessThan(DateTime.now().toInstant()));
+ assertThat(
+ output.commit(Instant.now()).getElements(),
+ containsInAnyOrder(
+ tgw(1L), tgw(2L), tgw(4L), tgw(8L), tgw(9L), tgw(7L), tgw(6L), tgw(5L), tgw(3L),
+ tgw(0L)));
+ }
+
+ /**
+ * Demonstrate that multiple sequential creations will produce additional elements if the source
+ * can provide them.
+ */
+ @Test
+ public void unboundedSourceInMemoryTransformEvaluatorMultipleSequentialCalls() throws Exception {
+ TransformEvaluator<?> evaluator =
+ factory.forApplication(longs.getProducingTransformInternal(), null, context);
+
+ InProcessTransformResult result = evaluator.finishBundle();
+ assertThat(
+ result.getWatermarkHold(), Matchers.<ReadableInstant>lessThan(DateTime.now().toInstant()));
+ assertThat(
+ output.commit(Instant.now()).getElements(),
+ containsInAnyOrder(
+ tgw(1L), tgw(2L), tgw(4L), tgw(8L), tgw(9L), tgw(7L), tgw(6L), tgw(5L), tgw(3L),
+ tgw(0L)));
+
+ UncommittedBundle<Long> secondOutput = bundleFactory.createRootBundle(longs);
+ when(context.createRootBundle(longs)).thenReturn(secondOutput);
+ TransformEvaluator<?> secondEvaluator =
+ factory.forApplication(longs.getProducingTransformInternal(), null, context);
+ InProcessTransformResult secondResult = secondEvaluator.finishBundle();
+ assertThat(
+ secondResult.getWatermarkHold(),
+ Matchers.<ReadableInstant>lessThan(DateTime.now().toInstant()));
+ assertThat(
+ secondOutput.commit(Instant.now()).getElements(),
+ containsInAnyOrder(tgw(11L), tgw(12L), tgw(14L), tgw(18L), tgw(19L), tgw(17L), tgw(16L),
+ tgw(15L), tgw(13L), tgw(10L)));
+ }
+
+ @Test
+ public void boundedSourceEvaluatorClosesReader() throws Exception {
+ TestUnboundedSource<Long> source =
+ new TestUnboundedSource<>(BigEndianLongCoder.of(), 1L, 2L, 3L);
+
+ TestPipeline p = TestPipeline.create();
+ PCollection<Long> pcollection = p.apply(Read.from(source));
+ AppliedPTransform<?, ?, ?> sourceTransform = pcollection.getProducingTransformInternal();
+
+ UncommittedBundle<Long> output = bundleFactory.createRootBundle(pcollection);
+ when(context.createRootBundle(pcollection)).thenReturn(output);
+
+ TransformEvaluator<?> evaluator = factory.forApplication(sourceTransform, null, context);
+ evaluator.finishBundle();
+ CommittedBundle<Long> committed = output.commit(Instant.now());
+ assertThat(ImmutableList.copyOf(committed.getElements()), hasSize(3));
+ assertThat(TestUnboundedSource.readerClosedCount, equalTo(1));
+ }
+
+ @Test
+ public void boundedSourceEvaluatorNoElementsClosesReader() throws Exception {
+ TestUnboundedSource<Long> source = new TestUnboundedSource<>(BigEndianLongCoder.of());
+
+ TestPipeline p = TestPipeline.create();
+ PCollection<Long> pcollection = p.apply(Read.from(source));
+ AppliedPTransform<?, ?, ?> sourceTransform = pcollection.getProducingTransformInternal();
+
+ UncommittedBundle<Long> output = bundleFactory.createRootBundle(pcollection);
+ when(context.createRootBundle(pcollection)).thenReturn(output);
+
+ TransformEvaluator<?> evaluator = factory.forApplication(sourceTransform, null, context);
+ evaluator.finishBundle();
+ CommittedBundle<Long> committed = output.commit(Instant.now());
+ assertThat(committed.getElements(), emptyIterable());
+ assertThat(TestUnboundedSource.readerClosedCount, equalTo(1));
+ }
+
+ // TODO: Once the source is split into multiple sources before evaluating, this test will have to
+ // be updated.
+ /**
+ * Demonstrate that only a single unfinished instance of TransformEvaluator can be created at a
+ * time, with other calls returning an empty evaluator.
+ */
+ @Test
+ public void unboundedSourceWithMultipleSimultaneousEvaluatorsIndependent() throws Exception {
+ UncommittedBundle<Long> secondOutput = bundleFactory.createRootBundle(longs);
+
+ TransformEvaluator<?> evaluator =
+ factory.forApplication(longs.getProducingTransformInternal(), null, context);
+
+ TransformEvaluator<?> secondEvaluator =
+ factory.forApplication(longs.getProducingTransformInternal(), null, context);
+
+ InProcessTransformResult secondResult = secondEvaluator.finishBundle();
+ InProcessTransformResult result = evaluator.finishBundle();
+
+ assertThat(
+ result.getWatermarkHold(), Matchers.<ReadableInstant>lessThan(DateTime.now().toInstant()));
+ assertThat(
+ output.commit(Instant.now()).getElements(),
+ containsInAnyOrder(
+ tgw(1L), tgw(2L), tgw(4L), tgw(8L), tgw(9L), tgw(7L), tgw(6L), tgw(5L), tgw(3L),
+ tgw(0L)));
+
+ assertThat(secondResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
+ assertThat(secondOutput.commit(Instant.now()).getElements(), emptyIterable());
+ }
+
+ /**
+ * A terse alias for producing timestamped longs in the {@link GlobalWindow}, where
+ * the timestamp is the epoch offset by the value of the element.
+ */
+ private static WindowedValue<Long> tgw(Long elem) {
+ return WindowedValue.timestampedValueInGlobalWindow(elem, new Instant(elem));
+ }
+
+ private static class LongToInstantFn implements SerializableFunction<Long, Instant> {
+ @Override
+ public Instant apply(Long input) {
+ return new Instant(input);
+ }
+ }
+
+ private static class TestUnboundedSource<T> extends UnboundedSource<T, TestCheckpointMark> {
+ static int readerClosedCount;
+ private final Coder<T> coder;
+ private final List<T> elems;
+
+ public TestUnboundedSource(Coder<T> coder, T... elems) {
+ readerClosedCount = 0;
+ this.coder = coder;
+ this.elems = Arrays.asList(elems);
+ }
+
+ @Override
+ public List<? extends UnboundedSource<T, TestCheckpointMark>> generateInitialSplits(
+ int desiredNumSplits, PipelineOptions options) throws Exception {
+ return ImmutableList.of(this);
+ }
+
+ @Override
+ public UnboundedSource.UnboundedReader<T> createReader(
+ PipelineOptions options, TestCheckpointMark checkpointMark) {
+ return new TestUnboundedReader(elems);
+ }
+
+ @Override
+ @Nullable
+ public Coder<TestCheckpointMark> getCheckpointMarkCoder() {
+ return new TestCheckpointMark.Coder();
+ }
+
+ @Override
+ public void validate() {}
+
+ @Override
+ public Coder<T> getDefaultOutputCoder() {
+ return coder;
+ }
+
+ private class TestUnboundedReader extends UnboundedReader<T> {
+ private final List<T> elems;
+ private int index;
+
+ public TestUnboundedReader(List<T> elems) {
+ this.elems = elems;
+ this.index = -1;
+ }
+
+ @Override
+ public boolean start() throws IOException {
+ return advance();
+ }
+
+ @Override
+ public boolean advance() throws IOException {
+ if (index + 1 < elems.size()) {
+ index++;
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public Instant getWatermark() {
+ return Instant.now();
+ }
+
+ @Override
+ public CheckpointMark getCheckpointMark() {
+ return new TestCheckpointMark();
+ }
+
+ @Override
+ public UnboundedSource<T, ?> getCurrentSource() {
+ TestUnboundedSource<T> source = TestUnboundedSource.this;
+ return source;
+ }
+
+ @Override
+ public T getCurrent() throws NoSuchElementException {
+ return elems.get(index);
+ }
+
+ @Override
+ public Instant getCurrentTimestamp() throws NoSuchElementException {
+ return Instant.now();
+ }
+
+ @Override
+ public void close() throws IOException {
+ readerClosedCount++;
+ }
+ }
+ }
+
+ private static class TestCheckpointMark implements CheckpointMark {
+ @Override
+ public void finalizeCheckpoint() throws IOException {}
+
+ public static class Coder extends AtomicCoder<TestCheckpointMark> {
+ @Override
+ public void encode(
+ TestCheckpointMark value,
+ OutputStream outStream,
+ org.apache.beam.sdk.coders.Coder.Context context)
+ throws CoderException, IOException {}
+
+ @Override
+ public TestCheckpointMark decode(
+ InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
+ throws CoderException, IOException {
+ return new TestCheckpointMark();
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java
new file mode 100644
index 0000000..859418b
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java
@@ -0,0 +1,101 @@
+/*
+ * 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.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.PCollectionViewWriter;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.Values;
+import org.apache.beam.sdk.transforms.View.CreatePCollectionView;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.util.PCollectionViews;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+
+import com.google.common.collect.ImmutableList;
+
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link ViewEvaluatorFactory}.
+ */
+@RunWith(JUnit4.class)
+public class ViewEvaluatorFactoryTest {
+ private BundleFactory bundleFactory = InProcessBundleFactory.create();
+
+ @Test
+ public void testInMemoryEvaluator() throws Exception {
+ TestPipeline p = TestPipeline.create();
+
+ PCollection<String> input = p.apply(Create.of("foo", "bar"));
+ CreatePCollectionView<String, Iterable<String>> createView =
+ CreatePCollectionView.of(
+ PCollectionViews.iterableView(p, input.getWindowingStrategy(), StringUtf8Coder.of()));
+ PCollection<Iterable<String>> concat =
+ input.apply(WithKeys.<Void, String>of((Void) null))
+ .setCoder(KvCoder.of(VoidCoder.of(), StringUtf8Coder.of()))
+ .apply(GroupByKey.<Void, String>create())
+ .apply(Values.<Iterable<String>>create());
+ PCollectionView<Iterable<String>> view =
+ concat.apply(new ViewEvaluatorFactory.WriteView<>(createView));
+
+ InProcessEvaluationContext context = mock(InProcessEvaluationContext.class);
+ TestViewWriter<String, Iterable<String>> viewWriter = new TestViewWriter<>();
+ when(context.createPCollectionViewWriter(concat, view)).thenReturn(viewWriter);
+
+ CommittedBundle<String> inputBundle =
+ bundleFactory.createRootBundle(input).commit(Instant.now());
+ TransformEvaluator<Iterable<String>> evaluator =
+ new ViewEvaluatorFactory()
+ .forApplication(view.getProducingTransformInternal(), inputBundle, context);
+
+ evaluator.processElement(
+ WindowedValue.<Iterable<String>>valueInGlobalWindow(ImmutableList.of("foo", "bar")));
+ assertThat(viewWriter.latest, nullValue());
+
+ evaluator.finishBundle();
+ assertThat(
+ viewWriter.latest,
+ containsInAnyOrder(
+ WindowedValue.valueInGlobalWindow("foo"), WindowedValue.valueInGlobalWindow("bar")));
+ }
+
+ private static class TestViewWriter<ElemT, ViewT> implements PCollectionViewWriter<ElemT, ViewT> {
+ private Iterable<WindowedValue<ElemT>> latest;
+
+ @Override
+ public void add(Iterable<WindowedValue<ElemT>> values) {
+ latest = values;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java
new file mode 100644
index 0000000..d47cf5e
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java
@@ -0,0 +1,128 @@
+/*
+ * 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.hamcrest.Matchers.equalTo;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+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.WindowFn;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.PCollection;
+
+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 java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Tests for {@link WatermarkCallbackExecutor}.
+ */
+@RunWith(JUnit4.class)
+public class WatermarkCallbackExecutorTest {
+ private WatermarkCallbackExecutor executor = WatermarkCallbackExecutor.create();
+ private AppliedPTransform<?, ?, ?> create;
+ private AppliedPTransform<?, ?, ?> sum;
+
+ @Before
+ public void setup() {
+ TestPipeline p = TestPipeline.create();
+ PCollection<Integer> created = p.apply(Create.of(1, 2, 3));
+ create = created.getProducingTransformInternal();
+ sum = created.apply(Sum.integersGlobally()).getProducingTransformInternal();
+ }
+
+ @Test
+ public void onGuaranteedFiringFiresAfterTrigger() throws Exception {
+ CountDownLatch latch = new CountDownLatch(1);
+ executor.callOnGuaranteedFiring(
+ create,
+ GlobalWindow.INSTANCE,
+ WindowingStrategy.globalDefault(),
+ new CountDownLatchCallback(latch));
+
+ executor.fireForWatermark(create, BoundedWindow.TIMESTAMP_MAX_VALUE);
+ assertThat(latch.await(500, TimeUnit.MILLISECONDS), equalTo(true));
+ }
+
+ @Test
+ public void multipleCallbacksShouldFireFires() throws Exception {
+ CountDownLatch latch = new CountDownLatch(2);
+ WindowFn<Object, IntervalWindow> windowFn = FixedWindows.of(Duration.standardMinutes(10));
+ IntervalWindow window =
+ new IntervalWindow(new Instant(0L), new Instant(0L).plus(Duration.standardMinutes(10)));
+ executor.callOnGuaranteedFiring(
+ create, window, WindowingStrategy.of(windowFn), new CountDownLatchCallback(latch));
+ executor.callOnGuaranteedFiring(
+ create, window, WindowingStrategy.of(windowFn), new CountDownLatchCallback(latch));
+
+ executor.fireForWatermark(create, new Instant(0L).plus(Duration.standardMinutes(10)));
+ assertThat(latch.await(500, TimeUnit.MILLISECONDS), equalTo(true));
+ }
+
+ @Test
+ public void noCallbacksShouldFire() throws Exception {
+ CountDownLatch latch = new CountDownLatch(1);
+ WindowFn<Object, IntervalWindow> windowFn = FixedWindows.of(Duration.standardMinutes(10));
+ IntervalWindow window =
+ new IntervalWindow(new Instant(0L), new Instant(0L).plus(Duration.standardMinutes(10)));
+ executor.callOnGuaranteedFiring(
+ create, window, WindowingStrategy.of(windowFn), new CountDownLatchCallback(latch));
+
+ executor.fireForWatermark(create, new Instant(0L).plus(Duration.standardMinutes(5)));
+ assertThat(latch.await(500, TimeUnit.MILLISECONDS), equalTo(false));
+ }
+
+ @Test
+ public void unrelatedStepShouldNotFire() throws Exception {
+ CountDownLatch latch = new CountDownLatch(1);
+ WindowFn<Object, IntervalWindow> windowFn = FixedWindows.of(Duration.standardMinutes(10));
+ IntervalWindow window =
+ new IntervalWindow(new Instant(0L), new Instant(0L).plus(Duration.standardMinutes(10)));
+ executor.callOnGuaranteedFiring(
+ sum, window, WindowingStrategy.of(windowFn), new CountDownLatchCallback(latch));
+
+ executor.fireForWatermark(create, new Instant(0L).plus(Duration.standardMinutes(20)));
+ assertThat(latch.await(500, TimeUnit.MILLISECONDS), equalTo(false));
+ }
+
+ private static class CountDownLatchCallback implements Runnable {
+ private final CountDownLatch latch;
+
+ public CountDownLatchCallback(CountDownLatch latch) {
+ this.latch = latch;
+ }
+
+ @Override
+ public void run() {
+ latch.countDown();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..64eb8ea
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java
@@ -0,0 +1,222 @@
+/*
+ * 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.hamcrest.Matchers.containsInAnyOrder;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+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.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.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.Window.Bound;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+
+import org.hamcrest.Matchers;
+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 WindowEvaluatorFactory}.
+ */
+@RunWith(JUnit4.class)
+public class WindowEvaluatorFactoryTest {
+ private static final Instant EPOCH = new Instant(0);
+
+ private PCollection<Long> input;
+ private WindowEvaluatorFactory factory;
+
+ @Mock private InProcessEvaluationContext evaluationContext;
+
+ private BundleFactory bundleFactory;
+
+ private WindowedValue<Long> first =
+ WindowedValue.timestampedValueInGlobalWindow(3L, new Instant(2L));
+ private WindowedValue<Long> second =
+ WindowedValue.timestampedValueInGlobalWindow(
+ Long.valueOf(1L), EPOCH.plus(Duration.standardDays(3)));
+ private WindowedValue<Long> third =
+ WindowedValue.of(
+ Long.valueOf(2L),
+ new Instant(-10L),
+ new IntervalWindow(new Instant(-100), EPOCH),
+ PaneInfo.NO_FIRING);
+
+ @Before
+ public void setup() {
+ MockitoAnnotations.initMocks(this);
+ TestPipeline p = TestPipeline.create();
+ input = p.apply(Create.of(1L, 2L, 3L));
+
+ bundleFactory = InProcessBundleFactory.create();
+ factory = new WindowEvaluatorFactory();
+ }
+
+ @Test
+ public void nullWindowFunSucceeds() throws Exception {
+ Bound<Long> transform =
+ Window.<Long>triggering(
+ AfterWatermark.pastEndOfWindow().withEarlyFirings(AfterPane.elementCountAtLeast(1)))
+ .accumulatingFiredPanes();
+ PCollection<Long> triggering = input.apply(transform);
+
+ CommittedBundle<Long> inputBundle = createInputBundle();
+
+ UncommittedBundle<Long> outputBundle = createOutputBundle(triggering, inputBundle);
+
+ InProcessTransformResult result = runEvaluator(triggering, inputBundle, transform);
+
+ assertThat(
+ Iterables.getOnlyElement(result.getOutputBundles()),
+ Matchers.<UncommittedBundle<?>>equalTo(outputBundle));
+ CommittedBundle<Long> committed = outputBundle.commit(Instant.now());
+ assertThat(committed.getElements(), containsInAnyOrder(third, first, second));
+ }
+
+ @Test
+ public void singleWindowFnSucceeds() throws Exception {
+ Duration windowDuration = Duration.standardDays(7);
+ Bound<Long> transform = Window.<Long>into(FixedWindows.of(windowDuration));
+ PCollection<Long> windowed = input.apply(transform);
+
+ CommittedBundle<Long> inputBundle = createInputBundle();
+
+ UncommittedBundle<Long> outputBundle = createOutputBundle(windowed, inputBundle);
+
+ BoundedWindow firstSecondWindow = new IntervalWindow(EPOCH, EPOCH.plus(windowDuration));
+ BoundedWindow thirdWindow = new IntervalWindow(EPOCH.minus(windowDuration), EPOCH);
+
+ InProcessTransformResult result = runEvaluator(windowed, inputBundle, transform);
+
+ assertThat(
+ Iterables.getOnlyElement(result.getOutputBundles()),
+ Matchers.<UncommittedBundle<?>>equalTo(outputBundle));
+ CommittedBundle<Long> committed = outputBundle.commit(Instant.now());
+
+ WindowedValue<Long> expectedNewFirst =
+ WindowedValue.of(3L, new Instant(2L), firstSecondWindow, PaneInfo.NO_FIRING);
+ WindowedValue<Long> expectedNewSecond =
+ WindowedValue.of(
+ 1L, EPOCH.plus(Duration.standardDays(3)), firstSecondWindow, PaneInfo.NO_FIRING);
+ WindowedValue<Long> expectedNewThird =
+ WindowedValue.of(2L, new Instant(-10L), thirdWindow, PaneInfo.NO_FIRING);
+ assertThat(
+ committed.getElements(),
+ containsInAnyOrder(expectedNewFirst, expectedNewSecond, expectedNewThird));
+ }
+
+ @Test
+ public void multipleWindowsWindowFnSucceeds() throws Exception {
+ Duration windowDuration = Duration.standardDays(6);
+ Duration slidingBy = Duration.standardDays(3);
+ Bound<Long> transform = Window.into(SlidingWindows.of(windowDuration).every(slidingBy));
+ PCollection<Long> windowed = input.apply(transform);
+
+ CommittedBundle<Long> inputBundle = createInputBundle();
+ UncommittedBundle<Long> outputBundle = createOutputBundle(windowed, inputBundle);
+
+ InProcessTransformResult result = runEvaluator(windowed, inputBundle, transform);
+
+ assertThat(
+ Iterables.getOnlyElement(result.getOutputBundles()),
+ Matchers.<UncommittedBundle<?>>equalTo(outputBundle));
+ CommittedBundle<Long> committed = outputBundle.commit(Instant.now());
+
+ BoundedWindow w1 = new IntervalWindow(EPOCH, EPOCH.plus(windowDuration));
+ BoundedWindow w2 =
+ new IntervalWindow(EPOCH.plus(slidingBy), EPOCH.plus(slidingBy).plus(windowDuration));
+ BoundedWindow wMinus1 = new IntervalWindow(EPOCH.minus(windowDuration), EPOCH);
+ BoundedWindow wMinusSlide =
+ new IntervalWindow(EPOCH.minus(windowDuration).plus(slidingBy), EPOCH.plus(slidingBy));
+
+ WindowedValue<Long> expectedFirst =
+ WindowedValue.of(
+ first.getValue(),
+ first.getTimestamp(),
+ ImmutableSet.of(w1, wMinusSlide),
+ PaneInfo.NO_FIRING);
+ WindowedValue<Long> expectedSecond =
+ WindowedValue.of(
+ second.getValue(), second.getTimestamp(), ImmutableSet.of(w1, w2), PaneInfo.NO_FIRING);
+ WindowedValue<Long> expectedThird =
+ WindowedValue.of(
+ third.getValue(),
+ third.getTimestamp(),
+ ImmutableSet.of(wMinus1, wMinusSlide),
+ PaneInfo.NO_FIRING);
+
+ assertThat(
+ committed.getElements(), containsInAnyOrder(expectedFirst, expectedSecond, expectedThird));
+ }
+
+ private CommittedBundle<Long> createInputBundle() {
+ CommittedBundle<Long> inputBundle =
+ bundleFactory
+ .createRootBundle(input)
+ .add(first)
+ .add(second)
+ .add(third)
+ .commit(Instant.now());
+ return inputBundle;
+ }
+
+ private UncommittedBundle<Long> createOutputBundle(
+ PCollection<Long> output, CommittedBundle<Long> inputBundle) {
+ UncommittedBundle<Long> outputBundle = bundleFactory.createBundle(inputBundle, output);
+ when(evaluationContext.createBundle(inputBundle, output)).thenReturn(outputBundle);
+ return outputBundle;
+ }
+
+ private InProcessTransformResult runEvaluator(
+ PCollection<Long> windowed,
+ CommittedBundle<Long> inputBundle,
+ Window.Bound<Long> windowTransform /* Required while Window.Bound is a composite */)
+ throws Exception {
+ TransformEvaluator<Long> evaluator =
+ factory.forApplication(
+ AppliedPTransform.of("Window", input, windowed, windowTransform),
+ inputBundle,
+ evaluationContext);
+
+ evaluator.processElement(first);
+ evaluator.processElement(second);
+ evaluator.processElement(third);
+ InProcessTransformResult result = evaluator.finishBundle();
+ return result;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/pom.xml
----------------------------------------------------------------------
diff --git a/runners/pom.xml b/runners/pom.xml
index 7b2f356..74812e8 100644
--- a/runners/pom.xml
+++ b/runners/pom.xml
@@ -35,6 +35,7 @@
<name>Apache Beam :: Runners</name>
<modules>
+ <module>direct-java</module>
<module>flink</module>
<module>spark</module>
</modules>
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AbstractModelEnforcement.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AbstractModelEnforcement.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AbstractModelEnforcement.java
deleted file mode 100644
index fe9c165..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AbstractModelEnforcement.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.util.WindowedValue;
-
-/**
- * An abstract {@link ModelEnforcement} that provides default empty implementations for each method.
- */
-abstract class AbstractModelEnforcement<T> implements ModelEnforcement<T> {
- @Override
- public void beforeElement(WindowedValue<T> element) {}
-
- @Override
- public void afterElement(WindowedValue<T> element) {}
-
- @Override
- public void afterFinish(
- CommittedBundle<T> input,
- InProcessTransformResult result,
- Iterable<? extends CommittedBundle<?>> outputs) {}
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactory.java
deleted file mode 100644
index e19ffe4..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactory.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.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.io.AvroIO;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.IOChannelUtils;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-
-class AvroIOShardedWriteFactory implements PTransformOverrideFactory {
- @Override
- public <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
- PTransform<InputT, OutputT> transform) {
- if (transform instanceof AvroIO.Write.Bound) {
- @SuppressWarnings("unchecked")
- AvroIO.Write.Bound<InputT> originalWrite = (AvroIO.Write.Bound<InputT>) transform;
- if (originalWrite.getNumShards() > 1
- || (originalWrite.getNumShards() == 1
- && !"".equals(originalWrite.getShardNameTemplate()))) {
- @SuppressWarnings("unchecked")
- PTransform<InputT, OutputT> override =
- (PTransform<InputT, OutputT>) new AvroIOShardedWrite<InputT>(originalWrite);
- return override;
- }
- }
- return transform;
- }
-
- private class AvroIOShardedWrite<InputT> extends ShardControlledWrite<InputT> {
- private final AvroIO.Write.Bound<InputT> initial;
-
- private AvroIOShardedWrite(AvroIO.Write.Bound<InputT> initial) {
- this.initial = initial;
- }
-
- @Override
- int getNumShards() {
- return initial.getNumShards();
- }
-
- @Override
- PTransform<? super PCollection<InputT>, PDone> getSingleShardTransform(int shardNum) {
- String shardName =
- IOChannelUtils.constructName(
- initial.getFilenamePrefix(),
- initial.getShardNameTemplate(),
- initial.getFilenameSuffix(),
- shardNum,
- getNumShards());
- return initial.withoutSharding().to(shardName).withSuffix("");
- }
-
- @Override
- protected PTransform<PCollection<InputT>, PDone> delegate() {
- return initial;
- }
- }
-}