aboutsummaryrefslogtreecommitdiff
path: root/core
diff options
context:
space:
mode:
authorMichael Armbrust <michael@databricks.com>2014-04-18 12:04:13 -0700
committerReynold Xin <rxin@apache.org>2014-04-18 12:04:13 -0700
commitc399baa0fc40be7aa51835aaeadcd5d768dfdb95 (patch)
tree11ba02f2c442cdd74594607d02c107f04c8eb223 /core
parent81a152c54bff21854de731476f62c8fd50dd29f7 (diff)
downloadspark-c399baa0fc40be7aa51835aaeadcd5d768dfdb95.tar.gz
spark-c399baa0fc40be7aa51835aaeadcd5d768dfdb95.tar.bz2
spark-c399baa0fc40be7aa51835aaeadcd5d768dfdb95.zip
SPARK-1456 Remove view bounds on Ordered in favor of a context bound on Ordering.
This doesn't require creating new Ordering objects per row. Additionally, [view bounds are going to be deprecated](https://issues.scala-lang.org/browse/SI-7629), so we should get rid of them while APIs are still flexible. Author: Michael Armbrust <michael@databricks.com> Closes #410 from marmbrus/viewBounds and squashes the following commits: c574221 [Michael Armbrust] fix example. 812008e [Michael Armbrust] Update Java API. 1b9b85c [Michael Armbrust] Update scala doc. 35798a8 [Michael Armbrust] Remove view bounds on Ordered in favor of a context bound on Ordering.
Diffstat (limited to 'core')
-rw-r--r--core/src/main/scala/org/apache/spark/Partitioner.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/SparkContext.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala26
-rw-r--r--core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala2
5 files changed, 30 insertions, 18 deletions
diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala
index ad99882264..9155159cf6 100644
--- a/core/src/main/scala/org/apache/spark/Partitioner.scala
+++ b/core/src/main/scala/org/apache/spark/Partitioner.scala
@@ -89,12 +89,14 @@ class HashPartitioner(partitions: Int) extends Partitioner {
* A [[org.apache.spark.Partitioner]] that partitions sortable records by range into roughly
* equal ranges. The ranges are determined by sampling the content of the RDD passed in.
*/
-class RangePartitioner[K <% Ordered[K]: ClassTag, V](
+class RangePartitioner[K : Ordering : ClassTag, V](
partitions: Int,
@transient rdd: RDD[_ <: Product2[K,V]],
private val ascending: Boolean = true)
extends Partitioner {
+ private val ordering = implicitly[Ordering[K]]
+
// An array of upper bounds for the first (partitions - 1) partitions
private val rangeBounds: Array[K] = {
if (partitions == 1) {
@@ -103,7 +105,7 @@ class RangePartitioner[K <% Ordered[K]: ClassTag, V](
val rddSize = rdd.count()
val maxSampleSize = partitions * 20.0
val frac = math.min(maxSampleSize / math.max(rddSize, 1), 1.0)
- val rddSample = rdd.sample(false, frac, 1).map(_._1).collect().sortWith(_ < _)
+ val rddSample = rdd.sample(false, frac, 1).map(_._1).collect().sorted
if (rddSample.length == 0) {
Array()
} else {
@@ -126,7 +128,7 @@ class RangePartitioner[K <% Ordered[K]: ClassTag, V](
var partition = 0
if (rangeBounds.length < 1000) {
// If we have less than 100 partitions naive search
- while (partition < rangeBounds.length && k > rangeBounds(partition)) {
+ while (partition < rangeBounds.length && ordering.gt(k, rangeBounds(partition))) {
partition += 1
}
} else {
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index ee5637371f..d3ef75bc73 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -1273,7 +1273,7 @@ object SparkContext extends Logging {
rdd: RDD[(K, V)]) =
new SequenceFileRDDFunctions(rdd)
- implicit def rddToOrderedRDDFunctions[K <% Ordered[K]: ClassTag, V: ClassTag](
+ implicit def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag](
rdd: RDD[(K, V)]) =
new OrderedRDDFunctions[K, V, (K, V)](rdd)
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
index e5b2c8a5e7..b3ec270281 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
@@ -626,10 +626,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])
* order of the keys).
*/
def sortByKey(comp: Comparator[K], ascending: Boolean): JavaPairRDD[K, V] = {
- class KeyOrdering(val a: K) extends Ordered[K] {
- override def compare(b: K) = comp.compare(a, b)
- }
- implicit def toOrdered(x: K): Ordered[K] = new KeyOrdering(x)
+ implicit val ordering = comp // Allow implicit conversion of Comparator to Ordering.
fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending))
}
@@ -640,10 +637,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])
* order of the keys).
*/
def sortByKey(comp: Comparator[K], ascending: Boolean, numPartitions: Int): JavaPairRDD[K, V] = {
- class KeyOrdering(val a: K) extends Ordered[K] {
- override def compare(b: K) = comp.compare(a, b)
- }
- implicit def toOrdered(x: K): Ordered[K] = new KeyOrdering(x)
+ implicit val ordering = comp // Allow implicit conversion of Comparator to Ordering.
fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending, numPartitions))
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
index d5691f2267..6a3f698444 100644
--- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
@@ -24,15 +24,31 @@ import org.apache.spark.{Logging, RangePartitioner}
/**
* Extra functions available on RDDs of (key, value) pairs where the key is sortable through
* an implicit conversion. Import `org.apache.spark.SparkContext._` at the top of your program to
- * use these functions. They will work with any key type that has a `scala.math.Ordered`
- * implementation.
+ * use these functions. They will work with any key type `K` that has an implicit `Ordering[K]` in
+ * scope. Ordering objects already exist for all of the standard primitive types. Users can also
+ * define their own orderings for custom types, or to override the default ordering. The implicit
+ * ordering that is in the closest scope will be used.
+ *
+ * {{{
+ * import org.apache.spark.SparkContext._
+ *
+ * val rdd: RDD[(String, Int)] = ...
+ * implicit val caseInsensitiveOrdering = new Ordering[String] {
+ * override def compare(a: String, b: String) = a.toLowerCase.compare(b.toLowerCase)
+ * }
+ *
+ * // Sort by key, using the above case insensitive ordering.
+ * rdd.sortByKey()
+ * }}}
*/
-class OrderedRDDFunctions[K <% Ordered[K]: ClassTag,
+class OrderedRDDFunctions[K : Ordering : ClassTag,
V: ClassTag,
P <: Product2[K, V] : ClassTag](
self: RDD[P])
extends Logging with Serializable {
+ private val ordering = implicitly[Ordering[K]]
+
/**
* Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling
* `collect` or `save` on the resulting RDD will return or output an ordered list of records
@@ -45,9 +61,9 @@ class OrderedRDDFunctions[K <% Ordered[K]: ClassTag,
shuffled.mapPartitions(iter => {
val buf = iter.toArray
if (ascending) {
- buf.sortWith((x, y) => x._1 < y._1).iterator
+ buf.sortWith((x, y) => ordering.lt(x._1, y._1)).iterator
} else {
- buf.sortWith((x, y) => x._1 > y._1).iterator
+ buf.sortWith((x, y) => ordering.gt(x._1, y._1)).iterator
}
}, preservesPartitioning = true)
}
diff --git a/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala b/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala
index 93235031f3..e4c254b9dd 100644
--- a/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala
+++ b/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala
@@ -23,7 +23,7 @@ import scala.Array
import scala.reflect._
private[spark] object CollectionsUtils {
- def makeBinarySearch[K <% Ordered[K] : ClassTag] : (Array[K], K) => Int = {
+ def makeBinarySearch[K : Ordering : ClassTag] : (Array[K], K) => Int = {
classTag[K] match {
case ClassTag.Float =>
(l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[Float]], x.asInstanceOf[Float])