Skip to content

Commit eb4ee0d

Browse files
committed
Remove customizable element type in ShuffledRDD
This feature allowed for more efficient serialization in some cases, but made it less efficient in the more common case of (K, V) because it required the data structures in shuffle to hold Tuple2 objects instead of unpacking them into parallel arrays. Moreover, it was broken for ShuffledRDDs with aggregation, because there wasn't any way to construct the user's Product2 type so we would return Tuple2. It seemed better to disable it. One unfortunate effect is that OrderedRDDFunctions used to have a public API that let it work on RDD[P <: Product2] and return RDD[P] from sortByKey. This can't work now, so I changed it to return RDD[(K, V)]. In practice anyone using OrderedRDDFunctions through the implicit conversions would get RDD[(K, V)] anyway, so it only mattered for people who somehow tried to create it directly. I've made OrderedRDDFunctions's constructor a @DeveloperAPI to discourage this. It will be possible to add back custom data types later, but we'll need a better interface that gives ShuffledRDD and such a mechanism to construct them and to manage buffers of them. It might also be possible to come up with an interface for "key-less" shuffles, where you shuffle an RDD[T] where each element is its own key. This would be a much better fit for sorting and some of the GraphX use cases.
1 parent fa2e8db commit eb4ee0d

File tree

13 files changed

+51
-48
lines changed

13 files changed

+51
-48
lines changed

core/src/main/scala/org/apache/spark/SparkContext.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -289,7 +289,7 @@ class SparkContext(config: SparkConf) extends Logging {
289289
value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} {
290290
executorEnvs(envKey) = value
291291
}
292-
Option(System.getenv("SPARK_PREPEND_CLASSES")).foreach { v =>
292+
Option(System.getenv("SPARK_PREPEND_CLASSES")).foreach { v =>
293293
executorEnvs("SPARK_PREPEND_CLASSES") = v
294294
}
295295
// The Mesos scheduler backend relies on this environment variable to set executor memory.
@@ -1203,10 +1203,10 @@ class SparkContext(config: SparkConf) extends Logging {
12031203
/**
12041204
* Clean a closure to make it ready to serialized and send to tasks
12051205
* (removes unreferenced variables in $outer's, updates REPL variables)
1206-
* If <tt>checkSerializable</tt> is set, <tt>clean</tt> will also proactively
1207-
* check to see if <tt>f</tt> is serializable and throw a <tt>SparkException</tt>
1206+
* If <tt>checkSerializable</tt> is set, <tt>clean</tt> will also proactively
1207+
* check to see if <tt>f</tt> is serializable and throw a <tt>SparkException</tt>
12081208
* if not.
1209-
*
1209+
*
12101210
* @param f the closure to clean
12111211
* @param checkSerializable whether or not to immediately check <tt>f</tt> for serializability
12121212
* @throws <tt>SparkException<tt> if <tt>checkSerializable</tt> is set but <tt>f</tt> is not

core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -122,7 +122,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])
122122
*/
123123
def sample(withReplacement: Boolean, fraction: Double): JavaPairRDD[K, V] =
124124
sample(withReplacement, fraction, Utils.random.nextLong)
125-
125+
126126
/**
127127
* Return a sampled subset of this RDD.
128128
*/

core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala

Lines changed: 9 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ package org.apache.spark.rdd
2020
import scala.reflect.ClassTag
2121

2222
import org.apache.spark.{Logging, RangePartitioner}
23+
import org.apache.spark.annotation.DeveloperApi
2324

2425
/**
2526
* Extra functions available on RDDs of (key, value) pairs where the key is sortable through
@@ -43,10 +44,10 @@ import org.apache.spark.{Logging, RangePartitioner}
4344
*/
4445
class OrderedRDDFunctions[K : Ordering : ClassTag,
4546
V: ClassTag,
46-
P <: Product2[K, V] : ClassTag](
47+
P <: Product2[K, V] : ClassTag] @DeveloperApi() (
4748
self: RDD[P])
48-
extends Logging with Serializable {
49-
49+
extends Logging with Serializable
50+
{
5051
private val ordering = implicitly[Ordering[K]]
5152

5253
/**
@@ -55,9 +56,12 @@ class OrderedRDDFunctions[K : Ordering : ClassTag,
5556
* (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in
5657
* order of the keys).
5758
*/
58-
def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.size): RDD[P] = {
59+
// TODO: this currently doesn't work on P other than Tuple2!
60+
def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.size)
61+
: RDD[(K, V)] =
62+
{
5963
val part = new RangePartitioner(numPartitions, self, ascending)
60-
new ShuffledRDD[K, V, V, P](self, part)
64+
new ShuffledRDD[K, V, V](self, part)
6165
.setKeyOrdering(if (ascending) ordering else ordering.reverse)
6266
}
6367
}

core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -90,7 +90,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
9090
new InterruptibleIterator(context, aggregator.combineValuesByKey(iter, context))
9191
}, preservesPartitioning = true)
9292
} else {
93-
new ShuffledRDD[K, V, C, (K, C)](self, partitioner)
93+
new ShuffledRDD[K, V, C](self, partitioner)
9494
.setSerializer(serializer)
9595
.setAggregator(aggregator)
9696
.setMapSideCombine(mapSideCombine)
@@ -425,7 +425,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
425425
if (self.partitioner == Some(partitioner)) {
426426
self
427427
} else {
428-
new ShuffledRDD[K, V, V, (K, V)](self, partitioner)
428+
new ShuffledRDD[K, V, V](self, partitioner)
429429
}
430430
}
431431

