This is an automated email from the ASF dual-hosted git repository.
yangjie01 pushed a commit to branch branch-4.0
in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/branch-4.0 by this push:
new 419a70316c34 [SPARK-51633][CORE][TESTS] Reset `Utils#customHostname`
in the `finally` block of `ExecutorSuite#withExecutor`
419a70316c34 is described below
commit 419a70316c3419e450e715565426671b301a2c31
Author: yangjie01 <[email protected]>
AuthorDate: Tue Apr 15 13:09:48 2025 +0800
[SPARK-51633][CORE][TESTS] Reset `Utils#customHostname` in the `finally`
block of `ExecutorSuite#withExecutor`
### What changes were proposed in this pull request?
This pull request adds a reset operation for `Utils#customHostname` in the
`finally` block of the `withExecutor` function in `ExecutorSuite`, in order to
prevent `withExecutor` from affecting other test cases.
### Why are the changes needed?
To avoid negative impacts on other test cases after the `withExecutor`
function in `ExecutorSuite` is called.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
- Pass GitHub Actions
- locally test
```
build/sbt clean "core/testOnly org.apache.spark.executor.ExecutorSuite
org.apache.spark.ExternalShuffleServiceSuite"
```
**Before**
```
[info] - SPARK-25888: using external shuffle service fetching disk
persisted blocks *** FAILED *** (1 second, 366 milliseconds)
[info]
ExternalShuffleServiceSuite.this.sc.env.blockManager.getRemoteValues[Nothing](blockId)((ClassTag.Nothing:
scala.reflect.ClassTag[Nothing])).isDefined was false
(ExternalShuffleServiceSuite.scala:176)
[info] org.scalatest.exceptions.TestFailedException:
[info] at
org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472)
[info] at
org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:471)
[info] at
org.scalatest.Assertions$.newAssertionFailedException(Assertions.scala:1231)
[info] at
org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:1295)
[info] at
org.apache.spark.ExternalShuffleServiceSuite.$anonfun$new$5(ExternalShuffleServiceSuite.scala:176)
[info] at
org.scalatest.enablers.Timed$$anon$1.timeoutAfter(Timed.scala:127)
[info] at
org.scalatest.concurrent.TimeLimits$.failAfterImpl(TimeLimits.scala:282)
[info] at
org.scalatest.concurrent.TimeLimits.failAfter(TimeLimits.scala:231)
[info] at
org.scalatest.concurrent.TimeLimits.failAfter$(TimeLimits.scala:230)
[info] at org.apache.spark.SparkFunSuite.failAfter(SparkFunSuite.scala:69)
[info] at
org.apache.spark.SparkFunSuite.$anonfun$test$2(SparkFunSuite.scala:155)
[info] at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
[info] at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
[info] at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
[info] at org.scalatest.Transformer.apply(Transformer.scala:22)
[info] at org.scalatest.Transformer.apply(Transformer.scala:20)
[info] at
org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226)
[info] at
org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:227)
[info] at
org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224)
[info] at
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236)
[info] at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
[info] at
org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236)
[info] at
org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218)
[info] at
org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:69)
[info] at
org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
[info] at
org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
[info] at org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:69)
```
**After**
```
[info] Run completed in 42 seconds, 544 milliseconds.
[info] Total number of tests run: 39
[info] Suites: completed 2, aborted 0
[info] Tests: succeeded 39, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.
```
### Was this patch authored or co-authored using generative AI tooling?
No
Closes #50580 from LuciferYang/ExecutorSuite-withExecutor.
Authored-by: yangjie01 <[email protected]>
Signed-off-by: yangjie01 <[email protected]>
(cherry picked from commit bb39e4ba8c093b52beafbd4905ed5b440165ede9)
Signed-off-by: yangjie01 <[email protected]>
---
core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala | 8 +++++++-
1 file changed, 7 insertions(+), 1 deletion(-)
diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala
b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala
index fa13092dc47a..6f525cf8b898 100644
--- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala
@@ -55,7 +55,7 @@ import org.apache.spark.scheduler.{DirectTaskResult,
FakeTask, ResultTask, Task,
import org.apache.spark.serializer.{JavaSerializer, SerializerInstance,
SerializerManager}
import org.apache.spark.shuffle.FetchFailedException
import org.apache.spark.storage.{BlockManager, BlockManagerId}
-import org.apache.spark.util.{LongAccumulator, SparkUncaughtExceptionHandler,
ThreadUtils, UninterruptibleThread}
+import org.apache.spark.util.{LongAccumulator, SparkUncaughtExceptionHandler,
ThreadUtils, UninterruptibleThread, Utils}
class ExecutorSuite extends SparkFunSuite
with LocalSparkContext with MockitoSugar with Eventually with
PrivateMethodTester {
@@ -81,6 +81,8 @@ class ExecutorSuite extends SparkFunSuite
resources: immutable.Map[String, ResourceInformation]
= immutable.Map.empty[String, ResourceInformation])(f: Executor =>
Unit): Unit = {
var executor: Executor = null
+ val getCustomHostname =
PrivateMethod[Option[String]](Symbol("customHostname"))
+ val defaultCustomHostNameValue = Utils.invokePrivate(getCustomHostname())
try {
executor = new Executor(executorId, executorHostname, env,
userClassPath, isLocal,
uncaughtExceptionHandler, resources)
@@ -90,6 +92,10 @@ class ExecutorSuite extends SparkFunSuite
if (executor != null) {
executor.stop()
}
+ // SPARK-51633: Reset the custom hostname to its default value in
finally block
+ // to avoid contaminating other tests
+ val setCustomHostname = PrivateMethod[Unit](Symbol("customHostname_$eq"))
+ Utils.invokePrivate(setCustomHostname(defaultCustomHostNameValue))
}
}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]