dongjoon-hyun commented on a change in pull request #32730: URL: https://github.com/apache/spark/pull/32730#discussion_r645264042
########## File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/shuffle/KubernetesLocalDiskShuffleExecutorComponents.scala ########## @@ -0,0 +1,105 @@ +/* + * 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 + +import java.io.File +import java.util.Optional + +import scala.reflect.ClassTag + +import org.apache.spark.{SparkConf, SparkEnv} +import org.apache.spark.internal.Logging +import org.apache.spark.shuffle.api.{ShuffleExecutorComponents, ShuffleMapOutputWriter, SingleSpillShuffleMapOutputWriter} +import org.apache.spark.shuffle.sort.io.LocalDiskShuffleExecutorComponents +import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, UnrecognizedBlockId} +import org.apache.spark.util.Utils + +class KubernetesLocalDiskShuffleExecutorComponents(sparkConf: SparkConf) + extends ShuffleExecutorComponents with Logging { + + private val delegate = new LocalDiskShuffleExecutorComponents(sparkConf) + private var blockManager: BlockManager = _ + + override def initializeExecutor( + appId: String, execId: String, extraConfigs: java.util.Map[String, String]): Unit = { + delegate.initializeExecutor(appId, execId, extraConfigs) + blockManager = SparkEnv.get.blockManager + if (sparkConf.getBoolean("spark.kubernetes.driver.reusePersistentVolumeClaim", false)) { + // Turn off the deletion of the shuffle data in order to reuse + blockManager.diskBlockManager.deleteFilesOnStop = false + logError("Recover shuffle data") + Utils.tryLogNonFatalError { + KubernetesLocalDiskShuffleExecutorComponents.recoverDiskStore(sparkConf, blockManager) + } + } + } + + override def createMapOutputWriter(shuffleId: Int, mapTaskId: Long, numPartitions: Int) + : ShuffleMapOutputWriter = { + delegate.createMapOutputWriter(shuffleId, mapTaskId, numPartitions) + } + + override def createSingleFileMapOutputWriter(shuffleId: Int, mapId: Long) + : Optional[SingleSpillShuffleMapOutputWriter] = { + delegate.createSingleFileMapOutputWriter(shuffleId, mapId) + } +} + +object KubernetesLocalDiskShuffleExecutorComponents extends Logging { + /** + * This tries to recover shuffle data of dead executors' local dirs if exists. + * Since the executors are already dead, we cannot use `getHostLocalDirs`. + * This is enabled only when spark.kubernetes.driver.reusePersistentVolumeClaim is true. + */ + def recoverDiskStore(conf: SparkConf, bm: BlockManager): Unit = { + // Find All files + val files = Utils.getConfiguredLocalDirs(conf) + .filter(_ != null) + .map(s => new File(new File(new File(s).getParent).getParent)) + .flatMap { dir => + logError(dir.toString) + val oldDirs = dir.listFiles().filter { f => + f.isDirectory && f.getName.startsWith("spark-") + } + val files = oldDirs + .flatMap(_.listFiles).filter(_.isDirectory) // executor-xxx + .flatMap(_.listFiles).filter(_.isDirectory) // blockmgr-xxx + .flatMap(_.listFiles).filter(_.isDirectory) // 00 + .flatMap(_.listFiles) + if (files != null) files.toSeq else Seq.empty + } + + logInfo(s"Found ${files.size} files") + + // This is not used. + val classTag = implicitly[ClassTag[Object]] + val level = StorageLevel.DISK_ONLY + val (indexFiles, dataFiles) = files.partition(_.getName.endsWith(".index")) + (dataFiles ++ indexFiles).foreach { f => Review comment: Thank you for review, @viirya . Apache Spark assumes that `dataFile` arrives first. - https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala#L599 ########## File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/shuffle/KubernetesLocalDiskShuffleExecutorComponents.scala ########## @@ -0,0 +1,105 @@ +/* + * 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 + +import java.io.File +import java.util.Optional + +import scala.reflect.ClassTag + +import org.apache.spark.{SparkConf, SparkEnv} +import org.apache.spark.internal.Logging +import org.apache.spark.shuffle.api.{ShuffleExecutorComponents, ShuffleMapOutputWriter, SingleSpillShuffleMapOutputWriter} +import org.apache.spark.shuffle.sort.io.LocalDiskShuffleExecutorComponents +import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, UnrecognizedBlockId} +import org.apache.spark.util.Utils + +class KubernetesLocalDiskShuffleExecutorComponents(sparkConf: SparkConf) + extends ShuffleExecutorComponents with Logging { + + private val delegate = new LocalDiskShuffleExecutorComponents(sparkConf) + private var blockManager: BlockManager = _ + + override def initializeExecutor( + appId: String, execId: String, extraConfigs: java.util.Map[String, String]): Unit = { + delegate.initializeExecutor(appId, execId, extraConfigs) + blockManager = SparkEnv.get.blockManager + if (sparkConf.getBoolean("spark.kubernetes.driver.reusePersistentVolumeClaim", false)) { + // Turn off the deletion of the shuffle data in order to reuse + blockManager.diskBlockManager.deleteFilesOnStop = false + logError("Recover shuffle data") Review comment: Yep. Right. This is a left-over of my debugging message. I used `logError`. :) ########## File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/shuffle/KubernetesLocalDiskShuffleExecutorComponents.scala ########## @@ -0,0 +1,105 @@ +/* + * 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 + +import java.io.File +import java.util.Optional + +import scala.reflect.ClassTag + +import org.apache.spark.{SparkConf, SparkEnv} +import org.apache.spark.internal.Logging +import org.apache.spark.shuffle.api.{ShuffleExecutorComponents, ShuffleMapOutputWriter, SingleSpillShuffleMapOutputWriter} +import org.apache.spark.shuffle.sort.io.LocalDiskShuffleExecutorComponents +import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, UnrecognizedBlockId} +import org.apache.spark.util.Utils + +class KubernetesLocalDiskShuffleExecutorComponents(sparkConf: SparkConf) + extends ShuffleExecutorComponents with Logging { + + private val delegate = new LocalDiskShuffleExecutorComponents(sparkConf) + private var blockManager: BlockManager = _ + + override def initializeExecutor( + appId: String, execId: String, extraConfigs: java.util.Map[String, String]): Unit = { + delegate.initializeExecutor(appId, execId, extraConfigs) + blockManager = SparkEnv.get.blockManager + if (sparkConf.getBoolean("spark.kubernetes.driver.reusePersistentVolumeClaim", false)) { + // Turn off the deletion of the shuffle data in order to reuse + blockManager.diskBlockManager.deleteFilesOnStop = false + logError("Recover shuffle data") Review comment: Yep. Right. This is a left-over of my personal debugging message. I used `logError`. :) I'll remove this. ########## File path: core/src/main/scala/org/apache/spark/MapOutputTracker.scala ########## @@ -146,6 +151,7 @@ private class ShuffleStatus( _numAvailableMapOutputs += 1 invalidateSerializedMapOutputStatusCache() } + mapStatusesDeleted(mapIndex) = mapStatuses(mapIndex) Review comment: Hi, @viirya and @mridulm . This is based on the semantic of this function description. With this, we will recover only the last one. ``` /** * Register a map output. If there is already a registered location for the map output then it * will be replaced by the new location. */ ``` ########## File path: core/src/main/scala/org/apache/spark/MapOutputTracker.scala ########## @@ -154,14 +160,25 @@ private class ShuffleStatus( */ def updateMapOutput(mapId: Long, bmAddress: BlockManagerId): Unit = withWriteLock { try { - val mapStatusOpt = mapStatuses.find(_.mapId == mapId) + val mapStatusOpt = mapStatuses.find(x => x != null && x.mapId == mapId) mapStatusOpt match { case Some(mapStatus) => logInfo(s"Updating map output for ${mapId} to ${bmAddress}") mapStatus.updateLocation(bmAddress) invalidateSerializedMapOutputStatusCache() case None => - logWarning(s"Asked to update map output ${mapId} for untracked map status.") + val index = mapStatusesDeleted.indexWhere(x => x != null && x.mapId == mapId) + if (index >= 0) { + val mapStatus = mapStatusesDeleted(index) + mapStatus.updateLocation(bmAddress) + assert(mapStatuses(index) == null) + mapStatuses(index) = mapStatus + _numAvailableMapOutputs += 1 + invalidateSerializedMapOutputStatusCache() + logDebug(s"Recover ${mapStatus.mapId} ${mapStatus.location}") Review comment: No, we should not clear `mapStatusesDeleted` after this recovery. We may need to recover more than once if this executor dies again. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
