You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/01/31 23:40:22 UTC

[GitHub] [iceberg] kbendick commented on a change in pull request #3977: [Core][Spark][Flink] Change partitioned fanout/delta writers map to caffine cache

kbendick commented on a change in pull request #3977:
URL: https://github.com/apache/iceberg/pull/3977#discussion_r796157999



##########
File path: core/src/main/java/org/apache/iceberg/io/PartitionedFanoutWriter.java
##########
@@ -19,19 +19,52 @@
 
 package org.apache.iceberg.io;
 
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
 import org.apache.iceberg.FileFormat;
 import org.apache.iceberg.PartitionKey;
 import org.apache.iceberg.PartitionSpec;
-import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
 
 public abstract class PartitionedFanoutWriter<T> extends BaseTaskWriter<T> {
-  private final Map<PartitionKey, RollingFileWriter> writers = Maps.newHashMap();
+  private Cache<PartitionKey, RollingFileWriter> writers;
 
   protected PartitionedFanoutWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory<T> appenderFactory,
-                          OutputFileFactory fileFactory, FileIO io, long targetFileSize) {
+                          OutputFileFactory fileFactory, FileIO io,
+                          long targetFileSize, Map<String, String> properties) {
     super(spec, format, appenderFactory, fileFactory, io, targetFileSize);
+    int writersCacheSize = PropertyUtil.propertyAsInt(
+        properties,
+        TableProperties.PARTITIONED_FANOUT_WRITERS_CACHE_SIZE,
+        TableProperties.PARTITIONED_FANOUT_WRITERS_CACHE_SIZE_DEFAULT);
+    long evictionTimeout = PropertyUtil.propertyAsLong(
+        properties,
+        TableProperties.PARTITIONED_FANOUT_WRITERS_CACHE_EVICT_MS,
+        TableProperties.PARTITIONED_FANOUT_WRITERS_CACHE_EVICT_MS_DEFAULT);
+    initWritersCache(writersCacheSize, evictionTimeout);
+  }
+
+  private synchronized void initWritersCache(int writersCacheSize, long evictionTimeout) {
+    if (writers == null) {
+      writers = Caffeine.newBuilder()
+          .maximumSize(writersCacheSize)
+          .expireAfterAccess(evictionTimeout, TimeUnit.MILLISECONDS)
+          .removalListener((key, value, cause) -> {

Review comment:
       This would instantiate a thread pool and evicted items would enter a queue to go in there.
   
   So I believe the removalListener would be run in a background thread instantiated by Cafffeine.




-- 
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-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org