viirya commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r649576592



##########
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, 60000)
+      // 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))
+      assert(master.shuffleStatuses(1).mapStatuses.forall(_.location == loc1))
+      assert(master.shuffleStatuses(0).mapStatuses.map(_.mapId).toSet == 
Set(0, 1, 2))
+      assert(master.shuffleStatuses(1).mapStatuses.map(_.mapId).toSet == 
Set(9, 10, 11, 12))
+
+      // Decommission all executors
+      val sched = sc.schedulerBackend.asInstanceOf[StandaloneSchedulerBackend]
+      sc.getExecutorIds().foreach { id =>
+        sched.killExecutor(id)
+      }
+      TestUtils.waitUntilExecutorsUp(sc, 1, 60000)
+      // Shuffle status are removed
+      eventually(timeout(60.second), interval(1.seconds)) {
+        assert(master.shuffleStatuses.keys.toSet == Set(0, 1))
+        assert(master.shuffleStatuses(0).mapStatuses.forall(_ == null))
+        assert(master.shuffleStatuses(1).mapStatuses.forall(_ == null))
+      }
+
+      rdd.collect()
+      assert(master.shuffleStatuses.keys.toSet == Set(0, 1))
+      assert(master.shuffleStatuses(0).mapStatuses.map(_.mapId).toSet == 
Set(17, 18, 19))
+      assert(master.shuffleStatuses(1).mapStatuses.map(_.mapId).toSet == 
Set(9, 10, 11, 12))

Review comment:
       Why rdd is recomputed, but rdd2 is to recover from old shuffle status?

##########
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:
       loc1 is not used?




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

Reply via email to