davseitsev opened a new issue, #11648: URL: https://github.com/apache/iceberg/issues/11648
### Apache Iceberg version 1.7.0 (latest release) ### Query engine Spark ### Please describe the bug 🐞 We have a scheduled Spark maintenance jobs which runs all necessary Spark actions to keep our data lake clean and healthy. Recently we have upgraded iceberg from 1.1.0 to 1.7.0 and enabled deletes compaction. We noticed that data compaction of tables with position deletes is really slow and it blocks almost all other data compaction jobs on the cluster. And here slow means it almost never ends, you can wait a few hours and se zero progress. Investigation showed that there are some threads on Spark workers which consumes all the CPU called `iceberg-delete-worker-pool-%`. Here is flame graph: <img width="1308" alt="image" src="https://github.com/user-attachments/assets/03dc1c61-2773-4699-b43b-a0f8e280cbe1"> Thread dump example: ``` "iceberg-delete-worker-pool-0" #190 daemon prio=5 os_prio=0 cpu=1108299.90ms elapsed=7369.57s tid=0x0000aaaaed08d370 nid=0x37a0 runnable [0x0000ffff2a0da000] java.lang.Thread.State: RUNNABLE at org.apache.iceberg.types.JavaHashes.hashCode(JavaHashes.java:34) at org.apache.iceberg.util.CharSequenceWrapper.hashCode(CharSequenceWrapper.java:85) at java.util.HashMap.hash(java.base@17.0.13/HashMap.java:338) at java.util.HashMap.getNode(java.base@17.0.13/HashMap.java:568) at java.util.HashMap.get(java.base@17.0.13/HashMap.java:556) at org.apache.iceberg.util.CharSequenceMap.get(CharSequenceMap.java:92) at java.util.Map.computeIfAbsent(java.base@17.0.13/Map.java:1052) at org.apache.iceberg.deletes.Deletes.toPositionIndexes(Deletes.java:152) at org.apache.iceberg.data.BaseDeleteLoader.readPosDeletes(BaseDeleteLoader.java:169) at org.apache.iceberg.data.BaseDeleteLoader.lambda$getOrReadPosDeletes$3(BaseDeleteLoader.java:160) at org.apache.iceberg.data.BaseDeleteLoader$$Lambda$3227/0x0000009002085c10.get(Unknown Source) at org.apache.iceberg.spark.SparkExecutorCache.lambda$loadFunc$0(SparkExecutorCache.java:122) at org.apache.iceberg.spark.SparkExecutorCache$$Lambda$3230/0x000000900208f190.apply(Unknown Source) at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.LocalCache.lambda$statsAware$0(LocalCache.java:139) at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.LocalCache$$Lambda$3231/0x000000900208f3d8.apply(Unknown Source) at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.lambda$doComputeIfAbsent$14(BoundedLocalCache.java:2406) at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache$$Lambda$2184/0x0000009001d62478.apply(Unknown Source) at java.util.concurrent.ConcurrentHashMap.compute(java.base@17.0.13/ConcurrentHashMap.java:1916) - locked <0x000000068b0009a0> (a java.util.concurrent.ConcurrentHashMap$ReservationNode) at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.doComputeIfAbsent(BoundedLocalCache.java:2404) at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.computeIfAbsent(BoundedLocalCache.java:2387) 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.spark.SparkExecutorCache.getOrLoad(SparkExecutorCache.java:114) at org.apache.iceberg.spark.source.BaseReader$SparkDeleteFilter$CachingDeleteLoader.getOrLoad(BaseReader.java:297) at org.apache.iceberg.data.BaseDeleteLoader.getOrReadPosDeletes(BaseDeleteLoader.java:160) at org.apache.iceberg.data.BaseDeleteLoader.lambda$loadPositionDeletes$2(BaseDeleteLoader.java:150) at org.apache.iceberg.data.BaseDeleteLoader$$Lambda$3224/0x0000009002084498.apply(Unknown Source) at org.apache.iceberg.data.BaseDeleteLoader.lambda$execute$7(BaseDeleteLoader.java:236) at org.apache.iceberg.data.BaseDeleteLoader$$Lambda$3226/0x0000009002085408.run(Unknown Source) at org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:413) at org.apache.iceberg.util.Tasks$Builder$1.run(Tasks.java:315) at java.util.concurrent.Executors$RunnableAdapter.call(java.base@17.0.13/Executors.java:539) at java.util.concurrent.FutureTask.run(java.base@17.0.13/FutureTask.java:264) at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@17.0.13/ThreadPoolExecutor.java:1136) at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@17.0.13/ThreadPoolExecutor.java:635) at java.lang.Thread.run(java.base@17.0.13/Thread.java:840) ``` After some research I figured out that there is a cache of delete files which loads the whole delete file even if you need position deletes only for single data file. As far as I understand in our case the size of the cache is really small and it's constantly evicted. It dramatically slows down reading deletes. When I turned it off `spark.sql.iceberg.executor-cache.enabled=false`, jobs which ran for a few hours without progress started to finish in about 1 minute. I don't see any benefit from having this cache for RewriteDataFilesSparkAction and I suggest to disable it by default for this action. ### Willingness to contribute - [ ] I can contribute a fix for this bug independently - [ ] I would be willing to contribute a fix for this bug with guidance from the Iceberg community - [ ] I cannot contribute a fix for this bug at this time -- 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: issues-unsubscr...@iceberg.apache.org.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org