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 2021/12/23 16:29:51 UTC

[GitHub] [iceberg] racevedoo opened a new pull request #3801: Core: fixes deadlock in CachingCatalog

racevedoo opened a new pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801


   Uses caffeine's `RemovalListener` to expire metadata tables asynchronously, avoiding modifying cache entries during `compute` HashMap functions (which cause deadlocks).
   For more details, check #3791
   
   Fixes #3791
   


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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774846264



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -105,14 +103,14 @@ public void delete(TableIdentifier tableIdentifier, Table table, RemovalCause ca
   }
 
   private Cache<TableIdentifier, Table> createTableCache(Ticker ticker) {
+    boolean hasExpiration = expirationIntervalMillis > 0;
     Caffeine<TableIdentifier, Table> cacheBuilder = Caffeine
         .newBuilder()
         .softValues()
-        .removalListener(identLoggingRemovalListener);
+        .removalListener(hasExpiration ? new MetadataTableInvalidatingRemovalListener() : identLoggingRemovalListener);

Review comment:
       Nit: all of the stuff that gets added due to expiration should ideally be inside of the `if (hasExpiration)` block or in the return statement after that section if it's for non-expiring cache (i.e. the `identLoggingRemovalListener` here).
   
   Can you split this line into that block (and the block after it for the `identLoggingRemovalListener`.

##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -86,16 +85,15 @@ protected CachingCatalog(Catalog catalog, boolean caseSensitive, long expiration
   }
 
   /**
-   * CacheWriter class for removing metadata tables when their associated data table is expired
+   * RemovalListener class for removing metadata tables when their associated data table is expired
    * via cache expiration.
+   *
+   * @see com.github.benmanes.caffeine.cache.RemovalListener

Review comment:
       Nit: This `@see` doesn't seem necessary  to me given that we're directly implementing `RemovalListener` on the next line.




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774843656



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       Side Note from my own investigation:
   
   If we're introducing `Awaitility` (which in general I think is a good idea), we should eventually update the rest of these tests to use `Awaitility` and remove the call to `cleanUp` in the `TestableCachingCatalog`, which was added to handle the async expiration.
   
   https://github.com/apache/iceberg/blob/9c3e340cd67699c3d2499762d794ab1bc1ee7f45/core/src/test/java/org/apache/iceberg/TestableCachingCatalog.java#L46-L51
    
   This larger refactor to using `Awaitility` entirely should be done in a separate PR though, as this fix is important and I'd like to get this PR in as soon as possible so people can use the updated snapshot and we'll continue to see if the race condition is fixed.
   
   I was curious why this change was needed so I went looking through the `Caffeine` issues and update notes and found the likely cause. Updating Caffeine to [2.8.5](https://github.com/ben-manes/caffeine/releases/tag/v2.8.5) would make this change not needed. The patch notes reference an issues that says `"Fixed expiration delay for scheduled cleanup"` which is likely brought in from the new / modified write path.
   
   I also have another PR open to upgrade the `caffeine` library version, as there are some important bug fixes for us and since we've been mucking around in here, we might as well upgrade instead of stay behind. https://github.com/apache/iceberg/pull/3803
   
   All that said, I think the use of `Awaitility` here is fine. =)
   
   Others might have different opinions based on a "smallest possible diff" principal, where we'd introduce `Awaitility` in one PR by itself to help people who backport or for keeping the assertion message etc. But I'm cool either way.
   
   Thanks again for all your work on this!




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


[GitHub] [iceberg] racevedoo commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
racevedoo commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774952006



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -105,14 +103,14 @@ public void delete(TableIdentifier tableIdentifier, Table table, RemovalCause ca
   }
 
   private Cache<TableIdentifier, Table> createTableCache(Ticker ticker) {
+    boolean hasExpiration = expirationIntervalMillis > 0;
     Caffeine<TableIdentifier, Table> cacheBuilder = Caffeine
         .newBuilder()
         .softValues()
-        .removalListener(identLoggingRemovalListener);
+        .removalListener(hasExpiration ? new MetadataTableInvalidatingRemovalListener() : identLoggingRemovalListener);

Review comment:
       Sure! Just had to change to `Caffeine<Object, Object>` on cache creation as we don't have the type yet

##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -86,16 +85,15 @@ protected CachingCatalog(Catalog catalog, boolean caseSensitive, long expiration
   }
 
   /**
-   * CacheWriter class for removing metadata tables when their associated data table is expired
+   * RemovalListener class for removing metadata tables when their associated data table is expired
    * via cache expiration.
+   *
+   * @see com.github.benmanes.caffeine.cache.RemovalListener

Review comment:
       removed




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


[GitHub] [iceberg] rdblue commented on pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#issuecomment-1002762363


   Looks good. Thanks for getting this done, @racevedoo!


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


[GitHub] [iceberg] racevedoo commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
racevedoo commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775455833



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -47,10 +44,6 @@
  */
 public class CachingCatalog implements Catalog {
 
-  private static final Logger LOG = LoggerFactory.getLogger(CachingCatalog.class);
-  private static final RemovalListener<TableIdentifier, Table> identLoggingRemovalListener =

Review comment:
       I'll put it back again




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775774386



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -257,6 +264,44 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
             catalog.cache().asMap().containsKey(metadataTable)));
   }
 
+  @Test
+  @Ignore("reproduces https://github.com/apache/iceberg/issues/3791")
+  public void testDeadlock() throws IOException, InterruptedException {
+    HadoopCatalog underlyingCatalog = hadoopCatalog();
+    TestableCachingCatalog catalog = TestableCachingCatalog.wrap(underlyingCatalog, Duration.ofSeconds(1), ticker);
+    Namespace namespace = Namespace.of("db", "ns1", "ns2");
+    int numThreads = 20;

Review comment:
       Would it make sense to use fewer threads, so the space for collision / hitting the deadlock is smaller? And then iterating like you are a few extra times? This way the `Random.nextInt` calls are more likely to collide and we don't have to spawn so many threads.




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


[GitHub] [iceberg] racevedoo commented on pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
racevedoo commented on pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#issuecomment-1002225835


   > I think this looks good to me for the fix, I think we should aim to try to get a more deterministic test, the faking of the hash value seems pretty doable but if that's not possible I am fine with the current state. Let's try to get that test cleaned up a bit and make sure it doesn't leave to much junk behind when it is done.
   
   I changed the test a little to remove the random stuff and cleanup the created tables :smile: 


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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775768849



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -86,16 +82,13 @@ protected CachingCatalog(Catalog catalog, boolean caseSensitive, long expiration
   }
 
   /**
-   * CacheWriter class for removing metadata tables when their associated data table is expired
+   * RemovalListener class for removing metadata tables when their associated data table is expired
    * via cache expiration.
    */
-  class MetadataTableInvalidatingCacheWriter implements CacheWriter<TableIdentifier, Table> {
-    @Override
-    public void write(TableIdentifier tableIdentifier, Table table) {
-    }
-
+  class MetadataTableInvalidatingRemovalListener implements RemovalListener<TableIdentifier, Table> {
     @Override
-    public void delete(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
+    public void onRemoval(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
+      LOG.debug("Evicted {} from the table cache ({})", tableIdentifier, cause);

Review comment:
       Given that this inner class is the only thing that uses the Logger, should we make the call to LoggerFactory.getLogger here instead?




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


[GitHub] [iceberg] ben-manes commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
ben-manes commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775781272



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -257,6 +264,44 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
             catalog.cache().asMap().containsKey(metadataTable)));
   }
 
+  @Test
+  @Ignore("reproduces https://github.com/apache/iceberg/issues/3791")
+  public void testDeadlock() throws IOException, InterruptedException {
+    HadoopCatalog underlyingCatalog = hadoopCatalog();
+    TestableCachingCatalog catalog = TestableCachingCatalog.wrap(underlyingCatalog, Duration.ofSeconds(1), ticker);
+    Namespace namespace = Namespace.of("db", "ns1", "ns2");
+    int numThreads = 20;

Review comment:
       Yes, the key. The map locks on a hashbin so locking multiple in an unpredictable order is a classic deadlock case. I don’t know if a single entry would suffice or if you need threads performing ABA vs BAB to get the desired test case.
   




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775771611



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -257,6 +264,44 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
             catalog.cache().asMap().containsKey(metadataTable)));
   }
 
