Remove DoFnTester#setSideOutputTags

Side Outputs are appended to the map of outputs on-demand.


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

Branch: refs/heads/master
Commit: b8cd5739492b16c3481281074586f891f4554999
Parents: 1c1af62
Author: Thomas Groh <tg...@google.com>
Authored: Thu Jul 14 15:34:02 2016 -0700
Committer: Kenneth Knowles <k...@google.com>
Committed: Thu Jul 14 17:13:11 2016 -0700

----------------------------------------------------------------------
 .../beam/runners/dataflow/DataflowRunnerTest.java   | 10 ----------
 .../org/apache/beam/sdk/transforms/DoFnTester.java  | 16 ----------------
 2 files changed, 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b8cd5739/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
----------------------------------------------------------------------
diff --git 
a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
 
b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
index f3cbb38..fe288ad 100644
--- 
a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
+++ 
b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
@@ -18,7 +18,6 @@
 package org.apache.beam.runners.dataflow;
 
 import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow;
-
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.equalTo;
@@ -84,7 +83,6 @@ import org.apache.beam.sdk.values.PDone;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.TimestampedValue;
 import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.sdk.values.TupleTagList;
 
 import com.google.api.services.dataflow.Dataflow;
 import com.google.api.services.dataflow.model.DataflowPackage;
@@ -1060,8 +1058,6 @@ public class DataflowRunnerTest {
             keyCoder,
             ismCoder,
             false /* unique keys */));
-    doFnTester.setSideOutputTags(TupleTagList.of(
-        ImmutableList.<TupleTag<?>>of(outputForSizeTag, 
outputForEntrySetTag)));
 
     IntervalWindow windowA = new IntervalWindow(new Instant(0), new 
Instant(10));
     IntervalWindow windowB = new IntervalWindow(new Instant(10), new 
Instant(20));
@@ -1162,8 +1158,6 @@ public class DataflowRunnerTest {
             keyCoder,
             ismCoder,
             true /* unique keys */));
-    doFnTester.setSideOutputTags(TupleTagList.of(
-        ImmutableList.<TupleTag<?>>of(outputForSizeTag, 
outputForEntrySetTag)));
 
     IntervalWindow windowA = new IntervalWindow(new Instant(0), new 
Instant(10));
 
@@ -1203,8 +1197,6 @@ public class DataflowRunnerTest {
                IsmRecord<WindowedValue<Long>>> doFnTester = DoFnTester.of(
         new BatchViewAsMultimap.ToIsmMetadataRecordForSizeDoFn<Long, Long, 
IntervalWindow>(
             windowCoder));
-    doFnTester.setSideOutputTags(TupleTagList.of(
-        ImmutableList.<TupleTag<?>>of(outputForSizeTag, 
outputForEntrySetTag)));
 
     IntervalWindow windowA = new IntervalWindow(new Instant(0), new 
Instant(10));
     IntervalWindow windowB = new IntervalWindow(new Instant(10), new 
Instant(20));
@@ -1256,8 +1248,6 @@ public class DataflowRunnerTest {
                IsmRecord<WindowedValue<Long>>> doFnTester = DoFnTester.of(
         new BatchViewAsMultimap.ToIsmMetadataRecordForKeyDoFn<Long, Long, 
IntervalWindow>(
             keyCoder, windowCoder));
-    doFnTester.setSideOutputTags(TupleTagList.of(
-        ImmutableList.<TupleTag<?>>of(outputForSizeTag, 
outputForEntrySetTag)));
 
     IntervalWindow windowA = new IntervalWindow(new Instant(0), new 
Instant(10));
     IntervalWindow windowB = new IntervalWindow(new Instant(10), new 
Instant(20));

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b8cd5739/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
----------------------------------------------------------------------
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
index a638feb..f8479de 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
@@ -24,7 +24,6 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.PTuple;
 import org.apache.beam.sdk.util.SerializableUtils;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.WindowedValue;
@@ -34,7 +33,6 @@ import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TimestampedValue;
 import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.sdk.values.TupleTagList;
 
 import com.google.common.base.Function;
 import com.google.common.base.MoreObjects;
@@ -133,20 +131,6 @@ public class DoFnTester<InputT, OutputT> {
   }
 
   /**
-   * Registers the list of {@code TupleTag}s that can be used by the
-   * {@code DoFn} under test to output to side output
-   * {@code PCollection}s.
-   *
-   * <p>If needed, first creates a fresh instance of the DoFn under test.
-   *
-   * <p>If this isn't called, {@code DoFnTester} assumes the
-   * {@code DoFn} doesn't emit to any side outputs.
-   */
-  public void setSideOutputTags(TupleTagList sideOutputTags) {
-    resetState();
-  }
-
-  /**
    * A convenience operation that first calls {@link #startBundle},
    * then calls {@link #processElement} on each of the input elements, then
    * calls {@link #finishBundle}, then returns the result of

Reply via email to