abbccdda commented on a change in pull request #11250: URL: https://github.com/apache/kafka/pull/11250#discussion_r694372189
########## 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 { + + private static final Logger LOGGER = LoggerFactory.getLogger(RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter.class); Review comment: Use `log` as the name of Logger instance here to be consistent with logger usage in general. ########## 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: We could refactor out a helper for the log-and-return behavior with parameter as the name of `option` ########## 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: I don't see why this change is required. -- 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