Delete unused ExecutionContext

Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/97c230af
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/97c230af
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/97c230af

Branch: refs/heads/master
Commit: 97c230af62151fdbe06ac622282d69c74db30b2f
Parents: 0be3cf3
Author: Kenneth Knowles <[email protected]>
Authored: Mon May 22 17:33:38 2017 -0700
Committer: Kenneth Knowles <[email protected]>
Committed: Tue May 23 11:16:27 2017 -0700

----------------------------------------------------------------------
 .../beam/runners/core/ExecutionContext.java     | 36 --------------------
 .../runners/direct/DirectExecutionContext.java  |  3 +-
 .../beam/runners/direct/EvaluationContext.java  | 29 +++++++---------
 3 files changed, 14 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/97c230af/runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java
----------------------------------------------------------------------
diff --git 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java
 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java
deleted file mode 100644
index eac3599..0000000
--- 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.core;
-
-import java.util.Collection;
-
-/**
- * Context for the current execution. This is guaranteed to exist during 
processing,
- * but does not necessarily persist between different batches of work.
- */
-public interface ExecutionContext {
-  /**
-   * Returns the {@link StepContext} associated with the given step.
-   */
-  StepContext getOrCreateStepContext(String stepName, String transformName);
-
-  /**
-   * Returns a collection view of all of the {@link StepContext}s.
-   */
-  Collection<? extends StepContext> getAllStepContexts();
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/97c230af/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java
----------------------------------------------------------------------
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java
index 9b68662..05dbebc 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java
@@ -22,7 +22,6 @@ import java.util.Collections;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import org.apache.beam.runners.core.BaseStepContext;
-import org.apache.beam.runners.core.ExecutionContext;
 import org.apache.beam.runners.core.StepContext;
 import org.apache.beam.runners.core.TimerInternals;
 import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
@@ -34,7 +33,7 @@ import 
org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks;
  * <p>This implementation is not thread safe. A new {@link 
DirectExecutionContext} must be created
  * for each thread that requires it.
  */
-class DirectExecutionContext implements ExecutionContext {
+class DirectExecutionContext {
   private final Clock clock;
   private final StructuralKey<?> key;
   private final CopyOnAccessInMemoryStateInternals existingState;

http://git-wip-us.apache.org/repos/asf/beam/blob/97c230af/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
----------------------------------------------------------------------
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
index c627119..88ce85a 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
@@ -31,7 +31,6 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import javax.annotation.Nullable;
-import org.apache.beam.runners.core.ExecutionContext;
 import org.apache.beam.runners.core.ReadyCheckingSideInputReader;
 import org.apache.beam.runners.core.SideInputReader;
 import org.apache.beam.runners.core.TimerInternals.TimerData;
@@ -52,22 +51,20 @@ import org.apache.beam.sdk.values.WindowingStrategy;
 import org.joda.time.Instant;
 
 /**
- * The evaluation context for a specific pipeline being executed by the
- * {@link DirectRunner}. Contains state shared within the execution across all
- * transforms.
+ * The evaluation context for a specific pipeline being executed by the {@link 
DirectRunner}.
+ * Contains state shared within the execution across all transforms.
  *
- * <p>{@link EvaluationContext} contains shared state for an execution of the
- * {@link DirectRunner} that can be used while evaluating a {@link 
PTransform}. This
- * consists of views into underlying state and watermark implementations, 
access to read and write
- * {@link PCollectionView PCollectionViews}, and managing the
- * {@link ExecutionContext ExecutionContexts}. This includes executing 
callbacks asynchronously when
- * state changes to the appropriate point (e.g. when a {@link PCollectionView} 
is requested and
- * known to be empty).
+ * <p>{@link EvaluationContext} contains shared state for an execution of the 
{@link DirectRunner}
+ * that can be used while evaluating a {@link PTransform}. This consists of 
views into underlying
+ * state and watermark implementations, access to read and write {@link 
PCollectionView
+ * PCollectionViews}, and managing the {@link DirectExecutionContext 
ExecutionContexts}. This
+ * includes executing callbacks asynchronously when state changes to the 
appropriate point (e.g.
+ * when a {@link PCollectionView} is requested and known to be empty).
  *
- * <p>{@link EvaluationContext} also handles results by committing finalizing 
bundles based
- * on the current global state and updating the global state appropriately. 
This includes updating
- * the per-{@link StepAndKey} state, updating global watermarks, and executing 
any callbacks that
- * can be executed.
+ * <p>{@link EvaluationContext} also handles results by committing finalizing 
bundles based on the
+ * current global state and updating the global state appropriately. This 
includes updating the
+ * per-{@link StepAndKey} state, updating global watermarks, and executing any 
callbacks that can be
+ * executed.
  */
 class EvaluationContext {
   /**
@@ -312,7 +309,7 @@ class EvaluationContext {
   }
 
   /**
-   * Get an {@link ExecutionContext} for the provided {@link 
AppliedPTransform} and key.
+   * Get a {@link DirectExecutionContext} for the provided {@link 
AppliedPTransform} and key.
    */
   public DirectExecutionContext getExecutionContext(
       AppliedPTransform<?, ?, ?> application, StructuralKey<?> key) {

Reply via email to