core/src/main/scala/org/apache/spark/rdd/RDD.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -332,7 +332,7 @@ abstract class RDD[T: ClassTag](
332332
val distributePartition = (index: Int, items: Iterator[T]) => {
333333
var position = (new Random(index)).nextInt(numPartitions)
334334
items.map { t =>
335-
// Note that the hash code of the key will just be the key itself. The HashPartitioner
335+
// Note that the hash code of the key will just be the key itself. The HashPartitioner
336336
// will mod it with the number of total partitions.
337337
position = position + 1
338338
(position, t)
@@ -341,7 +341,7 @@ abstract class RDD[T: ClassTag](
341341

342342
// include a shuffle step so that our upstream tasks are still distributed
343343
new CoalescedRDD(
344-
new ShuffledRDD[Int, T, T, (Int, T)](mapPartitionsWithIndex(distributePartition),
344+
new ShuffledRDD[Int, T, T](mapPartitionsWithIndex(distributePartition),
345345
new HashPartitioner(numPartitions)),
346346
numPartitions).values
347347
} else {
@@ -352,8 +352,8 @@ abstract class RDD[T: ClassTag](
352352
/**
353353
* Return a sampled subset of this RDD.
354354
*/
355-
def sample(withReplacement: Boolean,
356-
fraction: Double,
355+
def sample(withReplacement: Boolean,
356+
fraction: Double,
357357
seed: Long = Utils.random.nextLong): RDD[T] = {
358358
require(fraction >= 0.0, "Negative fraction value: " + fraction)
359359
if (withReplacement) {

core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala

Lines changed: 9 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -37,11 +37,12 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition {
3737
* @tparam V the value class.
3838
* @tparam C the combiner class.
3939
*/
40+
// TODO: Make this return RDD[Product2[K, C]] or have some way to configure mutable pairs
4041
@DeveloperApi
41-
class ShuffledRDD[K, V, C, P <: Product2[K, C] : ClassTag](
42+
class ShuffledRDD[K, V, C](
4243
@transient var prev: RDD[_ <: Product2[K, V]],
4344
part: Partitioner)
44-
extends RDD[P](prev.context, Nil) {
45+
extends RDD[(K, C)](prev.context, Nil) {
4546

4647
private var serializer: Option[Serializer] = None
4748

@@ -52,25 +53,25 @@ class ShuffledRDD[K, V, C, P <: Product2[K, C] : ClassTag](
5253
private var mapSideCombine: Boolean = false
5354

5455
/** Set a serializer for this RDD's shuffle, or null to use the default (spark.serializer) */
55-
def setSerializer(serializer: Serializer): ShuffledRDD[K, V, C, P] = {
56+
def setSerializer(serializer: Serializer): ShuffledRDD[K, V, C] = {
5657
this.serializer = Option(serializer)
5758
this
5859
}
5960

6061
/** Set key ordering for RDD's shuffle. */
61-
def setKeyOrdering(keyOrdering: Ordering[K]): ShuffledRDD[K, V, C, P] = {
62+
def setKeyOrdering(keyOrdering: Ordering[K]): ShuffledRDD[K, V, C] = {
6263
this.keyOrdering = Option(keyOrdering)
6364
this
6465
}
6566

6667
/** Set aggregator for RDD's shuffle. */
67-
def setAggregator(aggregator: Aggregator[K, V, C]): ShuffledRDD[K, V, C, P] = {
68+
def setAggregator(aggregator: Aggregator[K, V, C]): ShuffledRDD[K, V, C] = {
6869
this.aggregator = Option(aggregator)
6970
this
7071
}
7172

7273
/** Set mapSideCombine flag for RDD's shuffle. */
73-
def setMapSideCombine(mapSideCombine: Boolean): ShuffledRDD[K, V, C, P] = {
74+
def setMapSideCombine(mapSideCombine: Boolean): ShuffledRDD[K, V, C] = {
7475
this.mapSideCombine = mapSideCombine
7576
this
7677
}
@@ -85,11 +86,11 @@ class ShuffledRDD[K, V, C, P <: Product2[K, C] : ClassTag](
8586
Array.tabulate[Partition](part.numPartitions)(i => new ShuffledRDDPartition(i))
8687
}
8788

88-
override def compute(split: Partition, context: TaskContext): Iterator[P] = {
89+
override def compute(split: Partition, context: TaskContext): Iterator[(K, C)] = {
8990
val dep = dependencies.head.asInstanceOf[ShuffleDependency[K, V, C]]
9091
SparkEnv.get.shuffleManager.getReader(dep.shuffleHandle, split.index, split.index + 1, context)
9192
.read()
92-
.asInstanceOf[Iterator[P]]
93+
.asInstanceOf[Iterator[(K, C)]]
9394
}
9495

9596
override def clearDependencies() {

core/src/test/scala/org/apache/spark/CheckpointSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -99,7 +99,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
9999
test("ShuffledRDD") {
100100
testRDD(rdd => {
101101
// Creating ShuffledRDD directly as PairRDDFunctions.combineByKey produces a MapPartitionedRDD
102-
new ShuffledRDD[Int, Int, Int, (Int, Int)](rdd.map(x => (x % 2, 1)), partitioner)
102+
new ShuffledRDD[Int, Int, Int](rdd.map(x => (x % 2, 1)), partitioner)
103103
})
104104
}
105105

core/src/test/scala/org/apache/spark/ShuffleSuite.scala

Lines changed: 12 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -58,8 +58,7 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext {
5858
// default Java serializer cannot handle the non serializable class.
5959
val c = new ShuffledRDD[Int,
6060
NonJavaSerializableClass,
61-
NonJavaSerializableClass,
62-
(Int, NonJavaSerializableClass)](b, new HashPartitioner(NUM_BLOCKS))
61+
NonJavaSerializableClass](b, new HashPartitioner(NUM_BLOCKS))
6362
c.setSerializer(new KryoSerializer(conf))
6463
val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId
6564

@@ -83,8 +82,7 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext {
8382
// default Java serializer cannot handle the non serializable class.
8483
val c = new ShuffledRDD[Int,
8584
NonJavaSerializableClass,
86-
NonJavaSerializableClass,
87-
(Int, NonJavaSerializableClass)](b, new HashPartitioner(3))
85+
NonJavaSerializableClass](b, new HashPartitioner(3))
8886
c.setSerializer(new KryoSerializer(conf))
8987
assert(c.count === 10)
9088
}
@@ -100,7 +98,7 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext {
10098

10199
// NOTE: The default Java serializer doesn't create zero-sized blocks.
102100
// So, use Kryo
103-
val c = new ShuffledRDD[Int, Int, Int, (Int, Int)](b, new HashPartitioner(10))
101+
val c = new ShuffledRDD[Int, Int, Int](b, new HashPartitioner(10))
104102
.setSerializer(new KryoSerializer(conf))
105103

106104
val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId
@@ -126,7 +124,7 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext {
126124
val b = a.map(x => (x, x*2))
127125

128126
// NOTE: The default Java serializer should create zero-sized blocks
129-
val c = new ShuffledRDD[Int, Int, Int, (Int, Int)](b, new HashPartitioner(10))
127+
val c = new ShuffledRDD[Int, Int, Int](b, new HashPartitioner(10))
130128

131129
val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId
132130
assert(c.count === 4)
@@ -141,19 +139,19 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext {
141139
assert(nonEmptyBlocks.size <= 4)
142140
}
143141

144-
test("shuffle using mutable pairs") {
142+
test("shuffle on mutable pairs") {
145143
// Use a local cluster with 2 processes to make sure there are both local and remote blocks
146144
sc = new SparkContext("local-cluster[2,1,512]", "test")
147145
def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2)
148146
val data = Array(p(1, 1), p(1, 2), p(1, 3), p(2, 1))
149147
val pairs: RDD[MutablePair[Int, Int]] = sc.parallelize(data, 2)
150-
val results = new ShuffledRDD[Int, Int, Int, MutablePair[Int, Int]](pairs,
148+
val results = new ShuffledRDD[Int, Int, Int](pairs,
151149
new HashPartitioner(2)).collect()
152150

153-
data.foreach { pair => results should contain (pair) }
151+
data.foreach { pair => results should contain ((pair._1, pair._2)) }
154152
}
155153

156-
test("sorting using mutable pairs") {
154+
test("sorting on mutable pairs") {
157155
// This is not in SortingSuite because of the local cluster setup.
158156
// Use a local cluster with 2 processes to make sure there are both local and remote blocks
159157
sc = new SparkContext("local-cluster[2,1,512]", "test")
@@ -162,10 +160,10 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext {
162160
val pairs: RDD[MutablePair[Int, Int]] = sc.parallelize(data, 2)
163161
val results = new OrderedRDDFunctions[Int, Int, MutablePair[Int, Int]](pairs)
164162
.sortByKey().collect()
165-
results(0) should be (p(1, 11))
166-
results(1) should be (p(2, 22))
167-
results(2) should be (p(3, 33))
168-
results(3) should be (p(100, 100))
163+
results(0) should be ((1, 11))
164+
results(1) should be ((2, 22))
165+
results(2) should be ((3, 33))
166+
results(3) should be ((100, 100))
169167
}
170168

171169
test("cogroup using mutable pairs") {

core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -270,7 +270,7 @@ class RDDSuite extends FunSuite with SharedSparkContext {
270270
// we can optionally shuffle to keep the upstream parallel
271271
val coalesced5 = data.coalesce(1, shuffle = true)
272272
val isEquals = coalesced5.dependencies.head.rdd.dependencies.head.rdd.
273-
asInstanceOf[ShuffledRDD[_, _, _, _]] != null
273+
asInstanceOf[ShuffledRDD[_, _, _]] != null
274274
assert(isEquals)
275275

276276
// when shuffling, we can increase the number of partitions
@@ -730,9 +730,9 @@ class RDDSuite extends FunSuite with SharedSparkContext {
730730

731731
// Any ancestors before the shuffle are not considered
732732
assert(ancestors4.size === 0)
733-
assert(ancestors4.count(_.isInstanceOf[ShuffledRDD[_, _, _, _]]) === 0)
733+
assert(ancestors4.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 0)
734734
assert(ancestors5.size === 3)
735-
assert(ancestors5.count(_.isInstanceOf[ShuffledRDD[_, _, _, _]]) === 1)
735+
assert(ancestors5.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 1)
736736
assert(ancestors5.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 0)
737737
assert(ancestors5.count(_.isInstanceOf[MappedValuesRDD[_, _, _]]) === 2)
738738
}

graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,7 @@ import org.apache.spark.rdd.{ShuffledRDD, RDD}
2828
private[graphx]
2929
class VertexRDDFunctions[VD: ClassTag](self: RDD[(VertexId, VD)]) {
3030
def copartitionWithVertices(partitioner: Partitioner): RDD[(VertexId, VD)] = {
31-
val rdd = new ShuffledRDD[VertexId, VD, VD, (VertexId, VD)](self, partitioner)
31+
val rdd = new ShuffledRDD[VertexId, VD, VD](self, partitioner)
3232

3333
// Set a custom serializer if the data is of int or double type.
3434
if (classTag[VD] == ClassTag.Int) {

0 commit comments

Comments
 (0)