You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "zhenlineo (via GitHub)" <gi...@apache.org> on 2023/08/01 03:43:46 UTC

[GitHub] [spark] zhenlineo commented on a diff in pull request #42009: [SPARK-44422][CONNECT] Spark Connect fine grained interrupt

zhenlineo commented on code in PR #42009:
URL: https://github.com/apache/spark/pull/42009#discussion_r1280078719


##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionE2ESuite.scala:
##########
@@ -96,5 +103,148 @@ class SparkSessionE2ESuite extends RemoteSparkSession {
     assert(e2.getMessage.contains("OPERATION_CANCELED"), s"Unexpected exception: $e2")
     finished = true
     assert(ThreadUtils.awaitResult(interruptor, 10.seconds))
+    assert(interrupted.length == 2, s"Interrupted operations: $interrupted.")
+  }
+
+  test("interrupt tag") {
+    val session = spark
+    import session.implicits._
+
+    // global ExecutionContext has only 2 threads in Apache Spark CI
+    // create own thread pool for four Futures used in this test
+    val numThreads = 4
+    val fpool = ThreadUtils.newForkJoinPool("job-tags-test-thread-pool", numThreads)
+    val executionContext = ExecutionContext.fromExecutorService(fpool)
+
+    val q1 = Future {
+      assert(spark.getTags() == Set())
+      spark.addTag("two")
+      assert(spark.getTags() == Set("two"))
+      spark.clearTags() // check that clearing all tags works
+      assert(spark.getTags() == Set())
+      spark.addTag("one")
+      assert(spark.getTags() == Set("one"))
+      try {
+        spark
+          .range(10)
+          .map(n => {
+            Thread.sleep(30000); n
+          })
+          .collect()
+      } finally {
+        spark.clearTags() // clear for the case of thread reuse by another Future
+      }
+    }(executionContext)
+    val q2 = Future {
+      assert(spark.getTags() == Set())
+      spark.addTag("one")
+      spark.addTag("two")
+      spark.addTag("one")
+      spark.addTag("two") // duplicates shouldn't matter
+      assert(spark.getTags() == Set("one", "two"))
+      try {
+        spark
+          .range(10)
+          .map(n => {
+            Thread.sleep(30000); n
+          })
+          .collect()
+      } finally {
+        spark.clearTags() // clear for the case of thread reuse by another Future
+      }
+    }(executionContext)
+    val q3 = Future {
+      assert(spark.getTags() == Set())
+      spark.addTag("foo")
+      spark.removeTag("foo")
+      assert(spark.getTags() == Set()) // check that remove works removing the last tag
+      spark.addTag("two")
+      assert(spark.getTags() == Set("two"))
+      try {
+        spark
+          .range(10)
+          .map(n => {
+            Thread.sleep(30000); n
+          })
+          .collect()
+      } finally {
+        spark.clearTags() // clear for the case of thread reuse by another Future
+      }
+    }(executionContext)
+    val q4 = Future {
+      assert(spark.getTags() == Set())
+      spark.addTag("one")
+      spark.addTag("two")
+      spark.addTag("two")
+      assert(spark.getTags() == Set("one", "two"))
+      spark.removeTag("two") // check that remove works, despite duplicate add
+      assert(spark.getTags() == Set("one"))
+      try {
+        spark
+          .range(10)
+          .map(n => {
+            Thread.sleep(30000); n
+          })
+          .collect()
+      } finally {
+        spark.clearTags() // clear for the case of thread reuse by another Future
+      }
+    }(executionContext)
+    val interrupted = mutable.ListBuffer[String]()
+
+    // q2 and q3 should be cancelled
+    interrupted.clear()
+    eventually(timeout(20.seconds), interval(1.seconds)) {
+      val ids = spark.interruptTag("two")
+      interrupted ++= ids
+      assert(interrupted.length == 2, s"Interrupted operations: $interrupted.")
+    }
+    val e2 = intercept[SparkException] {
+      ThreadUtils.awaitResult(q2, 1.minute)
+    }
+    assert(e2.getCause.getMessage contains "OPERATION_CANCELED")
+    val e3 = intercept[SparkException] {
+      ThreadUtils.awaitResult(q3, 1.minute)
+    }
+    assert(e3.getCause.getMessage contains "OPERATION_CANCELED")
+    assert(interrupted.length == 2, s"Interrupted operations: $interrupted.")
+
+    // q1 and q4 should be cancelled
+    interrupted.clear()
+    eventually(timeout(20.seconds), interval(1.seconds)) {
+      val ids = spark.interruptTag("one")
+      interrupted ++= ids
+      assert(interrupted.length == 2, s"Interrupted operations: $interrupted.")
+    }
+    val e1 = intercept[SparkException] {
+      ThreadUtils.awaitResult(q1, 1.minute)
+    }
+    assert(e1.getCause.getMessage contains "OPERATION_CANCELED")
+    val e4 = intercept[SparkException] {
+      ThreadUtils.awaitResult(q4, 1.minute)
+    }
+    assert(e4.getCause.getMessage contains "OPERATION_CANCELED")
+    assert(interrupted.length == 2, s"Interrupted operations: $interrupted.")
+  }
+
+  test("interrupt operation") {

Review Comment:
   Hi @bogao007 it would be great when you get change to fix https://issues.apache.org/jira/browse/SPARK-44576
   After the fix, we can move the whole test to use addArtifact to fix the 4 maven tests. I think we need to fix them before the 3.5 release. 



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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