Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
26 changes: 17 additions & 9 deletions core/src/main/scala/org/apache/spark/MapOutputTracker.scala
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,6 @@ import org.apache.spark.shuffle.MetadataFetchFailedException
import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockId, ShuffleMergedBlockId}
import org.apache.spark.util._
import org.apache.spark.util.ArrayImplicits._
import org.apache.spark.util.collection.OpenHashMap
import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream}

/**
Expand Down Expand Up @@ -153,17 +152,22 @@ private class ShuffleStatus(
/**
* Mapping from a mapId to the mapIndex, this is required to reduce the searching overhead within
* the function updateMapOutput(mapId, bmAddress).
*
* Exposed for testing.
*/
private[this] val mapIdToMapIndex = new OpenHashMap[Long, Int]()
private[spark] val mapIdToMapIndex = new HashMap[Long, Int]()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

QQ: Why change to HashMap from OpenHashMap ? (it is specialized for Long and Int)

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 for the above question.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OpenHashMap doesn't support remove operation.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah yes !


/**
* Register a map output. If there is already a registered location for the map output then it
* will be replaced by the new location.
*/
def addMapOutput(mapIndex: Int, status: MapStatus): Unit = withWriteLock {
if (mapStatuses(mapIndex) == null) {
val currentMapStatus = mapStatuses(mapIndex)
if (currentMapStatus == null) {
_numAvailableMapOutputs += 1
invalidateSerializedMapOutputStatusCache()
} else {
mapIdToMapIndex.remove(currentMapStatus.mapId)
}
mapStatuses(mapIndex) = status
mapIdToMapIndex(status.mapId) = mapIndex
Expand Down Expand Up @@ -193,8 +197,8 @@ private class ShuffleStatus(
mapStatus.updateLocation(bmAddress)
invalidateSerializedMapOutputStatusCache()
case None =>
if (mapIndex.map(mapStatusesDeleted).exists(_.mapId == mapId)) {
val index = mapIndex.get
val index = mapStatusesDeleted.indexWhere(x => x != null && x.mapId == mapId)
if (index >= 0 && mapStatuses(index) == null) {
val mapStatus = mapStatusesDeleted(index)
mapStatus.updateLocation(bmAddress)
mapStatuses(index) = mapStatus
Expand Down Expand Up @@ -222,9 +226,11 @@ private class ShuffleStatus(
*/
def removeMapOutput(mapIndex: Int, bmAddress: BlockManagerId): Unit = withWriteLock {
logDebug(s"Removing existing map output ${mapIndex} ${bmAddress}")
if (mapStatuses(mapIndex) != null && mapStatuses(mapIndex).location == bmAddress) {
val currentMapStatus = mapStatuses(mapIndex)
if (currentMapStatus != null && currentMapStatus.location == bmAddress) {
_numAvailableMapOutputs -= 1
mapStatusesDeleted(mapIndex) = mapStatuses(mapIndex)
mapIdToMapIndex.remove(currentMapStatus.mapId)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removing it here will mean we cant query for it in mapStatusesDeleted, where we are relying on mapId -> mapIndex being in mapIdToMapIndex even when mapIndex is in mapStatusesDeleted

We should move this cleanup to when mapStatusesDeleted is being cleaned up.

Same applies to the cases below as well.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is a good point. But IIUC, mapStatusesDeleted will only be cleanedup when there is recovery happen using K8s. So it's not guaranteed to be always cleaned up in the end. I removed the dependency of mapIdToMapIndex for mapStatusesDeleted as it's not a common use case.

mapStatusesDeleted(mapIndex) = currentMapStatus
mapStatuses(mapIndex) = null
invalidateSerializedMapOutputStatusCache()
}
Expand Down Expand Up @@ -290,9 +296,11 @@ private class ShuffleStatus(
*/
def removeOutputsByFilter(f: BlockManagerId => Boolean): Unit = withWriteLock {
for (mapIndex <- mapStatuses.indices) {
if (mapStatuses(mapIndex) != null && f(mapStatuses(mapIndex).location)) {
val currentMapStatus = mapStatuses(mapIndex)
if (currentMapStatus != null && f(currentMapStatus.location)) {
_numAvailableMapOutputs -= 1
mapStatusesDeleted(mapIndex) = mapStatuses(mapIndex)
mapIdToMapIndex.remove(currentMapStatus.mapId)
mapStatusesDeleted(mapIndex) = currentMapStatus
mapStatuses(mapIndex) = null
invalidateSerializedMapOutputStatusCache()
}
Expand Down
55 changes: 55 additions & 0 deletions core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1110,4 +1110,59 @@ class MapOutputTrackerSuite extends SparkFunSuite with LocalSparkContext {
rpcEnv.shutdown()
}
}

test(
"SPARK-48394: mapIdToMapIndex should cleanup unused mapIndexes after removeOutputsByFilter"
) {
val rpcEnv = createRpcEnv("test")
val tracker = newTrackerMaster()
try {
tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME,
new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf))
tracker.registerShuffle(0, 1, 1)
tracker.registerMapOutput(0, 0, MapStatus(BlockManagerId("exec-1", "hostA", 1000),
Array(2L), 0))
tracker.removeOutputsOnHost("hostA")
assert(tracker.shuffleStatuses(0).mapIdToMapIndex.filter(_._2 == 0).size == 0)
} finally {
tracker.stop()
rpcEnv.shutdown()
}
}

test("SPARK-48394: mapIdToMapIndex should cleanup unused mapIndexes after unregisterMapOutput") {
val rpcEnv = createRpcEnv("test")
val tracker = newTrackerMaster()
try {
tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME,
new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf))
tracker.registerShuffle(0, 1, 1)
tracker.registerMapOutput(0, 0, MapStatus(BlockManagerId("exec-1", "hostA", 1000),
Array(2L), 0))
tracker.unregisterMapOutput(0, 0, BlockManagerId("exec-1", "hostA", 1000))
assert(tracker.shuffleStatuses(0).mapIdToMapIndex.filter(_._2 == 0).size == 0)
} finally {
tracker.stop()
rpcEnv.shutdown()
}
}

test("SPARK-48394: mapIdToMapIndex should cleanup unused mapIndexes after registerMapOutput") {
val rpcEnv = createRpcEnv("test")
val tracker = newTrackerMaster()
try {
tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME,
new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf))
tracker.registerShuffle(0, 1, 1)
tracker.registerMapOutput(0, 0, MapStatus(BlockManagerId("exec-1", "hostA", 1000),
Array(2L), 0))
// Another task also finished working on partition 0.
tracker.registerMapOutput(0, 0, MapStatus(BlockManagerId("exec-2", "hostB", 1000),
Array(2L), 1))
assert(tracker.shuffleStatuses(0).mapIdToMapIndex.filter(_._2 == 0).size == 1)
} finally {
tracker.stop()
rpcEnv.shutdown()
}
}
}