[ 
https://issues.apache.org/jira/browse/KAFKA-15443?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17906792#comment-17906792
 ] 

Swikar Patel edited comment on KAFKA-15443 at 12/18/24 6:15 PM:
----------------------------------------------------------------

[~cadonna] [~mjsax] 
 * The field *NO_FILE_CLOSES* is removed since rocksdb 8.0.0 version. Currently 
RocksDBMetricsRecorder.java uses 
[https://github.com/apache/kafka/blob/e551cb7bb36f76db7d2fe8105e964aa30d5c53d2/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorder.java#L466]
 * I investigated the reason behind removal of the field and found this issue 
on rocksdb: 
[https://github.com/search?q=repo%3Afacebook%2Frocksdb+NO_FILE_CLOSES&type=issues]

h4. *My Opinions/Suggestion:*
 * The simplest and most robust solution is to {*}remove the line of code that 
uses {{NO_FILE_CLOSES}}{*}. Since there's no direct replacement, trying to 
estimate or derive this value from other metrics might introduce inaccuracies 
or unnecessary complexity.

{code:java}
numberOfOpenFiles += 
valueProviders.statistics.getAndResetTickerCount(TickerType.NO_FILE_OPENS); // 
Only count opens{code}
 * By only counting {{{}NO_FILE_OPENS{}}}, we're now tracking the _total number 
of file opens_ since the last reset. This provides a different perspective but 
still gives us information about file activity. If the number of file opens 
continues to increase without corresponding decreases (which were previously 
tracked by {{{}NO_FILE_CLOSES{}}}), it might indicate a potential issue.
 * *Impact on Monitoring:* Evaluate how the removal of this specific metric 
affects our monitoring dashboards or alerts. If it was crucial for specific 
alerts, we'll need to adjust them or find alternative metrics to monitor.

{color:#0747a6}*{{So what do we want to do in this case? I've asked for any 
replacement of NO_FILE_CLOSES I am waiting for the response from rocksdb 
team.}}*{color}

 


was (Author: JIRAUSER307785):
[~cadonna] [~mjsax] 
 * The field *NO_FILE_CLOSES* is removed since rocksdb 8.0.0 version. Currently 
RocksDBMetricsRecorder.java uses 
[https://github.com/apache/kafka/blob/e551cb7bb36f76db7d2fe8105e964aa30d5c53d2/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorder.java#L466]
 * I investigated the reason behind removal of the field and found this issue 
on rocksdb: 
[https://github.com/search?q=repo%3Afacebook%2Frocksdb+NO_FILE_CLOSES&type=issues]

h4. *My Opinions/Suggestion:*
 * The simplest and most robust solution is to {*}remove the line of code that 
uses {{NO_FILE_CLOSES}}{*}. Since there's no direct replacement, trying to 
estimate or derive this value from other metrics might introduce inaccuracies 
or unnecessary complexity.

{code:java}
numberOfOpenFiles += 
valueProviders.statistics.getAndResetTickerCount(TickerType.NO_FILE_OPENS); // 
Only count opens{code}
 * By only counting {{{}NO_FILE_OPENS{}}}, we're now tracking the _total number 
of file opens_ since the last reset. This provides a different perspective but 
still gives us information about file activity. If the number of file opens 
continues to increase without corresponding decreases (which were previously 
tracked by {{{}NO_FILE_CLOSES{}}}), it might indicate a potential issue.
 * *Impact on Monitoring:* Evaluate how the removal of this specific metric 
affects our monitoring dashboards or alerts. If it was crucial for specific 
alerts, we'll need to adjust them or find alternative metrics to monitor.

*{{So what do we want to do in this case? I've asked for any replacement of 
NO_FILE_CLOSES I am waiting for the response from rocksdb team.}}*

 

> Upgrade RocksDB dependency
> --------------------------
>
>                 Key: KAFKA-15443
>                 URL: https://issues.apache.org/jira/browse/KAFKA-15443
>             Project: Kafka
>          Issue Type: Task
>          Components: streams
>            Reporter: Matthias J. Sax
>            Assignee: Swikar Patel
>            Priority: Blocker
>             Fix For: 4.0.0
>
>         Attachments: compat_report.html
>
>
> Kafka Streams currently depends on RocksDB 7.9.2
> However, the latest version of RocksDB is already 8.5.3. We should check the 
> RocksDB release notes to see what benefits we get to upgrade to the latest 
> version (and file corresponding tickets to exploit improvement of newer 
> releases as applicable).
> From the duplicate ticket KAFKA-18204:
> Kafka still uses rocksdbjni version 7.x (ref: 
> [https://github.com/apache/kafka/blob/trunk/gradle/dependencies.gradle#L120]) 
> which is no longer receiving backports from upstream.
> Please update to rocksdb version 9.x (latest version) so that security 
> updates are received.
> Examples for critical vulnerabilities (CVE score 9.8) in rocksdb version 7.x:
> [https://nvd.nist.gov/vuln/detail/CVE-2023-45853]
> [https://nvd.nist.gov/vuln/detail/CVE-2022-37434]
> (updating to the tip of 8.x release fixes these two vulnerabilities but for 
> any new security fixes, we will need to move to 9.x)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to