New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-15084: Remove lock contention from RemoteIndexCache #13850
Conversation
7c9686e
to
8877b20
Compare
I'll check it this week. |
I will review this PR in a couple of days. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @divijvaidya for the PR.
In our internal production environments, we ran around remote read requests for 1000+ partitions with bytes out ~500+ MBps per broker and did not see much latencies. It also depends on the cache size and how often the entries are missing the cache. But we observed an impact when there is a degradation in our remote storage(HDFS) clusters. We resolved it in a similar fashion as proposed in this PR by removing the global lock. Remote storages like Cloud object stores may cause longer latencies and encounter this issue.
One way to address is introducing Caffeine's concurrent LFU cache as done in this PR. But it adds a new dependency to the project.
Another way is to change the current LinkedHashMap to a synchronized map and use a slot based lock instead of a global lock. This workaround is less performant than Caffiene but it does not introduce a new dependency.
+1 on using Caffeine as it has a well optimized and reliable concurrent LFU cache. It is used in several other projects built for large scale like Druid, Cassandra, Solr etc. But we need to make sure that it does not bring a lot of other dependencies.
It seems caffeine brings a couple of dependencies like com.google.errorprone:error_prone_annotations
and org.checkerframework:checker-qual
.
@divijvaidya What dependencies does it bring?
fyi, Caffeine only depends on annotations for static analysis. This allows users to catch mistakes easier, e.g. if using a null checker. Traditionally these could be marked as optional dependencies. Unfortunately bugs in scalac, javac, Java modules, etc could cause runtime failures if annotations are not found on the classpath. While that is a spec violation, it’s less risky to include at compile scope and ask users who are adverse to exclude transitives in their build system. |
Thanks for the clarification @ben-manes. |
core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala
Outdated
Show resolved
Hide resolved
Thank you for looking into this @satishd
Yes, you are right, the impact of this lock contention is felt in cases when fetching from RSM is slow. We observed something similar.
I understand what you are saying. I did consider that approach. So that we are on the same page, the approach looks like:
The advantages of this approach are:
The disadvantages of this approach are:
Caffeine on the other hand doesn't have any of the disadvantage mentioned above, is used in high throughput low latency systems like Cassandra and has a constant release cadence with great support. Hence, I believe that using Caffeine is the right choice here. (I know that you already agreed to this but adding more context here for other readers of the PR) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM! Thanks for the improvement! Left some minor comments.
@@ -117,51 +128,60 @@ class RemoteIndexCacheTest { | |||
|
|||
@Test | |||
def testCacheEntryExpiry(): Unit = { | |||
val cache = new RemoteIndexCache(maxSize = 2, rsm, logDir = logDir.toString) | |||
// close existing cache created in test setup before creating a new one | |||
Utils.closeQuietly(cache, "RemoteIndexCache created for unit test") |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If we don't do this at the beginning of the test, will it fail the tests? I thought we've already done it at the AfterEach
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No, it wouldn't fail the test but it will lead to memory & thread leaks. This is because in some tests, we are forcefully assigning a new value to member variable cache
in the test. Hence, we end up with two instances of cache, one which was initialized in @BeforeEach
and another which the test created by itself. If we don't explicitly close the cache which is initialized at the @BeforeEach
, it will keep on consuming heap and have stray threads which won't be released until process is closed.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
OK, got it. Make sense.
core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala
Outdated
Show resolved
Hide resolved
Right, we are on the same page. I talked about the other possible approach without adding a dependency in the same comment mentioning I am not in favour of that because of its cons including less performant than caffeine. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM! Thanks for the improvement!
023ac38
to
9ee624e
Compare
Rebasing from trunk to fix flaky CI test failures. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @divijvaidya for addressing the review comments, LGTM.
Thank you @showuon @satishd for your review so far. I have added few more commits to de-flake some tests, improved thread safety for Requesting one last review cycle from you folks! |
@divijvaidya , sorry, which commits should I check? |
46912c0
to
8513b5f
Compare
I merged together new changes in one commit. It's this one: 8513b5f |
inReadLock(lock) { | ||
val cacheKey = remoteLogSegmentMetadata.remoteLogSegmentId().id() | ||
internalCache.get(cacheKey, (uuid: Uuid) => { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should we use writeLock
here since the get
means
If the specified key is not already associated with a value, attempts to compute its value using the given mapping function and enters it into this cache unless {@code null}. The entire method invocation is performed atomically, so the function is applied at most once per key. Some attempted update operations on this cache by other threads may be blocked while the computation is in progress, so the computation should be short and simple, and must not attempt to update any other mappings of this cache.
So it's actually updating value
https://github.com/ben-manes/caffeine/blob/master/caffeine/src/main/java/com/github/benmanes/caffeine/cache/Cache.java#L60-L65
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, it will update the cache with a new entry. But since the internalCache is thread safe, we don't need to prevent any other thread from read/writing to another entry in the cache, hence, we don't need to acquire a write lock over the entire RemoteIndexCache.
Does this answer your question?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM!
Unrelated test failures:
|
Problem
RemoteIndexCache cache is accessed from multiple threads concurrently in the fetch from consumer code path [1].
Currently, the RemoteIndexCache uses LinkedHashMap as the cache implementation internally. Since LinkedHashMap is not a thread safe data structure, we use coarse grained lock on the entire map/cache when writing to the cache.
This means that if a thread if fetching information from a particular segment from RemoteStorageManager, other threads who are trying to access a different segment from the cache will also wait for the former thread to complete. This is due to the usage of global lock in the cache.
This lock contentions leads to decrease in throughput for fetch from consumer for cases where RSM network call may take more time.
Solution
We need a data structure for the cache which satisfies the following requirements:
In Java, all non concurrent data structures (such as LinkedHashMap) violate condition 2. We can potentially use Concurrent data structures such as ConcurrentHashMap but we will have to implement the LRU eviction ourselves on top of this. OR we can implement a LRU cache from scratch ourselves which satisfy the above constraints.
Alternatively, (approach taken in this PR), we can use Caffeine cache which satisfies all the requirements mentioned above. Caffeine performs better than Google Guava cache [2] and is used by major open source projects such as Cassandra, HBase etc. Hence, it is safe to consider this a stable dependency.
Changes
File
API has been replaces withFiles
API introduced since JDK 7.Testing
[1]
kafka/core/src/main/java/kafka/log/remote/RemoteLogManager.java
Line 747 in dfe050c
[2] https://github.com/ben-manes/caffeine/wiki/Benchmarks