You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/05/10 10:39:21 UTC

[GitHub] [spark] pan3793 opened a new pull request, #36496: [SPARK-39104][SQL] Synchronize isCachedColumnBuffersLoaded to avoid concurrency issue

pan3793 opened a new pull request, #36496:
URL: https://github.com/apache/spark/pull/36496

   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the guideline first in
        'core/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   Add `synchronized` on method `isCachedColumnBuffersLoaded`
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   `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?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   No.
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   Existing UT.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] srowen commented on pull request #36496: [SPARK-39104][SQL] InMemoryRelation#isCachedColumnBuffersLoaded should be thread-safe

Posted by GitBox <gi...@apache.org>.
srowen commented on PR #36496:
URL: https://github.com/apache/spark/pull/36496#issuecomment-1128869583

   Hm, try tests again? I'm having trouble seeing the error. I thought it might be MiMa, because you make a method private, but not sure that is it


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] pan3793 commented on a diff in pull request #36496: [SPARK-39104][SQL] InMemoryRelation#isCachedColumnBuffersLoaded should be thread-safe

Posted by GitBox <gi...@apache.org>.
pan3793 commented on code in PR #36496:
URL: https://github.com/apache/spark/pull/36496#discussion_r874040979


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala:
##########
@@ -563,4 +563,33 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession {
       }
     }
   }
+
+  test("SPARK-39104: InMemoryRelation#isCachedColumnBuffersLoaded should be thread-safe") {

Review Comment:
   I updated the UT and reproduced the issue on local, but I would say it's not easy to reproduce
   ```
   [info] InMemoryColumnarQuerySuite:
   01:58:25.558 WARN org.apache.spark.util.Utils: Your hostname, Chengs-Mac-mini.local resolves to a loopback address: 127.0.0.1; using 10.221.96.10 instead (on interface en1)
   01:58:25.562 WARN org.apache.spark.util.Utils: Set SPARK_LOCAL_IP if you need to bind to another address
   01:58:25.808 WARN org.apache.hadoop.util.NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
   
   Exception: org.scalatest.exceptions.TestFailedException thrown from the UncaughtExceptionHandler in thread "Thread-10"
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] pan3793 commented on a diff in pull request #36496: [SPARK-39104][SQL] Synchronize isCachedColumnBuffersLoaded to avoid concurrency issue

Posted by GitBox <gi...@apache.org>.
pan3793 commented on code in PR #36496:
URL: https://github.com/apache/spark/pull/36496#discussion_r869458318


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala:
##########
@@ -238,7 +238,12 @@ case class CachedRDDBuilder(
   }
 
   def isCachedColumnBuffersLoaded: Boolean = {
-    _cachedColumnBuffers != null && isCachedRDDLoaded
+    if (_cachedColumnBuffers != null) {
+      _cachedColumnBuffers.synchronized {
+        return _cachedColumnBuffers != null && isCachedRDDLoaded
+      }
+    }
+    false
   }
 
   def isCachedRDDLoaded: Boolean = {

Review Comment:
   > Don't we just need a similar synchronized block?
   
   You are right, updated.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #36496: [SPARK-39104][SQL] Synchronize isCachedColumnBuffersLoaded to avoid concurrency issue

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on PR #36496:
URL: https://github.com/apache/spark/pull/36496#issuecomment-1123081531

   Can one of the admins verify this patch?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] pan3793 commented on a diff in pull request #36496: [SPARK-39104][SQL] Synchronize isCachedColumnBuffersLoaded to avoid concurrency issue

