Sylvain Le Gouellec created KAFKA-13973:
-------------------------------------------
Summary: block-cache-capacity metrics worth twice as much as normal
Key: KAFKA-13973
URL: https://issues.apache.org/jira/browse/KAFKA-13973
Project: Kafka
Issue Type: Bug
Components: streams
Affects Versions: 3.2.0
Reporter: Sylvain Le Gouellec
Attachments: Screenshot 2022-06-09 at 08.55.36.png
I have created a very simple kafka-streams application with 1 state store. I'm
very surprised that the block-cache-capacity metrics show a {{100MB}} block
cache capacity instead of the default one in kafka streams is {{{}50MB{}}}.
My topology :
StreamsBuilder sb = new StreamsBuilder();
sb.stream("input")
.groupByKey()
.count()
.toStream()
.to("output");
I checkout the {{kafka-streams}} code and I saw a strange thing. When the
{{{}RocksDBTimestampedStore{}}}store is created, we try to create two column
families for backward compatibility with a potentiel old key/value store.
In this method, {{setDbAccessor(col1, col2)}} if the first column is not valid,
well you close this one
([L102|https://github.com/apache/kafka/blob/4542acdc14d5ec3daa1f36d8dc24abc244ee24ff/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L102]).
But regarding the rocksdb instance, it's seems that the column families is not
deleted completely and the metrics exposed by [Rocksdb continue to aggregate
(L373)|https://github.com/apache/kafka/blob/4542acdc14d5ec3daa1f36d8dc24abc244ee24ff/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorder.java#L373]
{{block-cache-capacity }}for both column families (default and
keyValueWithTimestamp).
Maybe you have to drop explicitly the column family, in the
{{setDbAccessor(col1, col2)}} if the first column is not valid (like
{{{}db.dropColumnFamily(noTimestampColumnFamily);{}}})
I tried to drop the {{noTimestampColumnFamily in setDbAccessor if the first
column is not valid like : }}
{code:java}
private void setDbAccessor(final ColumnFamilyHandle noTimestampColumnFamily,
final ColumnFamilyHandle withTimestampColumnFamily)
throws RocksDBException {
final RocksIterator noTimestampsIter =
db.newIterator(noTimestampColumnFamily);
noTimestampsIter.seekToFirst();
if (noTimestampsIter.isValid()) {
log.info("Opening store {} in upgrade mode", name);
dbAccessor = new DualColumnFamilyAccessor(noTimestampColumnFamily,
withTimestampColumnFamily);
} else {
log.info("Opening store {} in regular mode", name);
dbAccessor = new SingleColumnFamilyAccessor(withTimestampColumnFamily);
noTimestampColumnFamily.close();
db.dropColumnFamily(noTimestampColumnFamily); // try fix it
}
noTimestampsIter.close();
}{code}
{{But it's seems that you can't drop the default column family in RocksDb (see
screenshot).}}
{{*So how can we have the real block-cache-capacity metrics value in Kafka
Streams monitoring ?* }}
--
This message was sent by Atlassian Jira
(v8.20.7#820007)