Repository: incubator-beam
Updated Branches:
  refs/heads/eclipse2 [created] 83f0b6436


Update to compile in eclipse again


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

Branch: refs/heads/eclipse2
Commit: a8cbd246c5f74e2c6066f424db19d84ad316b06e
Parents: 7322616
Author: Daniel Kulp <dk...@apache.org>
Authored: Thu Oct 13 12:41:32 2016 -0400
Committer: Daniel Kulp <dk...@apache.org>
Committed: Thu Oct 13 12:41:32 2016 -0400

----------------------------------------------------------------------
 .../beam/runners/direct/BoundedReadEvaluatorFactory.java       | 3 ++-
 .../apache/beam/runners/direct/TestStreamEvaluatorFactory.java | 2 +-
 .../beam/runners/direct/UnboundedReadEvaluatorFactory.java     | 5 +++--
 .../main/java/org/apache/beam/runners/spark/io/SourceRDD.java  | 6 ++++--
 4 files changed, 10 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a8cbd246/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java
----------------------------------------------------------------------
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java
index 326a535..8e06c3f 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java
@@ -33,6 +33,7 @@ import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 
 /**
@@ -131,7 +132,7 @@ final class BoundedReadEvaluatorFactory implements 
TransformEvaluatorFactory {
     }
 
     private <OutputT> Collection<CommittedBundle<?>> createInitialSplits(
-        AppliedPTransform<?, ?, Read.Bounded<OutputT>> transform) {
+        AppliedPTransform<PBegin, ?, Read.Bounded<OutputT>> transform) {
       BoundedSource<OutputT> source = transform.getTransform().getSource();
       return Collections.<CommittedBundle<?>>singleton(
           evaluationContext

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a8cbd246/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java
----------------------------------------------------------------------
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java
index 4a48a58..fc4f7c5 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java
@@ -204,7 +204,7 @@ class TestStreamEvaluatorFactory implements 
TransformEvaluatorFactory {
     }
 
     private <T> Collection<CommittedBundle<?>> createInputBundle(
-        AppliedPTransform<?, ?, TestStream<T>> transform) {
+        AppliedPTransform<PBegin, ?, TestStream<T>> transform) {
       CommittedBundle<TestStreamIndex<T>> initialBundle =
           evaluationContext
               .<TestStreamIndex<T>>createRootBundle()

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a8cbd246/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java
----------------------------------------------------------------------
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java
index 08dc286..183225f 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java
@@ -37,6 +37,7 @@ import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.joda.time.Instant;
 
@@ -70,7 +71,7 @@ class UnboundedReadEvaluatorFactory implements 
TransformEvaluatorFactory {
   }
 
   private <OutputT> TransformEvaluator<?> createEvaluator(
-      AppliedPTransform<?, PCollection<OutputT>, Read.Unbounded<OutputT>> 
application) {
+      AppliedPTransform<PBegin, PCollection<OutputT>, Read.Unbounded<OutputT>> 
application) {
     return new UnboundedReadEvaluator<>(
         application, evaluationContext, readerReuseChance);
   }
@@ -258,7 +259,7 @@ class UnboundedReadEvaluatorFactory implements 
TransformEvaluatorFactory {
     }
 
     private <OutputT> Collection<CommittedBundle<?>> createInitialSplits(
-        AppliedPTransform<?, ?, Read.Unbounded<OutputT>> transform) {
+        AppliedPTransform<PBegin, ?, Read.Unbounded<OutputT>> transform) {
       UnboundedSource<OutputT, ?> source = 
transform.getTransform().getSource();
       UnboundedReadDeduplicator deduplicator =
           source.requiresDeduping()

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a8cbd246/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java
----------------------------------------------------------------------
diff --git 
a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java 
b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java
index ec2d2cf..679b8b1 100644
--- 
a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java
+++ 
b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java
@@ -21,6 +21,7 @@ package org.apache.beam.runners.spark.io;
 import static com.google.common.base.Preconditions.checkArgument;
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import org.apache.beam.runners.spark.translation.SparkRuntimeContext;
@@ -56,8 +57,9 @@ public class SourceRDD {
     private final int numPartitions;
 
     // to satisfy Scala API.
-    private static final scala.collection.immutable.List<Dependency<?>> NIL =
-        scala.collection.immutable.List.empty();
+    private static final scala.collection.immutable.Seq<Dependency<?>> NIL =
+        scala.collection.JavaConversions
+          .asScalaBuffer(Collections.<Dependency<?>>emptyList()).toList();
 
     public Bounded(SparkContext sc,
                    BoundedSource<T> source,

Reply via email to