diff options
author | Chen Chao <crazyjvm@gmail.com> | 2014-04-16 17:58:42 -0700 |
---|---|---|
committer | Reynold Xin <rxin@apache.org> | 2014-04-16 17:58:42 -0700 |
commit | 9c40b9ead0d17ad836b3507c701198645c33d878 (patch) | |
tree | e1ead9ad23fe2bdfb15a45780ee70bd5ecc34860 /docs | |
parent | 38877ccf394a50bfd37c8433d4aafaa91683d3b8 (diff) | |
download | spark-9c40b9ead0d17ad836b3507c701198645c33d878.tar.gz spark-9c40b9ead0d17ad836b3507c701198645c33d878.tar.bz2 spark-9c40b9ead0d17ad836b3507c701198645c33d878.zip |
misleading task number of groupByKey
"By default, this uses only 8 parallel tasks to do the grouping." is a big misleading. Please refer to https://github.com/apache/spark/pull/389
detail is as following code :
def defaultPartitioner(rdd: RDD[_], others: RDD[_]*): Partitioner = {
val bySize = (Seq(rdd) ++ others).sortBy(_.partitions.size).reverse
for (r <- bySize if r.partitioner.isDefined) {
return r.partitioner.get
}
if (rdd.context.conf.contains("spark.default.parallelism")) {
new HashPartitioner(rdd.context.defaultParallelism)
} else {
new HashPartitioner(bySize.head.partitions.size)
}
}
Author: Chen Chao <crazyjvm@gmail.com>
Closes #403 from CrazyJvm/patch-4 and squashes the following commits:
42f6c9e [Chen Chao] fix format
829a995 [Chen Chao] fix format
1568336 [Chen Chao] misleading task number of groupByKey
Diffstat (limited to 'docs')
-rw-r--r-- | docs/scala-programming-guide.md | 4 |
1 files changed, 2 insertions, 2 deletions
diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index a07cd2e0a3..2b0a51e9df 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -189,8 +189,8 @@ The following tables list the transformations and actions currently supported (s <tr> <td> <b>groupByKey</b>([<i>numTasks</i>]) </td> <td> When called on a dataset of (K, V) pairs, returns a dataset of (K, Seq[V]) pairs. <br /> -<b>Note:</b> By default, this uses only 8 parallel tasks to do the grouping. You can pass an optional <code>numTasks</code> argument to set a different number of tasks. -</td> +<b>Note:</b> By default, if the RDD already has a partitioner, the task number is decided by the partition number of the partitioner, or else relies on the value of <code>spark.default.parallelism</code> if the property is set , otherwise depends on the partition number of the RDD. You can pass an optional <code>numTasks</code> argument to set a different number of tasks. + </td> </tr> <tr> <td> <b>reduceByKey</b>(<i>func</i>, [<i>numTasks</i>]) </td> |