Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8829#discussion_r42435124
  
    --- Diff: 
core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala ---
    @@ -52,38 +113,113 @@ private[spark] class SortShuffleManager(conf: 
SparkConf) extends ShuffleManager
           startPartition: Int,
           endPartition: Int,
           context: TaskContext): ShuffleReader[K, C] = {
    -    // We currently use the same block store shuffle fetcher as the 
hash-based shuffle.
         new BlockStoreShuffleReader(
           handle.asInstanceOf[BaseShuffleHandle[K, _, C]], startPartition, 
endPartition, context)
       }
     
       /** Get a writer for a given partition. Called on executors by map 
tasks. */
    -  override def getWriter[K, V](handle: ShuffleHandle, mapId: Int, context: 
TaskContext)
    -      : ShuffleWriter[K, V] = {
    -    val baseShuffleHandle = handle.asInstanceOf[BaseShuffleHandle[K, V, _]]
    -    shuffleMapNumber.putIfAbsent(baseShuffleHandle.shuffleId, 
baseShuffleHandle.numMaps)
    -    new SortShuffleWriter(
    -      shuffleBlockResolver, baseShuffleHandle, mapId, context)
    +  override def getWriter[K, V](
    +      handle: ShuffleHandle,
    +      mapId: Int,
    +      context: TaskContext): ShuffleWriter[K, V] = {
    +    numMapsForShuffle.putIfAbsent(
    +      handle.shuffleId, handle.asInstanceOf[BaseShuffleHandle[_, _, 
_]].numMaps)
    +    val env = SparkEnv.get
    +    handle match {
    +      case unsafeShuffleHandle: SerializedShuffleHandle[K @unchecked, V 
@unchecked] =>
    +        new UnsafeShuffleWriter(
    +          env.blockManager,
    +          shuffleBlockResolver.asInstanceOf[IndexShuffleBlockResolver],
    +          context.taskMemoryManager(),
    +          env.shuffleMemoryManager,
    +          unsafeShuffleHandle,
    +          mapId,
    +          context,
    +          env.conf)
    +      case bypassMergeSortHandle: BypassMergeSortShuffleHandle[K 
@unchecked, V @unchecked] =>
    +        new BypassMergeSortShuffleWriter(
    +          env.blockManager,
    +          shuffleBlockResolver.asInstanceOf[IndexShuffleBlockResolver],
    +          bypassMergeSortHandle,
    +          mapId,
    +          context,
    +          env.conf)
    +      case other: BaseShuffleHandle[K @unchecked, V @unchecked, _] =>
    +        new SortShuffleWriter(shuffleBlockResolver, other, mapId, context)
    +    }
       }
     
       /** Remove a shuffle's metadata from the ShuffleManager. */
       override def unregisterShuffle(shuffleId: Int): Boolean = {
    -    if (shuffleMapNumber.containsKey(shuffleId)) {
    -      val numMaps = shuffleMapNumber.remove(shuffleId)
    -      (0 until numMaps).map{ mapId =>
    +    Option(numMapsForShuffle.remove(shuffleId)).foreach { numMaps =>
    +      (0 until numMaps).foreach { mapId =>
             shuffleBlockResolver.removeDataByMap(shuffleId, mapId)
           }
         }
         true
       }
     
    -  override val shuffleBlockResolver: IndexShuffleBlockResolver = {
    -    indexShuffleBlockResolver
    -  }
    -
       /** Shut down this ShuffleManager. */
       override def stop(): Unit = {
         shuffleBlockResolver.stop()
       }
     }
     
    +
    +private[spark] object SortShuffleManager extends Logging {
    +
    +  /**
    +   * The maximum number of shuffle output partitions that 
SortShuffleManager supports when
    +   *
    --- End diff --
    
    this comment is incomplete? Also might be good to say this is an extreme 
defensive measure since it's extremely unlikely to be hit


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to