[SPARK-32436][CORE] Initialize numNonEmptyBlocks in HighlyCompressedMapStatus.readExternal#29231
[SPARK-32436][CORE] Initialize numNonEmptyBlocks in HighlyCompressedMapStatus.readExternal#29231dongjoon-hyun wants to merge 3 commits intoapache:masterfrom dongjoon-hyun:SPARK-32436
Conversation
…umNonEmptyBlocks" This reverts commit 9753676.
This comment has been minimized.
This comment has been minimized.
|
Test build #126535 has finished for PR 29231 at commit
|
|
Could you review this, @HyukjinKwon ? |
|
Also, cc @srowen . |
|
If the fix works, great. I think this might help me understand another failure in the streaming component. Other possible fixes? make it public (the class is private anyway). |
|
Let me try, @srowen .
|
|
I tried to remove all constraints; 1) public -private[spark] class HighlyCompressedMapStatus private (
+class HighlyCompressedMapStatus(
private[this] var loc: BlockManagerId,
private[this] var numNonEmptyBlocks: Int,
private[this] var emptyBlocks: RoaringBitmap,
@@ -181,7 +181,7 @@ private[spark] class HighlyCompressedMapStatus private (
|| numNonEmptyBlocks == 0 || _mapTaskId > 0,
"Average size can only be zero for map stages that produced no output")
- protected def this() = this(null, -1, null, -1, null, -1) // For deserialization only
+ def this() = this(null, -1, null, -1, null, -1) // For deserialization only
override def location: BlockManagerId = loc
@@ -217,7 +217,7 @@ private[spark] class HighlyCompressedMapStatus private (
override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException {
loc = BlockManagerId(in)
- numNonEmptyBlocks = -1 // SPARK-32436 Scala 2.13 doesn't initialize this during deserialization
+ // numNonEmptyBlocks = -1 // SPARK-32436 Scala 2.13 doesn't initialize this during deserialization |
|
This PR technically doesn't change the logic. In other words, the value will be the same in Scala 2.12 and 2.13. Can we move forward with AS-IS patch? BTW, |
|
I updated the PR description by focusing on |
|
Hm, weird. I still don't understand why this behavior is different in 2.13. OK, go ahead. |
|
Thanks! Merged to master. |
|
LGTM |
|
Thank you, @HyukjinKwon . |
|
@dongjoon-hyun Circling through older PR's ... do we know why this is happening ? |
|
Hi, @mridulm ! This was Scala 2.13 and 2.12.12 bug.
I believe they will fix the bug in the next releases. Then, we don't need to detect or change inside Apache Spark. |
|
Thanks @dongjoon-hyun ! |
|
@dongjoon-hyun Will this be included in a Spark 3.0.x release or is the plan to wait for a fix on the scala side? I ran into this very issue today, so just wondering. Thank you. |
|
This only seems to affect scala 2.13, regardless, and only 3.1.x supports scala 2.13, so no there isn't a need to put it in 3.0.x. The workaround doesn't require a scala fix if any, but, that may also resolve it anyway. |
|
Thank you @srowen, the environment I saw this on was running spark 3.0.1 and scala 2.12.12. If I can reproduce it today I can share a stack trace and other details if that would be helpful. |
|
Running on GCP's dataproc 2.0: I'm playing with the Databricks Delta Lake a simple If this is helpful I'm happy to provide more information. |
|
That is strange - it doesn't seem to happen in Spark unit tests in 2.12. But Spark is on 2.12.10. From the links above, it seems like it could be an issue in 2.12.12. Therefore @dongjoon-hyun it might be useful to backport this just in case? It's a small change, and i think what it does is prevent the compiler from (incorrectly?) eliding the field during compilation. Even when that's fixed this change doesn't hurt anything really. |
|
BTW, for the related Scala issues, I linked here. |
…apStatus.readExternal ### What changes were proposed in this pull request? This PR aims to initialize `numNonEmptyBlocks` in `HighlyCompressedMapStatus.readExternal`. In Scala 2.12, this is initialized to `-1` via the following. ```scala protected def this() = this(null, -1, null, -1, null, -1) // For deserialization only ``` ### Why are the changes needed? In Scala 2.13, this causes several UT failures because `HighlyCompressedMapStatus.readExternal` doesn't initialize this field. The following is one example. - org.apache.spark.scheduler.MapStatusSuite ``` MapStatusSuite: - compressSize - decompressSize *** RUN ABORTED *** java.lang.NoSuchFieldError: numNonEmptyBlocks at org.apache.spark.scheduler.HighlyCompressedMapStatus.<init>(MapStatus.scala:181) at org.apache.spark.scheduler.HighlyCompressedMapStatus$.apply(MapStatus.scala:281) at org.apache.spark.scheduler.MapStatus$.apply(MapStatus.scala:73) at org.apache.spark.scheduler.MapStatusSuite.$anonfun$new$8(MapStatusSuite.scala:64) at scala.runtime.java8.JFunction1$mcVD$sp.apply(JFunction1$mcVD$sp.scala:18) at scala.collection.immutable.List.foreach(List.scala:333) at org.apache.spark.scheduler.MapStatusSuite.$anonfun$new$7(MapStatusSuite.scala:61) at scala.runtime.java8.JFunction1$mcVJ$sp.apply(JFunction1$mcVJ$sp.scala:18) at scala.collection.immutable.List.foreach(List.scala:333) at org.apache.spark.scheduler.MapStatusSuite.$anonfun$new$6(MapStatusSuite.scala:60) ... ``` ### Does this PR introduce _any_ user-facing change? No. This is a private class. ### How was this patch tested? 1. Pass the GitHub Action or Jenkins with the existing tests. 2. Test with Scala-2.13 with `MapStatusSuite`. ``` $ dev/change-scala-version.sh 2.13 $ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.scheduler.MapStatusSuite ... MapStatusSuite: - compressSize - decompressSize - MapStatus should never report non-empty blocks' sizes as 0 - large tasks should use org.apache.spark.scheduler.HighlyCompressedMapStatus - HighlyCompressedMapStatus: estimated size should be the average non-empty block size - SPARK-22540: ensure HighlyCompressedMapStatus calculates correct avgSize - RoaringBitmap: runOptimize succeeded - RoaringBitmap: runOptimize failed - Blocks which are bigger than SHUFFLE_ACCURATE_BLOCK_THRESHOLD should not be underestimated. - SPARK-21133 HighlyCompressedMapStatus#writeExternal throws NPE Run completed in 7 seconds, 971 milliseconds. Total number of tests run: 10 Suites: completed 2, aborted 0 Tests: succeeded 10, failed 0, canceled 0, ignored 0, pending 0 All tests passed. ``` Closes #29231 from dongjoon-hyun/SPARK-32436. Authored-by: Dongjoon Hyun <dongjoon@apache.org> Signed-off-by: Dongjoon Hyun <dongjoon@apache.org> (cherry picked from commit f9f1867) Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
|
This lands at |
…apStatus.readExternal ### What changes were proposed in this pull request? This PR aims to initialize `numNonEmptyBlocks` in `HighlyCompressedMapStatus.readExternal`. In Scala 2.12, this is initialized to `-1` via the following. ```scala protected def this() = this(null, -1, null, -1, null, -1) // For deserialization only ``` ### Why are the changes needed? In Scala 2.13, this causes several UT failures because `HighlyCompressedMapStatus.readExternal` doesn't initialize this field. The following is one example. - org.apache.spark.scheduler.MapStatusSuite ``` MapStatusSuite: - compressSize - decompressSize *** RUN ABORTED *** java.lang.NoSuchFieldError: numNonEmptyBlocks at org.apache.spark.scheduler.HighlyCompressedMapStatus.<init>(MapStatus.scala:181) at org.apache.spark.scheduler.HighlyCompressedMapStatus$.apply(MapStatus.scala:281) at org.apache.spark.scheduler.MapStatus$.apply(MapStatus.scala:73) at org.apache.spark.scheduler.MapStatusSuite.$anonfun$new$8(MapStatusSuite.scala:64) at scala.runtime.java8.JFunction1$mcVD$sp.apply(JFunction1$mcVD$sp.scala:18) at scala.collection.immutable.List.foreach(List.scala:333) at org.apache.spark.scheduler.MapStatusSuite.$anonfun$new$7(MapStatusSuite.scala:61) at scala.runtime.java8.JFunction1$mcVJ$sp.apply(JFunction1$mcVJ$sp.scala:18) at scala.collection.immutable.List.foreach(List.scala:333) at org.apache.spark.scheduler.MapStatusSuite.$anonfun$new$6(MapStatusSuite.scala:60) ... ``` ### Does this PR introduce _any_ user-facing change? No. This is a private class. ### How was this patch tested? 1. Pass the GitHub Action or Jenkins with the existing tests. 2. Test with Scala-2.13 with `MapStatusSuite`. ``` $ dev/change-scala-version.sh 2.13 $ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.scheduler.MapStatusSuite ... MapStatusSuite: - compressSize - decompressSize - MapStatus should never report non-empty blocks' sizes as 0 - large tasks should use org.apache.spark.scheduler.HighlyCompressedMapStatus - HighlyCompressedMapStatus: estimated size should be the average non-empty block size - SPARK-22540: ensure HighlyCompressedMapStatus calculates correct avgSize - RoaringBitmap: runOptimize succeeded - RoaringBitmap: runOptimize failed - Blocks which are bigger than SHUFFLE_ACCURATE_BLOCK_THRESHOLD should not be underestimated. - SPARK-21133 HighlyCompressedMapStatus#writeExternal throws NPE Run completed in 7 seconds, 971 milliseconds. Total number of tests run: 10 Suites: completed 2, aborted 0 Tests: succeeded 10, failed 0, canceled 0, ignored 0, pending 0 All tests passed. ``` Closes apache#29231 from dongjoon-hyun/SPARK-32436. Authored-by: Dongjoon Hyun <dongjoon@apache.org> Signed-off-by: Dongjoon Hyun <dongjoon@apache.org> (cherry picked from commit f9f1867) Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
What changes were proposed in this pull request?
This PR aims to initialize
numNonEmptyBlocksinHighlyCompressedMapStatus.readExternal.In Scala 2.12, this is initialized to
-1via the following.Why are the changes needed?
In Scala 2.13, this causes several UT failures because
HighlyCompressedMapStatus.readExternaldoesn't initialize this field. The following is one example.Does this PR introduce any user-facing change?
No. This is a private class.
How was this patch tested?
MapStatusSuite.