Posted by GitBox <gi...@apache.org>.
pan3793 commented on code in PR #36496:
URL: https://github.com/apache/spark/pull/36496#discussion_r869831806


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala:
##########
@@ -238,20 +238,27 @@ case class CachedRDDBuilder(
   }
 
   def isCachedColumnBuffersLoaded: Boolean = {
-    _cachedColumnBuffers != null && isCachedRDDLoaded
+    if (_cachedColumnBuffers != null) {

Review Comment:
   The double-checked locking follows other methods in the class. Yes, it's a perf optimization.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] pan3793 commented on a diff in pull request #36496: [SPARK-39104][SQL] Synchronize isCachedColumnBuffersLoaded to avoid concurrency issue

Posted by GitBox <gi...@apache.org>.
pan3793 commented on code in PR #36496:
URL: https://github.com/apache/spark/pull/36496#discussion_r869459395


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala:
##########
@@ -238,7 +238,12 @@ case class CachedRDDBuilder(
   }
 
   def isCachedColumnBuffersLoaded: Boolean = {
-    _cachedColumnBuffers != null && isCachedRDDLoaded
+    if (_cachedColumnBuffers != null) {
+      _cachedColumnBuffers.synchronized {
+        return _cachedColumnBuffers != null && isCachedRDDLoaded
+      }
+    }
+    false
   }
 
   def isCachedRDDLoaded: Boolean = {

Review Comment:
   > This change is really helpful, is it a good idea to add another synchronized block in `isCacheRDDLoaded` too? It seems `_cachedColumnBuffersAreLoaded` is also volatile and while there is no other call to `isCacheRDDLoaded`, adding that should be safe for another method to call it.
   
   Added, anyway adding a synchronized block is not a bad idea.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] pan3793 commented on a diff in pull request #36496: [SPARK-39104][SQL] InMemoryRelation#isCachedColumnBuffersLoaded should be thread-safe

Posted by GitBox <gi...@apache.org>.
pan3793 commented on code in PR #36496:
URL: https://github.com/apache/spark/pull/36496#discussion_r874423234


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala:
##########
@@ -563,4 +564,51 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession {
       }
     }
   }
+
+  test("SPARK-39104: InMemoryRelation#isCachedColumnBuffersLoaded should be thread-safe") {

Review Comment:
   Thanks, the change makes it easy to reproduce, adopted.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] pan3793 commented on a diff in pull request #36496: [SPARK-39104][SQL] Synchronize isCachedColumnBuffersLoaded to avoid concurrency issue

Posted by GitBox <gi...@apache.org>.
pan3793 commented on code in PR #36496:
URL: https://github.com/apache/spark/pull/36496#discussion_r873803127


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala:
##########
@@ -238,7 +238,12 @@ case class CachedRDDBuilder(
   }
 
   def isCachedColumnBuffersLoaded: Boolean = {
-    _cachedColumnBuffers != null && isCachedRDDLoaded
+    if (_cachedColumnBuffers != null) {
+      _cachedColumnBuffers.synchronized {
+        return _cachedColumnBuffers != null && isCachedRDDLoaded
+      }
+    }
+    false
   }
 
   def isCachedRDDLoaded: Boolean = {

Review Comment:
   After second thought, I think it doesn't make sense to add `synchronized` block in `isCacheRDDLoaded`, becuase we need to check `_cachedColumnBuffers != null` again and that make `isCacheRDDLoaded` as same as `isCachedColumnBuffersLoaded`. 
   Alternatively, I change `isCacheRDDLoaded` to private to avoid accident un-thread-safe invokes.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] pan3793 commented on pull request #36496: [SPARK-39104][SQL] InMemoryRelation#isCachedColumnBuffersLoaded should be thread-safe

Posted by GitBox <gi...@apache.org>.
pan3793 commented on PR #36496:
URL: https://github.com/apache/spark/pull/36496#issuecomment-1128945680

   Two jobs failed, hive slow test failed because of OOM, another is pyspark(not familiar with python), re-triggered


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] srowen closed pull request #36496: [SPARK-39104][SQL] InMemoryRelation#isCachedColumnBuffersLoaded should be thread-safe

Posted by GitBox <gi...@apache.org>.
srowen closed pull request #36496: [SPARK-39104][SQL] InMemoryRelation#isCachedColumnBuffersLoaded should be thread-safe
URL: https://github.com/apache/spark/pull/36496


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] srowen commented on a diff in pull request #36496: [SPARK-39104][SQL] Synchronize isCachedColumnBuffersLoaded to avoid concurrency issue