+  @Test
+  @Ignore("reproduces https://github.com/apache/iceberg/issues/3791")
+  public void testDeadlock() throws IOException, InterruptedException {
+    HadoopCatalog underlyingCatalog = hadoopCatalog();
+    TestableCachingCatalog catalog = TestableCachingCatalog.wrap(underlyingCatalog, Duration.ofSeconds(1), ticker);
+    Namespace namespace = Namespace.of("db", "ns1", "ns2");
+    int numThreads = 20;
+    for (int i = 0; i < numThreads; i++) {
+      TableIdentifier tableIdent = TableIdentifier.of(namespace, "tbl" + i);
+      catalog.createTable(tableIdent, SCHEMA, SPEC, ImmutableMap.of("key", "value"));

Review comment:
       Nit / being good citizens: Please make sure all of these tables get dropped at the end test! Maybe make a list of table names and then you can place a finally block or an `@After` function somehow.
   
   Let me know if you'd like any help in sorting that out structurally etc 🙂




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775091445



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       Great. Our work in the callback is relatively negligible (building a list of maybe at most 10 or so possible keys and calling invalidateAll with that list) and disabling async would be the desired behavior to reduce the possibility of users getting a stale associated value (the ones we’re invalidating in the callback). But it wouldn’t be the end of the world if they did. Really appreciate the input and knowledge sharing! 😊




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774845284



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       Actually, this will balloon up the PR and this is an important fix. I'd say let's move everything in this test suite to awaitility in a separate PR (though the use of awaitlity is fine here). The priority should be on fixing the possible deadlock :) 
   
   That way we can review this change and get it out without having to review the much larger diff that will come from adding `awaitility` to a lot of this file.




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


[GitHub] [iceberg] racevedoo commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
racevedoo commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774785440



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -36,6 +35,7 @@
 import org.apache.iceberg.exceptions.AlreadyExistsException;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.checkerframework.checker.nullness.qual.NonNull;

Review comment:
       done

##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -86,33 +86,32 @@ protected CachingCatalog(Catalog catalog, boolean caseSensitive, long expiration
   }
 
   /**
-   * CacheWriter class for removing metadata tables when their associated data table is expired
+   * RemovalListener class for removing metadata tables when their associated data table is expired
    * via cache expiration.
+   *
+   * @see com.github.benmanes.caffeine.cache.RemovalListener
    */
