Skip to content
Snippets Groups Projects
Commit 588a1695 authored by Patrick Wendell's avatar Patrick Wendell
Browse files

Merge pull request #297 from tdas/window-improvement

Improvements to DStream window ops and refactoring of Spark's CheckpointSuite

- Added a new RDD - PartitionerAwareUnionRDD. Using this RDD, one can take multiple RDDs partitioned by the same partitioner and unify them into a single RDD while preserving the partitioner. So m RDDs with p partitions each will be unified to a single RDD with p partitions and the same partitioner. The preferred location for each partition of the unified RDD will be the most common preferred location of the corresponding partitions of the parent RDDs. For example, location of partition 0 of the unified RDD will be where most of partition 0 of the parent RDDs are located.
- Improved the performance of DStream's reduceByKeyAndWindow and groupByKeyAndWindow. Both these operations work by doing per-batch reduceByKey/groupByKey and then using PartitionerAwareUnionRDD to union the RDDs across the window. This eliminates a shuffle related to the window operation, which can reduce batch processing time by 30-40% for simple workloads.
- Fixed bugs and simplified Spark's CheckpointSuite. Some of the tests were incorrect and unreliable. Added missing tests for ZippedRDD. I can go into greater detail if necessary.
- Added mapSideCombine option to combineByKeyAndWindow.
parents 7bafb68d 5fde4566
No related branches found
No related tags found
No related merge requests found
Showing
with 388 additions and 172 deletions
/*
* 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.spark.rdd
import scala.reflect.ClassTag
import java.io.{ObjectOutputStream, IOException}
import org.apache.spark.{TaskContext, OneToOneDependency, SparkContext, Partition}
/**
* Class representing partitions of PartitionerAwareUnionRDD, which maintains the list of corresponding partitions
* of parent RDDs.
*/
private[spark]
class PartitionerAwareUnionRDDPartition(
@transient val rdds: Seq[RDD[_]],
val idx: Int
) extends Partition {
var parents = rdds.map(_.partitions(idx)).toArray
override val index = idx
override def hashCode(): Int = idx
@throws(classOf[IOException])
private def writeObject(oos: ObjectOutputStream) {
// Update the reference to parent partition at the time of task serialization
parents = rdds.map(_.partitions(index)).toArray
oos.defaultWriteObject()
}
}
/**
* Class representing an RDD that can take multiple RDDs partitioned by the same partitioner and
* unify them into a single RDD while preserving the partitioner. So m RDDs with p partitions each
* will be unified to a single RDD with p partitions and the same partitioner. The preferred
* location for each partition of the unified RDD will be the most common preferred location
* of the corresponding partitions of the parent RDDs. For example, location of partition 0
* of the unified RDD will be where most of partition 0 of the parent RDDs are located.
*/
private[spark]
class PartitionerAwareUnionRDD[T: ClassTag](
sc: SparkContext,
var rdds: Seq[RDD[T]]
) extends RDD[T](sc, rdds.map(x => new OneToOneDependency(x))) {
require(rdds.length > 0)
require(rdds.flatMap(_.partitioner).toSet.size == 1,
"Parent RDDs have different partitioners: " + rdds.flatMap(_.partitioner))
override val partitioner = rdds.head.partitioner
override def getPartitions: Array[Partition] = {
val numPartitions = partitioner.get.numPartitions
(0 until numPartitions).map(index => {
new PartitionerAwareUnionRDDPartition(rdds, index)
}).toArray
}
// Get the location where most of the partitions of parent RDDs are located
override def getPreferredLocations(s: Partition): Seq[String] = {
logDebug("Finding preferred location for " + this + ", partition " + s.index)
val parentPartitions = s.asInstanceOf[PartitionerAwareUnionRDDPartition].parents
val locations = rdds.zip(parentPartitions).flatMap {
case (rdd, part) => {
val parentLocations = currPrefLocs(rdd, part)
logDebug("Location of " + rdd + " partition " + part.index + " = " + parentLocations)
parentLocations
}
}
val location = if (locations.isEmpty) {
None
} else {
// Find the location that maximum number of parent partitions prefer
Some(locations.groupBy(x => x).maxBy(_._2.length)._1)
}
logDebug("Selected location for " + this + ", partition " + s.index + " = " + location)
location.toSeq
}
override def compute(s: Partition, context: TaskContext): Iterator[T] = {
val parentPartitions = s.asInstanceOf[PartitionerAwareUnionRDDPartition].parents
rdds.zip(parentPartitions).iterator.flatMap {
case (rdd, p) => rdd.iterator(p, context)
}
}
override def clearDependencies() {
super.clearDependencies()
rdds = null
}
// Get the *current* preferred locations from the DAGScheduler (as opposed to the static ones)
private def currPrefLocs(rdd: RDD[_], part: Partition): Seq[String] = {
rdd.context.getPreferredLocs(rdd, part.index).map(tl => tl.host)
}
}
......@@ -40,7 +40,7 @@ private[spark] object CheckpointState extends Enumeration {
* manages the post-checkpoint state by providing the updated partitions, iterator and preferred locations
* of the checkpointed RDD.
*/
private[spark] class RDDCheckpointData[T: ClassTag](rdd: RDD[T])
private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T])
extends Logging with Serializable {
import CheckpointState._
......
This diff is collapsed.
......@@ -84,6 +84,33 @@ class RDDSuite extends FunSuite with SharedSparkContext {
assert(sc.union(Seq(nums, nums)).collect().toList === List(1, 2, 3, 4, 1, 2, 3, 4))
}
test("partitioner aware union") {
import SparkContext._
def makeRDDWithPartitioner(seq: Seq[Int]) = {
sc.makeRDD(seq, 1)
.map(x => (x, null))
.partitionBy(new HashPartitioner(2))
.mapPartitions(_.map(_._1), true)
}
val nums1 = makeRDDWithPartitioner(1 to 4)
val nums2 = makeRDDWithPartitioner(5 to 8)
assert(nums1.partitioner == nums2.partitioner)
assert(new PartitionerAwareUnionRDD(sc, Seq(nums1)).collect().toSet === Set(1, 2, 3, 4))
val union = new PartitionerAwareUnionRDD(sc, Seq(nums1, nums2))
assert(union.collect().toSet === Set(1, 2, 3, 4, 5, 6, 7, 8))
val nums1Parts = nums1.collectPartitions()
val nums2Parts = nums2.collectPartitions()
val unionParts = union.collectPartitions()
assert(nums1Parts.length === 2)
assert(nums2Parts.length === 2)
assert(unionParts.length === 2)
assert((nums1Parts(0) ++ nums2Parts(0)).toList === unionParts(0).toList)
assert((nums1Parts(1) ++ nums2Parts(1)).toList === unionParts(1).toList)
assert(union.partitioner === nums1.partitioner)
}
test("aggregate") {
val pairs = sc.makeRDD(Array(("a", 1), ("b", 2), ("a", 2), ("c", 5), ("a", 3)))
type StringMap = HashMap[String, Int]
......
......@@ -108,8 +108,9 @@ extends Serializable {
createCombiner: V => C,
mergeValue: (C, V) => C,
mergeCombiner: (C, C) => C,
partitioner: Partitioner) : DStream[(K, C)] = {
new ShuffledDStream[K, V, C](self, createCombiner, mergeValue, mergeCombiner, partitioner)
partitioner: Partitioner,
mapSideCombine: Boolean = true): DStream[(K, C)] = {
new ShuffledDStream[K, V, C](self, createCombiner, mergeValue, mergeCombiner, partitioner, mapSideCombine)
}
/**
......@@ -173,7 +174,13 @@ extends Serializable {
slideDuration: Duration,
partitioner: Partitioner
): DStream[(K, Seq[V])] = {
self.window(windowDuration, slideDuration).groupByKey(partitioner)
val createCombiner = (v: Seq[V]) => new ArrayBuffer[V] ++= v
val mergeValue = (buf: ArrayBuffer[V], v: Seq[V]) => buf ++= v
val mergeCombiner = (buf1: ArrayBuffer[V], buf2: ArrayBuffer[V]) => buf1 ++= buf2
self.groupByKey(partitioner)
.window(windowDuration, slideDuration)
.combineByKey[ArrayBuffer[V]](createCombiner, mergeValue, mergeCombiner, partitioner)
.asInstanceOf[DStream[(K, Seq[V])]]
}
/**
......
......@@ -155,7 +155,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
/**
* Combine elements of each key in DStream's RDDs using custom function. This is similar to the
* combineByKey for RDDs. Please refer to combineByKey in [[org.apache.spark.PairRDDFunctions]] for more
* combineByKey for RDDs. Please refer to combineByKey in [[org.apache.spark.rdd.PairRDDFunctions]] for more
* information.
*/
def combineByKey[C](createCombiner: JFunction[V, C],
......@@ -168,6 +168,22 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
dstream.combineByKey(createCombiner, mergeValue, mergeCombiners, partitioner)
}
/**
* Combine elements of each key in DStream's RDDs using custom function. This is similar to the
* combineByKey for RDDs. Please refer to combineByKey in [[org.apache.spark.rdd.PairRDDFunctions]] for more
* information.
*/
def combineByKey[C](createCombiner: JFunction[V, C],
mergeValue: JFunction2[C, V, C],
mergeCombiners: JFunction2[C, C, C],
partitioner: Partitioner,
mapSideCombine: Boolean
): JavaPairDStream[K, C] = {
implicit val cm: ClassTag[C] =
implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]]
dstream.combineByKey(createCombiner, mergeValue, mergeCombiners, partitioner, mapSideCombine)
}
/**
* Return a new DStream by applying `groupByKey` over a sliding window. This is similar to
* `DStream.groupByKey()` but applies it over a sliding window. The new DStream generates RDDs
......
......@@ -29,8 +29,9 @@ class ShuffledDStream[K: ClassTag, V: ClassTag, C: ClassTag](
createCombiner: V => C,
mergeValue: (C, V) => C,
mergeCombiner: (C, C) => C,
partitioner: Partitioner
) extends DStream [(K,C)] (parent.ssc) {
partitioner: Partitioner,
mapSideCombine: Boolean = true
) extends DStream[(K,C)] (parent.ssc) {
override def dependencies = List(parent)
......@@ -38,8 +39,8 @@ class ShuffledDStream[K: ClassTag, V: ClassTag, C: ClassTag](
override def compute(validTime: Time): Option[RDD[(K,C)]] = {
parent.getOrCompute(validTime) match {
case Some(rdd) =>
Some(rdd.combineByKey[C](createCombiner, mergeValue, mergeCombiner, partitioner))
case Some(rdd) => Some(rdd.combineByKey[C](
createCombiner, mergeValue, mergeCombiner, partitioner, mapSideCombine))
case None => None
}
}
......
......@@ -17,10 +17,10 @@
package org.apache.spark.streaming.dstream
import org.apache.spark.rdd.RDD
import org.apache.spark.rdd.UnionRDD
import org.apache.spark.rdd.{PartitionerAwareUnionRDD, RDD, UnionRDD}
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.{Duration, Interval, Time, DStream}
import org.apache.spark.streaming._
import org.apache.spark.streaming.Duration
import scala.reflect.ClassTag
......@@ -51,6 +51,14 @@ class WindowedDStream[T: ClassTag](
override def compute(validTime: Time): Option[RDD[T]] = {
val currentWindow = new Interval(validTime - windowDuration + parent.slideDuration, validTime)
Some(new UnionRDD(ssc.sc, parent.slice(currentWindow)))
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)
}
}
......@@ -224,9 +224,7 @@ class WindowOperationsSuite extends TestSuiteBase {
val slideDuration = Seconds(1)
val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt
val operation = (s: DStream[(String, Int)]) => {
s.groupByKeyAndWindow(windowDuration, slideDuration)
.map(x => (x._1, x._2.toSet))
.persist()
s.groupByKeyAndWindow(windowDuration, slideDuration).map(x => (x._1, x._2.toSet))
}
testOperation(input, operation, expectedOutput, numBatches, true)
}
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment