Skip to content

Core: fix deadlock in CachingCatalog#3801

Merged
rdblue merged 1 commit intoapache:masterfrom
racevedoo:fix-caching-catalog-deadlock
Dec 29, 2021
Merged

Core: fix deadlock in CachingCatalog#3801
rdblue merged 1 commit intoapache:masterfrom
racevedoo:fix-caching-catalog-deadlock

Conversation

@racevedoo
Copy link
Contributor

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

@racevedoo racevedoo changed the title Core: fixes deadlock in CachingCatalog Core: fix deadlock in CachingCatalog Dec 23, 2021
@racevedoo racevedoo force-pushed the fix-caching-catalog-deadlock branch from f1b04a0 to 59f355e Compare December 23, 2021 16:38
@racevedoo racevedoo force-pushed the fix-caching-catalog-deadlock branch from 59f355e to 76f9667 Compare December 23, 2021 21:32
@racevedoo racevedoo requested a review from rdblue December 23, 2021 21:33
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)));
Copy link
Contributor

@kbendick kbendick Dec 24, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

public Cache<TableIdentifier, Table> cache() {
// cleanUp must be called as tests apply assertions directly on the underlying map, but metadata table
// map entries are cleaned up asynchronously.
tableCache.cleanUp();
return tableCache;
}

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 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. #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!

Copy link
Contributor

@kbendick kbendick Dec 24, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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!

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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!

Copy link

@ben-manes ben-manes Dec 24, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Contributor

@kbendick kbendick Dec 25, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

just updated the code to use Runnable::run 😄

Copy link
Contributor

@kbendick kbendick left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for quickly reporting this issue and then also finding a fix.

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.

Aside from a few small nits, I'm +1. Thank you @racevedoo for reporting the issue and for the quick repro and patch!

@Override
public void delete(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
public void onRemoval(TableIdentifier tableIdentifier, Table table, RemovalCause cause) {
identLoggingRemovalListener.onRemoval(tableIdentifier, table, cause);
Copy link
Contributor

@kbendick kbendick Dec 24, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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));
        }
      }
    }

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

removed identLoggingRemovalListener

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
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

@racevedoo racevedoo force-pushed the fix-caching-catalog-deadlock branch from 76f9667 to 867650c Compare December 24, 2021 09:39
@racevedoo
Copy link
Contributor Author

racevedoo commented Dec 24, 2021

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.

@racevedoo
Copy link
Contributor Author

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

@racevedoo racevedoo force-pushed the fix-caching-catalog-deadlock branch from 867650c to 149b5b2 Compare December 24, 2021 13:41
@racevedoo
Copy link
Contributor Author

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

@racevedoo racevedoo force-pushed the fix-caching-catalog-deadlock branch from 149b5b2 to 2bda14f Compare December 24, 2021 13:52
@racevedoo racevedoo requested a review from kbendick December 24, 2021 13:53
@racevedoo racevedoo force-pushed the fix-caching-catalog-deadlock branch 2 times, most recently from d6a74f3 to de72d8d Compare December 24, 2021 14:07
@racevedoo
Copy link
Contributor Author

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 Core: add test for deadlock in CachingCatalog #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

@kbendick
Copy link
Contributor

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

@racevedoo racevedoo force-pushed the fix-caching-catalog-deadlock branch 2 times, most recently from f8aa34d to 181022c Compare December 25, 2021 12:29
Copy link
Contributor

@kbendick kbendick left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This looks great, thank you so much @racevedoo!

Just a few minor nits.

@kbendick kbendick self-requested a review December 25, 2021 22:02
@racevedoo racevedoo force-pushed the fix-caching-catalog-deadlock branch from 181022c to cca7f51 Compare December 26, 2021 00:38
Copy link
Contributor

@kbendick kbendick left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 this looks great. Thank you @racevedoo for the report and the quick fix. Hugely appreciated, especially during the holidays.

Copy link
Member

@RussellSpitzer RussellSpitzer left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

}

@Test
@Ignore("reproduces https://github.com/apache/iceberg/issues/3791")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Contributor

@kbendick kbendick Dec 28, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

@racevedoo racevedoo force-pushed the fix-caching-catalog-deadlock branch 2 times, most recently from 0d94efd to 1f54cd8 Compare December 27, 2021 17:22
@kbendick
Copy link
Contributor

kbendick commented Dec 28, 2021

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.

@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);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

static declarations are not supported in inner classes :(

HadoopCatalog underlyingCatalog = hadoopCatalog();
TestableCachingCatalog catalog = TestableCachingCatalog.wrap(underlyingCatalog, Duration.ofSeconds(1), ticker);
Namespace namespace = Namespace.of("db", "ns1", "ns2");
int numThreads = 20;
Copy link
Contributor

@kbendick kbendick Dec 28, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Member

@RussellSpitzer RussellSpitzer left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Uses caffeine's `RemovalListener` to expire metadata tables, avoiding modifying cache entries during `compute` HashMap functions (which cause deadlocks).

Also changes caffeine's executor to make `RemovalListener` run sync
For more details, check apache#3791

Fixes apache#3791

Co-authored-by: Kyle Bendickson <kjbendickson@gmail.com>
@racevedoo racevedoo force-pushed the fix-caching-catalog-deadlock branch from 1f54cd8 to 4b22c97 Compare December 28, 2021 18:06
@racevedoo
Copy link
Contributor Author

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 😄

@rdblue rdblue merged commit 63aa349 into apache:master Dec 29, 2021
@rdblue
Copy link
Contributor

rdblue commented Dec 29, 2021

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

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Deadlock when using CachingCatalog with Iceberg 0.13.0-SNAPSHOT

5 participants