Skip to content

KAFKA-2594 Added InMemoryLRUCacheStore#256

Closed
rhauch wants to merge 1 commit into
apache:trunkfrom
rhauch:kafka-2594
Closed

KAFKA-2594 Added InMemoryLRUCacheStore#256
rhauch wants to merge 1 commit into
apache:trunkfrom
rhauch:kafka-2594

Conversation

@rhauch

@rhauch rhauch commented Sep 28, 2015

Copy link
Copy Markdown
Contributor

Added a new KeyValueStore implementation called InMemoryLRUCacheStore that keeps a maximum number of entries in-memory, and as the size exceeds the capacity the least-recently used entry is removed from the store and the backing topic. Also added unit tests for this new store and the existing InMemoryKeyValueStore and RocksDBKeyValueStore implementations. A new KeyValueStoreTestDriver class simplifies all of the other tests, and can be used by other libraries to help test their own custom implementations.

This PR depends upon KAFKA-2593 and its PR at #255. Once that PR is merged, I can rebase this PR if desired.

Two issues were uncovered when creating these new unit tests, and both are also addressed as separate (small) commits in this PR:

  • The RocksDBKeyValueStore initialization was not creating the file system directory if missing.
  • MeteredKeyValueStore was casting to ProcessorContextImpl to access the RecordCollector, which prevent using MeteredKeyValueStore implementations in tests where something other than ProcessorContextImpl was used. The fix was to introduce a RecordCollector.Supplier interface to define this recordCollector() method, and change ProcessorContextImpl and MockProcessorContext to both implement this interface. Now, MeteredKeyValueStore can cast to the new interface to access the record collector rather than to a single concrete implementation, making it possible to use any and all current stores inside unit tests.

@asfbot

asfbot commented Sep 28, 2015

Copy link
Copy Markdown

kafka-trunk-git-pr #586 FAILURE
Looks like there's a problem with this pull request

@asfbot

asfbot commented Sep 29, 2015

Copy link
Copy Markdown

kafka-trunk-git-pr #593 FAILURE
Looks like there's a problem with this pull request

@rhauch

rhauch commented Sep 29, 2015

Copy link
Copy Markdown
Contributor Author

Looks like there's an occasional test failure in core:

542 tests completed, 1 failed
:core:test FAILED

FAILURE: Build failed with an exception.

@rhauch

rhauch commented Oct 1, 2015

Copy link
Copy Markdown
Contributor Author

@guozhangwang, I've updated this PR to involve a single commit that does nothing but add the InMemoryLRUCacheStore. It's based on the updated #255 (which is where the first 4 commits come from), so once that's merged I can rebase this PR.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

@rhauch I am wondering if this class could not benefit from a static Builder class. Something along the lines of

InMemoryLRUCacheStore cacheStore = InMemoryLRUCacheStore.Builder.create()
                                                                .name(name)
                                                                .capacity(capacity)
                                                                .context(context)
                                                                .keySerDe(keySerializer, keyDeserializer)
                                                                .valueSerDe(valueSerializer, valueDeserializer)
                                                                .time(time)
                                                                .build();

OTOH, the constructor arguments are not the same type (say, String), so no good chances of inverting parameter order or so. Wdyt?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Yeah, it'd be nice, but KeyValueStore is parameterized, and that makes builders complicated (but not impossible). See the discussion on #255.

@guozhangwang

Copy link
Copy Markdown
Contributor

@rhauch Do you think the new org.apache.kafka.common.cache.LRUCache can be reused here?

@rhauch

rhauch commented Oct 7, 2015

Copy link
Copy Markdown
Contributor Author

@guozhangwang, the new org.apache.kafka.common.cache.LRUCache class just wraps a LinkedHashMap, whereas we're using one directly. Also, we require a hook into the cache to respond when an entry is removed, and the org.apache.kafka.common.cache.LRUCache doesn't currently provide a way to do that.

@rhauch rhauch force-pushed the kafka-2594 branch 3 times, most recently from d59b2e6 to 7678126 Compare October 7, 2015 20:27
@rhauch

rhauch commented Oct 9, 2015

Copy link
Copy Markdown
Contributor Author

@guozhangwang, rebased this PR.

@guozhangwang

Copy link
Copy Markdown
Contributor

This PR seems still contains some changes from #255, could you double check?

@rhauch

rhauch commented Oct 10, 2015

Copy link
Copy Markdown
Contributor Author

@guozhangwang, yes, it is currently based upon #255, so after #255 is merged I will rebase this so it is a single commit.

Added a new KeyValueStore implementation that keeps a maximum number of entries in-memory, and as the size exceeds the capacity the least-recently used entry is removed from the store and the backing topic. Also added unit tests for this new store.
@rhauch

