Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-39104][SQL] InMemoryRelation#isCachedColumnBuffersLoaded shoul…
…d be thread-safe ### What changes were proposed in this pull request? Add `synchronized` on method `isCachedColumnBuffersLoaded` ### Why are the changes needed? `isCachedColumnBuffersLoaded` should has `synchronized` wrapped, otherwise may cause NPE when modify `_cachedColumnBuffers` concurrently. ``` def isCachedColumnBuffersLoaded: Boolean = { _cachedColumnBuffers != null && isCachedRDDLoaded } def isCachedRDDLoaded: Boolean = { _cachedColumnBuffersAreLoaded || { val bmMaster = SparkEnv.get.blockManager.master val rddLoaded = _cachedColumnBuffers.partitions.forall { partition => bmMaster.getBlockStatus(RDDBlockId(_cachedColumnBuffers.id, partition.index), false) .exists { case(_, blockStatus) => blockStatus.isCached } } if (rddLoaded) { _cachedColumnBuffersAreLoaded = rddLoaded } rddLoaded } } ``` ``` java.lang.NullPointerException at org.apache.spark.sql.execution.columnar.CachedRDDBuilder.isCachedRDDLoaded(InMemoryRelation.scala:247) at org.apache.spark.sql.execution.columnar.CachedRDDBuilder.isCachedColumnBuffersLoaded(InMemoryRelation.scala:241) at org.apache.spark.sql.execution.CacheManager.$anonfun$uncacheQuery$8(CacheManager.scala:189) at org.apache.spark.sql.execution.CacheManager.$anonfun$uncacheQuery$8$adapted(CacheManager.scala:176) at scala.collection.TraversableLike.$anonfun$filterImpl$1(TraversableLike.scala:304) at scala.collection.Iterator.foreach(Iterator.scala:943) at scala.collection.Iterator.foreach$(Iterator.scala:943) at scala.collection.AbstractIterator.foreach(Iterator.scala:1431) at scala.collection.IterableLike.foreach(IterableLike.scala:74) at scala.collection.IterableLike.foreach$(IterableLike.scala:73) at scala.collection.AbstractIterable.foreach(Iterable.scala:56) at scala.collection.TraversableLike.filterImpl(TraversableLike.scala:303) at scala.collection.TraversableLike.filterImpl$(TraversableLike.scala:297) at scala.collection.AbstractTraversable.filterImpl(Traversable.scala:108) at scala.collection.TraversableLike.filter(TraversableLike.scala:395) at scala.collection.TraversableLike.filter$(TraversableLike.scala:395) at scala.collection.AbstractTraversable.filter(Traversable.scala:108) at org.apache.spark.sql.execution.CacheManager.recacheByCondition(CacheManager.scala:219) at org.apache.spark.sql.execution.CacheManager.uncacheQuery(CacheManager.scala:176) at org.apache.spark.sql.Dataset.unpersist(Dataset.scala:3220) at org.apache.spark.sql.Dataset.unpersist(Dataset.scala:3231) ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing UT. Closes apache#36496 from pan3793/SPARK-39104. Authored-by: Cheng Pan <chengpan@apache.org> Signed-off-by: Sean Owen <srowen@gmail.com> (cherry picked from commit 3c8d8d7) Signed-off-by: Sean Owen <srowen@gmail.com>
- Loading branch information