cadonna commented on a change in pull request #11250: URL: https://github.com/apache/kafka/pull/11250#discussion_r694622105
########## File path: streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter.java ########## @@ -249,7 +251,7 @@ public int maxFileOpeningThreads() { @Override public Options setMaxTotalWalSize(final long maxTotalWalSize) { - dbOptions.setMaxTotalWalSize(maxTotalWalSize); + LOGGER.warn("WAL is explicitly disabled by Streams in RocksDB. Setting option 'maxTotalWalSize' will be ignored"); Review comment: Could you please add tests that verify the log messages? You can find an example how to verify log messages in `KTableSourceTest#kTableShouldLogAndMeterOnSkippedRecords()`. ########## File path: streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter.java ########## @@ -61,12 +62,13 @@ * * This class do the translation between generic {@link Options} into {@link DBOptions} and {@link ColumnFamilyOptions}. */ -public class RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter extends Options { +class RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter extends Options { Review comment: I think this is not strictly needed since the constructor is already package-private. ########## File path: streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter.java ########## @@ -304,7 +306,7 @@ public String walDir() { @Override public Options setWalDir(final String walDir) { - dbOptions.setWalDir(walDir); + LOGGER.warn("WAL is explicitly disabled by Streams in RocksDB. Setting option 'walDir' will be ignored"); Review comment: A logging helper method sounds reasonable. IMO the return can stay as it is, because we would not save too much. But I do not have strong feelings about it. ########## File path: streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java ########## @@ -178,15 +178,15 @@ void openDB(final Map<String, Object> configs, final File stateDir) { // Setup statistics before the database is opened, otherwise the statistics are not updated // with the measurements from Rocks DB - maybeSetUpStatistics(configs); + setStatisticsIfNeeded(configs); openRocksDB(dbOptions, columnFamilyOptions); open = true; addValueProvidersToMetricsRecorder(); } - private void maybeSetUpStatistics(final Map<String, Object> configs) { + private void setStatisticsIfNeeded(final Map<String, Object> configs) { Review comment: `maybeDoSomething()` is also used in a lot of places. So I agree with @abbccdda. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org