Posted by GitBox <gi...@apache.org>.
srowen commented on code in PR #36496:
URL: https://github.com/apache/spark/pull/36496#discussion_r869822485


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala:
##########
@@ -238,20 +238,27 @@ case class CachedRDDBuilder(
   }
 
   def isCachedColumnBuffersLoaded: Boolean = {
-    _cachedColumnBuffers != null && isCachedRDDLoaded
+    if (_cachedColumnBuffers != null) {

Review Comment:
   Do you need the double-checked locking anymore? I suppose it's a perf optimization?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] pan3793 commented on pull request #36496: [SPARK-39104][SQL] Synchronize isCachedColumnBuffersLoaded to avoid concurrency issue

Posted by GitBox <gi...@apache.org>.
pan3793 commented on PR #36496:
URL: https://github.com/apache/spark/pull/36496#issuecomment-1127755192

   > Is there a chance to add a new test?
   
   Sorry I missed this comment, added.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] LuciferYang commented on a diff in pull request #36496: [SPARK-39104][SQL] InMemoryRelation#isCachedColumnBuffersLoaded should be thread-safe

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on code in PR #36496:
URL: https://github.com/apache/spark/pull/36496#discussion_r873817057


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala:
##########
@@ -563,4 +563,33 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession {
       }
     }
   }
+
+  test("SPARK-39104: InMemoryRelation#isCachedColumnBuffersLoaded should be thread-safe") {

Review Comment:
   I tested this UT manually, seems it can't reproduce the issue?
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] pan3793 commented on pull request #36496: [SPARK-39104][SQL] InMemoryRelation#isCachedColumnBuffersLoaded should be thread-safe

Posted by GitBox <gi...@apache.org>.
pan3793 commented on PR #36496:
URL: https://github.com/apache/spark/pull/36496#issuecomment-1129007630

   All tests past now https://github.com/pan3793/spark/runs/6471801942?check_suite_focus=true


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] LuciferYang commented on pull request #36496: [SPARK-39104][SQL] Synchronize isCachedColumnBuffersLoaded to avoid concurrency issue

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on PR #36496:
URL: https://github.com/apache/spark/pull/36496#issuecomment-1123738954

   Is there a chance to  add a new test?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] neilagupta commented on a diff in pull request #36496: [SPARK-39104][SQL] Synchronize isCachedColumnBuffersLoaded to avoid concurrency issue

Posted by GitBox <gi...@apache.org>.
neilagupta commented on code in PR #36496:
URL: https://github.com/apache/spark/pull/36496#discussion_r869294184


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala:
##########
@@ -238,7 +238,12 @@ case class CachedRDDBuilder(
   }
 
   def isCachedColumnBuffersLoaded: Boolean = {
-    _cachedColumnBuffers != null && isCachedRDDLoaded
+    if (_cachedColumnBuffers != null) {
+      _cachedColumnBuffers.synchronized {
+        return _cachedColumnBuffers != null && isCachedRDDLoaded
+      }
+    }
+    false
   }
 
   def isCachedRDDLoaded: Boolean = {

Review Comment:
   This change is really helpful, is it a good idea to add another synchronized block in `isCacheRDDLoaded` too? It seems `_cachedColumnBuffersAreLoaded` is also volatile and while there is no other call to `isCacheRDDLoaded`, adding that should be safe for another method to call it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] srowen commented on a diff in pull request #36496: [SPARK-39104][SQL] Synchronize isCachedColumnBuffersLoaded to avoid concurrency issue