-  class MetadataTableInvalidatingCacheWriter implements CacheWriter<TableIdentifier, Table> {
+  class MetadataTableInvalidatingRemovalListener implements RemovalListener<TableIdentifier, Table> {
     @Override
-    public void write(TableIdentifier tableIdentifier, Table table) {
-    }
-
-    @Override
-    public void delete(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
+    public void onRemoval(TableIdentifier tableIdentifier, Table table, @NonNull RemovalCause cause) {
+      identLoggingRemovalListener.onRemoval(tableIdentifier, table, cause);
       if (RemovalCause.EXPIRED.equals(cause)) {
-        if (!MetadataTableUtils.hasMetadataTableName(tableIdentifier)) {
+        if (tableIdentifier != null && !MetadataTableUtils.hasMetadataTableName(tableIdentifier)) {

Review comment:
       done




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774844834



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility

Review comment:
       Nit: I don't think this comment is necessary. The use of awaitility on the next line somewhat signifies this.

##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility

Review comment:
       Nit: I don't think this comment is necessary. The use of awaitility on the next line signifies this.




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774852961



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -86,16 +85,15 @@ protected CachingCatalog(Catalog catalog, boolean caseSensitive, long expiration
   }
 
   /**
-   * CacheWriter class for removing metadata tables when their associated data table is expired
+   * RemovalListener class for removing metadata tables when their associated data table is expired
    * via cache expiration.
+   *
+   * @see com.github.benmanes.caffeine.cache.RemovalListener
    */
-  class MetadataTableInvalidatingCacheWriter implements CacheWriter<TableIdentifier, Table> {
+  class MetadataTableInvalidatingRemovalListener implements RemovalListener<TableIdentifier, Table> {
     @Override
-    public void write(TableIdentifier tableIdentifier, Table table) {
-    }
-
-    @Override
-    public void delete(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
+    public void onRemoval(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
+      identLoggingRemovalListener.onRemoval(tableIdentifier, table, cause);

Review comment:
       Nit: Instead of relying on `identLoggingRemovalListener#onRemoval`, as we're already inside of an `onRemoval` function, would it make sense to cut out the mental overhead and just add the log statement here directly? The double `onRemoval` was odd to me on first glance, and is added overhead for the reader.
   
   EDIT: As mentioned elsewhere, `identLoggingRemovalLIstener` is no longer needed (we only added it to log about cache expiration). Realistically, logging has additional overhead and caffeine logs itself as well. Was this specific log crucial to your analysis of the issue? If the log message doesn't seem _critical_, I think the class should look as follows:
   
   ```
     /**
      * RemovalListener class for removing metadata tables when their associated data table is expired
      * via cache expiration.
      */
     class MetadataTableInvalidatingRemovalListener implements RemovalListener<TableIdentifier, Table> {
       @Override
       public void onRemoval(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
         if (RemovalCause.EXPIRED.equals(cause) && !MetadataTableUtils.hasMetadataTableName(tableIdentifier)) {
             tableCache.invalidateAll(metadataTableIdentifiers(tableIdentifier));
           }
         }
       }
   ```




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774843656



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       Question: Since you're introducing `Awaitility`, do you want to try to remove the call to `cleanUp` in the `TestableCachingCatalog` (the `cleanUp` call was added to handle the async expiration).
    https://github.com/apache/iceberg/blob/9c3e340cd67699c3d2499762d794ab1bc1ee7f45/core/src/test/java/org/apache/iceberg/TestableCachingCatalog.java#L46-L51
    
    This could likely be done in a separate PR though.




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774852961



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -86,16 +85,15 @@ protected CachingCatalog(Catalog catalog, boolean caseSensitive, long expiration
   }
 
   /**
-   * CacheWriter class for removing metadata tables when their associated data table is expired
+   * RemovalListener class for removing metadata tables when their associated data table is expired
    * via cache expiration.
+   *
+   * @see com.github.benmanes.caffeine.cache.RemovalListener
    */
-  class MetadataTableInvalidatingCacheWriter implements CacheWriter<TableIdentifier, Table> {
+  class MetadataTableInvalidatingRemovalListener implements RemovalListener<TableIdentifier, Table> {
     @Override
-    public void write(TableIdentifier tableIdentifier, Table table) {
-    }
-
-    @Override
-    public void delete(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
+    public void onRemoval(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
+      identLoggingRemovalListener.onRemoval(tableIdentifier, table, cause);

Review comment:
       Nit: Instead of relying on `identLoggingRemovalListener`, since we're already inside of an `onRemoval` function, would it make sense to cut out the overhead and just add the log statement here if need be?
   
   The double `onRemoval` call looks strange on first glance, and is added overhead for the reader or, if Caffeine changes, a potential source of issue (Caffeine is much more complex than I had pictured).




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


[GitHub] [iceberg] racevedoo removed a comment on pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
racevedoo removed a comment on pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#issuecomment-1000846725


   Actually when using `evictionListener` we don't need awaitility, so I upgraded to caffeine 2.9.x and removed awaitility.


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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774845284



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       Actually, this will balloon up the PR and this is an important fix. I'd say let's move everything in this test suite to awaitility in a separate PR (though the use of awaitlity is fine here).
   
   That way we can review this change and get it out without having to review the much larger diff that will come from adding `awaitility` to a lot of this file.




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774843656



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       Side note from my own investigation into the need for these changes:
   
   If we're introducing `Awaitility` (which in general I think is a good idea), we should eventually update the rest of these tests to use `Awaitility` and remove the call to `cleanUp` in the `TestableCachingCatalog`, which was added to handle the async expiration.
   
   https://github.com/apache/iceberg/blob/9c3e340cd67699c3d2499762d794ab1bc1ee7f45/core/src/test/java/org/apache/iceberg/TestableCachingCatalog.java#L46-L51
    
   This larger refactor to using `Awaitility` entirely should be done in a separate PR though, as this fix is important and I'd like to get this PR in as soon as possible so people can use the updated snapshot and we'll continue to see if the race condition is fixed.
   
   I was curious why this change was needed so I went looking through the `Caffeine` issues and update notes and found the likely cause. Updating Caffeine to [2.8.5](https://github.com/ben-manes/caffeine/releases/tag/v2.8.5) would make this change not needed. The patch notes reference an issues that says `"Fixed expiration delay for scheduled cleanup"` which is likely brought in from the new / modified write path.
   
   I also have another PR open to upgrade the `caffeine` library version, as there are some important bug fixes for us and since we've been mucking around in here, we might as well upgrade instead of stay behind. https://github.com/apache/iceberg/pull/3803
   
   All that said, I think the use of `Awaitility` here is fine. =)
   
   Others might have different opinions based on a "smallest possible diff" principal, where we'd introduce `Awaitility` in one PR by itself to help people who backport or for keeping the assertion message etc. But I'm cool either way.
   
   Thanks again for all your work on this!




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775771611



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -257,6 +264,44 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
             catalog.cache().asMap().containsKey(metadataTable)));
   }
 
+  @Test
+  @Ignore("reproduces https://github.com/apache/iceberg/issues/3791")
+  public void testDeadlock() throws IOException, InterruptedException {
+    HadoopCatalog underlyingCatalog = hadoopCatalog();
+    TestableCachingCatalog catalog = TestableCachingCatalog.wrap(underlyingCatalog, Duration.ofSeconds(1), ticker);
+    Namespace namespace = Namespace.of("db", "ns1", "ns2");
+    int numThreads = 20;
+    for (int i = 0; i < numThreads; i++) {
+      TableIdentifier tableIdent = TableIdentifier.of(namespace, "tbl" + i);
+      catalog.createTable(tableIdent, SCHEMA, SPEC, ImmutableMap.of("key", "value"));

Review comment:
       Nit / being good citizens: Please make sure all of these tables get dropped at the end of the test. Maybe you can iterate over the underlying catalog in a finally block or an `@After` function somehow (if it's not too disruptive to the class).
   
   Unfortunately `DROP NAMESPACE cascade` isn't quite implemented yet.
   
   Let me know if you'd like any help in sorting that out structurally etc 🙂




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775771611



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -257,6 +264,44 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
             catalog.cache().asMap().containsKey(metadataTable)));
   }
 
+  @Test
+  @Ignore("reproduces https://github.com/apache/iceberg/issues/3791")
+  public void testDeadlock() throws IOException, InterruptedException {
+    HadoopCatalog underlyingCatalog = hadoopCatalog();
+    TestableCachingCatalog catalog = TestableCachingCatalog.wrap(underlyingCatalog, Duration.ofSeconds(1), ticker);
+    Namespace namespace = Namespace.of("db", "ns1", "ns2");
+    int numThreads = 20;
+    for (int i = 0; i < numThreads; i++) {
+      TableIdentifier tableIdent = TableIdentifier.of(namespace, "tbl" + i);
+      catalog.createTable(tableIdent, SCHEMA, SPEC, ImmutableMap.of("key", "value"));

Review comment:
       Nit / being good citizens: Please make sure all of these tables get dropped at the end test. Maybe maybe you can iterate over the underlying catalog in a finally block or an `@After` function somehow (if it's not too disruptive to the class).
   
   Let me know if you'd like any help in sorting that out structurally etc 🙂




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775769623



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -257,6 +264,44 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
             catalog.cache().asMap().containsKey(metadataTable)));
   }
 
+  @Test
+  @Ignore("reproduces https://github.com/apache/iceberg/issues/3791")

Review comment:
       If it's just 2 seconds, I'd say enable it. We share infra with all of ASF,  but arguably we have a lot of Spark tests in this repo that should be refactored that are more resource intensive than this. The risk of thread thrashing is likely minimal and we can deal with it if it comes up.




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774843656



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       Side Note: Since you're introducing `Awaitility` (which in general I think is a good idea), we should update the rest of the tests to use `Awaitility` and remove the call to `cleanUp` in the `TestableCachingCatalog`, which was added to handle the async expiration.
   
   https://github.com/apache/iceberg/blob/9c3e340cd67699c3d2499762d794ab1bc1ee7f45/core/src/test/java/org/apache/iceberg/TestableCachingCatalog.java#L46-L51
    
   This larger refactor to using `Awaitility` entirely should be done in a separate PR though, as this fix is important and I'd like to get this PR in as soon as possible so people can use the updated snapshot and we'll continue to see if the race condition is fixed.




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


[GitHub] [iceberg] racevedoo commented on pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
racevedoo commented on pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#issuecomment-1000852750


   Guys, sorry for the lots of force-pushes. The main changes/discoveries from the previous state are:
   - Upgraded to caffeine 2.9.x
   - Added the test present in #3798 (I guess we can close that PR if the test is fine)
   - Kept using `removalListener` since `evictionListener` is synchronous and also causes the deadlock (the test fails)
   - Kept awaitility since removalListener is async


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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774843656



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       Side Note: Since you're introducing `Awaitility` (which in general I think is a good idea), we should update the rest of the tests to use `Awaitility` and remove the call to `cleanUp` in the `TestableCachingCatalog`, which was added to handle the async expiration.
   
   https://github.com/apache/iceberg/blob/9c3e340cd67699c3d2499762d794ab1bc1ee7f45/core/src/test/java/org/apache/iceberg/TestableCachingCatalog.java#L46-L51
    
   This larger refactor to using `Awaitility` entirely should be done in a separate PR though, as this fix is important and I'd like to get this PR in as soon as possible so people can use the updated snapshot and we'll continue to see if the race condition is fixed.
   
   I was curious why this change was needed so I went looking through the `Caffeine` issues and update notes and found the likely cause. Updating Caffeine to [2.8.5](https://github.com/ben-manes/caffeine/releases/tag/v2.8.5) would make this change not needed. The patch notes reference an issues that says `"Fixed expiration delay for scheduled cleanup"` which is likely brought in from the new / modified write path.
   
   I also have another PR open to upgrade the `caffeine` library version, as there are some important bug fixes for us and since we've been mucking around in here, we might as well upgrade instead of stay behind. https://github.com/apache/iceberg/pull/3803
   
   All that said, I think the use of `Awaitility` here is fine. =)
   
   Others might have different opinions based on a "smallest possible diff" principal, where we'd introduce `Awaitility` in one PR by itself to help people who backport, but I'm cool either way.
   
   Thanks again for all your work on this!




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775088227



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       Oh awesome thank you! You're right, after running the tests again several more times on 2.8.5, it didn't fix the issue. I just somehow got lucky and was able to get several successful runs in a row.
   
   Appreciate the input regarding disabling async!




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775339360



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -257,6 +264,44 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
             catalog.cache().asMap().containsKey(metadataTable)));
   }
 
+  @Test
+  @Ignore("reproduces https://github.com/apache/iceberg/issues/3791")

Review comment:
       This test should pass now right, so should we turn this on?

##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -47,10 +44,6 @@
  */
 public class CachingCatalog implements Catalog {
 
-  private static final Logger LOG = LoggerFactory.getLogger(CachingCatalog.class);
-  private static final RemovalListener<TableIdentifier, Table> identLoggingRemovalListener =

Review comment:
       Should we be logging this in the new removal listener?

##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -105,14 +94,14 @@ public void delete(TableIdentifier tableIdentifier, Table table, RemovalCause ca
   }
 
   private Cache<TableIdentifier, Table> createTableCache(Ticker ticker) {
-    Caffeine<TableIdentifier, Table> cacheBuilder = Caffeine
+    Caffeine<Object, Object> cacheBuilder = Caffeine

Review comment:
       Is there a reason we need the generic types here?




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


[GitHub] [iceberg] ben-manes commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
ben-manes commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775775182



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -257,6 +264,44 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
             catalog.cache().asMap().containsKey(metadataTable)));
   }
 
+  @Test
+  @Ignore("reproduces https://github.com/apache/iceberg/issues/3791")
+  public void testDeadlock() throws IOException, InterruptedException {
+    HadoopCatalog underlyingCatalog = hadoopCatalog();
+    TestableCachingCatalog catalog = TestableCachingCatalog.wrap(underlyingCatalog, Duration.ofSeconds(1), ticker);
+    Namespace namespace = Namespace.of("db", "ns1", "ns2");
+    int numThreads = 20;

Review comment:
       Can you hack the hashCode (e.g. make constant) to coerce collisions?




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775764402



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -257,6 +264,44 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
             catalog.cache().asMap().containsKey(metadataTable)));
   }
 
+  @Test
+  @Ignore("reproduces https://github.com/apache/iceberg/issues/3791")

Review comment:
       I figured to turn it off normally since it uses a lot of threads. I was hoping that we might come up with a system of tagging tests as resource intensive etc that we run nightly or something.
   
   I've seen a handful of tests lately that start many threads and that seems not great for CI.
   
   But if we're ok with running this on every push, we can remove the `Ignore`. We could either open an issue for expensive tests that we flag or hold off on that as we might not be there yet.
   
   But tests that spawn 20+ threads make me nervous about increasing oddities in CI like HMS timeouts etc.




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


[GitHub] [iceberg] kbendick edited a comment on pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick edited a comment on pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#issuecomment-1001917591


   > Had a few questions, but in general this looks good to me. I'll need some time after break to really dig into the Caffine library but invalidating cache in the removal listener seems to make more sense to me
   
   Agreed on digging into Caffeine. I've begun doing that as well and plan to upgrade our version after this PR. It is ubiquitous in the rest of our stack too so it's worth knowing well.
   
   But given that the snapshot currently can deadlock under common usage patterns, I'd prefer to ship this and then revisit it if need be. If the deadlock doesn't happen anymore with the snapshot, that will be great.
   
   Then we'll just need to revisit with more knowledge the choice of executor service (possibly making it optionally not synchronous or something). But the deadlock seems important to fix as soon as possible.


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


[GitHub] [iceberg] racevedoo commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
racevedoo commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775181920



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -86,16 +79,13 @@ protected CachingCatalog(Catalog catalog, boolean caseSensitive, long expiration
   }
 
   /**
-   * CacheWriter class for removing metadata tables when their associated data table is expired
+   * RemovalListener class for removing metadata tables when their associated data table is expired
    * via cache expiration.
+   *

Review comment:
       removed

##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -105,14 +95,14 @@ public void delete(TableIdentifier tableIdentifier, Table table, RemovalCause ca
   }
 
   private Cache<TableIdentifier, Table> createTableCache(Ticker ticker) {
-    Caffeine<TableIdentifier, Table> cacheBuilder = Caffeine
+    Caffeine<Object, Object> cacheBuilder = Caffeine
         .newBuilder()
-        .softValues()
-        .removalListener(identLoggingRemovalListener);
+        .softValues();
 
     if (expirationIntervalMillis > 0) {
       return cacheBuilder
-          .writer(new CachingCatalog.MetadataTableInvalidatingCacheWriter())
+          .removalListener(new MetadataTableInvalidatingRemovalListener())
+          .executor(Runnable::run)

Review comment:
       done :D




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


[GitHub] [iceberg] racevedoo commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
racevedoo commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774952207



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -86,16 +85,15 @@ protected CachingCatalog(Catalog catalog, boolean caseSensitive, long expiration
   }
 
   /**
-   * CacheWriter class for removing metadata tables when their associated data table is expired
+   * RemovalListener class for removing metadata tables when their associated data table is expired
    * via cache expiration.
+   *
+   * @see com.github.benmanes.caffeine.cache.RemovalListener
    */
-  class MetadataTableInvalidatingCacheWriter implements CacheWriter<TableIdentifier, Table> {
+  class MetadataTableInvalidatingRemovalListener implements RemovalListener<TableIdentifier, Table> {
     @Override
-    public void write(TableIdentifier tableIdentifier, Table table) {
-    }
-
-    @Override
-    public void delete(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
+    public void onRemoval(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
+      identLoggingRemovalListener.onRemoval(tableIdentifier, table, cause);

Review comment:
       removed `identLoggingRemovalListener`




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


[GitHub] [iceberg] rdblue commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774756168



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -86,33 +86,32 @@ protected CachingCatalog(Catalog catalog, boolean caseSensitive, long expiration
   }
 
   /**
-   * CacheWriter class for removing metadata tables when their associated data table is expired
+   * RemovalListener class for removing metadata tables when their associated data table is expired
    * via cache expiration.
+   *
+   * @see com.github.benmanes.caffeine.cache.RemovalListener
    */
-  class MetadataTableInvalidatingCacheWriter implements CacheWriter<TableIdentifier, Table> {
+  class MetadataTableInvalidatingRemovalListener implements RemovalListener<TableIdentifier, Table> {
     @Override
-    public void write(TableIdentifier tableIdentifier, Table table) {
-    }
-
-    @Override
-    public void delete(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
+    public void onRemoval(TableIdentifier tableIdentifier, Table table, @NonNull RemovalCause cause) {
+      identLoggingRemovalListener.onRemoval(tableIdentifier, table, cause);
       if (RemovalCause.EXPIRED.equals(cause)) {
-        if (!MetadataTableUtils.hasMetadataTableName(tableIdentifier)) {
+        if (tableIdentifier != null && !MetadataTableUtils.hasMetadataTableName(tableIdentifier)) {

Review comment:
       Why would `tableIdentifier` be null?




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


[GitHub] [iceberg] rdblue commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774756414



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -36,6 +35,7 @@
 import org.apache.iceberg.exceptions.AlreadyExistsException;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.checkerframework.checker.nullness.qual.NonNull;

Review comment:
       Can you remove this? I we don't typically use these annotations.




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774878350



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility

Review comment:
       Nit: This comment seems unnecessary, as we're using `awaitility` immediately below.




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


[GitHub] [iceberg] racevedoo commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
racevedoo commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r776021656



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -257,6 +264,44 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
             catalog.cache().asMap().containsKey(metadataTable)));
   }
 
+  @Test
+  @Ignore("reproduces https://github.com/apache/iceberg/issues/3791")
+  public void testDeadlock() throws IOException, InterruptedException {
+    HadoopCatalog underlyingCatalog = hadoopCatalog();
+    TestableCachingCatalog catalog = TestableCachingCatalog.wrap(underlyingCatalog, Duration.ofSeconds(1), ticker);
+    Namespace namespace = Namespace.of("db", "ns1", "ns2");
+    int numThreads = 20;
+    for (int i = 0; i < numThreads; i++) {
+      TableIdentifier tableIdent = TableIdentifier.of(namespace, "tbl" + i);
+      catalog.createTable(tableIdent, SCHEMA, SPEC, ImmutableMap.of("key", "value"));

Review comment:
       done. I simply got the table names and dropped them at the end of this test. Maybe we can add a `@After` function that lists the tables and drops them all, but it's probably better to do it in a separate PR




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


[GitHub] [iceberg] racevedoo commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
racevedoo commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r776020816



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -86,16 +82,13 @@ protected CachingCatalog(Catalog catalog, boolean caseSensitive, long expiration
   }
 
   /**
-   * CacheWriter class for removing metadata tables when their associated data table is expired
+   * RemovalListener class for removing metadata tables when their associated data table is expired
    * via cache expiration.
    */
-  class MetadataTableInvalidatingCacheWriter implements CacheWriter<TableIdentifier, Table> {
-    @Override
-    public void write(TableIdentifier tableIdentifier, Table table) {
-    }
-
+  class MetadataTableInvalidatingRemovalListener implements RemovalListener<TableIdentifier, Table> {
     @Override
-    public void delete(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
+    public void onRemoval(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
+      LOG.debug("Evicted {} from the table cache ({})", tableIdentifier, cause);

Review comment:
       static declarations are not supported in inner classes :(
   




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


[GitHub] [iceberg] rdblue merged pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801


   


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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774843656



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       Side Note: Since you're introducing `Awaitility` (which in general I think is a good idea), we should update the rest of the tests to use `Awaitility` and remove the call to `cleanUp` in the `TestableCachingCatalog`, which was added to handle the async expiration.
   
   https://github.com/apache/iceberg/blob/9c3e340cd67699c3d2499762d794ab1bc1ee7f45/core/src/test/java/org/apache/iceberg/TestableCachingCatalog.java#L46-L51
    
   This larger refactor to using `Awaitility` entirely should be done in a separate PR though, as this fix is important and I'd like to get this PR in as soon as possible so people can use the updated snapshot and we'll continue to see if the race condition is fixed.
   
   I was curious why this change was needed so I went looking through the `Caffeine` issues and update notes and found the likely cause. Updating Caffeine to [2.8.5](https://github.com/ben-manes/caffeine/releases/tag/v2.8.5) would make this change not needed. The patch notes reference an issues that says `"Fixed expiration delay for scheduled cleanup"` which is likely brought in from the new / modified write path.
   
   I also have another PR open to upgrade the `caffeine` library version, as there are some important bug fixes for us and since we've been mucking around in here, we might as well upgrade instead of stay behind. https://github.com/apache/iceberg/pull/3803
   
   All that said, I think the use of `Awaitility` here is fine. =)
   
   Others might have different opinions based on a "smallest possible diff" principal, where we'd introduce `Awaitility` in one PR by itself to help people who backport or for keeping the assertion message etc. But I'm cool either way.
   
   Thanks again for all your work on this!




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774843656



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       Side Note: Since you're introducing `Awaitility` (which in general I think is a good idea), we should update the rest of the tests to use `Awaitility` and remove the call to `cleanUp` in the `TestableCachingCatalog`, which was added to handle the async expiration.
   
   https://github.com/apache/iceberg/blob/9c3e340cd67699c3d2499762d794ab1bc1ee7f45/core/src/test/java/org/apache/iceberg/TestableCachingCatalog.java#L46-L51
    
   This larger refactor to using `Awaitility` entirely should be done in a separate PR though, as this fix is important and I'd like to get this PR in as soon as possible so people can use the updated snapshot and we'll continue to see if the race condition is fixed.
   
   The use of awaitility is fine here though, but I was curious why it was needed so I went looking through the `Caffeine` issues and update notes and found the likely cause. Updating Caffeine to 2.8.5 would make this change not needed.
   
   All that said, I think the use of `Awaitility` here is fine. =)
   
   Thanks again for all your work on this!




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774866772



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility

Review comment:
       If we update the Caffeine library from 2.8.4 to [2.8.5](https://github.com/ben-manes/caffeine/releases/tag/v2.8.5) and make the proposed changes, we won't have to introduce `Awaitility` yet, because the `TestableCachingCatalog` calls `cleanUp` on the cache already. I was wondering why this change was needed and noticed this bug fix in 2.8.5.
   
   ```
   Fixed expiration delay for scheduled cleanup
   ```
   
   But as mentioned, I think we should stick with the `Awaitility` introduction and remove the call to `cleanUp` and depend on `Awaitility` entirely in a separate PR.
   
   I also have another PR open to upgrade the `caffeine` library version, as there are some important bug fixes for us and since we've been mucking around in here, we might as well upgrade instead of stay behind. https://github.com/apache/iceberg/pull/3803




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774898150



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       Thinking on it further, **Can we upgrade Caffeine**  to the next minor version (2.8.5) in this PR, which makes this `Awaitility` introduction unnecessary?
   
   My reasoning is that the bug that causes you to have to use `Awaitility` here, is fixed byy upgrading to Caffeine 2.8.5. This indicates that the write path we're going through _also_ receives the fix and so we should provide that fix to our actual code too (even if we're not explicitly calling `cleanUp` ever, the bug could still affect us).
   
   It's the[ only patch mentioned in the release notes](https://github.com/ben-manes/caffeine/releases/tag/v2.8.5) for 2.8.4 -> 2.8.5
   
   We can and should still convert this to not call `cleanUp` in the `TestableCachingCache`, but I think it's best that the fix from 2.8.5 is also applied to the final code as it's a scheduling bug that could affect our new write path for this cache as well. This way also, we can introduce `Awaitility` in a follow up PR and focus only on this bug.




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


[GitHub] [iceberg] racevedoo edited a comment on pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
racevedoo edited a comment on pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#issuecomment-1000762129


   Oops, I missed some of the comments/edits.
   
   @kbendick from my perspective, the proper fix is indeed upgrading to caffeine 2.9.x and using `evictionListener`. This would get us rid of awaitility and the `cleanUp` calls. I still have to check if the test in #3798 passes though
   
   I'll try to work on this today.


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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775088658



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       We typically don't expect to see a _huge_ amount of cache usage. It's for caching tables that are accessed, and a job usually accesses a table via the catalog one time at the start of the job when query planning happens.
   
   There are cases where they can be accessed several times (as mentioned in the issue), but it's usually a pretty limited number of values and times to use it.
   
   Would it be a terrible idea to use `Caffeine.executor(Runnable::run)` to disable async in the actual production code? Ideally the tables we're removing in the `RemovalListener` would expire synchronously when the main table they reference expires.
   
   Thanks again for your input and the great library @ben-manes!




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


[GitHub] [iceberg] ben-manes commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
ben-manes commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775089305



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       It would be perfectly fine to disable async in production code (Redhat's Infinispan does that, for example). The async is primarily because we don't know the cost of running user-supplied callbacks like the removal listener, which might impact user-facing latencies. The cache's own logic is very fast and strictly uses amortized O(1) algorithms.




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775170448



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -86,16 +79,13 @@ protected CachingCatalog(Catalog catalog, boolean caseSensitive, long expiration
   }
 
   /**
-   * CacheWriter class for removing metadata tables when their associated data table is expired
+   * RemovalListener class for removing metadata tables when their associated data table is expired
    * via cache expiration.
+   *

Review comment:
       Nit: Unneeded extra line.

##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -105,14 +95,14 @@ public void delete(TableIdentifier tableIdentifier, Table table, RemovalCause ca
   }
 
   private Cache<TableIdentifier, Table> createTableCache(Ticker ticker) {
-    Caffeine<TableIdentifier, Table> cacheBuilder = Caffeine
+    Caffeine<Object, Object> cacheBuilder = Caffeine
         .newBuilder()
-        .softValues()
-        .removalListener(identLoggingRemovalListener);
+        .softValues();
 
     if (expirationIntervalMillis > 0) {
       return cacheBuilder
-          .writer(new CachingCatalog.MetadataTableInvalidatingCacheWriter())
+          .removalListener(new MetadataTableInvalidatingRemovalListener())
+          .executor(Runnable::run)

Review comment:
       Can you put a comment inline next to this so we don't forget why we did this in the future?
   
   ```
         return cacheBuilder
             .removalListener(new MetadataTableInvalidatingRemovalListener())
             .executor(Runnable::run)  // Makes the callbacks to removal listener synchronous
   ```




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


[GitHub] [iceberg] racevedoo commented on pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
racevedoo commented on pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#issuecomment-1000844385


   Caffeine upgraded to 2.8.5
   
   > I've been out of office due to the holidays. In addition to the unit test, have you verified that this change actually fixes the issue that you reported (e.g. compile it and use the jar from your local build)? That would be the best form of verification, but given the reproduction test case in #3798, that's not required.
   
   I'm out of office too, so I haven't verified if this change fixes the issue, but I guess the unit test is good enough


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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774843656



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       Side Note: If we're introducing `Awaitility` (which in general I think is a good idea), we should eventually update the rest of these tests to use `Awaitility` and remove the call to `cleanUp` in the `TestableCachingCatalog`, which was added to handle the async expiration.
   
   https://github.com/apache/iceberg/blob/9c3e340cd67699c3d2499762d794ab1bc1ee7f45/core/src/test/java/org/apache/iceberg/TestableCachingCatalog.java#L46-L51
    
   This larger refactor to using `Awaitility` entirely should be done in a separate PR though, as this fix is important and I'd like to get this PR in as soon as possible so people can use the updated snapshot and we'll continue to see if the race condition is fixed.
   
   I was curious why this change was needed so I went looking through the `Caffeine` issues and update notes and found the likely cause. Updating Caffeine to [2.8.5](https://github.com/ben-manes/caffeine/releases/tag/v2.8.5) would make this change not needed. The patch notes reference an issues that says `"Fixed expiration delay for scheduled cleanup"` which is likely brought in from the new / modified write path.
   
   I also have another PR open to upgrade the `caffeine` library version, as there are some important bug fixes for us and since we've been mucking around in here, we might as well upgrade instead of stay behind. https://github.com/apache/iceberg/pull/3803
   
   All that said, I think the use of `Awaitility` here is fine. =)
   
   Others might have different opinions based on a "smallest possible diff" principal, where we'd introduce `Awaitility` in one PR by itself to help people who backport or for keeping the assertion message etc. But I'm cool either way.
   
   Thanks again for all your work on this!




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


[GitHub] [iceberg] racevedoo commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
racevedoo commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775458719



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -47,10 +44,6 @@
  */
 public class CachingCatalog implements Catalog {
 
-  private static final Logger LOG = LoggerFactory.getLogger(CachingCatalog.class);
-  private static final RemovalListener<TableIdentifier, Table> identLoggingRemovalListener =

Review comment:
       done :D




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775769623



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -257,6 +264,44 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
             catalog.cache().asMap().containsKey(metadataTable)));
   }
 
+  @Test
+  @Ignore("reproduces https://github.com/apache/iceberg/issues/3791")

Review comment:
       If it's just 2 seconds, I'd say enable it. We share infra with all of ASF,  but arguably we have a lot of Spark tests that should be refactored that are more resource intensive than this. The risk of thread thrashing is likely minimal and we can deal with it if it comes up.




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775771611



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -257,6 +264,44 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
             catalog.cache().asMap().containsKey(metadataTable)));
   }
 
+  @Test
+  @Ignore("reproduces https://github.com/apache/iceberg/issues/3791")
+  public void testDeadlock() throws IOException, InterruptedException {
+    HadoopCatalog underlyingCatalog = hadoopCatalog();
+    TestableCachingCatalog catalog = TestableCachingCatalog.wrap(underlyingCatalog, Duration.ofSeconds(1), ticker);
+    Namespace namespace = Namespace.of("db", "ns1", "ns2");
+    int numThreads = 20;
+    for (int i = 0; i < numThreads; i++) {
+      TableIdentifier tableIdent = TableIdentifier.of(namespace, "tbl" + i);
+      catalog.createTable(tableIdent, SCHEMA, SPEC, ImmutableMap.of("key", "value"));

Review comment:
       Nit / being good citizens: Please make sure all of these tables get dropped at the end test. Maybe make a list of table names and then you can place a finally block or an `@After` function somehow.
   
   Let me know if you'd like any help in sorting that out structurally etc 🙂




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774898150



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       Thinking on it further, **Can we upgrade Caffeine**  to the next minor version (2.8.5) in this PR, which makes this `Awaitility` introduction unnecessary?
   
   My reasoning is that the bug that causes you to have to use `Awaitility` here, is fixed byy upgrading to Caffeine 2.8.5. This indicates that the write path we're going through _also_ receives the fix and so we should provide that fix to our actual code too (even if we're not explicitly calling `cleanUp` ever, the bug could still affect us).
   
   It's the[ only patch mentioned in the release notes](https://github.com/ben-manes/caffeine/releases/tag/v2.8.5) for 2.8.4 -> 2.8.5
   
   We can and should still convert this to not call `cleanUp` in the `TestableCachingCache`, but I think it's best that the fix from 2.8.5 is also applied to the final code as it's a scheduling bug that could affect our new write path for this cache as well. This way also, we can introduce `Awaitility` in a follow up PR and focus only on this bug.
   
   Also, when we upgrade Caffeine to 2.9.x or 3.0 (same release more or less), we should look into changing this to use the newly added `.evictionListener`, which would make the metadata table drops fully atomic. It might make the `Awaitility` changes not fully needed. But my main thinking is upgrade to caffeine 2.8.5 for now to get the scheduling fix that we know otherwise likely affects us.




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774852961



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -86,16 +85,15 @@ protected CachingCatalog(Catalog catalog, boolean caseSensitive, long expiration
   }
 
   /**
-   * CacheWriter class for removing metadata tables when their associated data table is expired
+   * RemovalListener class for removing metadata tables when their associated data table is expired
    * via cache expiration.
+   *
+   * @see com.github.benmanes.caffeine.cache.RemovalListener
    */
-  class MetadataTableInvalidatingCacheWriter implements CacheWriter<TableIdentifier, Table> {
+  class MetadataTableInvalidatingRemovalListener implements RemovalListener<TableIdentifier, Table> {
     @Override
-    public void write(TableIdentifier tableIdentifier, Table table) {
-    }
-
-    @Override
-    public void delete(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
+    public void onRemoval(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
+      identLoggingRemovalListener.onRemoval(tableIdentifier, table, cause);

Review comment:
       Nit: Instead of relying on `identLoggingRemovalListener#onRemoval`, as we're already inside of an `onRemoval` function, would it make sense to cut out the mental overhead and just add the log statement here directly? The double `onRemoval` was odd to me on first glance, and is added overhead for the reader.
   
   EDIT: As mentioned elsewhere, `identLoggingRemovalLIstener` is no longer needed (we only added it to log about cache expiration). Realistically, logging has additional overhead and caffeine logs itself as well. How helpful to debugging the situation for you was this specific log? If the log message doesn't seem _critical_, I think the class should look as follows and avoid the extra logging that was added when the `MetadataTableInvalidatingCacheWriter` was introduced:
   
   ```
     /**
      * RemovalListener class for removing metadata tables when their associated data table is expired
      * via cache expiration.
      */
     class MetadataTableInvalidatingRemovalListener implements RemovalListener<TableIdentifier, Table> {
       @Override
       public void onRemoval(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
         if (RemovalCause.EXPIRED.equals(cause) && !MetadataTableUtils.hasMetadataTableName(tableIdentifier)) {
             tableCache.invalidateAll(metadataTableIdentifiers(tableIdentifier));
           }
         }
       }
   ```

##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -86,16 +85,15 @@ protected CachingCatalog(Catalog catalog, boolean caseSensitive, long expiration
   }
 
   /**
-   * CacheWriter class for removing metadata tables when their associated data table is expired
+   * RemovalListener class for removing metadata tables when their associated data table is expired
    * via cache expiration.
+   *
+   * @see com.github.benmanes.caffeine.cache.RemovalListener
    */
-  class MetadataTableInvalidatingCacheWriter implements CacheWriter<TableIdentifier, Table> {
+  class MetadataTableInvalidatingRemovalListener implements RemovalListener<TableIdentifier, Table> {
     @Override
-    public void write(TableIdentifier tableIdentifier, Table table) {
-    }
-
-    @Override
-    public void delete(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
+    public void onRemoval(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
+      identLoggingRemovalListener.onRemoval(tableIdentifier, table, cause);

Review comment:
       Nit: Instead of relying on `identLoggingRemovalListener#onRemoval`, as we're already inside of an `onRemoval` function, would it make sense to cut out the mental overhead and just add the log statement here directly? The double `onRemoval` was odd to me on first glance, and is added overhead for the reader.
   
   EDIT: As mentioned elsewhere, `identLoggingRemovalListener` is no longer needed (we only added it to log about cache expiration). Realistically, logging has additional overhead and caffeine logs itself as well. How helpful to debugging the situation for you was this specific log? If the log message doesn't seem _critical_, I think the class should look as follows and avoid the extra logging that was added when the `MetadataTableInvalidatingCacheWriter` was introduced:
   
   ```
     /**
      * RemovalListener class for removing metadata tables when their associated data table is expired
      * via cache expiration.
      */
     class MetadataTableInvalidatingRemovalListener implements RemovalListener<TableIdentifier, Table> {
       @Override
       public void onRemoval(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
         if (RemovalCause.EXPIRED.equals(cause) && !MetadataTableUtils.hasMetadataTableName(tableIdentifier)) {
             tableCache.invalidateAll(metadataTableIdentifiers(tableIdentifier));
           }
         }
       }
   ```




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774866772



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility

Review comment:
       If we update the Caffeine library from 2.8.4 to [2.8.5](https://github.com/ben-manes/caffeine/releases/tag/v2.8.5) and make the proposed changes, we won't have to introduce `Awaitility` yet, because the `TestableCachingCatalog` calls `cleanUp` on the cache already. I was wondering why this change was needed and noticed this bug fix in 2.8.5.
   
   ```
   Fixed expiration delay for scheduled cleanup
   ```
   
   But as mentioned, I think we should stick with the `Awaitility` introduction and remove the call to `cleanUp` and depend on `Awaitility` entirely in a separate PR.
   
   I also have another PR open to upgrade the `caffeine` library version, as there are some important bug fixes for us and since we've been mucking around in here, we might as well upgrade instead of stay behind. https://github.com/apache/iceberg/pull/3803




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774846264



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -105,14 +103,14 @@ public void delete(TableIdentifier tableIdentifier, Table table, RemovalCause ca
   }
 
   private Cache<TableIdentifier, Table> createTableCache(Ticker ticker) {
+    boolean hasExpiration = expirationIntervalMillis > 0;
     Caffeine<TableIdentifier, Table> cacheBuilder = Caffeine
         .newBuilder()
         .softValues()
-        .removalListener(identLoggingRemovalListener);
+        .removalListener(hasExpiration ? new MetadataTableInvalidatingRemovalListener() : identLoggingRemovalListener);

Review comment:
       Nit: all of the stuff that gets added due to expiration should ideally be inside of the `if (hasExpiration)` block and things that are applied only when expiration isn't enabled should ideally be on the return statement just after that block.
   
   Can you move some of this around? Also, there's no reason to have a `removalListener` if cache-expiration isn't enabled. So we can remove the existing `identLoggingRemovalLIstener` entirely.
   
   Like this would be great as it involves fewer changes for people who cherry-pick.
   
   ```java
     private Cache<TableIdentifier, Table> createTableCache(Ticker ticker) {
       Caffeine<TableIdentifier, Table> cacheBuilder = Caffeine
           .newBuilder()
           .softValues();
   
       if (expirationIntervalMillis > 0) {
         return cacheBuilder
             .removalListener(new MetadataTableInvalidatingRemovalListener())
             .expireAfterAccess(Duration.ofMillis(expirationIntervalMillis))
             .ticker(ticker)
             .build();
       }
   
       return cacheBuilder.build();
     }
   ```




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


[GitHub] [iceberg] racevedoo commented on pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
racevedoo commented on pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#issuecomment-1000846725


   Actually when using `evictionListener` we don't need awaitility, so I upgraded to caffeine 2.9.x and removed awaitility.


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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775771611



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -257,6 +264,44 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
             catalog.cache().asMap().containsKey(metadataTable)));
   }
 
+  @Test
+  @Ignore("reproduces https://github.com/apache/iceberg/issues/3791")
+  public void testDeadlock() throws IOException, InterruptedException {
+    HadoopCatalog underlyingCatalog = hadoopCatalog();
+    TestableCachingCatalog catalog = TestableCachingCatalog.wrap(underlyingCatalog, Duration.ofSeconds(1), ticker);
+    Namespace namespace = Namespace.of("db", "ns1", "ns2");
+    int numThreads = 20;
+    for (int i = 0; i < numThreads; i++) {
+      TableIdentifier tableIdent = TableIdentifier.of(namespace, "tbl" + i);
+      catalog.createTable(tableIdent, SCHEMA, SPEC, ImmutableMap.of("key", "value"));

Review comment:
       Nit / being good citizens: Please make sure all of these tables get dropped at the end test. Maybe you can iterate over the underlying catalog in a finally block or an `@After` function somehow (if it's not too disruptive to the class).
   
   Let me know if you'd like any help in sorting that out structurally etc 🙂




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775091445



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       Great. Our work in the callback is relatively negligible (building a list of maybe and disabling async would be the desired behavior. Really appreciate the input and knowledge sharing! 😊




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775763070



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -105,14 +94,14 @@ public void delete(TableIdentifier tableIdentifier, Table table, RemovalCause ca
   }
 
   private Cache<TableIdentifier, Table> createTableCache(Ticker ticker) {
-    Caffeine<TableIdentifier, Table> cacheBuilder = Caffeine
+    Caffeine<Object, Object> cacheBuilder = Caffeine

Review comment:
       So technically it works if you put the types, but intellij does kind of freak out (even if you put the types on the right side and the left).
   
   This is normal, to use `<Object, Object>` until something gives it type definitions (after which there are strict rules about using that chained fluent builder because it stores type info from the responseListener, which doesn't come until later now).
   
   Starting with `<Object, Object>` is very normal with Caffeine's builder.




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775778541



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -257,6 +264,44 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
             catalog.cache().asMap().containsKey(metadataTable)));
   }
 
+  @Test
+  @Ignore("reproduces https://github.com/apache/iceberg/issues/3791")
+  public void testDeadlock() throws IOException, InterruptedException {
+    HadoopCatalog underlyingCatalog = hadoopCatalog();
+    TestableCachingCatalog catalog = TestableCachingCatalog.wrap(underlyingCatalog, Duration.ofSeconds(1), ticker);
+    Namespace namespace = Namespace.of("db", "ns1", "ns2");
+    int numThreads = 20;

Review comment:
       I don't mean to take up too much of your time, but you mean for the cache key's (the TableIdentifier)?
   
   Would it be equivalent to just make one cache entry and then just only operate on that? That would make dropping the table (cache entry) free.




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


[GitHub] [iceberg] racevedoo commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
racevedoo commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r776021934



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -257,6 +264,44 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
             catalog.cache().asMap().containsKey(metadataTable)));
   }
 
+  @Test
+  @Ignore("reproduces https://github.com/apache/iceberg/issues/3791")
+  public void testDeadlock() throws IOException, InterruptedException {
+    HadoopCatalog underlyingCatalog = hadoopCatalog();
+    TestableCachingCatalog catalog = TestableCachingCatalog.wrap(underlyingCatalog, Duration.ofSeconds(1), ticker);
+    Namespace namespace = Namespace.of("db", "ns1", "ns2");
+    int numThreads = 20;

Review comment:
       The thing here is to make the cache miss (from my tests). I changed this a little to get rid of the random stuff, making cache misses more certain.




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


[GitHub] [iceberg] racevedoo commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
racevedoo commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774784839



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -86,33 +86,32 @@ protected CachingCatalog(Catalog catalog, boolean caseSensitive, long expiration
   }
 
   /**
-   * CacheWriter class for removing metadata tables when their associated data table is expired
+   * RemovalListener class for removing metadata tables when their associated data table is expired
    * via cache expiration.
+   *
+   * @see com.github.benmanes.caffeine.cache.RemovalListener
    */
-  class MetadataTableInvalidatingCacheWriter implements CacheWriter<TableIdentifier, Table> {
+  class MetadataTableInvalidatingRemovalListener implements RemovalListener<TableIdentifier, Table> {
     @Override
-    public void write(TableIdentifier tableIdentifier, Table table) {
-    }
-
-    @Override
-    public void delete(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
+    public void onRemoval(TableIdentifier tableIdentifier, Table table, @NonNull RemovalCause cause) {
+      identLoggingRemovalListener.onRemoval(tableIdentifier, table, cause);
       if (RemovalCause.EXPIRED.equals(cause)) {
-        if (!MetadataTableUtils.hasMetadataTableName(tableIdentifier)) {
+        if (tableIdentifier != null && !MetadataTableUtils.hasMetadataTableName(tableIdentifier)) {

Review comment:
       That's when the entry is GC'ed (according to caffeine javadocs:
   ![image](https://user-images.githubusercontent.com/5193063/147292986-d305cc86-7c9e-4c3e-a948-d39bc9b132eb.png)
   
   Since we're dealing only with the `EXPIRED` cause, it's safe. I'll remove this check




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


[GitHub] [iceberg] racevedoo commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
racevedoo commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774784894



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -36,6 +35,7 @@
 import org.apache.iceberg.exceptions.AlreadyExistsException;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.checkerframework.checker.nullness.qual.NonNull;

Review comment:
       sure :smile: 




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


[GitHub] [iceberg] ben-manes commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
ben-manes commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775085494



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       The evictionListener replaces CacheWriter’s delete for an atomic action under the map’s computation. Since your previous code wrote back into the cache I think you would suffer the same problem if switching.
   
   Since you don’t use a scheduler, I don’t see how that bug fix affects you.
   
   You can use Caffeine.executor(Runnable::run) to disable async, which simplifies tests.




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


[GitHub] [iceberg] racevedoo commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
racevedoo commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775136671



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       just updated the code to use `Runnable::run` :smile: 




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774843656



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       Side Note: Since you're introducing `Awaitility` (which in general I think is a good idea), we should update the rest of the tests to use `Awaitility` and remove the call to `cleanUp` in the `TestableCachingCatalog`, which was added to handle the async expiration.
   
   https://github.com/apache/iceberg/blob/9c3e340cd67699c3d2499762d794ab1bc1ee7f45/core/src/test/java/org/apache/iceberg/TestableCachingCatalog.java#L46-L51
    
   This larger refactor to using `Awaitility` entirely should be done in a separate PR though, as this fix is important and I'd like to get this PR in as soon as possible so people can use the updated snapshot and we'll continue to see if the race condition is fixed.
   
   The use of awaitility is fine here though, but I was curious why it was needed so I went looking through the `Caffeine` issues and update notes and found the likely cause. Updating Caffeine to [2.8.5](https://github.com/ben-manes/caffeine/releases/tag/v2.8.5) would make this change not needed.
   
   The version bump says `Fixed expiration delay for scheduled cleanup` which must be why this test started failing with the new write path into the cache. I only mention it as I was very curious why this particular change was needed given the existing calls to `cleanUp`.
   
   I also have another PR open to upgrade the `caffeine` library version, as there are some important bug fixes for us and since we've been mucking around in here, we might as well upgrade instead of stay behind. https://github.com/apache/iceberg/pull/3803
   
   All that said, I think the use of `Awaitility` here is fine. =)
   
   Others might have different opinions based on a "smallest possible diff" principal, where we'd introduce `Awaitility` in one PR by itself to help people who backport, but I'm cool either way.
   
   Thanks again for all your work on this!




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


[GitHub] [iceberg] kbendick commented on pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#issuecomment-1000937447


   Thanks for the follow up @racevedoo! No worries on the force pushes, just be sure to rebase off of master when needed. 
   
   Also, @racevedoo and I are syncing up offline about some of the changes in the tests. Will comment here with more detail later. 🙂


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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774843656



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       Question: Since you're introducing `Awaitility`, do you want to try to remove the call to `cleanUp` in the `TestableCachingCatalog` (the `cleanUp` call was added to handle the async expiration).
    https://github.com/apache/iceberg/blob/9c3e340cd67699c3d2499762d794ab1bc1ee7f45/core/src/test/java/org/apache/iceberg/TestableCachingCatalog.java#L46-L51
    
    This could likely be done in a separate PR though if it gets too big.




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774844834



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility

Review comment:
       Nit: I don't think this comment is necessary. The use of awaitility on the next line somewhat signifies this.
   
   Also, as mentioned, the `TestableCachingCatalog` (which is used here in the tests) already has a comment to that effect.




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


[GitHub] [iceberg] racevedoo commented on pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
racevedoo commented on pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#issuecomment-1000762129


   Oops, I missed some of the comments/edits.
   
   @kbendick from my perspective, the proper fix is indeed upgrading to caffeine 2.9.x and using `evictionListener`. This would get us rid of awaitility and the `cleanUp` calls. I still have to check if the test in #3798 passes though


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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774852961



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -86,16 +85,15 @@ protected CachingCatalog(Catalog catalog, boolean caseSensitive, long expiration
   }
 
   /**
-   * CacheWriter class for removing metadata tables when their associated data table is expired
+   * RemovalListener class for removing metadata tables when their associated data table is expired
    * via cache expiration.
+   *
+   * @see com.github.benmanes.caffeine.cache.RemovalListener
    */
-  class MetadataTableInvalidatingCacheWriter implements CacheWriter<TableIdentifier, Table> {
+  class MetadataTableInvalidatingRemovalListener implements RemovalListener<TableIdentifier, Table> {
     @Override
-    public void write(TableIdentifier tableIdentifier, Table table) {
-    }
-
-    @Override
-    public void delete(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
+    public void onRemoval(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
+      identLoggingRemovalListener.onRemoval(tableIdentifier, table, cause);

Review comment:
       Nit: Instead of relying on `identLoggingRemovalListener`, since we're already inside of an `onRemoval` function, would it make sense to cut out the overhead and just add the log statement here if need be?
   
   The double `onRemoval` call makes me nervous about more potential race conditions etc by looking like a double callback.




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774844834



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility

Review comment:
       Nit: I don't think this comment is necessary. The use of awaitility on the next line signifies this.




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


[GitHub] [iceberg] racevedoo commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
racevedoo commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775455602



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -257,6 +264,44 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
             catalog.cache().asMap().containsKey(metadataTable)));
   }
 
+  @Test
+  @Ignore("reproduces https://github.com/apache/iceberg/issues/3791")

Review comment:
       That's an open question. I think we should enable it, as it runs relatively fast (~2 seconds) and we would be getting the safety of not hitting the deadlock issue again.
   
   I guess @kbendick thinks differently though




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


[GitHub] [iceberg] racevedoo commented on pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
racevedoo commented on pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#issuecomment-1000843016


   Actually `evictionListener` seems to be called only within the `cleanUp` task. I'm not sure if I understood correctly, but I guess we should move forward with the upgrade to caffeine 2.8.5 and keep the `cleanUp` in `cache()` until we investigate this in more detail


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


[GitHub] [iceberg] ben-manes commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
ben-manes commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775089305



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       It would be perfectly fine to disable async in production code (Redhat's Infinispan does that, for example). The async is primarily because we don't know the cost of running user-supplied callbacks like the removal listener, which might impact user-facing latencies if run on the calling threads. The cache's own logic is very fast and strictly uses amortized O(1) algorithms.




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774846264



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -105,14 +103,14 @@ public void delete(TableIdentifier tableIdentifier, Table table, RemovalCause ca
   }
 
   private Cache<TableIdentifier, Table> createTableCache(Ticker ticker) {
+    boolean hasExpiration = expirationIntervalMillis > 0;
     Caffeine<TableIdentifier, Table> cacheBuilder = Caffeine
         .newBuilder()
         .softValues()
-        .removalListener(identLoggingRemovalListener);
+        .removalListener(hasExpiration ? new MetadataTableInvalidatingRemovalListener() : identLoggingRemovalListener);

Review comment:
       Nit: all of the stuff that gets added due to expiration should ideally be inside of the `if (hasExpiration)` block or in the return statement after that section if it's for non-expiring cache (i.e. the `identLoggingRemovalListener` here).
   
   Can you split this line into that block?
   
   ```java
     private Cache<TableIdentifier, Table> createTableCache(Ticker ticker) {
       Caffeine<TableIdentifier, Table> cacheBuilder = Caffeine
           .newBuilder()
           .softValues();
   
       if (expirationIntervalMillis > 0) {
         return cacheBuilder
             .removalListener(new MetadataTableInvalidatingRemovalListener())
             .expireAfterAccess(Duration.ofMillis(expirationIntervalMillis))
             .ticker(ticker)
             .build();
       }
   
       return cacheBuilder.removalListener(identLoggingRemovalListener).build();
     }
   ```




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774845284



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       Actually, this will balloon up the PR and this is an important fix.
   
   I'd say let's move everything in this test suite to `awaitility` and see if we can remove the above call in a separate PR (though the present use of awaitlity is fine). The priority should be on preventing the deadlock from occurring :) 
   
   That way we can review this change and get it out without having to review the much larger diff that will come from adding `awaitility` to a lot of this file.
   
   If you don't want to do the separate PR, let me know and I'll take care of it. I'll leave that up to you. But it should definitely be a separate PR.




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774846264



##########
File path: core/src/main/java/org/apache/iceberg/CachingCatalog.java
##########
@@ -105,14 +103,14 @@ public void delete(TableIdentifier tableIdentifier, Table table, RemovalCause ca
   }
 
   private Cache<TableIdentifier, Table> createTableCache(Ticker ticker) {
+    boolean hasExpiration = expirationIntervalMillis > 0;
     Caffeine<TableIdentifier, Table> cacheBuilder = Caffeine
         .newBuilder()
         .softValues()
-        .removalListener(identLoggingRemovalListener);
+        .removalListener(hasExpiration ? new MetadataTableInvalidatingRemovalListener() : identLoggingRemovalListener);

Review comment:
       Nit: all of the stuff that gets added due to expiration should ideally be inside of the `if (hasExpiration)` block or in the return statement after that section if it's for non-expiring cache (i.e. the `identLoggingRemovalListener` here).
   
   Can you split this line into that block (and the block after it for the `identLoggingRemovalListener`.
   
   This would also remove the need for the local variable `hasExpiration`.




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r774845284



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -252,9 +254,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
     ticker.advance(HALF_OF_EXPIRATION);
     Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent);
 
-    Arrays.stream(metadataTables(tableIdent)).forEach(metadataTable ->
-        Assert.assertFalse("When a data table expires, its metadata tables should expire regardless of age",
-            catalog.cache().asMap().containsKey(metadataTable)));
+    // Removal of metadata tables from cache is async, use awaitility
+    await().untilAsserted(() ->
+        Assertions.assertThat(catalog.cache().asMap()).doesNotContainKeys(metadataTables(tableIdent)));

Review comment:
       Actually, this will balloon up the PR and this is an important fix.
   
   I'd say let's move everything in this test suite to `awaitility` and see if we can remove the above call in a separate PR (though the present use of awaitlity is fine). The priority should be on preventing the deadlock from occurring :) 
   
   That way we can review this change and get it out without having to review the much larger diff that will come from adding `awaitility` to a lot of this file.
   
   If you don't want to do the separate PR, let me know and I'll take care of it. I'll leave that up to you. But it should definitely be a separate PR.




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


[GitHub] [iceberg] kbendick commented on pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#issuecomment-1001917591


   > Had a few questions, but in general this looks good to me. I'll need some time after break to really dig into the Caffine library but invalidating cache in the removal listener seems to make more sense to me
   
   Agreed on digging into Caffeine. I've begun doing that as well and plan to upgrade our version after this PR. It is ubiquitous in the rest of our stack too so it's worth knowing well.
   
   But given that the snapshot currently can deadlock, I'd prefer to ship this and then revisit it if need be. If the deadlock doesn't happen anymore with the snapshot, that will be great.
   
   Then we'll just need to revisit with more knowledge the choice of executor service (possibly making it optionally not synchronous or something). But the deadlock seems important to fix as soon as possible.


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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775771611



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -257,6 +264,44 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
             catalog.cache().asMap().containsKey(metadataTable)));
   }
 
+  @Test
+  @Ignore("reproduces https://github.com/apache/iceberg/issues/3791")
+  public void testDeadlock() throws IOException, InterruptedException {
+    HadoopCatalog underlyingCatalog = hadoopCatalog();
+    TestableCachingCatalog catalog = TestableCachingCatalog.wrap(underlyingCatalog, Duration.ofSeconds(1), ticker);
+    Namespace namespace = Namespace.of("db", "ns1", "ns2");
+    int numThreads = 20;
+    for (int i = 0; i < numThreads; i++) {
+      TableIdentifier tableIdent = TableIdentifier.of(namespace, "tbl" + i);
+      catalog.createTable(tableIdent, SCHEMA, SPEC, ImmutableMap.of("key", "value"));

Review comment:
       Nit / being good citizens: Please make sure all of these tables get dropped at the end test. Maybe you can iterate over the underlying catalog in a finally block or an `@After` function somehow (if it's not too disruptive to the class).
   
   Unfortunately `DROP NAMESPACE cascade` isn't quite implemented yet.
   
   Let me know if you'd like any help in sorting that out structurally etc 🙂




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


[GitHub] [iceberg] kbendick commented on a change in pull request #3801: Core: fix deadlock in CachingCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3801:
URL: https://github.com/apache/iceberg/pull/3801#discussion_r775774386



##########
File path: core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java
##########
@@ -257,6 +264,44 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException
             catalog.cache().asMap().containsKey(metadataTable)));
   }
 
+  @Test
+  @Ignore("reproduces https://github.com/apache/iceberg/issues/3791")
+  public void testDeadlock() throws IOException, InterruptedException {
+    HadoopCatalog underlyingCatalog = hadoopCatalog();
+    TestableCachingCatalog catalog = TestableCachingCatalog.wrap(underlyingCatalog, Duration.ofSeconds(1), ticker);
+    Namespace namespace = Namespace.of("db", "ns1", "ns2");
+    int numThreads = 20;

Review comment:
       Would it make sense to use fewer threads, so the space for collision / hitting the deadlock is smaller? And then iterating like you are a few extra times (so possibly switching to a different fixed threadpool)? This way the `Random.nextInt` calls are more likely to collide and we don't have to spawn so many threads.
   
   Or even just creating two tables and then calling Random.nextInt(2) 20 times, which is highly likely to collide and use the same value twice in a row.




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