beam-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From da...@apache.org
Subject [05/74] [partial] incubator-beam git commit: Rename com/google/cloud/dataflow->org/apache/beam
Date Thu, 14 Apr 2016 04:47:52 GMT
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessEvaluationContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessEvaluationContext.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessEvaluationContext.java
deleted file mode 100644
index 078827d..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessEvaluationContext.java
+++ /dev/null
@@ -1,411 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.FiredTimers;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.PCollectionViewWriter;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger;
-import com.google.cloud.dataflow.sdk.util.ExecutionContext;
-import com.google.cloud.dataflow.sdk.util.SideInputReader;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.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 Iterable<? extends CommittedBundle<?>> 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 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 SideInputReader} 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 SideInputReader createSideInputReader(final List<PCollectionView<?>> sideInputs) {
-    return sideInputContainer.createReaderForViews(sideInputs);
-  }
-
-  /**
-   * 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/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessExecutionContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessExecutionContext.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessExecutionContext.java
deleted file mode 100644
index adea2a6..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessExecutionContext.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
-import com.google.cloud.dataflow.sdk.util.BaseExecutionContext;
-import com.google.cloud.dataflow.sdk.util.ExecutionContext;
-import com.google.cloud.dataflow.sdk.util.TimerInternals;
-import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler;
-import com.google.cloud.dataflow.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, StateSampler stateSampler) {
-    return new InProcessStepContext(this, stepName, transformName);
-  }
-
-  /**
-   * Step Context for the {@link InProcessPipelineRunner}.
-   */
-  public class InProcessStepContext
-      extends com.google.cloud.dataflow.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/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessExecutor.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessExecutor.java
deleted file mode 100644
index 2792fd3..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.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/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessPipelineOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessPipelineOptions.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessPipelineOptions.java
deleted file mode 100644
index d44ea78..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.options.ApplicationNameOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.Hidden;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Validation.Required;
-import com.google.cloud.dataflow.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/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessPipelineRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessPipelineRunner.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessPipelineRunner.java
deleted file mode 100644
index 797a533..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessPipelineRunner.java
+++ /dev/null
@@ -1,366 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.annotations.Experimental;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineExecutionException;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.AggregatorPipelineExtractor;
-import com.google.cloud.dataflow.sdk.runners.AggregatorRetrievalException;
-import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
-import com.google.cloud.dataflow.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOverrideFactory;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessCreate.InProcessCreateOverrideFactory;
-import com.google.cloud.dataflow.sdk.runners.inprocess.ViewEvaluatorFactory.InProcessViewOverrideFactory;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View.CreatePCollectionView;
-import com.google.cloud.dataflow.sdk.util.MapAggregatorValues;
-import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.common.Counter;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.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(Create.Values.class, new InProcessCreateOverrideFactory())
-              .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 whether this bundle is keyed. A bundle that is part of a {@link PCollection} that
-     * occurs after a {@link GroupByKey} is keyed by the result of the last {@link GroupByKey}.
-     */
-    boolean isKeyed();
-
-    /**
-     * 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();
-  }
-
-  /**
-   * 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 com.google.cloud.dataflow.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/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessRegistrar.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessRegistrar.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessRegistrar.java
deleted file mode 100644
index e031aa6..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessRegistrar.java
+++ /dev/null
@@ -1,54 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.auto.service.AutoService;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar;
-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/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessSideInputContainer.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessSideInputContainer.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessSideInputContainer.java
deleted file mode 100644
index 70c2f20..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessSideInputContainer.java
+++ /dev/null
@@ -1,232 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.PCollectionViewWindow;
-import com.google.cloud.dataflow.sdk.util.SideInputReader;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.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 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 SideInputReader 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 SideInputReader {
-    private final Collection<PCollectionView<?>> readerViews;
-
-    private SideInputContainerSideInputReader(Collection<PCollectionView<?>> readerViews) {
-      this.readerViews = ImmutableSet.copyOf(readerViews);
-    }
-
-    @Override
-    @Nullable
-    public <T> T get(final PCollectionView<T> view, final BoundedWindow window) {
-      checkArgument(
-          readerViews.contains(view), "calling get(PCollectionView) with unknown view: " + view);
-      PCollectionViewWindow<T> windowedView = PCollectionViewWindow.of(view, window);
-      try {
-        final SettableFuture<Iterable<? extends WindowedValue<?>>> future =
-            viewByWindows.get(windowedView);
-
-        WindowingStrategy<?, ?> windowingStrategy = view.getWindowingStrategyInternal();
-        evaluationContext.scheduleAfterOutputWouldBeProduced(
-            view, window, windowingStrategy, new Runnable() {
-              @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("InProcessSideInputContainerEmptyCallback")
-                .add("view", view)
-                .add("window", window)
-                .toString();
-          }
-        });
-        // 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);
-      }
-    }
-
-    @Override
-    public <T> boolean contains(PCollectionView<T> view) {
-      return readerViews.contains(view);
-    }
-
-    @Override
-    public boolean isEmpty() {
-      return readerViews.isEmpty();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessTimerInternals.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessTimerInternals.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessTimerInternals.java
deleted file mode 100644
index 9cd6069..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessTimerInternals.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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate.TimerUpdateBuilder;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
-import com.google.cloud.dataflow.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/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessTransformResult.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessTransformResult.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/InProcessTransformResult.java
deleted file mode 100644
index 41d6672..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.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/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/KeyedPValueTrackingVisitor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/KeyedPValueTrackingVisitor.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/KeyedPValueTrackingVisitor.java
deleted file mode 100644
index 81a4801..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/KeyedPValueTrackingVisitor.java
+++ /dev/null
@@ -1,97 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor;
-import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.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/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ModelEnforcement.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ModelEnforcement.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ModelEnforcement.java
deleted file mode 100644
index de2a77a..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.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/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ModelEnforcementFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ModelEnforcementFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ModelEnforcementFactory.java
deleted file mode 100644
index 2bc01ce..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import com.google.cloud.dataflow.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/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/NanosOffsetClock.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/NanosOffsetClock.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/NanosOffsetClock.java
deleted file mode 100644
index d033e4a..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/NanosOffsetClock.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.options.DefaultValueFactory;
-import com.google.cloud.dataflow.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/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/PTransformOverrideFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/PTransformOverrideFactory.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/PTransformOverrideFactory.java
deleted file mode 100644
index 77fa585..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.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/0393a791/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ParDoInProcessEvaluator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ParDoInProcessEvaluator.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ParDoInProcessEvaluator.java
deleted file mode 100644
index 4b4d699..0000000
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ParDoInProcessEvaluator.java
+++ /dev/null
@@ -1,120 +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 com.google.cloud.dataflow.sdk.runners.inprocess;
-
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessExecutionContext.InProcessStepContext;
-import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.util.DoFnRunner;
-import com.google.cloud.dataflow.sdk.util.DoFnRunners.OutputManager;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.common.CounterSet;
-import com.google.cloud.dataflow.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.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> {
-  private final DoFnRunner<T, ?> fnRunner;
-  private final AppliedPTransform<PCollection<T>, ?, ?> transform;
-  private final CounterSet counters;
-  private final Collection<UncommittedBundle<?>> outputBundles;
-  private final InProcessStepContext stepContext;
-
-  public 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);
-      }
-    }
-  }
-}



Mime
View raw message