racevedoo opened a new issue #3791:
URL: https://github.com/apache/iceberg/issues/3791
Hi all!
When running maintenance procedures with spark sql, namely
`expire_snapshots, rewrite_data_files, rewrite_manifests and
remove_orphan_files`, I'm getting a deadlock within the caffeine cache in
CachingCatalog. This does not happen for every run, but happens for most of
them (specifically when running the `rewrite_data_files` procedure).
The blocked threads and their traces are below:
```
"ForkJoinPool.commonPool-worker-13" #1103 daemon prio=5 os_prio=0 cpu=7.93ms
elapsed=674.73s tid=0x000055c8bb5e9000 nid=0x5a0 waiting for monitor entry
[0x00007f0291ee4000]
java.lang.Thread.State: BLOCKED (on object monitor)
at
java.util.concurrent.ConcurrentHashMap.computeIfPresent([email protected]/ConcurrentHashMap.java:1814)
- waiting to lock <0x00000006c9b9fca0> (a
java.util.concurrent.ConcurrentHashMap$Node)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.remove(BoundedLocalCache.java:2102)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.LocalCache.invalidateAll(LocalCache.java:121)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.LocalManualCache.invalidateAll(LocalManualCache.java:145)
at
org.apache.iceberg.CachingCatalog$MetadataTableInvalidatingCacheWriter.delete(CachingCatalog.java:101)
at
org.apache.iceberg.CachingCatalog$MetadataTableInvalidatingCacheWriter.delete(CachingCatalog.java:92)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.lambda$evictEntry$2(BoundedLocalCache.java:937)
- locked <0x00000006c7576498> (a
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.PDA)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache$$Lambda$4879/0x0000000801e49440.apply(Unknown
Source)
at
java.util.concurrent.ConcurrentHashMap.computeIfPresent([email protected]/ConcurrentHashMap.java:1822)
- locked <0x00000006cf86f1c8> (a
java.util.concurrent.ConcurrentHashMap$Node)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.evictEntry(BoundedLocalCache.java:905)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.expireAfterAccessEntries(BoundedLocalCache.java:818)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.expireAfterAccessEntries(BoundedLocalCache.java:802)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.expireEntries(BoundedLocalCache.java:782)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.maintenance(BoundedLocalCache.java:1479)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.performCleanUp(BoundedLocalCache.java:1448)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache$PerformCleanupTask.run(BoundedLocalCache.java:3293)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache$PerformCleanupTask.exec(BoundedLocalCache.java:3280)
at
java.util.concurrent.ForkJoinTask.doExec([email protected]/ForkJoinTask.java:290)
at
java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec([email protected]/ForkJoinPool.java:1020)
at
java.util.concurrent.ForkJoinPool.scan([email protected]/ForkJoinPool.java:1656)
at
java.util.concurrent.ForkJoinPool.runWorker([email protected]/ForkJoinPool.java:1594)
at
java.util.concurrent.ForkJoinWorkerThread.run([email protected]/ForkJoinWorkerThread.java:183)
Locked ownable synchronizers:
- <0x00000006c5103038> (a
java.util.concurrent.locks.ReentrantLock$NonfairSync)
"HiveServer2-Background-Pool: Thread-1682" #1682 prio=5 os_prio=0
cpu=13.77ms elapsed=412.87s tid=0x00007f02a8034800 nid=0x845 waiting for
monitor entry [0x00007f01ed9ee000]
java.lang.Thread.State: BLOCKED (on object monitor)
at
java.util.concurrent.ConcurrentHashMap.computeIfPresent([email protected]/ConcurrentHashMap.java:1814)
- waiting to lock <0x00000006cf86f1c8> (a
java.util.concurrent.ConcurrentHashMap$Node)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.remove(BoundedLocalCache.java:2102)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.LocalCache.invalidateAll(LocalCache.java:121)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.LocalManualCache.invalidateAll(LocalManualCache.java:145)
at
org.apache.iceberg.CachingCatalog$MetadataTableInvalidatingCacheWriter.delete(CachingCatalog.java:101)
at
org.apache.iceberg.CachingCatalog$MetadataTableInvalidatingCacheWriter.delete(CachingCatalog.java:92)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.lambda$doComputeIfAbsent$14(BoundedLocalCache.java:2368)
- locked <0x00000006c9a0f0d8> (a
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.PDA)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache$$Lambda$2256/0x00000008014aac40.apply(Unknown
Source)
at
java.util.concurrent.ConcurrentHashMap.compute([email protected]/ConcurrentHashMap.java:1932)
- locked <0x00000006c9b9fca0> (a
java.util.concurrent.ConcurrentHashMap$Node)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.doComputeIfAbsent(BoundedLocalCache.java:2342)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.computeIfAbsent(BoundedLocalCache.java:2325)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.LocalCache.computeIfAbsent(LocalCache.java:108)
at
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.LocalManualCache.get(LocalManualCache.java:62)
at
org.apache.iceberg.CachingCatalog.loadTable(CachingCatalog.java:168)
at org.apache.iceberg.spark.SparkCatalog.load(SparkCatalog.java:490)
at
org.apache.iceberg.spark.SparkCatalog.loadTable(SparkCatalog.java:135)
at
org.apache.iceberg.spark.SparkCatalog.loadTable(SparkCatalog.java:92)
at
org.apache.iceberg.spark.procedures.BaseProcedure.loadSparkTable(BaseProcedure.java:115)
at
org.apache.iceberg.spark.procedures.BaseProcedure.execute(BaseProcedure.java:82)
at
org.apache.iceberg.spark.procedures.BaseProcedure.modifyIcebergTable(BaseProcedure.java:74)
at
org.apache.iceberg.spark.procedures.RewriteDataFilesProcedure.call(RewriteDataFilesProcedure.java:94)
at
org.apache.spark.sql.execution.datasources.v2.CallExec.run(CallExec.scala:33)
at
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result$lzycompute(V2CommandExec.scala:43)
- locked <0x00000006d8f18c68> (a
org.apache.spark.sql.execution.datasources.v2.CallExec)
at
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result(V2CommandExec.scala:43)
at
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.executeCollect(V2CommandExec.scala:49)
at
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:110)
at
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1$$Lambda$1933/0x00000008012b3840.apply(Unknown
Source)
at
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
at
org.apache.spark.sql.execution.SQLExecution$$$Lambda$1941/0x00000008012b7440.apply(Unknown
Source)
at
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
at
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
at
org.apache.spark.sql.execution.SQLExecution$$$Lambda$1934/0x00000008012b3c40.apply(Unknown
Source)
at
org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775)
at
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
at
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:110)
at
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:106)
at
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:481)
at
org.apache.spark.sql.catalyst.trees.TreeNode$$Lambda$1923/0x00000008012ad840.apply(Unknown
Source)
at
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:82)
at
org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:481)
at
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30)
at
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
at
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
at
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
at
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
at
org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:457)
at
org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:106)
at
org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:93)
- locked <0x00000006d8f18ea8> (a
org.apache.spark.sql.execution.QueryExecution)
at
org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:91)
at org.apache.spark.sql.Dataset.<init>(Dataset.scala:219)
at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:99)
at
org.apache.spark.sql.Dataset$$$Lambda$1551/0x0000000800f6cc40.apply(Unknown
Source)
at
org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775)
at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:96)
at
org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:618)
at
org.apache.spark.sql.SparkSession$$Lambda$1471/0x0000000800f28840.apply(Unknown
Source)
at
org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775)
at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:613)
at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:651)
at
org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.org$apache$spark$sql$hive$thriftserver$SparkExecuteStatementOperation$$execute(SparkExecuteStatementOperation.scala:291)
at
org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$2$$anon$3.$anonfun$run$2(SparkExecuteStatementOperation.scala:230)
at
org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$2$$anon$3$$Lambda$2166/0x00000008013e5440.apply$mcV$sp(Unknown
Source)
at
scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
at
org.apache.spark.sql.hive.thriftserver.SparkOperation.withLocalProperties(SparkOperation.scala:79)
at
org.apache.spark.sql.hive.thriftserver.SparkOperation.withLocalProperties$(SparkOperation.scala:63)
at
org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.withLocalProperties(SparkExecuteStatementOperation.scala:43)
at
org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$2$$anon$3.run(SparkExecuteStatementOperation.scala:230)
at
org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$2$$anon$3.run(SparkExecuteStatementOperation.scala:225)
at
java.security.AccessController.doPrivileged([email protected]/Native Method)
at
javax.security.auth.Subject.doAs([email protected]/Subject.java:423)
at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1878)
at
org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$2.run(SparkExecuteStatementOperation.scala:239)
at
java.util.concurrent.Executors$RunnableAdapter.call([email protected]/Executors.java:515)
at
java.util.concurrent.FutureTask.run([email protected]/FutureTask.java:264)
at
java.util.concurrent.ThreadPoolExecutor.runWorker([email protected]/ThreadPoolExecutor.java:1128)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run([email protected]/ThreadPoolExecutor.java:628)
at java.lang.Thread.run([email protected]/Thread.java:829)
Locked ownable synchronizers:
- <0x00000006d8f19118> (a
java.util.concurrent.ThreadPoolExecutor$Worker)
```
Full thread dump available
[here](https://gist.github.com/racevedoo/e90a23ffdad3534522773501f47efb04)
From what I'm seeing, this happens when running multiple procedure calls in
a row (sequentially) and for multiple tables. Running a single
`rewrite_data_files` works fine (probably because the cache has not expired
yet).
The workaround on my end was to disable caching.
Environment details: Spark 3.2 and Iceberg 0.13.0-SNAPSHOT
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]