This is an automated email from the ASF dual-hosted git repository. dongjoon pushed a commit to branch branch-3.0 in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/branch-3.0 by this push: new 1bec8a3 [SPARK-32436][CORE] Initialize numNonEmptyBlocks in HighlyCompressedMapStatus.readExternal 1bec8a3 is described below commit 1bec8a3e17ffd81126b36037a852cfc648764006 Author: Dongjoon Hyun <dongj...@apache.org> AuthorDate: Sat Jul 25 10:16:01 2020 -0700 [SPARK-32436][CORE] Initialize numNonEmptyBlocks in HighlyCompressedMapStatus.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 <dongj...@apache.org> Signed-off-by: Dongjoon Hyun <dongj...@apache.org> (cherry picked from commit f9f18673dca16c78a77c3386a1629136031bf2ba) Signed-off-by: Dongjoon Hyun <dh...@apple.com> --- core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 7f8893f..6005240 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -206,6 +206,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 emptyBlocks = new RoaringBitmap() emptyBlocks.readExternal(in) avgSize = in.readLong() --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org