Posted by GitBox <gi...@apache.org>.
srowen commented on code in PR #36496:
URL: https://github.com/apache/spark/pull/36496#discussion_r869427441


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala:
##########
@@ -238,7 +238,12 @@ case class CachedRDDBuilder(
   }
 
   def isCachedColumnBuffersLoaded: Boolean = {
-    _cachedColumnBuffers != null && isCachedRDDLoaded
+    if (_cachedColumnBuffers != null) {
+      _cachedColumnBuffers.synchronized {
+        return _cachedColumnBuffers != null && isCachedRDDLoaded
+      }
+    }
+    false
   }
 
   def isCachedRDDLoaded: Boolean = {

Review Comment:
   Wait a sec though, _cachedColumnBuffers is set to null in a synchronized block (not synchronized on the object itself). Don't we just need a similar synchronized block? I'm always worried about holding two different locks in this class, as it can lead to deadlock



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] pan3793 commented on pull request #36496: [SPARK-39104][SQL] InMemoryRelation#isCachedColumnBuffersLoaded should be thread-safe

Posted by GitBox <gi...@apache.org>.
pan3793 commented on PR #36496:
URL: https://github.com/apache/spark/pull/36496#issuecomment-1128885052

   Hmm, let me check the error message first


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] srowen commented on pull request #36496: [SPARK-39104][SQL] InMemoryRelation#isCachedColumnBuffersLoaded should be thread-safe

Posted by GitBox <gi...@apache.org>.
srowen commented on PR #36496:
URL: https://github.com/apache/spark/pull/36496#issuecomment-1129416530

   Merged to master/3.3/3.2


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] LuciferYang commented on a diff in pull request #36496: [SPARK-39104][SQL] InMemoryRelation#isCachedColumnBuffersLoaded should be thread-safe

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on code in PR #36496:
URL: https://github.com/apache/spark/pull/36496#discussion_r874402726


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala:
##########
@@ -563,4 +564,51 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession {
       }
     }
   }
+
+  test("SPARK-39104: InMemoryRelation#isCachedColumnBuffersLoaded should be thread-safe") {

Review Comment:
   @pan3793 I try to change the UT as follow:
   ```
   test("SPARK-39104: InMemoryRelation#isCachedColumnBuffersLoaded should be thread-safe") {
       val plan = spark.range(1).queryExecution.executedPlan
       val serializer = new TestCachedBatchSerializer(true, 1)
       val cachedRDDBuilder = CachedRDDBuilder(serializer, MEMORY_ONLY, plan, None)
   
       @volatile var isCachedColumnBuffersLoaded = false
       @volatile var stopped = false
   
       val th1 = new Thread {
         override def run(): Unit = {
           while (!isCachedColumnBuffersLoaded && !stopped) {
             cachedRDDBuilder.cachedColumnBuffers
             cachedRDDBuilder.clearCache()
           }
         }
       }
   
       val th2 = new Thread {
         override def run(): Unit = {
           while (!isCachedColumnBuffersLoaded && !stopped) {
             isCachedColumnBuffersLoaded = cachedRDDBuilder.isCachedColumnBuffersLoaded
           }
         }
       }
   
       val th3 = new Thread {
         override def run(): Unit = {
           Thread.sleep(3000L)
           stopped = true;
         }
       }
   
       val exceptionCnt = new AtomicInteger
       val exceptionHandler: Thread.UncaughtExceptionHandler = (_: Thread, cause: Throwable) => {
           exceptionCnt.incrementAndGet
           fail(cause)
         }
   
       th1.setUncaughtExceptionHandler(exceptionHandler)
       th2.setUncaughtExceptionHandler(exceptionHandler)
       th1.start()
       th2.start()
       th3.start()
       th1.join()
       th2.join()
       th3.join()
   
       cachedRDDBuilder.clearCache()
   
       assert(exceptionCnt.get == 0)
     }
   ```
   
   then 
   
   ```
   Exception: org.scalatest.exceptions.TestFailedException thrown from the UncaughtExceptionHandler in thread "Thread-16"
   
   
   1 did not equal 0
   ScalaTestFailureLocation: org.apache.spark.sql.execution.columnar.InMemoryColumnarQuerySuite at (InMemoryColumnarQuerySuite.scala:617)
   Expected :0
   Actual   :1
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org