-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-9043] Serialize key, value and combiner classes in ShuffleDependency #7403
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
ed1afac
eccb0ed
2906e74
adcdfaf
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -57,7 +57,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) | |
| with SparkHadoopMapReduceUtil | ||
| with Serializable | ||
| { | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * Generic function to combine the elements for each key using a custom set of aggregation | ||
| * functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C | ||
| * Note that V and C can be different -- for example, one might group an RDD of type | ||
|
|
@@ -70,12 +72,14 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) | |
| * In addition, users can control the partitioning of the output RDD, and whether to perform | ||
| * map-side aggregation (if a mapper can produce multiple items with the same key). | ||
| */ | ||
| def combineByKey[C](createCombiner: V => C, | ||
| @Experimental | ||
| def combineByKeyWithClassTag[C]( | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. actually, why call this something else? Does it not compile if you just called this
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Because If you know as way of doing this more cleanly, I would be happy to make that change. |
||
| createCombiner: V => C, | ||
| mergeValue: (C, V) => C, | ||
| mergeCombiners: (C, C) => C, | ||
| partitioner: Partitioner, | ||
| mapSideCombine: Boolean = true, | ||
| serializer: Serializer = null): RDD[(K, C)] = self.withScope { | ||
| serializer: Serializer = null)(implicit ct: ClassTag[C]): RDD[(K, C)] = self.withScope { | ||
| require(mergeCombiners != null, "mergeCombiners must be defined") // required as of Spark 0.9.0 | ||
| if (keyClass.isArray) { | ||
| if (mapSideCombine) { | ||
|
|
@@ -103,13 +107,50 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) | |
| } | ||
|
|
||
| /** | ||
| * Simplified version of combineByKey that hash-partitions the output RDD. | ||
| * Generic function to combine the elements for each key using a custom set of aggregation | ||
| * functions. This method is here for backward compatibility. It does not provide combiner | ||
| * classtag information to the shuffle. | ||
| * | ||
| * @see [[combineByKeyWithClassTag]] | ||
| */ | ||
| def combineByKey[C](createCombiner: V => C, | ||
| def combineByKey[C]( | ||
| createCombiner: V => C, | ||
| mergeValue: (C, V) => C, | ||
| mergeCombiners: (C, C) => C, | ||
| partitioner: Partitioner, | ||
| mapSideCombine: Boolean = true, | ||
| serializer: Serializer = null): RDD[(K, C)] = self.withScope { | ||
| combineByKeyWithClassTag(createCombiner, mergeValue, mergeCombiners, | ||
| partitioner, mapSideCombine, serializer)(null) | ||
| } | ||
|
|
||
| /** | ||
| * Simplified version of combineByKeyWithClassTag that hash-partitions the output RDD. | ||
| * This method is here for backward compatibility. It does not provide combiner | ||
| * classtag information to the shuffle. | ||
| * | ||
| * @see [[combineByKeyWithClassTag]] | ||
| */ | ||
| def combineByKey[C]( | ||
| createCombiner: V => C, | ||
| mergeValue: (C, V) => C, | ||
| mergeCombiners: (C, C) => C, | ||
| numPartitions: Int): RDD[(K, C)] = self.withScope { | ||
| combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numPartitions)) | ||
| combineByKeyWithClassTag(createCombiner, mergeValue, mergeCombiners, numPartitions)(null) | ||
| } | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * Simplified version of combineByKeyWithClassTag that hash-partitions the output RDD. | ||
| */ | ||
| @Experimental | ||
| def combineByKeyWithClassTag[C]( | ||
| createCombiner: V => C, | ||
| mergeValue: (C, V) => C, | ||
| mergeCombiners: (C, C) => C, | ||
| numPartitions: Int)(implicit ct: ClassTag[C]): RDD[(K, C)] = self.withScope { | ||
| combineByKeyWithClassTag(createCombiner, mergeValue, mergeCombiners, | ||
| new HashPartitioner(numPartitions)) | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -133,7 +174,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) | |
|
|
||
| // We will clean the combiner closure later in `combineByKey` | ||
| val cleanedSeqOp = self.context.clean(seqOp) | ||
| combineByKey[U]((v: V) => cleanedSeqOp(createZero(), v), cleanedSeqOp, combOp, partitioner) | ||
| combineByKeyWithClassTag[U]((v: V) => cleanedSeqOp(createZero(), v), | ||
| cleanedSeqOp, combOp, partitioner) | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -182,7 +224,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) | |
| val createZero = () => cachedSerializer.deserialize[V](ByteBuffer.wrap(zeroArray)) | ||
|
|
||
| val cleanedFunc = self.context.clean(func) | ||
| combineByKey[V]((v: V) => cleanedFunc(createZero(), v), cleanedFunc, cleanedFunc, partitioner) | ||
| combineByKeyWithClassTag[V]((v: V) => cleanedFunc(createZero(), v), | ||
| cleanedFunc, cleanedFunc, partitioner) | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -268,7 +311,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) | |
| * "combiner" in MapReduce. | ||
| */ | ||
| def reduceByKey(partitioner: Partitioner, func: (V, V) => V): RDD[(K, V)] = self.withScope { | ||
| combineByKey[V]((v: V) => v, func, func, partitioner) | ||
| combineByKeyWithClassTag[V]((v: V) => v, func, func, partitioner) | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -392,7 +435,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) | |
| h1 | ||
| } | ||
|
|
||
| combineByKey(createHLL, mergeValueHLL, mergeHLL, partitioner).mapValues(_.cardinality()) | ||
| combineByKeyWithClassTag(createHLL, mergeValueHLL, mergeHLL, partitioner) | ||
| .mapValues(_.cardinality()) | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -466,7 +510,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) | |
| val createCombiner = (v: V) => CompactBuffer(v) | ||
| val mergeValue = (buf: CompactBuffer[V], v: V) => buf += v | ||
| val mergeCombiners = (c1: CompactBuffer[V], c2: CompactBuffer[V]) => c1 ++= c2 | ||
| val bufs = combineByKey[CompactBuffer[V]]( | ||
| val bufs = combineByKeyWithClassTag[CompactBuffer[V]]( | ||
| createCombiner, mergeValue, mergeCombiners, partitioner, mapSideCombine = false) | ||
| bufs.asInstanceOf[RDD[(K, Iterable[V])]] | ||
| } | ||
|
|
@@ -565,12 +609,30 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) | |
| } | ||
|
|
||
| /** | ||
| * Simplified version of combineByKey that hash-partitions the resulting RDD using the | ||
| * Simplified version of combineByKeyWithClassTag that hash-partitions the resulting RDD using the | ||
| * existing partitioner/parallelism level. This method is here for backward compatibility. It | ||
| * does not provide combiner classtag information to the shuffle. | ||
| * | ||
| * @see [[combineByKeyWithClassTag]] | ||
| */ | ||
| def combineByKey[C]( | ||
| createCombiner: V => C, | ||
| mergeValue: (C, V) => C, | ||
| mergeCombiners: (C, C) => C): RDD[(K, C)] = self.withScope { | ||
| combineByKeyWithClassTag(createCombiner, mergeValue, mergeCombiners)(null) | ||
| } | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * Simplified version of combineByKeyWithClassTag that hash-partitions the resulting RDD using the | ||
| * existing partitioner/parallelism level. | ||
| */ | ||
| def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) | ||
| : RDD[(K, C)] = self.withScope { | ||
| combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(self)) | ||
| @Experimental | ||
| def combineByKeyWithClassTag[C]( | ||
| createCombiner: V => C, | ||
| mergeValue: (C, V) => C, | ||
| mergeCombiners: (C, C) => C)(implicit ct: ClassTag[C]): RDD[(K, C)] = self.withScope { | ||
| combineByKeyWithClassTag(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(self)) | ||
| } | ||
|
|
||
| /** | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,67 @@ | ||
| /* | ||
| * 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.shuffle | ||
|
|
||
| import org.apache.spark._ | ||
|
|
||
| case class KeyClass() | ||
|
|
||
| case class ValueClass() | ||
|
|
||
| case class CombinerClass() | ||
|
|
||
| class ShuffleDependencySuite extends SparkFunSuite with LocalSparkContext { | ||
|
|
||
| val conf = new SparkConf(loadDefaults = false) | ||
|
|
||
| test("key, value, and combiner classes correct in shuffle dependency without aggregation") { | ||
| sc = new SparkContext("local", "test", conf.clone()) | ||
| val rdd = sc.parallelize(1 to 5, 4) | ||
| .map(key => (KeyClass(), ValueClass())) | ||
| .groupByKey() | ||
| val dep = rdd.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]] | ||
| assert(!dep.mapSideCombine, "Test requires that no map-side aggregator is defined") | ||
| assert(dep.keyClassName == classOf[KeyClass].getName) | ||
| assert(dep.valueClassName == classOf[ValueClass].getName) | ||
| } | ||
|
|
||
| test("key, value, and combiner classes available in shuffle dependency with aggregation") { | ||
| sc = new SparkContext("local", "test", conf.clone()) | ||
| val rdd = sc.parallelize(1 to 5, 4) | ||
| .map(key => (KeyClass(), ValueClass())) | ||
| .aggregateByKey(CombinerClass())({ case (a, b) => a }, { case (a, b) => a }) | ||
| val dep = rdd.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]] | ||
| assert(dep.mapSideCombine && dep.aggregator.isDefined, "Test requires map-side aggregation") | ||
| assert(dep.keyClassName == classOf[KeyClass].getName) | ||
| assert(dep.valueClassName == classOf[ValueClass].getName) | ||
| assert(dep.combinerClassName == Some(classOf[CombinerClass].getName)) | ||
| } | ||
|
|
||
| test("combineByKey null combiner class tag handled correctly") { | ||
| sc = new SparkContext("local", "test", conf.clone()) | ||
| val rdd = sc.parallelize(1 to 5, 4) | ||
| .map(key => (KeyClass(), ValueClass())) | ||
| .combineByKey((v: ValueClass) => v, | ||
| (c: AnyRef, v: ValueClass) => c, | ||
| (c1: AnyRef, c2: AnyRef) => c1) | ||
| val dep = rdd.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]] | ||
| assert(dep.keyClassName == classOf[KeyClass].getName) | ||
| assert(dep.valueClassName == classOf[ValueClass].getName) | ||
| assert(dep.combinerClassName == None) | ||
| } | ||
|
|
||
| } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
FYI, this is a binary-incompatible change to a DeveloperAPI.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
According to the source for DeveloperApi, a Developer API is unstable and can change between minor releases.