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

    https://github.com/apache/spark/pull/1499#discussion_r15494109
  
    --- Diff: 
core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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.sort
    +
    +import java.io.{DataInputStream, FileInputStream}
    +
    +import org.apache.spark.shuffle._
    +import org.apache.spark.{TaskContext, ShuffleDependency}
    +import org.apache.spark.shuffle.hash.HashShuffleReader
    +import org.apache.spark.storage.{DiskBlockManager, FileSegment, 
ShuffleBlockId}
    +
    +private[spark] class SortShuffleManager extends ShuffleManager {
    +  /**
    +   * Register a shuffle with the manager and obtain a handle for it to 
pass to tasks.
    +   */
    +  override def registerShuffle[K, V, C](
    +      shuffleId: Int,
    +      numMaps: Int,
    +      dependency: ShuffleDependency[K, V, C]): ShuffleHandle = {
    +    new BaseShuffleHandle(shuffleId, numMaps, dependency)
    +  }
    +
    +  /**
    +   * Get a reader for a range of reduce partitions (startPartition to 
endPartition-1, inclusive).
    +   * Called on executors by reduce tasks.
    +   */
    +  override def getReader[K, C](
    +      handle: ShuffleHandle,
    +      startPartition: Int,
    +      endPartition: Int,
    +      context: TaskContext): ShuffleReader[K, C] = {
    +    // We currently use the same block store shuffle fetcher as the 
hash-based shuffle.
    +    new HashShuffleReader(
    +      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] = {
    +    new SortShuffleWriter(handle.asInstanceOf[BaseShuffleHandle[K, V, _]], 
mapId, context)
    +  }
    +
    +  /** Remove a shuffle's metadata from the ShuffleManager. */
    +  override def unregisterShuffle(shuffleId: Int): Unit = {}
    +
    +  /** Shut down this ShuffleManager. */
    +  override def stop(): Unit = {}
    +
    +  /** Get the location of a block in a map output file. Uses the index 
file we create for it. */
    +  def getBlockLocation(blockId: ShuffleBlockId, diskManager: 
DiskBlockManager): FileSegment = {
    +    // The block is actually going to be a range of a single map output 
file for this map, so
    +    // figure out the ID of the consolidated file, then the offset within 
that from our index
    +    val realId = ShuffleBlockId(blockId.shuffleId, blockId.mapId, 0)
    +    val indexFile = diskManager.getFile(realId.name + ".index")
    +    val in = new DataInputStream(new FileInputStream(indexFile))
    --- End diff --
    
    This index should be pretty small, so I suppose we could keep it in memory 
instead of writing it to a file.  On the other hand, it might be likely to 
remain in the filesystem cache.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to