You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Zoltán Zvara (JIRA)" <ji...@apache.org> on 2015/10/05 15:40:26 UTC

[jira] [Commented] (SPARK-10797) RDD's coalesce should not write out the temporary key

    [ https://issues.apache.org/jira/browse/SPARK-10797?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14943366#comment-14943366 ] 

Zoltán Zvara commented on SPARK-10797:
--------------------------------------

I think we have the following options to solve this problem:

# [General, but too complex] Create a new ShuffledRDD, add support to shuffle system to read values instead of key-value pairs.
# [General, too complex] Create a new CoalescedRDD, that bypasses ShuffledRDD and reads from shuffle-layer directly - still need to add support to read values.
# [Minimal impact on code] Add option for shuffle-layer to write out values only and cast the key-value iterator at read-site to {{Iterator[T]}}.

I've implemented and I'm using the 3rd option. You will experience speed-up based on the size of current payload (values).

> RDD's coalesce should not write out the temporary key
> -----------------------------------------------------
>
>                 Key: SPARK-10797
>                 URL: https://issues.apache.org/jira/browse/SPARK-10797
>             Project: Spark
>          Issue Type: Improvement
>          Components: Spark Core
>            Reporter: Zoltán Zvara
>
> It seems that {{RDD.coalesce}} will unnecessarily write out (to shuffle files) temporary keys used on the shuffle code path. Consider the following code:
> {code:title=RDD.scala|borderStyle=solid}
> if (shuffle) {
>       /** Distributes elements evenly across output partitions, starting from a random partition. */
>       val distributePartition = (index: Int, items: Iterator[T]) => {
>         var position = (new Random(index)).nextInt(numPartitions)
>         items.map { t =>
>           // Note that the hash code of the key will just be the key itself. The HashPartitioner
>           // will mod it with the number of total partitions.
>           position = position + 1
>           (position, t)
>         }
>       } : Iterator[(Int, T)]
>       // include a shuffle step so that our upstream tasks are still distributed
>       new CoalescedRDD(
>         new ShuffledRDD[Int, T, T](mapPartitionsWithIndex(distributePartition),
>         new HashPartitioner(numPartitions)),
>         numPartitions).values
>     } else {
> {code}
> {{ShuffledRDD}} will hash using {{position}} as keys as in the {{distributePartition}} function. After the bucket has been chosen by the sorter {{ExternalSorter}} or {{BypassMergeSortShuffleWriter}}, the {{DiskBlockObjectWriter}} writes out both the (temporary) key and value to the specified partition. On the next stage, after reading we take only the values with {{PairRDDFunctions}}.
> This certainly has a performance impact, as we unnecessarily write/read {{Int}} and transform the data.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org