Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-25297][Streaming][Test] Fix blocking unit tests for Scala 2.12 #22304

Closed
wants to merge 1 commit into from

Conversation

da-liii
Copy link
Contributor

@da-liii da-liii commented Aug 31, 2018

What changes were proposed in this pull request?

Customize ExecutorContext's reporter to fix blocking unit tests for Scala 2.12

How was this patch tested?

./dev/change-scala-version.sh 2.12
$ sbt -Dscala-2.12
> ++2.12.6
> project streaming
> testOnly *FileBasedWriteAheadLogWithFileCloseAfterWriteSuite

@da-liii
Copy link
Contributor Author

da-liii commented Aug 31, 2018

There is simplified project for this specific problem.

https://github.com/sadhen/blocking-future

@@ -65,7 +65,8 @@ private[streaming] class FileBasedWriteAheadLog(
"WriteAheadLogManager" + callerName.map(c => s" for $c").getOrElse("")
}
private val forkJoinPool = ThreadUtils.newForkJoinPool(threadpoolName, 20)
private val executionContext = ExecutionContext.fromExecutorService(forkJoinPool)
private val executionContext = ExecutionContext
.fromExecutorService(forkJoinPool, { e: Throwable => throw e })
Copy link
Contributor

Choose a reason for hiding this comment

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

do you mean in scala 2.12 the default reporter is not throwing the exception?

Copy link
Contributor Author

@da-liii da-liii Aug 31, 2018

Choose a reason for hiding this comment

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

The default reporter does not throw the exception in Scala 2.11 either.

I guess there are bugs in the Future impl in Scala 2.12, since there is a big change at Future.scala in Scala 2.12.

This is a temporary fix.

I will dig into Future.scala, and hopefully propose a PR for Scala.

Copy link
Member

Choose a reason for hiding this comment

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

Thanks @sadhen I was just going to start looking into this. This seems like a reasonable solution.

Choose a reason for hiding this comment

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

The suggested solution will not work as intended as it will prevent subsequent callbacks (esp for other ExecutionContexts) to be submitted, esp. in the case of RejectedExecutionException.

See:
https://github.com/scala/scala/blob/2.12.x/src/library/scala/concurrent/impl/Promise.scala#L68
https://github.com/scala/scala/blob/2.12.x/src/library/scala/concurrent/impl/Promise.scala#L284

Or, in other words, rethrowing is not "reporting" the error.

The new Future encoding included in Scala 2.13.x deals much better with RejectedExecutionExceptions and InterruptedExceptions, but it is unclear right now what parts could be backported within the binary compatibility constraints.

@SparkQA
Copy link

SparkQA commented Aug 31, 2018

Test build #95540 has finished for PR 22304 at commit 60eec77.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@srowen
Copy link
Member

srowen commented Aug 31, 2018

More info ... the test failure symptom is that this test never completes, just repeating a RejectedExecutionException:

- FileBasedWriteAheadLog - handling file errors while reading rotating logs
- FileBasedWriteAheadLog - do not create directories or files unless write
java.util.concurrent.RejectedExecutionException
	at java.util.concurrent.ForkJoinPool.externalSubmit(ForkJoinPool.java:2328)
	at java.util.concurrent.ForkJoinPool.externalPush(ForkJoinPool.java:2419)
	at java.util.concurrent.ForkJoinPool.execute(ForkJoinPool.java:2648)
	at scala.concurrent.impl.ExecutionContextImpl$$anon$1.execute(ExecutionContextImpl.scala:134)
	at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:68)
	at scala.concurrent.impl.Promise$KeptPromise$Kept.onComplete(Promise.scala:368)
	at scala.concurrent.impl.Promise$KeptPromise$Kept.onComplete$(Promise.scala:367)
	at scala.concurrent.impl.Promise$KeptPromise$Successful.onComplete(Promise.scala:375)
	at scala.concurrent.impl.Promise.transform(Promise.scala:29)
	at scala.concurrent.impl.Promise.transform$(Promise.scala:27)
	at scala.concurrent.impl.Promise$KeptPromise$Successful.transform(Promise.scala:375)
	at scala.concurrent.Future.map(Future.scala:288)
	at scala.concurrent.Future.map$(Future.scala:288)
	at scala.concurrent.impl.Promise$KeptPromise$Successful.map(Promise.scala:375)
	at scala.concurrent.Future$.apply(Future.scala:654)
	at org.apache.spark.util.ThreadUtils$.$anonfun$parmap$1(ThreadUtils.scala:314)
	at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:234)
	at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:59)
	at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:52)
	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
	at scala.collection.TraversableLike.map(TraversableLike.scala:234)
	at scala.collection.TraversableLike.map$(TraversableLike.scala:227)
	at scala.collection.AbstractTraversable.map(Traversable.scala:104)
	at org.apache.spark.util.ThreadUtils$.parmap(ThreadUtils.scala:314)
	at org.apache.spark.streaming.util.FileBasedWriteAheadLog$.$anonfun$seqToParIterator$1(FileBasedWriteAheadLog.scala:318)
	at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:482)
	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:488)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:456)
	at scala.collection.TraversableOnce$FlattenOps$$anon$1.hasNext(TraversableOnce.scala:464)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:456)
	at scala.collection.Iterator.foreach(Iterator.scala:944)
	at scala.collection.Iterator.foreach$(Iterator.scala:944)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1432)
	at scala.collection.generic.Growable.$plus$plus$eq(Growable.scala:59)
	at scala.collection.generic.Growable.$plus$plus$eq$(Growable.scala:50)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48)
	at scala.collection.TraversableOnce.to(TraversableOnce.scala:310)
	at scala.collection.TraversableOnce.to$(TraversableOnce.scala:308)
	at scala.collection.AbstractIterator.to(Iterator.scala:1432)
	at scala.collection.TraversableOnce.toBuffer(TraversableOnce.scala:302)
	at scala.collection.TraversableOnce.toBuffer$(TraversableOnce.scala:302)
	at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1432)
	at scala.collection.TraversableOnce.toArray(TraversableOnce.scala:289)
	at scala.collection.TraversableOnce.toArray$(TraversableOnce.scala:283)
	at scala.collection.AbstractIterator.toArray(Iterator.scala:1432)
	at org.apache.spark.streaming.util.CommonWriteAheadLogTests.$anonfun$new$16(WriteAheadLogSuite.scala:213)
	at org.scalatest.Assertions.intercept(Assertions.scala:805)
	at org.scalatest.Assertions.intercept$(Assertions.scala:802)
	at org.scalatest.FunSuite.intercept(FunSuite.scala:1560)
	at org.apache.spark.streaming.util.CommonWriteAheadLogTests.$anonfun$new$13(WriteAheadLogSuite.scala:213)

@srowen
Copy link
Member

srowen commented Aug 31, 2018

The one concern is whether this behavior change impacts anything else. Now some exceptions will cause an actual exception in a caller to methods like Await, rather than just logging and continuing. That might even be a good thing, and is crucial here. But that's what I have in mind. That said, tests all pass.

@cloud-fan
Copy link
Contributor

@srowen are we targeting this patch to Spark 2.4?

@da-liii
Copy link
Contributor Author

da-liii commented Sep 1, 2018

@srowen @cloud-fan The merged #22292 already fixed it.

@da-liii da-liii closed this Sep 1, 2018
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants