aboutsummaryrefslogtreecommitdiff
path: root/core/src/main/scala/spark/rdd/UnionRDD.scala
blob: 2c52a67e22635bf196a2d5341bf0972a2ab3075c (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
package spark.rdd

import scala.collection.mutable.ArrayBuffer
import spark.{Dependency, RangeDependency, RDD, SparkContext, Partition, TaskContext}
import java.io.{ObjectOutputStream, IOException}

private[spark] class UnionPartition[T: ClassManifest](idx: Int, rdd: RDD[T], splitIndex: Int)
  extends Partition {

  var split: Partition = rdd.partitions(splitIndex)

  def iterator(context: TaskContext) = rdd.iterator(split, context)

  def preferredLocations() = rdd.preferredLocations(split)

  override val index: Int = idx

  @throws(classOf[IOException])
  private def writeObject(oos: ObjectOutputStream) {
    // Update the reference to parent split at the time of task serialization
    split = rdd.partitions(splitIndex)
    oos.defaultWriteObject()
  }
}

class UnionRDD[T: ClassManifest](
    sc: SparkContext,
    @transient var rdds: Seq[RDD[T]])
  extends RDD[T](sc, Nil) {  // Nil since we implement getDependencies

  override def getPartitions: Array[Partition] = {
    val array = new Array[Partition](rdds.map(_.partitions.size).sum)
    var pos = 0
    for (rdd <- rdds; split <- rdd.partitions) {
      array(pos) = new UnionPartition(pos, rdd, split.index)
      pos += 1
    }
    array
  }

  override def getDependencies: Seq[Dependency[_]] = {
    val deps = new ArrayBuffer[Dependency[_]]
    var pos = 0
    for (rdd <- rdds) {
      deps += new RangeDependency(rdd, 0, pos, rdd.partitions.size)
      pos += rdd.partitions.size
    }
    deps
  }

  override def compute(s: Partition, context: TaskContext): Iterator[T] =
    s.asInstanceOf[UnionPartition[T]].iterator(context)

  override def getPreferredLocations(s: Partition): Seq[String] =
    s.asInstanceOf[UnionPartition[T]].preferredLocations()
}