Skip to content

Conversation

sarutak
Copy link
Member

@sarutak sarutak commented Sep 23, 2025

What changes were proposed in this pull request?

This PR aims to fix one of the issues which block SPARK-48139.
In the problematic test interrupt tag in SparkSessionE2ESuite, four futures run on threads in ForkJoinPool and try to interrupt through tags.
A thread in a ForkJoinPool can create a spare thread and make it available in the pool so any of threads can be parent and child. It can happen when a thread performs a blocking operation. One example is ArrayBlockingQueue.take and it is called in a method provided by gRPC.

On the other hand, tags are inplemented as InheritableThreadLocal.
So, if the futures q1 and q4, or q2 and q3 are parent and child, tags should be inheritd, which causes the flaky test faulure.

You can easily reprodue the issue by inserting a sleep into the problematic test like as follows (don't forget to replace ignore with test).

   // TODO(SPARK-48139): Re-enable `SparkSessionE2ESuite.interrupt tag`
-  ignore("interrupt tag") {
+  test("interrupt tag") {
     val session = spark
     import session.implicits._
 
@@ -204,6 +204,7 @@ class SparkSessionE2ESuite extends ConnectFunSuite with RemoteSparkSession {
         spark.clearTags() // clear for the case of thread reuse by another Future
       }
     }(executionContext)
+    Thread.sleep(1000)
     val q4 = Future {
       assert(spark.getTags() == Set())
       spark.addTag("one")

And then, run the test.

$ build/sbt 'connect-client-jvm/testOnly org.apache.spark.sql.connect.SparkSessionE2ESuite -- -z "interrupt tag"'

Why are the changes needed?

For test stability.

Does this PR introduce any user-facing change?

No.

How was this patch tested?

Ran the problematic test with inserting sleep like mentioned above and it passed.

Was this patch authored or co-authored using generative AI tooling?

No.

@sarutak
Copy link
Member Author

sarutak commented Sep 23, 2025

Intentionally keep the test as ignore because there is another cause of the flakiness.

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

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

It sounds reasonable to me as a partial improvement. Thank you, @sarutak .

@dongjoon-hyun
Copy link
Member

Feel free to merge since the CI result is irrelevant to this PR because the test case is still ignored.

@sarutak sarutak closed this in 0e42b95 Sep 23, 2025
@sarutak
Copy link
Member Author

sarutak commented Sep 23, 2025

Thank yoou @dongjoon-hyun @zhengruifeng @yaooqinn for the review!
Merged to master.

BTW, as far as I know, there is the last one issue which blocks SPARK-48139 and a PR tries to resolve it.
@zhengruifeng I'm happy if you are familiar with the area and can take a look at the PR.

dongjoon-hyun pushed a commit that referenced this pull request Sep 27, 2025
…SessionE2ESuite - interrupt tag` caused by the usage of `ForkJoinPool`

### What changes were proposed in this pull request?
This PR backports #52417 to `branch-4.0`.

This PR aims to fix one of the issues which block SPARK-48139.
In the problematic test `interrupt tag` in `SparkSessionE2ESuite`, four futures run on threads in `ForkJoinPool` and try to interrupt through tags.
A thread in a `ForkJoinPool` can create a spare thread and make it available in the pool so any of threads can be parent and child. It can happen when a thread performs a blocking operation. One example is `ArrayBlockingQueue.take` and it is called in a method provided by [gRPC](https://github.com/grpc/grpc-java/blob/24085103b926559659ecd3941a3308479876f084/stub/src/main/java/io/grpc/stub/ClientCalls.java#L607).

On the other hand, tags are implemented as [InheritableThreadLocal](https://github.com/apache/spark/blob/13e70100426233e62fd9edf13e229f91f4941ff8/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala#L285).
So, if the futures q1 and q4, or q2 and q3 are parent and child, tags should be inherited, which causes the flaky test failure.

You can easily reproduce the issue by inserting a sleep into the problematic test like as follows (don't forget to replace `ignore` with `test`).

```
   // TODO(SPARK-48139): Re-enable `SparkSessionE2ESuite.interrupt tag`
-  ignore("interrupt tag") {
+  test("interrupt tag") {
     val session = spark
     import session.implicits._

 -204,6 +204,7  class SparkSessionE2ESuite extends ConnectFunSuite with RemoteSparkSession {
         spark.clearTags() // clear for the case of thread reuse by another Future
       }
     }(executionContext)
+    Thread.sleep(1000)
     val q4 = Future {
       assert(spark.getTags() == Set())
       spark.addTag("one")
```

And then, run the test.
```
$ build/sbt 'connect-client-jvm/testOnly org.apache.spark.sql.connect.SparkSessionE2ESuite -- -z "interrupt tag"'
```

### Why are the changes needed?
For test stability.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Ran the problematic test with inserting sleep like mentioned above and it passed.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #52476 from sarutak/fix-thread-pool-issue-4.0.

Authored-by: Kousuke Saruta <sarutak@amazon.co.jp>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
dongjoon-hyun pushed a commit that referenced this pull request Sep 27, 2025
…SessionE2ESuite - interrupt tag` caused by the usage of `ForkJoinPool`

### What changes were proposed in this pull request?
This PR backports #52417 to `branch-3.5`.

Different from `master` and `branch-4.0`, the SPARK-53673 doesn't seem to affect the `branch-3.5` at this time because the implementation of `ClientCalls#waitForNext` in `gRPC 1.56.0` which `branch-3.5` depends on is different from the one in `gRPC 1.67.1` which `master` and `branch-4.0` depend on.
More specifically,  the test doesn't go through the pass which calls `ArrayBlockingQueue#take` but go through [this else block](https://github.com/grpc/grpc-java/blob/v1.56.0/stub/src/main/java/io/grpc/stub/ClientCalls.java#L634).
But I think it's better to backport #52417 to `branch-3.5` to prevent future changes from causing that issue.

### Why are the changes needed?
Just in case to prevent future changes from causing that issue.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
1.  Temporarily enable the test and insert a sleep into the test like as follows

```
   // TODO(SPARK-48139): Re-enable `SparkSessionE2ESuite.interrupt tag`
-  ignore("interrupt tag") {
+  test("interrupt tag") {
     val session = spark
     import session.implicits._

 -204,6 +204,7  class SparkSessionE2ESuite extends ConnectFunSuite with RemoteSparkSession {
         spark.clearTags() // clear for the case of thread reuse by another Future
       }
     }(executionContext)
+    Thread.sleep(1000)
     val q4 = Future {
       assert(spark.getTags() == Set())
       spark.addTag("one")
```

2. Run the test and confirm it passes
```
$ build/sbt 'connect-client-jvm/testOnly org.apache.spark.sql.SparkSessionE2ESuite -- -z "interrupt tag"'
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #52477 from sarutak/fix-thread-pool-issue-3.5.

Authored-by: Kousuke Saruta <sarutak@amazon.co.jp>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants