ahmedabu98 commented on code in PR #33504:
URL: https://github.com/apache/beam/pull/33504#discussion_r2005255454


##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/TableCache.java:
##########
@@ -17,44 +17,62 @@
  */
 package org.apache.beam.sdk.io.iceberg;
 
+import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
-import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache;
 import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheLoader;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.LoadingCache;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Futures;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ListenableFuture;
 import org.apache.iceberg.Table;
 import org.apache.iceberg.catalog.Catalog;
 import org.apache.iceberg.catalog.TableIdentifier;
-import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Utility to fetch and cache Iceberg {@link Table}s. */
 class TableCache {
-  private static final Cache<TableIdentifier, Table> CACHE =
-      CacheBuilder.newBuilder().expireAfterWrite(3, TimeUnit.MINUTES).build();
+  private static final Map<String, Catalog> CATALOG_CACHE = new 
ConcurrentHashMap<>();
+  private static final LoadingCache<String, Table> INTERNAL_CACHE =
+      CacheBuilder.newBuilder()
+          .expireAfterAccess(1, TimeUnit.HOURS)
+          .refreshAfterWrite(3, TimeUnit.MINUTES)
+          .build(
+              new CacheLoader<String, Table>() {
+                @Override
+                public Table load(String identifier) {
+                  return checkStateNotNull(CATALOG_CACHE.get(identifier))
+                      .loadTable(TableIdentifier.parse(identifier));
+                }
+
+                @Override
+                public ListenableFuture<Table> reload(String unusedIdentifier, 
Table table) {
+                  table.refresh();
+                  return Futures.immediateFuture(table);
+                }
+              });;
 
-  static Table get(TableIdentifier identifier, Catalog catalog) {
+  static Table get(String identifier) {
+    checkStateNotNull(INTERNAL_CACHE, "Please call TableCache.setup() first.");

Review Comment:
   Ah thx I forgot to clean this up (I was flipping between different 
implementations) 



-- 
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]

Reply via email to