Github user colorant commented on a diff in the pull request:
https://github.com/apache/spark/pull/1499#discussion_r15331341
--- 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 = {}
--- End diff --
shuffle output file in sortShuffleWritter do not get cleaned. We might need
to add map to save registered shuffle handle, Then try to remove the data file
in unregisterShuffle method. Though at present, in HashShuffleManager, this is
also not implemented. But HashShuffleManager depends on shuffleBlockManager and
the file will be cleaned there.
I have a PR to generalize shuffleBlockManager and hide it behind
shuffleMananger. as in #1241 , and upon blockMananger do remove shuffle, will
call into this unregisterShuffle method. Will rebase upon this PR been merged.
Should we fix this issue in my PR, or add the store/clean shuffleHandle
logic here in this PR firstly?
---
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.
---