Skip to content
Snippets Groups Projects
Commit 1e10df0a authored by Matei Zaharia's avatar Matei Zaharia
Browse files

Merge pull request #111 from alupher/master

Adding sorting to RDDs
parents aa04f87c 0d93d95b
No related branches found
No related tags found
No related merge requests found
......@@ -359,6 +359,29 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
def getValueClass() = implicitly[ClassManifest[V]].erasure
}
class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest](
self: RDD[(K, V)])
extends Logging
with Serializable {
def sortByKey(ascending: Boolean = true): RDD[(K,V)] = {
val rangePartitionedRDD = self.partitionBy(new RangePartitioner(self.splits.size, self, ascending))
new SortedRDD(rangePartitionedRDD, ascending)
}
}
class SortedRDD[K <% Ordered[K], V](prev: RDD[(K, V)], ascending: Boolean)
extends RDD[(K, V)](prev.context) {
override def splits = prev.splits
override val partitioner = prev.partitioner
override val dependencies = List(new OneToOneDependency(prev))
override def compute(split: Split) = {
prev.iterator(split).toArray
.sortWith((x, y) => if (ascending) x._1 < y._1 else x._1 > y._1).iterator
}
}
class MappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => U) extends RDD[(K, U)](prev.context) {
override def splits = prev.splits
override val dependencies = List(new OneToOneDependency(prev))
......
......@@ -23,4 +23,36 @@ class HashPartitioner(partitions: Int) extends Partitioner {
case _ =>
false
}
}
\ No newline at end of file
}
class RangePartitioner[K <% Ordered[K],V](partitions: Int, rdd: RDD[(K,V)], ascending: Boolean = true)
extends Partitioner {
def numPartitions = partitions
val rddSize = rdd.count()
val maxSampleSize = partitions*10.0
val frac = 1.0.min(maxSampleSize / rddSize)
val rddSample = rdd.sample(true, frac, 1).collect.toList
.sortWith((x, y) => if (ascending) x._1 < y._1 else x._1 > y._1)
.map(_._1)
val bucketSize = rddSample.size / partitions
val rangeBounds = rddSample.zipWithIndex.filter(_._2 % bucketSize == 0)
.map(_._1).slice(1, partitions)
def getPartition(key: Any): Int = {
val k = key.asInstanceOf[K]
val p = rangeBounds.zipWithIndex.foldLeft(0) {
case (part, (bound, index)) =>
if (k > bound) index + 1 else part
}
if (ascending) p else numPartitions-1-p
}
override def equals(other: Any): Boolean = other match {
case r: RangePartitioner[_,_] =>
r.numPartitions == numPartitions & r.rangeBounds == rangeBounds
case _ => false
}
}
......@@ -349,10 +349,13 @@ object SparkContext {
// TODO: Add AccumulatorParams for other types, e.g. lists and strings
implicit def rddToPairRDDFunctions[K: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)]) =
new PairRDDFunctions(rdd)
implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable: ClassManifest](rdd: RDD[(K, V)]) =
new SequenceFileRDDFunctions(rdd)
implicit def rddToOrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)]) =
new OrderedRDDFunctions(rdd)
// Implicit conversions to common Writable types, for saveAsSequenceFile
implicit def intToIntWritable(i: Int) = new IntWritable(i)
......
package spark
import org.scalatest.FunSuite
import SparkContext._
class SortingSuite extends FunSuite {
test("sortByKey") {
val sc = new SparkContext("local", "test")
val pairs = sc.parallelize(Array((1, 0), (2, 0), (0, 0), (3, 0)))
assert(pairs.sortByKey().collect() === Array((0,0), (1,0), (2,0), (3,0)))
sc.stop()
}
test("sortLargeArray") {
val sc = new SparkContext("local", "test")
val rand = new scala.util.Random()
val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) }
val pairs = sc.parallelize(pairArr)
assert(pairs.sortByKey().collect() === pairArr.sortBy(_._1))
sc.stop()
}
test("sortDescending") {
val sc = new SparkContext("local", "test")
val rand = new scala.util.Random()
val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) }
val pairs = sc.parallelize(pairArr)
assert(pairs.sortByKey(false).collect() === pairArr.sortWith((x, y) => x._1 > y._1))
sc.stop()
}
test("sortHighParallelism") {
val sc = new SparkContext("local", "test")
val rand = new scala.util.Random()
val pairArr = Array.fill(3000) { (rand.nextInt(), rand.nextInt()) }
val pairs = sc.parallelize(pairArr, 300)
assert(pairs.sortByKey().collect() === pairArr.sortBy(_._1))
sc.stop()
}
}
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