rhauch commented Oct 15, 2015

Copy link
Copy Markdown
Contributor Author

@guozhangwang: I've rebased and cleaned up this PR, and it now has only one commit that adds the in-memory cache via the new method in the Stores builder. A local build succeeds with this PR.

Creating a limited-size store is nearly identical to creating an unlimited-sized in-memory store. The difference is that the interface returned by inMemory() now has an optional method maxEntries(int) with which the caller can specify the maximum number of entries the store should allow:

    KeyValueStore<Integer, String> store = Stores.create("my-store", driver.context())
                                                 .withIntegerKeys().withStringValues()
                                                 .inMemory()
                                                 .maxEntries(2047)
                                                 .build();

The default maximum number of entries is Integer.MAX_VALUE, and that corresponds to an unlimited in-memory key-value store.

@guozhangwang

Copy link
Copy Markdown
Contributor

I think this is good since for any in-memory stores, we need some general mechanism to bound the memory usage. LRU is a good starting point to do it, but moving forward we may need to 1) bound the capacity based on bytes instead of number of records (we have seen quite many cases where bounding footprint with number of messages brings ops problems with variable record length), and 2) have customizable evicting methods beyond LRU.

@rhauch

rhauch commented Oct 16, 2015

Copy link
Copy Markdown
Contributor Author

@guozhangwang: I completely agree. Perhaps this is a good start, though.

@asfgit asfgit closed this in c553249 Oct 16, 2015
@guozhangwang

Copy link
Copy Markdown
Contributor

Yeah just trying to remind ourselves of generalizing it in the future, this patch LGTM.

@rhauch rhauch deleted the kafka-2594 branch October 27, 2015 15:08
jsancio pushed a commit to jsancio/kafka that referenced this pull request Aug 6, 2019
…ion (apache#256)

Callback handler for Metadata Service to perform LDAP username/password authentication for clients using HTTP Basic authentication method. This uses the Kafka AuthenticateCallbackHandler interface so that it can also be used in future for SASL/PLAIN authentication using LDAP for Kafka clients when custom configs are supported for callback handlers.

To avoid external dependencies, password comparison is performed using an extensible interface. Encrypted password verification based on the implementation in Jetty is included in Metadata Service.

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
0pdd referenced this pull request in objectionary/kafka Aug 28, 2025
0pdd referenced this pull request in objectionary/kafka Aug 28, 2025
0pdd referenced this pull request in objectionary/kafka Sep 1, 2025
0pdd referenced this pull request in objectionary/kafka Sep 1, 2025
0pdd referenced this pull request in objectionary/kafka Sep 15, 2025
0pdd referenced this pull request in objectionary/kafka Sep 15, 2025
0pdd referenced this pull request in objectionary/kafka Sep 15, 2025
0pdd referenced this pull request in objectionary/kafka Sep 15, 2025
0pdd referenced this pull request in objectionary/kafka Sep 15, 2025
0pdd referenced this pull request in objectionary/kafka Sep 16, 2025
0pdd referenced this pull request in objectionary/kafka Sep 16, 2025
0pdd referenced this pull request in objectionary/kafka Sep 16, 2025
0pdd referenced this pull request in objectionary/kafka Sep 16, 2025
0pdd referenced this pull request in objectionary/kafka Sep 16, 2025
0pdd referenced this pull request in objectionary/kafka Sep 17, 2025
0pdd referenced this pull request in objectionary/kafka Sep 17, 2025
0pdd referenced this pull request in objectionary/kafka Sep 17, 2025
0pdd referenced this pull request in objectionary/kafka Sep 17, 2025
0pdd referenced this pull request in objectionary/kafka Sep 17, 2025
0pdd referenced this pull request in objectionary/kafka Sep 19, 2025
0pdd referenced this pull request in objectionary/kafka Sep 22, 2025
0pdd referenced this pull request in objectionary/kafka Sep 22, 2025
0pdd referenced this pull request in objectionary/kafka Sep 23, 2025
0pdd referenced this pull request in objectionary/kafka Sep 23, 2025
0pdd referenced this pull request in objectionary/kafka Sep 23, 2025
0pdd referenced this pull request in objectionary/kafka Sep 23, 2025
0pdd referenced this pull request in objectionary/kafka Sep 23, 2025
0pdd referenced this pull request in objectionary/kafka Sep 23, 2025
0pdd referenced this pull request in objectionary/kafka Sep 23, 2025
0pdd referenced this pull request in objectionary/kafka Sep 23, 2025
0pdd referenced this pull request in objectionary/kafka Sep 23, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants