aboutsummaryrefslogtreecommitdiff
path: root/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala
diff options
context:
space:
mode:
Diffstat (limited to 'streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala')
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala11
1 files changed, 2 insertions, 9 deletions
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala
index ee50a8d024..fe0f875525 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala
@@ -19,7 +19,7 @@ package org.apache.spark.streaming.dstream
import scala.reflect.ClassTag
-import org.apache.spark.rdd.{PartitionerAwareUnionRDD, RDD, UnionRDD}
+import org.apache.spark.rdd.RDD
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming._
import org.apache.spark.streaming.Duration
@@ -63,13 +63,6 @@ class WindowedDStream[T: ClassTag](
override def compute(validTime: Time): Option[RDD[T]] = {
val currentWindow = new Interval(validTime - windowDuration + parent.slideDuration, validTime)
val rddsInWindow = parent.slice(currentWindow)
- val windowRDD = if (rddsInWindow.flatMap(_.partitioner).distinct.length == 1) {
- logDebug("Using partition aware union for windowing at " + validTime)
- new PartitionerAwareUnionRDD(ssc.sc, rddsInWindow)
- } else {
- logDebug("Using normal union for windowing at " + validTime)
- new UnionRDD(ssc.sc, rddsInWindow)
- }
- Some(windowRDD)
+ Some(ssc.sc.union(rddsInWindow))
}
}