[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-10 Thread GitBox


dongjoon-hyun commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r649578467



##
File path: 
resource-managers/kubernetes/core/src/test/scala/org/apache/spark/shuffle/KubernetesLocalDiskShuffleDataIOSuite.scala
##
@@ -0,0 +1,219 @@
+/*
+ * 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 scala.concurrent.duration._
+
+import org.scalatest.concurrent.Eventually.{eventually, interval, timeout}
+
+import org.apache.spark.{LocalSparkContext, MapOutputTrackerMaster, SparkConf, 
SparkContext, SparkFunSuite, TestUtils}
+import org.apache.spark.LocalSparkContext.withSpark
+import org.apache.spark.deploy.k8s.Config.KUBERNETES_DRIVER_REUSE_PVC
+import org.apache.spark.internal.config._
+import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend
+
+class KubernetesLocalDiskShuffleDataIOSuite extends SparkFunSuite with 
LocalSparkContext {
+
+  val conf = new SparkConf()
+.setAppName("ShuffleExecutorComponentsSuite")
+.setMaster("local-cluster[1,1,1024]")
+.set(UI.UI_ENABLED, false)
+.set(DYN_ALLOCATION_ENABLED, true)
+.set(DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED, true)
+.set(DYN_ALLOCATION_INITIAL_EXECUTORS, 1)
+.set(DYN_ALLOCATION_MIN_EXECUTORS, 1)
+.set(IO_ENCRYPTION_ENABLED, false)
+.set(KUBERNETES_DRIVER_REUSE_PVC, true)
+.set(SHUFFLE_IO_PLUGIN_CLASS, 
classOf[KubernetesLocalDiskShuffleDataIO].getName)
+
+  test("recompute is not blocked by the recovery") {
+sc = new SparkContext(conf)
+withSpark(sc) { sc =>
+  val master = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster]
+  assert(master.shuffleStatuses.isEmpty)
+
+  val rdd = sc.parallelize(Seq((1, "one"), (2, "two"), (3, "three")), 3)
+.groupByKey()
+  rdd.collect()
+  assert(master.shuffleStatuses.keys.toSet == Set(0))
+  val loc1 = master.shuffleStatuses(0).mapStatuses(0).location
+  assert(master.shuffleStatuses(0).mapStatuses.map(_.mapId).toSet == 
Set(0, 1, 2))
+
+  // Reuse the existing shuffle data
+  rdd.collect()
+  assert(master.shuffleStatuses.keys.toSet == Set(0))
+  assert(master.shuffleStatuses(0).mapStatuses.map(_.mapId).toSet == 
Set(0, 1, 2))
+
+  // Decommission all executors
+  val sched = sc.schedulerBackend.asInstanceOf[StandaloneSchedulerBackend]
+  sc.getExecutorIds().foreach { id =>
+sched.killExecutor(id)
+  }
+  TestUtils.waitUntilExecutorsUp(sc, 1, 6)
+  // Shuffle status are removed
+  eventually(timeout(60.second), interval(1.seconds)) {
+assert(master.shuffleStatuses.keys.toSet == Set(0))
+assert(master.shuffleStatuses(0).mapStatuses.forall(_ == null))
+  }
+
+  rdd.collect()
+  assert(master.shuffleStatuses.keys.toSet == Set(0))
+  assert(master.shuffleStatuses(0).mapStatuses.map(_.mapId).toSet == 
Set(9, 10, 11))
+}
+  }
+
+  test("Partial recompute shuffle data") {
+sc = new SparkContext(conf)
+withSpark(sc) { sc =>
+  val master = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster]
+  assert(master.shuffleStatuses.isEmpty)
+
+  val rdd = sc.parallelize(Seq((1, "one"), (2, "two"), (3, "three")), 
3).groupByKey()
+  rdd.collect()
+  assert(master.shuffleStatuses.keys.toSet == Set(0))
+  val loc1 = master.shuffleStatuses(0).mapStatuses(0).location
+  assert(master.shuffleStatuses(0).mapStatuses.forall(_.location == loc1))
+  assert(master.shuffleStatuses(0).mapStatuses.map(_.mapId).toSet == 
Set(0, 1, 2))
+
+  // Reuse the existing shuffle data
+  rdd.collect()
+  assert(master.shuffleStatuses.keys.toSet == Set(0))
+  assert(master.shuffleStatuses(0).mapStatuses.forall(_.location == loc1))
+  assert(master.shuffleStatuses(0).mapStatuses.map(_.mapId).toSet == 
Set(0, 1, 2))
+
+  val rdd2 = sc.parallelize(Seq((4, "four"), (5, "five"), (6, "six"), (7, 
"seven")), 4)
+.groupByKey()
+  rdd2.collect()
+  assert(master.shuffleStatuses.keys.toSet == Set(0, 1))
+  assert(master.shuffleStatuses(0).mapStatuses.forall(_.location == loc1))
+  

[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-10 Thread GitBox


dongjoon-hyun commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r649577655



##
File path: 
resource-managers/kubernetes/core/src/test/scala/org/apache/spark/shuffle/KubernetesLocalDiskShuffleDataIOSuite.scala
##
@@ -0,0 +1,219 @@
+/*
+ * 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 scala.concurrent.duration._
+
+import org.scalatest.concurrent.Eventually.{eventually, interval, timeout}
+
+import org.apache.spark.{LocalSparkContext, MapOutputTrackerMaster, SparkConf, 
SparkContext, SparkFunSuite, TestUtils}
+import org.apache.spark.LocalSparkContext.withSpark
+import org.apache.spark.deploy.k8s.Config.KUBERNETES_DRIVER_REUSE_PVC
+import org.apache.spark.internal.config._
+import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend
+
+class KubernetesLocalDiskShuffleDataIOSuite extends SparkFunSuite with 
LocalSparkContext {
+
+  val conf = new SparkConf()
+.setAppName("ShuffleExecutorComponentsSuite")
+.setMaster("local-cluster[1,1,1024]")
+.set(UI.UI_ENABLED, false)
+.set(DYN_ALLOCATION_ENABLED, true)
+.set(DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED, true)
+.set(DYN_ALLOCATION_INITIAL_EXECUTORS, 1)
+.set(DYN_ALLOCATION_MIN_EXECUTORS, 1)
+.set(IO_ENCRYPTION_ENABLED, false)
+.set(KUBERNETES_DRIVER_REUSE_PVC, true)
+.set(SHUFFLE_IO_PLUGIN_CLASS, 
classOf[KubernetesLocalDiskShuffleDataIO].getName)
+
+  test("recompute is not blocked by the recovery") {
+sc = new SparkContext(conf)
+withSpark(sc) { sc =>
+  val master = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster]
+  assert(master.shuffleStatuses.isEmpty)
+
+  val rdd = sc.parallelize(Seq((1, "one"), (2, "two"), (3, "three")), 3)
+.groupByKey()
+  rdd.collect()
+  assert(master.shuffleStatuses.keys.toSet == Set(0))
+  val loc1 = master.shuffleStatuses(0).mapStatuses(0).location

Review comment:
   Yes, `loc1` is not used in this test case. I'll remove it.




-- 
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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-10 Thread GitBox


dongjoon-hyun commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r649519438



##
File path: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/shuffle/KubernetesLocalDiskShuffleExecutorComponents.scala
##
@@ -0,0 +1,102 @@
+/*
+ * 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
+  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 =>
+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

Review comment:
   Sure, we can check the pattern. I will do later~




-- 
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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-10 Thread GitBox


dongjoon-hyun commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r649362673



##
File path: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/shuffle/KubernetesLocalDiskShuffleExecutorComponents.scala
##
@@ -0,0 +1,102 @@
+/*
+ * 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
+  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))

Review comment:
   The current directory is the local directory for this executor. Here, we 
are trying to find dead executor's local directory. 
   > We store shuffle files outside local dirs?




-- 
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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-10 Thread GitBox


dongjoon-hyun commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r649357020



##
File path: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/shuffle/KubernetesLocalDiskShuffleExecutorComponents.scala
##
@@ -0,0 +1,102 @@
+/*
+ * 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
+  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 =>
+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 =>
+  try {
+val id = BlockId(f.getName)
+val decryptedSize = f.length()
+bm.TempFileBasedBlockStoreUpdater(id, level, classTag, f, 
decryptedSize).save()
+  } catch {
+case _: UnrecognizedBlockId =>

Review comment:
   No, we will not lose any valid block files. `UnrecognizedBlockId` means 
the file name is not in Spark-generated pattern. We are excluding non-Spark 
files. This is also a preventive approach.
   ```scala
   @DeveloperApi
   class UnrecognizedBlockId(name: String)
   extends SparkException(s"Failed to parse $name into a block ID")
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the 

[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-10 Thread GitBox


dongjoon-hyun commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r649357020



##
File path: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/shuffle/KubernetesLocalDiskShuffleExecutorComponents.scala
##
@@ -0,0 +1,102 @@
+/*
+ * 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
+  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 =>
+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 =>
+  try {
+val id = BlockId(f.getName)
+val decryptedSize = f.length()
+bm.TempFileBasedBlockStoreUpdater(id, level, classTag, f, 
decryptedSize).save()
+  } catch {
+case _: UnrecognizedBlockId =>

Review comment:
   No, `UnrecognizedBlockId` means the file name is not in Spark-generated 
pattern. We are excluding non-Spark files. This is also a preventive approach.
   ```scala
   @DeveloperApi
   class UnrecognizedBlockId(name: String)
   extends SparkException(s"Failed to parse $name into a block ID")
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the

[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-10 Thread GitBox


dongjoon-hyun commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r649357020



##
File path: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/shuffle/KubernetesLocalDiskShuffleExecutorComponents.scala
##
@@ -0,0 +1,102 @@
+/*
+ * 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
+  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 =>
+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 =>
+  try {
+val id = BlockId(f.getName)
+val decryptedSize = f.length()
+bm.TempFileBasedBlockStoreUpdater(id, level, classTag, f, 
decryptedSize).save()
+  } catch {
+case _: UnrecognizedBlockId =>

Review comment:
   No, `UnrecognizedBlockId` means the file name is not in Spark-generated 
pattern.
   ```scala
   @DeveloperApi
   class UnrecognizedBlockId(name: String)
   extends SparkException(s"Failed to parse $name into a block ID")
   ```




-- 
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 

[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-10 Thread GitBox


dongjoon-hyun commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r649355661



##
File path: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/shuffle/KubernetesLocalDiskShuffleExecutorComponents.scala
##
@@ -0,0 +1,102 @@
+/*
+ * 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
+  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 =>
+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

Review comment:
   Yes, it's a protected approach which excludes some random files in the 
intermediate level.




-- 
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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-06 Thread GitBox


dongjoon-hyun commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r646245154



##
File path: core/src/main/scala/org/apache/spark/MapOutputTracker.scala
##
@@ -154,14 +159,26 @@ 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)

Review comment:
   > I have not looked into decomissioning in detail - but this assumption 
will hold right (assertion) ?
   
   For the worker decommission, yes. There are three cases.
   - When the worker is decommissioning, it should be handled at the previous 
`Some(mapStatus)`.
   - When the worker is killed before finishing and new executor is started 
before handling the dead executors, it should be handled at the previous 
`Some(mapStatus)`.
   - When the worker is killed before finishing and the data of the deleted 
executors are removed, `Asked to update map output ${mapId} for untracked map 
status`  happened before. And, this PR is fixing it.
   
   For the following, if something is filled here already with the different 
mapId, we can update this or ignore this. However, in both case, `assert` is 
too strong. In that case, we had better skip in that case. Thanks! I'll update 
this PR.
   > Can there be an interleaving recomputation or speculative task which 
updates MOT for that partition index ?




-- 
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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-06 Thread GitBox


dongjoon-hyun commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r646245154



##
File path: core/src/main/scala/org/apache/spark/MapOutputTracker.scala
##
@@ -154,14 +159,26 @@ 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)

Review comment:
   > I have not looked into decomissioning in detail - but this assumption 
will hold right (assertion) ?
   
   For the worker decommission, yes. There are three cases.
   - When the worker is decommissioning, it should be handled at the previous 
`Some(mapStatus)`.
   - When the worker is killed before finishing, new executor is started before 
handling the dead executors. It should be handled at the previous 
`Some(mapStatus)`.
   - When the worker is killed before finishing, the data of the deleted 
executors are removed and `Asked to update map output ${mapId} for untracked 
map status`  happened before. And, this PR is fixing it.
   
   For the following, if something is filled here already with the different 
mapId, we can update this or ignore this. However, in both case, `assert` is 
too strong. In that case, we had better skip in that case. Thanks! I'll update 
this PR.
   > Can there be an interleaving recomputation or speculative task which 
updates MOT for that partition index ?




-- 
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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-04 Thread GitBox


dongjoon-hyun commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r645698008



##
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:
   Right, it will be matched at that time. I'll update the PR.




-- 
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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-04 Thread GitBox


dongjoon-hyun commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r645690836



##
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:
   Ah, I missed your intention. Right, AS-IS code is matching 
`mapStatusesDeleted` every places where `mapStatuses` is changed. I'll remove 
this part. Thanks, @viirya and @mridulm .




-- 
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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-04 Thread GitBox


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 

[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-03 Thread GitBox


dongjoon-hyun commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r645275244



##
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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-03 Thread GitBox


dongjoon-hyun commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r645274836



##
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.
  */
   ```




-- 
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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-03 Thread GitBox


dongjoon-hyun commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r645264427



##
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.




-- 
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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-03 Thread GitBox


dongjoon-hyun commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r645264427



##
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`. :)




-- 
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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-03 Thread GitBox


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




-- 
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:
us...@infra.apache.org



-
To unsubscribe, e-mail: 

[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-01 Thread GitBox


dongjoon-hyun commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r643669887



##
File path: core/src/main/scala/org/apache/spark/MapOutputTracker.scala
##
@@ -234,6 +252,7 @@ private class ShuffleStatus(
 for (mapIndex <- mapStatuses.indices) {
   if (mapStatuses(mapIndex) != null && f(mapStatuses(mapIndex).location)) {
 _numAvailableMapOutputs -= 1
+mapStatusesDeleted(mapIndex) = mapStatuses(mapIndex)

Review comment:
   Thank you for review. In that case, `mapId` is not the same, isn't it? 
We are reusing with `mapId` at [line 
170](https://github.com/apache/spark/pull/32730/files#diff-a3b15298f97577c1fadcc2d76d015eebd6343e246c6717417d33f3c458847f46R170),
 @mridulm .
   ```
   val index = mapStatusesDeleted.indexWhere(x => x != null && x.mapId == mapId)
   ```




-- 
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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-01 Thread GitBox


dongjoon-hyun commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r643670680



##
File path: core/src/main/scala/org/apache/spark/MapOutputTracker.scala
##
@@ -234,6 +252,7 @@ private class ShuffleStatus(
 for (mapIndex <- mapStatuses.indices) {
   if (mapStatuses(mapIndex) != null && f(mapStatuses(mapIndex).location)) {
 _numAvailableMapOutputs -= 1
+mapStatusesDeleted(mapIndex) = mapStatuses(mapIndex)

Review comment:
   BTW, I agree with you that we don't have a test coverage for the 
indeterministic stage case. Let me try to add some.




-- 
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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-01 Thread GitBox


dongjoon-hyun commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r643670680



##
File path: core/src/main/scala/org/apache/spark/MapOutputTracker.scala
##
@@ -234,6 +252,7 @@ private class ShuffleStatus(
 for (mapIndex <- mapStatuses.indices) {
   if (mapStatuses(mapIndex) != null && f(mapStatuses(mapIndex).location)) {
 _numAvailableMapOutputs -= 1
+mapStatusesDeleted(mapIndex) = mapStatuses(mapIndex)

Review comment:
   BTW, I agree with you that we don't have a test coverage for the 
indeterministic stage case.




-- 
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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] dongjoon-hyun commented on a change in pull request #32730: [SPARK-35593][K8S][CORE] Support shuffle data recovery on the reused PVCs

2021-06-01 Thread GitBox


dongjoon-hyun commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r643669887



##
File path: core/src/main/scala/org/apache/spark/MapOutputTracker.scala
##
@@ -234,6 +252,7 @@ private class ShuffleStatus(
 for (mapIndex <- mapStatuses.indices) {
   if (mapStatuses(mapIndex) != null && f(mapStatuses(mapIndex).location)) {
 _numAvailableMapOutputs -= 1
+mapStatusesDeleted(mapIndex) = mapStatuses(mapIndex)

Review comment:
   In that case, `mapId` is not the same, isn't it? We are reusing with 
`mapId` at [line 
170](https://github.com/apache/spark/pull/32730/files#diff-a3b15298f97577c1fadcc2d76d015eebd6343e246c6717417d33f3c458847f46R170),
 @mridulm .
   ```
   val index = mapStatusesDeleted.indexWhere(x => x != null && x.mapId == mapId)
   ```




-- 
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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org