kafka-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mj...@apache.org
Subject [kafka] branch trunk updated: MINOR: Require final variables in Streams (#5452)
Date Fri, 03 Aug 2018 20:19:54 GMT
This is an automated email from the ASF dual-hosted git repository.

mjsax pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 3637b2c  MINOR: Require final variables in Streams (#5452)
3637b2c is described below

commit 3637b2c374d48e99e0a7be37605d3c79f2661007
Author: John Roesler <vvcephei@users.noreply.github.com>
AuthorDate: Fri Aug 3 15:19:46 2018 -0500

    MINOR: Require final variables in Streams (#5452)
    
    Reviewers: Bill Bejeck <bill@confluent.io>, Guozhang Wang <guozhang@confluent.io>, Matthias J. Sax <matthias@confluent.io>
---
 checkstyle/checkstyle.xml                          |   6 +
 checkstyle/suppressions.xml                        |   4 +
 .../examples/pageview/JsonPOJODeserializer.java    |   8 +-
 .../examples/pageview/JsonPOJOSerializer.java      |   6 +-
 .../examples/pageview/PageViewTypedDemo.java       |  26 ++--
 .../examples/pageview/PageViewUntypedDemo.java     |  22 +--
 .../kafka/streams/examples/pipe/PipeDemo.java      |   8 +-
 .../examples/temperature/TemperatureDemo.java      |  14 +-
 .../examples/wordcount/WordCountProcessorDemo.java |   3 +-
 .../org/apache/kafka/streams/KafkaStreams.java     |  12 +-
 .../apache/kafka/streams/kstream/Serialized.java   |   4 +-
 .../streams/kstream/SessionWindowedSerializer.java |   4 +-
 .../streams/kstream/internals/AbstractStream.java  |   6 +-
 .../kafka/streams/kstream/internals/Change.java    |   2 +-
 .../kstream/internals/ChangedDeserializer.java     |  12 +-
 .../kstream/internals/ChangedSerializer.java       |  14 +-
 .../internals/GroupedStreamAggregateBuilder.java   |  10 +-
 .../kstream/internals/InternalStreamsBuilder.java  |   8 +-
 .../kstream/internals/KGroupedTableImpl.java       |  16 +--
 .../streams/kstream/internals/KStreamFilter.java   |   4 +-
 .../streams/kstream/internals/KStreamFlatMap.java  |   6 +-
 .../kstream/internals/KStreamFlatMapValues.java    |   4 +-
 .../streams/kstream/internals/KStreamImpl.java     |   4 +-
 .../kstream/internals/KStreamJoinWindow.java       |   6 +-
 .../streams/kstream/internals/KStreamMap.java      |   6 +-
 .../kstream/internals/KStreamPassThrough.java      |   2 +-
 .../kstream/internals/KStreamTransform.java        |  10 +-
 .../kstream/internals/KStreamTransformValues.java  |   2 +-
 .../streams/kstream/internals/KTableFilter.java    |  10 +-
 .../internals/KTableKTableAbstractJoin.java        |   6 +-
 .../kstream/internals/KTableKTableJoinMerger.java  |   2 +-
 .../streams/kstream/internals/KTableMapValues.java |   4 +-
 .../internals/KTableSourceValueGetterSupplier.java |   6 +-
 .../internals/graph/KTableKTableJoinNode.java      |   2 +-
 .../kstream/internals/graph/StreamsGraphNode.java  |   6 +-
 .../kstream/internals/graph/TableSourceNode.java   |   2 +-
 .../streams/processor/DefaultPartitionGrouper.java |  26 ++--
 .../org/apache/kafka/streams/processor/TaskId.java |  28 ++--
 .../kafka/streams/processor/TaskMetadata.java      |   4 +-
 .../kafka/streams/processor/ThreadMetadata.java    |   4 +-
 .../internals/CompositeRestoreListener.java        |   2 +-
 .../internals/GlobalProcessorContextImpl.java      |   2 +-
 .../internals/InternalTopologyBuilder.java         |  12 +-
 .../processor/internals/ProcessorTopology.java     |  16 +--
 .../processor/internals/PunctuationQueue.java      |   4 +-
 .../processor/internals/PunctuationSchedule.java   |   4 +-
 .../streams/processor/internals/QuickUnion.java    |  20 +--
 .../kafka/streams/processor/internals/Stamped.java |  10 +-
 .../streams/processor/internals/StampedRecord.java |   2 +-
 .../processor/internals/StandbyContextImpl.java    |   2 +-
 .../processor/internals/StateDirectory.java        |   8 +-
 .../streams/processor/internals/StateRestorer.java |   4 +-
 .../processor/internals/StreamsMetadataState.java  |  16 +--
 .../internals/StreamsPartitionAssignor.java        |   6 +-
 .../streams/processor/internals/TaskManager.java   |   8 +-
 .../WrappedBatchingStateRestoreCallback.java       |   2 +-
 .../internals/assignment/AssignmentInfo.java       |   2 +-
 .../internals/assignment/StickyTaskAssignor.java   |   8 +-
 .../internals/metrics/StreamsMetricsImpl.java      |   4 +-
 .../kafka/streams/state/QueryableStoreTypes.java   |   2 +-
 .../apache/kafka/streams/state/StateSerdes.java    |   8 +-
 .../AbstractMergedSortedCacheStoreIterator.java    |   4 +-
 .../state/internals/CachingKeyValueStore.java      |   8 +-
 .../state/internals/CachingSessionStore.java       |   4 +-
 .../state/internals/CachingWindowStore.java        |   2 +-
 .../internals/ChangeLoggingKeyValueBytesStore.java |   4 +-
 .../internals/ChangeLoggingWindowBytesStore.java   |   2 +-
 .../internals/CompositeReadOnlyKeyValueStore.java  |  10 +-
 .../state/internals/FilteredCacheIterator.java     |   2 +-
 .../state/internals/GlobalStateStoreProvider.java  |   2 +-
 .../internals/InMemoryKeyValueLoggedStore.java     |  30 ++--
 .../streams/state/internals/MemoryLRUCache.java    |  12 +-
 .../state/internals/MemoryNavigableLRUCache.java   |   6 +-
 .../MergedSortedCacheWindowStoreIterator.java      |   4 +-
 .../state/internals/MeteredSessionStore.java       |   2 +-
 .../streams/state/internals/OffsetCheckpoint.java  |   4 +-
 .../streams/state/internals/OrderedBytes.java      |  10 +-
 .../state/internals/QueryableStoreProvider.java    |   2 +-
 .../internals/RocksDBSegmentedBytesStore.java      |   4 +-
 .../state/internals/RocksDBSessionStore.java       |   2 +-
 .../streams/state/internals/RocksDBStore.java      |   6 +-
 .../kafka/streams/state/internals/Segment.java     |   8 +-
 .../streams/state/internals/SegmentIterator.java   |   4 +-
 .../state/internals/SegmentedCacheFunction.java    |  24 ++--
 .../streams/state/internals/SessionKeySchema.java  |   4 +-
 .../streams/state/internals/StoreChangeLogger.java |   4 +-
 .../internals/StreamThreadStateStoreProvider.java  |   2 +-
 .../kafka/streams/state/internals/ThreadCache.java |  20 +--
 .../internals/WindowStoreIteratorWrapper.java      |   2 +-
 .../state/internals/WrappingStoreProvider.java     |   4 +-
 .../org/apache/kafka/streams/KafkaStreamsTest.java |  16 +--
 .../org/apache/kafka/streams/KeyValueTest.java     |  18 +--
 .../apache/kafka/streams/StreamsConfigTest.java    |   4 +-
 .../org/apache/kafka/streams/TopologyWrapper.java  |   2 +-
 .../integration/AbstractJoinIntegrationTest.java   |   8 +-
 .../integration/AbstractResetIntegrationTest.java  |  36 ++---
 .../FineGrainedAutoResetIntegrationTest.java       |  10 +-
 .../GlobalKTableEOSIntegrationTest.java            |  16 +--
 .../integration/GlobalThreadShutDownOrderTest.java |  16 +--
 .../integration/InternalTopicIntegrationTest.java  |   6 +-
 .../KStreamAggregationDedupIntegrationTest.java    |  26 ++--
 .../KStreamAggregationIntegrationTest.java         |   5 +-
 .../KTableSourceTopicRestartIntegrationTest.java   |   2 +-
 .../PurgeRepartitionTopicIntegrationTest.java      |  18 +--
 .../integration/RegexSourceIntegrationTest.java    |   2 +-
 .../integration/RestoreIntegrationTest.java        |   4 +-
 .../StreamStreamJoinIntegrationTest.java           |   2 +-
 .../StreamTableJoinIntegrationTest.java            |   2 +-
 .../integration/TableTableJoinIntegrationTest.java |  28 ++--
 .../streams/integration/utils/KafkaEmbedded.java   |   4 +-
 .../kafka/streams/kstream/JoinWindowsTest.java     |  14 +-
 .../kstream/SessionWindowedDeserializerTest.java   |   4 +-
 .../kstream/SessionWindowedSerializerTest.java     |   4 +-
 .../kstream/TimeWindowedDeserializerTest.java      |   4 +-
 .../kstream/TimeWindowedSerializerTest.java        |   4 +-
 .../kafka/streams/kstream/TimeWindowsTest.java     |  22 +--
 .../streams/kstream/UnlimitedWindowsTest.java      |  16 +--
 .../apache/kafka/streams/kstream/WindowsTest.java  |   2 +-
 .../kstream/internals/AbstractStreamTest.java      |  14 +-
 .../internals/InternalStreamsBuilderTest.java      |   4 +-
 .../kstream/internals/KGroupedTableImplTest.java   |   6 +-
 .../kstream/internals/KStreamBranchTest.java       |  26 ++--
 .../kstream/internals/KStreamFilterTest.java       |  22 +--
 .../kstream/internals/KStreamFlatMapTest.java      |  16 +--
 .../internals/KStreamFlatMapValuesTest.java        |  18 +--
 .../kstream/internals/KStreamForeachTest.java      |  22 +--
 .../streams/kstream/internals/KStreamImplTest.java |  48 +++----
 .../kstream/internals/KStreamKStreamJoinTest.java  |  92 ++++++------
 .../internals/KStreamKStreamLeftJoinTest.java      |  24 ++--
 .../streams/kstream/internals/KStreamMapTest.java  |  18 +--
 .../kstream/internals/KStreamMapValuesTest.java    |  22 +--
 .../streams/kstream/internals/KStreamPeekTest.java |   2 +-
 .../kstream/internals/KStreamPrintTest.java        |   4 +-
 .../kstream/internals/KStreamSelectKeyTest.java    |  16 +--
 .../kstream/internals/KStreamTransformTest.java    |  24 ++--
 .../internals/KStreamTransformValuesTest.java      |  24 ++--
 .../kstream/internals/KTableAggregateTest.java     |  55 ++++----
 .../kstream/internals/KTableFilterTest.java        | 124 ++++++++--------
 .../streams/kstream/internals/KTableImplTest.java  |  26 ++--
 .../kstream/internals/KTableMapKeysTest.java       |  12 +-
 .../kstream/internals/KTableMapValuesTest.java     |  24 ++--
 .../internals/KTableTransformValuesTest.java       |   4 +-
 .../internals/WindowedStreamPartitionerTest.java   |  16 +--
 .../kstream/internals/graph/StreamsGraphTest.java  |   2 +-
 .../apache/kafka/streams/perf/SimpleBenchmark.java |  30 ++--
 .../apache/kafka/streams/perf/YahooBenchmark.java  |  58 ++++----
 .../streams/processor/TimestampExtractorTest.java  |   2 +-
 .../processor/WallclockTimestampExtractorTest.java |   8 +-
 .../internals/AbstractProcessorContextTest.java    |   4 +-
 .../internals/AssignedStreamsTasksTest.java        |   4 +-
 .../internals/CompositeRestoreListenerTest.java    |   8 +-
 .../internals/GlobalStateManagerImplTest.java      |  10 +-
 .../internals/GlobalStreamThreadTest.java          |   4 +-
 .../internals/InternalTopicManagerTest.java        |   8 +-
 .../processor/internals/ProcessorTopologyTest.java |   6 +-
 .../processor/internals/PunctuationQueueTest.java  |  16 +--
 .../processor/internals/QuickUnionTest.java        |  18 +--
 .../internals/SingleGroupPartitionGrouperStub.java |   8 +-
 .../processor/internals/StreamThreadTest.java      |  12 +-
 .../internals/StreamsMetadataStateTest.java        |   6 +-
 .../processor/internals/TaskManagerTest.java       |  20 +--
 .../WrappedBatchingStateRestoreCallbackTest.java   |   2 +-
 .../kafka/streams/state/NoOpWindowStore.java       |   4 +-
 .../kafka/streams/state/StateSerdesTest.java       |  12 +-
 .../state/internals/AbstractKeyValueStoreTest.java |   4 +-
 .../state/internals/CachingKeyValueStoreTest.java  |  14 +-
 .../ChangeLoggingKeyValueBytesStoreTest.java       |  14 +-
 .../ChangeLoggingSessionBytesStoreTest.java        |  14 +-
 .../ChangeLoggingWindowBytesStoreTest.java         |  14 +-
 .../CompositeReadOnlyKeyValueStoreTest.java        |  16 +--
 .../CompositeReadOnlySessionStoreTest.java         |   2 +-
 .../CompositeReadOnlyWindowStoreTest.java          |   8 +-
 .../DelegatingPeekingKeyValueIteratorTest.java     |   2 +-
 .../state/internals/FilteredCacheIteratorTest.java |   4 +-
 .../state/internals/InMemoryLRUCacheStoreTest.java |   4 +-
 ...dSortedCacheKeyValueBytesStoreIteratorTest.java |  14 +-
 ...SortedCacheWrappedSessionStoreIteratorTest.java |   2 +-
 ...dSortedCacheWrappedWindowStoreIteratorTest.java |   2 +-
 ...acheWrappedWindowStoreKeyValueIteratorTest.java |   2 +-
 .../streams/state/internals/NamedCacheTest.java    |  10 +-
 .../state/internals/OffsetCheckpointTest.java      |   2 +-
 .../state/internals/ReadOnlyWindowStoreStub.java   |  16 +--
 .../state/internals/RocksDBSessionStoreTest.java   |   2 +-
 .../streams/state/internals/RocksDBStoreTest.java  |  22 +--
 .../internals/SegmentedCacheFunctionTest.java      |   2 +-
 .../state/internals/SessionKeySchemaTest.java      |  10 +-
 .../streams/state/internals/ThreadCacheTest.java   | 109 +++++++-------
 .../state/internals/WindowKeySchemaTest.java       |  14 +-
 .../state/internals/WrappingStoreProviderTest.java |   2 +-
 .../streams/tests/BrokerCompatibilityTest.java     |   4 +-
 .../apache/kafka/streams/tests/EosTestClient.java  |   2 +-
 .../kafka/streams/tests/ShutdownDeadlockTest.java  |   2 +-
 .../kafka/streams/tests/SmokeTestClient.java       |   2 +-
 .../kafka/streams/tests/SmokeTestDriver.java       | 156 ++++++++++-----------
 .../tests/StreamsBrokerDownResilienceTest.java     |  18 +--
 .../kafka/streams/tests/StreamsSmokeTest.java      |   2 +-
 .../streams/tests/StreamsStandByReplicaTest.java   |   4 +-
 .../kafka/streams/tools/StreamsResetterTest.java   |  10 +-
 .../apache/kafka/test/GlobalStateManagerStub.java  |   2 +-
 .../kafka/test/InternalMockProcessorContext.java   |  10 +-
 .../apache/kafka/test/KTableValueGetterStub.java   |   2 +-
 .../java/org/apache/kafka/test/MockAggregator.java |   2 +-
 .../kafka/test/MockInternalTopicManager.java       |   2 +-
 .../java/org/apache/kafka/test/MockMapper.java     |  10 +-
 .../org/apache/kafka/test/MockProcessorNode.java   |   6 +-
 .../apache/kafka/test/MockProcessorSupplier.java   |   4 +-
 .../java/org/apache/kafka/test/MockReducer.java    |   4 +-
 .../org/apache/kafka/test/MockRestoreConsumer.java |  24 ++--
 .../java/org/apache/kafka/test/MockSourceNode.java |   4 +-
 .../kafka/test/MockStateRestoreListener.java       |   2 +-
 .../org/apache/kafka/test/MockValueJoiner.java     |   2 +-
 .../apache/kafka/test/NoOpProcessorContext.java    |   2 +-
 .../kafka/test/ReadOnlySessionStoreStub.java       |   2 +-
 .../apache/kafka/test/SegmentedBytesStoreStub.java |   4 +-
 214 files changed, 1229 insertions(+), 1221 deletions(-)

diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml
index ad85450..6eb1a82 100644
--- a/checkstyle/checkstyle.xml
+++ b/checkstyle/checkstyle.xml
@@ -71,6 +71,12 @@
     <module name="TypeName"/>
     <module name="AvoidStarImport"/>
 
+    <!-- variables that can be final should be final (suppressed except for Streams) -->
+    <module name="FinalLocalVariable">
+      <property name="tokens" value="VARIABLE_DEF,PARAMETER_DEF"/>
+      <property name="validateEnhancedForLoopVariable" value="true"/>
+    </module>
+
     <!-- dependencies -->
     <module name="ImportControl">
       <property name="file" value="${importControlFile}"/>
diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml
index e80d5bf..3ed4a9c 100644
--- a/checkstyle/suppressions.xml
+++ b/checkstyle/suppressions.xml
@@ -151,6 +151,10 @@
     <suppress checks="NPathComplexity"
               files="(ProcessorStateManager|InternalTopologyBuilder|StreamsPartitionAssignor|StreamThread).java"/>
 
+    <!-- suppress FinalLocalVariable outside of the streams package. -->
+    <suppress checks="FinalLocalVariable"
+              files="^(?!.*[\\/]org[\\/]apache[\\/]kafka[\\/]streams[\\/].*$)"/>
+
     <!-- Streams tests -->
     <suppress checks="ClassFanOutComplexity"
               files="(StreamThreadTest|StreamTaskTest|ProcessorTopologyTestDriver).java"/>
diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonPOJODeserializer.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonPOJODeserializer.java
index b2ef6dc..d55246c 100644
--- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonPOJODeserializer.java
+++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonPOJODeserializer.java
@@ -35,19 +35,19 @@ public class JsonPOJODeserializer<T> implements Deserializer<T> {
 
     @SuppressWarnings("unchecked")
     @Override
-    public void configure(Map<String, ?> props, boolean isKey) {
+    public void configure(final Map<String, ?> props, final boolean isKey) {
         tClass = (Class<T>) props.get("JsonPOJOClass");
     }
 
     @Override
-    public T deserialize(String topic, byte[] bytes) {
+    public T deserialize(final String topic, final byte[] bytes) {
         if (bytes == null)
             return null;
 
-        T data;
+        final T data;
         try {
             data = objectMapper.readValue(bytes, tClass);
-        } catch (Exception e) {
+        } catch (final Exception e) {
             throw new SerializationException(e);
         }
 
diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonPOJOSerializer.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonPOJOSerializer.java
index 625bda9..81ccf1e 100644
--- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonPOJOSerializer.java
+++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonPOJOSerializer.java
@@ -33,17 +33,17 @@ public class JsonPOJOSerializer<T> implements Serializer<T> {
     }
     
     @Override
-    public void configure(Map<String, ?> props, boolean isKey) {
+    public void configure(final Map<String, ?> props, final boolean isKey) {
     }
 
     @Override
-    public byte[] serialize(String topic, T data) {
+    public byte[] serialize(final String topic, final T data) {
         if (data == null)
             return null;
 
         try {
             return objectMapper.writeValueAsBytes(data);
-        } catch (Exception e) {
+        } catch (final Exception e) {
             throw new SerializationException("Error serializing JSON message", e);
         }
     }
diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java
index 234d3fc..503dbeb 100644
--- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java
+++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java
@@ -81,8 +81,8 @@ public class PageViewTypedDemo {
         public String region;
     }
 
-    public static void main(String[] args) {
-        Properties props = new Properties();
+    public static void main(final String[] args) {
+        final Properties props = new Properties();
         props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pageview-typed");
         props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
         props.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, JsonTimestampExtractor.class);
@@ -91,10 +91,10 @@ public class PageViewTypedDemo {
         // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data
         props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
 
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
         // TODO: the following can be removed with a serialization factory
-        Map<String, Object> serdeProps = new HashMap<>();
+        final Map<String, Object> serdeProps = new HashMap<>();
 
         final Serializer<PageView> pageViewSerializer = new JsonPOJOSerializer<>();
         serdeProps.put("JsonPOJOClass", PageView.class);
@@ -143,14 +143,14 @@ public class PageViewTypedDemo {
         pageViewByRegionDeserializer.configure(serdeProps, false);
         final Serde<PageViewByRegion> pageViewByRegionSerde = Serdes.serdeFrom(pageViewByRegionSerializer, pageViewByRegionDeserializer);
 
-        KStream<String, PageView> views = builder.stream("streams-pageview-input", Consumed.with(Serdes.String(), pageViewSerde));
+        final KStream<String, PageView> views = builder.stream("streams-pageview-input", Consumed.with(Serdes.String(), pageViewSerde));
 
-        KTable<String, UserProfile> users = builder.table("streams-userprofile-input",
-                                                          Consumed.with(Serdes.String(), userProfileSerde));
+        final KTable<String, UserProfile> users = builder.table("streams-userprofile-input",
+                                                                Consumed.with(Serdes.String(), userProfileSerde));
 
-        KStream<WindowedPageViewByRegion, RegionCount> regionCount = views
+        final KStream<WindowedPageViewByRegion, RegionCount> regionCount = views
             .leftJoin(users, (view, profile) -> {
-                PageViewByRegion viewByRegion = new PageViewByRegion();
+                final PageViewByRegion viewByRegion = new PageViewByRegion();
                 viewByRegion.user = view.user;
                 viewByRegion.page = view.page;
 
@@ -167,11 +167,11 @@ public class PageViewTypedDemo {
             .count()
             .toStream()
             .map((key, value) -> {
-                WindowedPageViewByRegion wViewByRegion = new WindowedPageViewByRegion();
+                final WindowedPageViewByRegion wViewByRegion = new WindowedPageViewByRegion();
                 wViewByRegion.windowStart = key.window().start();
                 wViewByRegion.region = key.key();
 
-                RegionCount rCount = new RegionCount();
+                final RegionCount rCount = new RegionCount();
                 rCount.region = key.key();
                 rCount.count = value;
 
@@ -181,7 +181,7 @@ public class PageViewTypedDemo {
         // write to the result topic
         regionCount.to("streams-pageviewstats-typed-output", Produced.with(wPageViewByRegionSerde, regionCountSerde));
 
-        KafkaStreams streams = new KafkaStreams(builder.build(), props);
+        final KafkaStreams streams = new KafkaStreams(builder.build(), props);
         final CountDownLatch latch = new CountDownLatch(1);
 
         // attach shutdown handler to catch control-c
@@ -196,7 +196,7 @@ public class PageViewTypedDemo {
         try {
             streams.start();
             latch.await();
-        } catch (Throwable e) {
+        } catch (final Throwable e) {
             System.exit(1);
         }
         System.exit(0);
diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java
index dddb542..e2b7c37 100644
--- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java
+++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java
@@ -54,8 +54,8 @@ import java.util.Properties;
  */
 public class PageViewUntypedDemo {
 
-    public static void main(String[] args) throws Exception {
-        Properties props = new Properties();
+    public static void main(final String[] args) throws Exception {
+        final Properties props = new Properties();
         props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pageview-untyped");
         props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
         props.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, JsonTimestampExtractor.class);
@@ -64,22 +64,22 @@ public class PageViewUntypedDemo {
         // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data
         props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
 
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
         final Serializer<JsonNode> jsonSerializer = new JsonSerializer();
         final Deserializer<JsonNode> jsonDeserializer = new JsonDeserializer();
         final Serde<JsonNode> jsonSerde = Serdes.serdeFrom(jsonSerializer, jsonDeserializer);
 
         final Consumed<String, JsonNode> consumed = Consumed.with(Serdes.String(), jsonSerde);
-        KStream<String, JsonNode> views = builder.stream("streams-pageview-input", consumed);
+        final KStream<String, JsonNode> views = builder.stream("streams-pageview-input", consumed);
 
-        KTable<String, JsonNode> users = builder.table("streams-userprofile-input", consumed);
+        final KTable<String, JsonNode> users = builder.table("streams-userprofile-input", consumed);
 
-        KTable<String, String> userRegions = users.mapValues(record -> record.get("region").textValue());
+        final KTable<String, String> userRegions = users.mapValues(record -> record.get("region").textValue());
 
-        KStream<JsonNode, JsonNode> regionCount = views
+        final KStream<JsonNode, JsonNode> regionCount = views
             .leftJoin(userRegions, (view, region) -> {
-                ObjectNode jNode = JsonNodeFactory.instance.objectNode();
+                final ObjectNode jNode = JsonNodeFactory.instance.objectNode();
                 return (JsonNode) jNode.put("user", view.get("user").textValue())
                         .put("page", view.get("page").textValue())
                         .put("region", region == null ? "UNKNOWN" : region);
@@ -91,11 +91,11 @@ public class PageViewUntypedDemo {
             .count()
             .toStream()
             .map((key, value) -> {
-                ObjectNode keyNode = JsonNodeFactory.instance.objectNode();
+                final ObjectNode keyNode = JsonNodeFactory.instance.objectNode();
                 keyNode.put("window-start", key.window().start())
                         .put("region", key.key());
 
-                ObjectNode valueNode = JsonNodeFactory.instance.objectNode();
+                final ObjectNode valueNode = JsonNodeFactory.instance.objectNode();
                 valueNode.put("count", value);
 
                 return new KeyValue<>((JsonNode) keyNode, (JsonNode) valueNode);
@@ -104,7 +104,7 @@ public class PageViewUntypedDemo {
         // write to the result topic
         regionCount.to("streams-pageviewstats-untyped-output", Produced.with(jsonSerde, jsonSerde));
 
-        KafkaStreams streams = new KafkaStreams(builder.build(), props);
+        final KafkaStreams streams = new KafkaStreams(builder.build(), props);
         streams.start();
 
         // usually the stream application would be running forever,
diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java
index d61e174..99505fd 100644
--- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java
+++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java
@@ -38,8 +38,8 @@ import java.util.concurrent.CountDownLatch;
  */
 public class PipeDemo {
 
-    public static void main(String[] args) {
-        Properties props = new Properties();
+    public static void main(final String[] args) {
+        final Properties props = new Properties();
         props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pipe");
         props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
         props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
@@ -48,7 +48,7 @@ public class PipeDemo {
         // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data
         props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
 
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
         builder.stream("streams-plaintext-input").to("streams-pipe-output");
 
@@ -67,7 +67,7 @@ public class PipeDemo {
         try {
             streams.start();
             latch.await();
-        } catch (Throwable e) {
+        } catch (final Throwable e) {
             System.exit(1);
         }
         System.exit(0);
diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/temperature/TemperatureDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/temperature/TemperatureDemo.java
index 4607d75..c1c7a42 100644
--- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/temperature/TemperatureDemo.java
+++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/temperature/TemperatureDemo.java
@@ -68,9 +68,9 @@ public class TemperatureDemo {
     // window size within which the filtering is applied
     private static final int TEMPERATURE_WINDOW_SIZE = 5;
 
-    public static void main(String[] args) {
+    public static void main(final String[] args) {
 
-        Properties props = new Properties();
+        final Properties props = new Properties();
         props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-temperature");
         props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
         props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
@@ -79,11 +79,11 @@ public class TemperatureDemo {
         props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
         props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
 
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
-        KStream<String, String> source = builder.stream("iot-temperature");
+        final KStream<String, String> source = builder.stream("iot-temperature");
 
-        KStream<Windowed<String>, String> max = source
+        final KStream<Windowed<String>, String> max = source
             // temperature values are sent without a key (null), so in order
             // to group and reduce them, a key is needed ("temp" has been chosen)
             .selectKey((key, value) -> "temp")
@@ -98,7 +98,7 @@ public class TemperatureDemo {
             .toStream()
             .filter((key, value) -> Integer.parseInt(value) > TEMPERATURE_THRESHOLD);
 
-        Serde<Windowed<String>> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class);
+        final Serde<Windowed<String>> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class);
 
         // need to override key serde to Windowed<String> type
         max.to("iot-temperature-max", Produced.with(windowedSerde, Serdes.String()));
@@ -118,7 +118,7 @@ public class TemperatureDemo {
         try {
             streams.start();
             latch.await();
-        } catch (Throwable e) {
+        } catch (final Throwable e) {
             System.exit(1);
         }
         System.exit(0);
diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java
index c05e8d8..4fa38dc 100644
--- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java
+++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java
@@ -80,8 +80,7 @@ public final class WordCountProcessorDemo {
                 }
 
                 @Override
-                public void process(final String dummy,
-                                    final String line) {
+                public void process(final String dummy, final String line) {
                     final String[] words = line.toLowerCase(Locale.getDefault()).split(" ");
 
                     for (final String word : words) {
diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
index 8ed80dc..2cefb35 100644
--- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
+++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
@@ -212,7 +212,7 @@ public class KafkaStreams {
     protected volatile State state = State.CREATED;
 
     private boolean waitOnState(final State targetState, final long waitMs) {
-        long begin = time.milliseconds();
+        final long begin = time.milliseconds();
         synchronized (stateLock) {
             long elapsedMs = 0L;
             while (state != targetState) {
@@ -223,7 +223,7 @@ public class KafkaStreams {
                         // it is ok: just move on to the next iteration
                     }
                 } else if (waitMs > elapsedMs) {
-                    long remainingMs = waitMs - elapsedMs;
+                    final long remainingMs = waitMs - elapsedMs;
                     try {
                         stateLock.wait(remainingMs);
                     } catch (final InterruptedException e) {
@@ -449,7 +449,7 @@ public class KafkaStreams {
                                           final ThreadStateTransitionValidator abstractOldState) {
             // StreamThreads first
             if (thread instanceof StreamThread) {
-                StreamThread.State newState = (StreamThread.State) abstractNewState;
+                final StreamThread.State newState = (StreamThread.State) abstractNewState;
                 threadState.put(thread.getId(), newState);
 
                 if (newState == StreamThread.State.PARTITIONS_REVOKED && state != State.REBALANCING) {
@@ -461,7 +461,7 @@ public class KafkaStreams {
                 }
             } else if (thread instanceof GlobalStreamThread) {
                 // global stream thread has different invariants
-                GlobalStreamThread.State newState = (GlobalStreamThread.State) abstractNewState;
+                final GlobalStreamThread.State newState = (GlobalStreamThread.State) abstractNewState;
                 globalThreadState = newState;
 
                 // special case when global thread is dead
@@ -729,7 +729,7 @@ public class KafkaStreams {
         if (globalTaskTopology != null) {
             globalStreamThread.setStateListener(streamStateListener);
         }
-        for (StreamThread thread : threads) {
+        for (final StreamThread thread : threads) {
             thread.setStateListener(streamStateListener);
         }
 
@@ -1046,7 +1046,7 @@ public class KafkaStreams {
     public Set<ThreadMetadata> localThreadsMetadata() {
         validateIsRunning();
         final Set<ThreadMetadata> threadMetadata = new HashSet<>();
-        for (StreamThread thread : threads) {
+        for (final StreamThread thread : threads) {
             threadMetadata.add(thread.threadMetadata());
         }
         return threadMetadata;
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Serialized.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Serialized.java
index d69aabd..0b04696 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/Serialized.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Serialized.java
@@ -27,8 +27,8 @@ import org.apache.kafka.common.serialization.Serde;
  */
 public class Serialized<K, V> {
 
-    protected Serde<K> keySerde;
-    protected Serde<V> valueSerde;
+    protected final Serde<K> keySerde;
+    protected final Serde<V> valueSerde;
 
     private Serialized(final Serde<K> keySerde,
                        final Serde<V> valueSerde) {
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindowedSerializer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindowedSerializer.java
index 484b3af..430fd63 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindowedSerializer.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindowedSerializer.java
@@ -47,8 +47,8 @@ public class SessionWindowedSerializer<T> implements WindowedSerializer<T> {
     @Override
     public void configure(final Map<String, ?> configs, final boolean isKey) {
         if (inner == null) {
-            String propertyName = isKey ? StreamsConfig.DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS : StreamsConfig.DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS;
-            String value = (String) configs.get(propertyName);
+            final String propertyName = isKey ? StreamsConfig.DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS : StreamsConfig.DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS;
+            final String value = (String) configs.get(propertyName);
             try {
                 inner = Serde.class.cast(Utils.newInstance(value, Serde.class)).serializer();
                 inner.configure(configs, isKey);
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java
index 9fce80f..a0724eb 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java
@@ -40,7 +40,7 @@ public abstract class AbstractStream<K> {
 
     // This copy-constructor will allow to extend KStream
     // and KTable APIs with new methods without impacting the public interface.
-    public AbstractStream(AbstractStream<K> stream) {
+    public AbstractStream(final AbstractStream<K> stream) {
         this.builder = stream.builder;
         this.name = stream.name;
         this.sourceNodes = stream.sourceNodes;
@@ -68,7 +68,7 @@ public abstract class AbstractStream<K> {
     }
 
     Set<String> ensureJoinableWith(final AbstractStream<K> other) {
-        Set<String> allSourceNodes = new HashSet<>();
+        final Set<String> allSourceNodes = new HashSet<>();
         allSourceNodes.addAll(sourceNodes);
         allSourceNodes.addAll(other.sourceNodes);
 
@@ -80,7 +80,7 @@ public abstract class AbstractStream<K> {
     static <T2, T1, R> ValueJoiner<T2, T1, R> reverseJoiner(final ValueJoiner<T1, T2, R> joiner) {
         return new ValueJoiner<T2, T1, R>() {
             @Override
-            public R apply(T2 value2, T1 value1) {
+            public R apply(final T2 value2, final T1 value1) {
                 return joiner.apply(value1, value2);
             }
         };
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/Change.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/Change.java
index a328f5d..d513f2b 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/Change.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/Change.java
@@ -23,7 +23,7 @@ public class Change<T> {
     public final T newValue;
     public final T oldValue;
 
-    public Change(T newValue, T oldValue) {
+    public Change(final T newValue, final T oldValue) {
         this.newValue = newValue;
         this.oldValue = oldValue;
     }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedDeserializer.java
index 1363a0b..29180f2 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedDeserializer.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedDeserializer.java
@@ -31,7 +31,7 @@ public class ChangedDeserializer<T> implements ExtendedDeserializer<Change<T>> {
 
     private ExtendedDeserializer<T> inner;
 
-    public ChangedDeserializer(Deserializer<T> inner) {
+    public ChangedDeserializer(final Deserializer<T> inner) {
         this.inner = ensureExtended(inner);
     }
 
@@ -39,19 +39,19 @@ public class ChangedDeserializer<T> implements ExtendedDeserializer<Change<T>> {
         return inner;
     }
 
-    public void setInner(Deserializer<T> inner) {
+    public void setInner(final Deserializer<T> inner) {
         this.inner = ensureExtended(inner);
     }
 
     @Override
-    public void configure(Map<String, ?> configs, boolean isKey) {
+    public void configure(final Map<String, ?> configs, final boolean isKey) {
         // do nothing
     }
 
     @Override
-    public Change<T> deserialize(String topic, Headers headers, byte[] data) {
+    public Change<T> deserialize(final String topic, final Headers headers, final byte[] data) {
 
-        byte[] bytes = new byte[data.length - NEWFLAG_SIZE];
+        final byte[] bytes = new byte[data.length - NEWFLAG_SIZE];
 
         System.arraycopy(data, 0, bytes, 0, bytes.length);
 
@@ -63,7 +63,7 @@ public class ChangedDeserializer<T> implements ExtendedDeserializer<Change<T>> {
     }
 
     @Override
-    public Change<T> deserialize(String topic, byte[] data) {
+    public Change<T> deserialize(final String topic, final byte[] data) {
         return deserialize(topic, null, data);
     }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java
index fa261cb..83ac8e0 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java
@@ -32,7 +32,7 @@ public class ChangedSerializer<T> implements ExtendedSerializer<Change<T>> {
 
     private ExtendedSerializer<T> inner;
 
-    public ChangedSerializer(Serializer<T> inner) {
+    public ChangedSerializer(final Serializer<T> inner) {
         this.inner = ensureExtended(inner);
     }
 
@@ -40,12 +40,12 @@ public class ChangedSerializer<T> implements ExtendedSerializer<Change<T>> {
         return inner;
     }
 
-    public void setInner(Serializer<T> inner) {
+    public void setInner(final Serializer<T> inner) {
         this.inner = ensureExtended(inner);
     }
 
     @Override
-    public void configure(Map<String, ?> configs, boolean isKey) {
+    public void configure(final Map<String, ?> configs, final boolean isKey) {
         // do nothing
     }
 
@@ -54,8 +54,8 @@ public class ChangedSerializer<T> implements ExtendedSerializer<Change<T>> {
      * both values are not null
      */
     @Override
-    public byte[] serialize(String topic, Headers headers, Change<T> data) {
-        byte[] serializedKey;
+    public byte[] serialize(final String topic, final Headers headers, final Change<T> data) {
+        final byte[] serializedKey;
 
         // only one of the old / new values would be not null
         if (data.newValue != null) {
@@ -71,7 +71,7 @@ public class ChangedSerializer<T> implements ExtendedSerializer<Change<T>> {
             serializedKey = inner.serialize(topic, headers, data.oldValue);
         }
 
-        ByteBuffer buf = ByteBuffer.allocate(serializedKey.length + NEWFLAG_SIZE);
+        final ByteBuffer buf = ByteBuffer.allocate(serializedKey.length + NEWFLAG_SIZE);
         buf.put(serializedKey);
         buf.put((byte) (data.newValue != null ? 1 : 0));
 
@@ -79,7 +79,7 @@ public class ChangedSerializer<T> implements ExtendedSerializer<Change<T>> {
     }
 
     @Override
-    public byte[] serialize(String topic, Change<T> data) {
+    public byte[] serialize(final String topic, final Change<T> data) {
         return serialize(topic, null, data);
     }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GroupedStreamAggregateBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GroupedStreamAggregateBuilder.java
index d02a569..a191c5a 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GroupedStreamAggregateBuilder.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GroupedStreamAggregateBuilder.java
@@ -71,27 +71,27 @@ class GroupedStreamAggregateBuilder<K, V> {
 
         final String aggFunctionName = builder.newProcessorName(functionName);
 
-        OptimizableRepartitionNode.OptimizableRepartitionNodeBuilder<K, V> repartitionNodeBuilder = OptimizableRepartitionNode.optimizableRepartitionNodeBuilder();
+        final OptimizableRepartitionNode.OptimizableRepartitionNodeBuilder<K, V> repartitionNodeBuilder = OptimizableRepartitionNode.optimizableRepartitionNodeBuilder();
 
         final String sourceName = repartitionIfRequired(storeBuilder.name(), repartitionNodeBuilder);
 
         StreamsGraphNode parentNode = streamsGraphNode;
 
         if (!sourceName.equals(this.name)) {
-            StreamsGraphNode repartitionNode = repartitionNodeBuilder.build();
+            final StreamsGraphNode repartitionNode = repartitionNodeBuilder.build();
             builder.addGraphNode(parentNode, repartitionNode);
             parentNode = repartitionNode;
         }
-        StatefulProcessorNode.StatefulProcessorNodeBuilder<K, T> statefulProcessorNodeBuilder = StatefulProcessorNode.statefulProcessorNodeBuilder();
+        final StatefulProcessorNode.StatefulProcessorNodeBuilder<K, T> statefulProcessorNodeBuilder = StatefulProcessorNode.statefulProcessorNodeBuilder();
 
-        ProcessorParameters processorParameters = new ProcessorParameters<>(aggregateSupplier, aggFunctionName);
+        final ProcessorParameters processorParameters = new ProcessorParameters<>(aggregateSupplier, aggFunctionName);
         statefulProcessorNodeBuilder
             .withProcessorParameters(processorParameters)
             .withNodeName(aggFunctionName)
             .withRepartitionRequired(repartitionRequired)
             .withStoreBuilder(storeBuilder);
 
-        StatefulProcessorNode<K, T> statefulProcessorNode = statefulProcessorNodeBuilder.build();
+        final StatefulProcessorNode<K, T> statefulProcessorNode = statefulProcessorNodeBuilder.build();
 
         builder.addGraphNode(parentNode, statefulProcessorNode);
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java
index b502153..2e55e9f 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java
@@ -204,14 +204,14 @@ public class InternalStreamsBuilder implements InternalNameProvider {
                        stateUpdateSupplier);
     }
 
-    void addGraphNode(StreamsGraphNode parent, StreamsGraphNode child) {
+    void addGraphNode(final StreamsGraphNode parent, final StreamsGraphNode child) {
         Objects.requireNonNull(parent, "parent node can't be null");
         Objects.requireNonNull(child, "child node can't be null");
         parent.addChildNode(child);
         maybeAddNodeForOptimizationMetadata(child);
     }
 
-    void addGraphNode(Collection<StreamsGraphNode> parents, StreamsGraphNode child) {
+    void addGraphNode(final Collection<StreamsGraphNode> parents, final StreamsGraphNode child) {
         Objects.requireNonNull(parents, "parent node can't be null");
         Objects.requireNonNull(child, "child node can't be null");
 
@@ -219,7 +219,7 @@ public class InternalStreamsBuilder implements InternalNameProvider {
             throw new StreamsException("Parent node collection can't be empty");
         }
 
-        for (StreamsGraphNode parent : parents) {
+        for (final StreamsGraphNode parent : parents) {
             addGraphNode(parent, child);
         }
     }
@@ -246,7 +246,7 @@ public class InternalStreamsBuilder implements InternalNameProvider {
                 streamGraphNode.setHasWrittenToTopology(true);
             }
 
-            for (StreamsGraphNode graphNode : streamGraphNode.children()) {
+            for (final StreamsGraphNode graphNode : streamGraphNode.children()) {
                 graphNodePriorityQueue.offer(graphNode);
             }
         }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java
index 5858a8c..08fb605 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java
@@ -53,7 +53,7 @@ public class KGroupedTableImpl<K, V> extends AbstractStream<K> implements KGroup
 
     private final Aggregator<K, V, Long> countAdder = (aggKey, value, aggregate) -> aggregate + 1L;
 
-    private Aggregator<K, V, Long> countSubtractor = (aggKey, value, aggregate) -> aggregate - 1L;
+    private final Aggregator<K, V, Long> countSubtractor = (aggKey, value, aggregate) -> aggregate - 1L;
 
     KGroupedTableImpl(final InternalStreamsBuilder builder,
                       final String name,
@@ -75,16 +75,16 @@ public class KGroupedTableImpl<K, V> extends AbstractStream<K> implements KGroup
         final String funcName = builder.newProcessorName(functionName);
         final String topic = materialized.storeName() + KStreamImpl.REPARTITION_TOPIC_SUFFIX;
 
-        StreamsGraphNode repartitionNode = createRepartitionNode(sinkName,
-                                                                 sourceName,
-                                                                 topic);
+        final StreamsGraphNode repartitionNode = createRepartitionNode(sinkName,
+                                                                       sourceName,
+                                                                       topic);
 
         // the passed in StreamsGraphNode must be the parent of the repartition node
         builder.addGraphNode(this.streamsGraphNode, repartitionNode);
 
-        StatefulProcessorNode statefulProcessorNode = getStatefulProcessorNode(materialized,
-                                                                               funcName,
-                                                                               aggregateSupplier);
+        final StatefulProcessorNode statefulProcessorNode = getStatefulProcessorNode(materialized,
+                                                                                     funcName,
+                                                                                     aggregateSupplier);
 
         // now the repartition node must be the parent of the StateProcessorNode
         builder.addGraphNode(repartitionNode, statefulProcessorNode);
@@ -104,7 +104,7 @@ public class KGroupedTableImpl<K, V> extends AbstractStream<K> implements KGroup
                                                                final String functionName,
                                                                final ProcessorSupplier aggregateSupplier) {
 
-        ProcessorParameters aggregateFunctionProcessorParams = new ProcessorParameters<>(aggregateSupplier, functionName);
+        final ProcessorParameters aggregateFunctionProcessorParams = new ProcessorParameters<>(aggregateSupplier, functionName);
 
         return StatefulProcessorNode.statefulProcessorNodeBuilder()
             .withNodeName(functionName)
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java
index 0349426..ac03c18 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java
@@ -26,7 +26,7 @@ class KStreamFilter<K, V> implements ProcessorSupplier<K, V> {
     private final Predicate<K, V> predicate;
     private final boolean filterNot;
 
-    public KStreamFilter(Predicate<K, V> predicate, boolean filterNot) {
+    public KStreamFilter(final Predicate<K, V> predicate, final boolean filterNot) {
         this.predicate = predicate;
         this.filterNot = filterNot;
     }
@@ -38,7 +38,7 @@ class KStreamFilter<K, V> implements ProcessorSupplier<K, V> {
 
     private class KStreamFilterProcessor extends AbstractProcessor<K, V> {
         @Override
-        public void process(K key, V value) {
+        public void process(final K key, final V value) {
             if (filterNot ^ predicate.test(key, value)) {
                 context().forward(key, value);
             }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java
index f511461..e20ec90 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java
@@ -26,7 +26,7 @@ class KStreamFlatMap<K, V, K1, V1> implements ProcessorSupplier<K, V> {
 
     private final KeyValueMapper<? super K, ? super V, ? extends Iterable<? extends KeyValue<? extends K1, ? extends V1>>> mapper;
 
-    KStreamFlatMap(KeyValueMapper<? super K, ? super V, ? extends Iterable<? extends KeyValue<? extends K1, ? extends V1>>> mapper) {
+    KStreamFlatMap(final KeyValueMapper<? super K, ? super V, ? extends Iterable<? extends KeyValue<? extends K1, ? extends V1>>> mapper) {
         this.mapper = mapper;
     }
 
@@ -37,8 +37,8 @@ class KStreamFlatMap<K, V, K1, V1> implements ProcessorSupplier<K, V> {
 
     private class KStreamFlatMapProcessor extends AbstractProcessor<K, V> {
         @Override
-        public void process(K key, V value) {
-            for (KeyValue<? extends K1, ? extends V1> newPair : mapper.apply(key, value)) {
+        public void process(final K key, final V value) {
+            for (final KeyValue<? extends K1, ? extends V1> newPair : mapper.apply(key, value)) {
                 context().forward(newPair.key, newPair.value);
             }
         }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java
index 7d0d270..fedfe39 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java
@@ -36,9 +36,9 @@ class KStreamFlatMapValues<K, V, V1> implements ProcessorSupplier<K, V> {
 
     private class KStreamFlatMapValuesProcessor extends AbstractProcessor<K, V> {
         @Override
-        public void process(K key, V value) {
+        public void process(final K key, final V value) {
             final Iterable<? extends V1> newValues = mapper.apply(key, value);
-            for (V1 v : newValues) {
+            for (final V1 v : newValues) {
                 context().forward(key, v);
             }
         }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java
index 5451a86..2b37f24 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java
@@ -862,8 +862,8 @@ public class KStreamImpl<K, V> extends AbstractStream<K> implements KStream<K, V
             final String joinOtherName = leftOuter ? builder.newProcessorName(OUTEROTHER_NAME) : builder.newProcessorName(JOINOTHER_NAME);
             final String joinMergeName = builder.newProcessorName(MERGE_NAME);
 
-            StreamsGraphNode thisStreamsGraphNode = ((AbstractStream) lhs).streamsGraphNode;
-            StreamsGraphNode otherStreamsGraphNode = ((AbstractStream) other).streamsGraphNode;
+            final StreamsGraphNode thisStreamsGraphNode = ((AbstractStream) lhs).streamsGraphNode;
+            final StreamsGraphNode otherStreamsGraphNode = ((AbstractStream) other).streamsGraphNode;
 
             final StoreBuilder<WindowStore<K1, V1>> thisWindowStore =
                 createWindowedStateStore(windows, joined.keySerde(), joined.valueSerde(), joinThisName + "-store");
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java
index e21f702..895dab4 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java
@@ -30,7 +30,7 @@ class KStreamJoinWindow<K, V> implements ProcessorSupplier<K, V> {
     /**
      * @throws TopologyException if retention period of the join window is less than expected
      */
-    KStreamJoinWindow(String windowName, long windowSizeMs, long retentionPeriodMs) {
+    KStreamJoinWindow(final String windowName, final long windowSizeMs, final long retentionPeriodMs) {
         this.windowName = windowName;
 
         if (windowSizeMs > retentionPeriodMs)
@@ -49,14 +49,14 @@ class KStreamJoinWindow<K, V> implements ProcessorSupplier<K, V> {
 
         @SuppressWarnings("unchecked")
         @Override
-        public void init(ProcessorContext context) {
+        public void init(final ProcessorContext context) {
             super.init(context);
 
             window = (WindowStore<K, V>) context.getStateStore(windowName);
         }
 
         @Override
-        public void process(K key, V value) {
+        public void process(final K key, final V value) {
             // if the key is null, we do not need to put the record into window store
             // since it will never be considered for join operations
             if (key != null) {
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java
index ef9a95e..8179ca8 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java
@@ -26,7 +26,7 @@ class KStreamMap<K, V, K1, V1> implements ProcessorSupplier<K, V> {
 
     private final KeyValueMapper<? super K, ? super V, ? extends KeyValue<? extends K1, ? extends V1>> mapper;
 
-    public KStreamMap(KeyValueMapper<? super K, ? super V, ? extends KeyValue<? extends K1, ? extends V1>> mapper) {
+    public KStreamMap(final KeyValueMapper<? super K, ? super V, ? extends KeyValue<? extends K1, ? extends V1>> mapper) {
         this.mapper = mapper;
     }
 
@@ -37,8 +37,8 @@ class KStreamMap<K, V, K1, V1> implements ProcessorSupplier<K, V> {
 
     private class KStreamMapProcessor extends AbstractProcessor<K, V> {
         @Override
-        public void process(K key, V value) {
-            KeyValue<? extends K1, ? extends V1> newPair = mapper.apply(key, value);
+        public void process(final K key, final V value) {
+            final KeyValue<? extends K1, ? extends V1> newPair = mapper.apply(key, value);
             context().forward(newPair.key, newPair.value);
         }
     }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java
index ebaf027..2afe507 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java
@@ -29,7 +29,7 @@ class KStreamPassThrough<K, V> implements ProcessorSupplier<K, V> {
 
     private static final class KStreamPassThroughProcessor<K, V> extends AbstractProcessor<K, V> {
         @Override
-        public void process(K key, V value) {
+        public void process(final K key, final V value) {
             context().forward(key, value);
         }
     }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransform.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransform.java
index 1ae8ede..8e91469 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransform.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransform.java
@@ -28,7 +28,7 @@ public class KStreamTransform<K, V, K1, V1> implements ProcessorSupplier<K, V> {
 
     private final TransformerSupplier<? super K, ? super V, ? extends KeyValue<? extends K1, ? extends V1>> transformerSupplier;
 
-    public KStreamTransform(TransformerSupplier<? super K, ? super V, ? extends KeyValue<? extends K1, ? extends V1>> transformerSupplier) {
+    public KStreamTransform(final TransformerSupplier<? super K, ? super V, ? extends KeyValue<? extends K1, ? extends V1>> transformerSupplier) {
         this.transformerSupplier = transformerSupplier;
     }
 
@@ -41,19 +41,19 @@ public class KStreamTransform<K, V, K1, V1> implements ProcessorSupplier<K, V> {
 
         private final Transformer<? super K1, ? super V1, ? extends KeyValue<? extends K2, ? extends V2>> transformer;
 
-        public KStreamTransformProcessor(Transformer<? super K1, ? super V1, ? extends KeyValue<? extends K2, ? extends V2>> transformer) {
+        public KStreamTransformProcessor(final Transformer<? super K1, ? super V1, ? extends KeyValue<? extends K2, ? extends V2>> transformer) {
             this.transformer = transformer;
         }
 
         @Override
-        public void init(ProcessorContext context) {
+        public void init(final ProcessorContext context) {
             super.init(context);
             transformer.init(context);
         }
 
         @Override
-        public void process(K1 key, V1 value) {
-            KeyValue<? extends K2, ? extends V2> pair = transformer.transform(key, value);
+        public void process(final K1 key, final V1 value) {
+            final KeyValue<? extends K2, ? extends V2> pair = transformer.transform(key, value);
 
             if (pair != null)
                 context().forward(pair.key, pair.value);
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java
index b645960..843606b 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java
@@ -52,7 +52,7 @@ public class KStreamTransformValues<K, V, R> implements ProcessorSupplier<K, V>
         }
 
         @Override
-        public void process(K key, V value) {
+        public void process(final K key, final V value) {
             context.forward(key, valueTransformer.transform(key, value));
         }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java
index d020e4b..75fba99 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java
@@ -51,7 +51,7 @@ class KTableFilter<K, V> implements KTableProcessorSupplier<K, V, V> {
         sendOldValues = true;
     }
 
-    private V computeValue(K key, V value) {
+    private V computeValue(final K key, final V value) {
         V newValue = null;
 
         if (value != null && (filterNot ^ predicate.test(key, value)))
@@ -66,7 +66,7 @@ class KTableFilter<K, V> implements KTableProcessorSupplier<K, V, V> {
 
         @SuppressWarnings("unchecked")
         @Override
-        public void init(ProcessorContext context) {
+        public void init(final ProcessorContext context) {
             super.init(context);
             if (queryableName != null) {
                 store = (KeyValueStore<K, V>) context.getStateStore(queryableName);
@@ -75,9 +75,9 @@ class KTableFilter<K, V> implements KTableProcessorSupplier<K, V, V> {
         }
 
         @Override
-        public void process(K key, Change<V> change) {
-            V newValue = computeValue(key, change.newValue);
-            V oldValue = sendOldValues ? computeValue(key, change.oldValue) : null;
+        public void process(final K key, final Change<V> change) {
+            final V newValue = computeValue(key, change.newValue);
+            final V oldValue = sendOldValues ? computeValue(key, change.oldValue) : null;
 
             if (sendOldValues && oldValue == null && newValue == null)
                 return; // unnecessary to forward here.
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java
index bdc1dca..b0189df 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java
@@ -28,9 +28,9 @@ abstract class KTableKTableAbstractJoin<K, R, V1, V2> implements KTableProcessor
 
     boolean sendOldValues = false;
 
-    KTableKTableAbstractJoin(KTableImpl<K, ?, V1> table1,
-                             KTableImpl<K, ?, V2> table2,
-                             ValueJoiner<? super V1, ? super V2, ? extends R> joiner) {
+    KTableKTableAbstractJoin(final KTableImpl<K, ?, V1> table1,
+                             final KTableImpl<K, ?, V2> table2,
+                             final ValueJoiner<? super V1, ? super V2, ? extends R> joiner) {
         this.table1 = table1;
         this.table2 = table2;
         this.valueGetterSupplier1 = table1.valueGetterSupplier();
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java
index 6400750..5c464b9 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java
@@ -95,7 +95,7 @@ class KTableKTableJoinMerger<K, V> implements KTableProcessorSupplier<K, V, V> {
         }
 
         @Override
-        public void process(K key, Change<V> value) {
+        public void process(final K key, final Change<V> value) {
             if (queryableName != null) {
                 store.put(key, value.newValue);
                 tupleForwarder.maybeForward(key, value.newValue, value.oldValue);
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java
index 1106a94..9ae2b65 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java
@@ -95,7 +95,7 @@ class KTableMapValues<K, V, V1> implements KTableProcessorSupplier<K, V, V1> {
         }
 
         @Override
-        public void process(K key, Change<V> change) {
+        public void process(final K key, final Change<V> change) {
             final V1 newValue = computeValue(key, change.newValue);
             final V1 oldValue = sendOldValues ? computeValue(key, change.oldValue) : null;
 
@@ -112,7 +112,7 @@ class KTableMapValues<K, V, V1> implements KTableProcessorSupplier<K, V, V1> {
 
         private final KTableValueGetter<K, V> parentGetter;
 
-        KTableMapValuesValueGetter(KTableValueGetter<K, V> parentGetter) {
+        KTableMapValuesValueGetter(final KTableValueGetter<K, V> parentGetter) {
             this.parentGetter = parentGetter;
         }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java
index ebf16d1..a5f68c3 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java
@@ -23,7 +23,7 @@ public class KTableSourceValueGetterSupplier<K, V> implements KTableValueGetterS
 
     private final String storeName;
 
-    public KTableSourceValueGetterSupplier(String storeName) {
+    public KTableSourceValueGetterSupplier(final String storeName) {
         this.storeName = storeName;
     }
 
@@ -41,11 +41,11 @@ public class KTableSourceValueGetterSupplier<K, V> implements KTableValueGetterS
         ReadOnlyKeyValueStore<K, V> store = null;
 
         @SuppressWarnings("unchecked")
-        public void init(ProcessorContext context) {
+        public void init(final ProcessorContext context) {
             store = (ReadOnlyKeyValueStore<K, V>) context.getStateStore(storeName);
         }
 
-        public V get(K key) {
+        public V get(final K key) {
             return store.get(key);
         }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java
index ddebeac..b63b66d 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java
@@ -129,7 +129,7 @@ public class KTableKTableJoinNode<K, V1, V2, VR> extends BaseJoinProcessorNode<K
             return this;
         }
 
-        public KTableKTableJoinNodeBuilder<K, V1, V2, VR> withNodeName(String nodeName) {
+        public KTableKTableJoinNodeBuilder<K, V1, V2, VR> withNodeName(final String nodeName) {
             this.nodeName = nodeName;
             return this;
         }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphNode.java
index 4604807..902a4e9 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphNode.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphNode.java
@@ -29,7 +29,7 @@ public abstract class StreamsGraphNode {
     private final Collection<StreamsGraphNode> childNodes = new LinkedHashSet<>();
     private final Collection<StreamsGraphNode> parentNodes = new LinkedHashSet<>();
     private final String nodeName;
-    private boolean repartitionRequired;
+    private final boolean repartitionRequired;
     private boolean keyChangingOperation;
     private Integer id;
     private boolean hasWrittenToTopology = false;
@@ -105,13 +105,13 @@ public abstract class StreamsGraphNode {
         return hasWrittenToTopology;
     }
 
-    public void setHasWrittenToTopology(boolean hasWrittenToTopology) {
+    public void setHasWrittenToTopology(final boolean hasWrittenToTopology) {
         this.hasWrittenToTopology = hasWrittenToTopology;
     }
 
     @Override
     public String toString() {
-        String[] parentNames = parentNodeNames();
+        final String[] parentNames = parentNodeNames();
         return "StreamsGraphNode{" +
                "nodeName='" + nodeName + '\'' +
                ", id=" + id +
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java
index 41daf9d..8576ee2 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java
@@ -31,7 +31,7 @@ import java.util.Collections;
  */
 public class TableSourceNode<K, V, S extends StateStore> extends StreamSourceNode<K, V> {
 
-    private StoreBuilder<S> storeBuilder;
+    private final StoreBuilder<S> storeBuilder;
     private final ProcessorParameters<K, V> processorParameters;
     private final String sourceName;
     private final boolean isGlobalKTable;
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java b/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java
index 712f8a7..5b735bb 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java
@@ -47,20 +47,20 @@ public class DefaultPartitionGrouper implements PartitionGrouper {
      * @param metadata      metadata of the consuming cluster
      * @return The map from generated task ids to the assigned partitions
      */
-    public Map<TaskId, Set<TopicPartition>> partitionGroups(Map<Integer, Set<String>> topicGroups, Cluster metadata) {
-        Map<TaskId, Set<TopicPartition>> groups = new HashMap<>();
+    public Map<TaskId, Set<TopicPartition>> partitionGroups(final Map<Integer, Set<String>> topicGroups, final Cluster metadata) {
+        final Map<TaskId, Set<TopicPartition>> groups = new HashMap<>();
 
-        for (Map.Entry<Integer, Set<String>> entry : topicGroups.entrySet()) {
-            Integer topicGroupId = entry.getKey();
-            Set<String> topicGroup = entry.getValue();
+        for (final Map.Entry<Integer, Set<String>> entry : topicGroups.entrySet()) {
+            final Integer topicGroupId = entry.getKey();
+            final Set<String> topicGroup = entry.getValue();
 
-            int maxNumPartitions = maxNumPartitions(metadata, topicGroup);
+            final int maxNumPartitions = maxNumPartitions(metadata, topicGroup);
 
             for (int partitionId = 0; partitionId < maxNumPartitions; partitionId++) {
-                Set<TopicPartition> group = new HashSet<>(topicGroup.size());
+                final Set<TopicPartition> group = new HashSet<>(topicGroup.size());
 
-                for (String topic : topicGroup) {
-                    List<PartitionInfo> partitions = metadata.partitionsForTopic(topic);
+                for (final String topic : topicGroup) {
+                    final List<PartitionInfo> partitions = metadata.partitionsForTopic(topic);
                     if (partitionId < partitions.size()) {
                         group.add(new TopicPartition(topic, partitionId));
                     }
@@ -75,16 +75,16 @@ public class DefaultPartitionGrouper implements PartitionGrouper {
     /**
      * @throws StreamsException if no metadata can be received for a topic
      */
-    protected int maxNumPartitions(Cluster metadata, Set<String> topics) {
+    protected int maxNumPartitions(final Cluster metadata, final Set<String> topics) {
         int maxNumPartitions = 0;
-        for (String topic : topics) {
-            List<PartitionInfo> partitions = metadata.partitionsForTopic(topic);
+        for (final String topic : topics) {
+            final List<PartitionInfo> partitions = metadata.partitionsForTopic(topic);
             if (partitions.isEmpty()) {
                 log.error("Empty partitions for topic {}", topic);
                 throw new RuntimeException("Empty partitions for topic " + topic);
             }
 
-            int numPartitions = partitions.size();
+            final int numPartitions = partitions.size();
             if (numPartitions > maxNumPartitions) {
                 maxNumPartitions = numPartitions;
             }
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java b/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java
index f4c9ce0..44550ae 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java
@@ -33,7 +33,7 @@ public class TaskId implements Comparable<TaskId> {
     /** The ID of the partition. */
     public final int partition;
 
-    public TaskId(int topicGroupId, int partition) {
+    public TaskId(final int topicGroupId, final int partition) {
         this.topicGroupId = topicGroupId;
         this.partition = partition;
     }
@@ -45,16 +45,16 @@ public class TaskId implements Comparable<TaskId> {
     /**
      * @throws TaskIdFormatException if the string is not a valid {@link TaskId}
      */
-    public static TaskId parse(String string) {
-        int index = string.indexOf('_');
+    public static TaskId parse(final String string) {
+        final int index = string.indexOf('_');
         if (index <= 0 || index + 1 >= string.length()) throw new TaskIdFormatException(string);
 
         try {
-            int topicGroupId = Integer.parseInt(string.substring(0, index));
-            int partition = Integer.parseInt(string.substring(index + 1));
+            final int topicGroupId = Integer.parseInt(string.substring(0, index));
+            final int partition = Integer.parseInt(string.substring(index + 1));
 
             return new TaskId(topicGroupId, partition);
-        } catch (Exception e) {
+        } catch (final Exception e) {
             throw new TaskIdFormatException(string);
         }
     }
@@ -62,7 +62,7 @@ public class TaskId implements Comparable<TaskId> {
     /**
      * @throws IOException if cannot write to output stream
      */
-    public void writeTo(DataOutputStream out) throws IOException {
+    public void writeTo(final DataOutputStream out) throws IOException {
         out.writeInt(topicGroupId);
         out.writeInt(partition);
     }
@@ -70,26 +70,26 @@ public class TaskId implements Comparable<TaskId> {
     /**
      * @throws IOException if cannot read from input stream
      */
-    public static TaskId readFrom(DataInputStream in) throws IOException {
+    public static TaskId readFrom(final DataInputStream in) throws IOException {
         return new TaskId(in.readInt(), in.readInt());
     }
 
-    public void writeTo(ByteBuffer buf) {
+    public void writeTo(final ByteBuffer buf) {
         buf.putInt(topicGroupId);
         buf.putInt(partition);
     }
 
-    public static TaskId readFrom(ByteBuffer buf) {
+    public static TaskId readFrom(final ByteBuffer buf) {
         return new TaskId(buf.getInt(), buf.getInt());
     }
 
     @Override
-    public boolean equals(Object o) {
+    public boolean equals(final Object o) {
         if (this == o)
             return true;
 
         if (o instanceof TaskId) {
-            TaskId other = (TaskId) o;
+            final TaskId other = (TaskId) o;
             return other.topicGroupId == this.topicGroupId && other.partition == this.partition;
         } else {
             return false;
@@ -98,12 +98,12 @@ public class TaskId implements Comparable<TaskId> {
 
     @Override
     public int hashCode() {
-        long n = ((long) topicGroupId << 32) | (long) partition;
+        final long n = ((long) topicGroupId << 32) | (long) partition;
         return (int) (n % 0xFFFFFFFFL);
     }
 
     @Override
-    public int compareTo(TaskId other) {
+    public int compareTo(final TaskId other) {
         return
             this.topicGroupId < other.topicGroupId ? -1 :
                 (this.topicGroupId > other.topicGroupId ? 1 :
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TaskMetadata.java b/streams/src/main/java/org/apache/kafka/streams/processor/TaskMetadata.java
index 15337a4..a087004 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/TaskMetadata.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/TaskMetadata.java
@@ -47,14 +47,14 @@ public class TaskMetadata {
     }
 
     @Override
-    public boolean equals(Object o) {
+    public boolean equals(final Object o) {
         if (this == o) {
             return true;
         }
         if (o == null || getClass() != o.getClass()) {
             return false;
         }
-        TaskMetadata that = (TaskMetadata) o;
+        final TaskMetadata that = (TaskMetadata) o;
         return Objects.equals(taskId, that.taskId) &&
                Objects.equals(topicPartitions, that.topicPartitions);
     }
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ThreadMetadata.java b/streams/src/main/java/org/apache/kafka/streams/processor/ThreadMetadata.java
index 865d289..7f75c8d 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/ThreadMetadata.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/ThreadMetadata.java
@@ -62,14 +62,14 @@ public class ThreadMetadata {
     }
 
     @Override
-    public boolean equals(Object o) {
+    public boolean equals(final Object o) {
         if (this == o) {
             return true;
         }
         if (o == null || getClass() != o.getClass()) {
             return false;
         }
-        ThreadMetadata that = (ThreadMetadata) o;
+        final ThreadMetadata that = (ThreadMetadata) o;
         return Objects.equals(threadName, that.threadName) &&
                Objects.equals(threadState, that.threadState) &&
                Objects.equals(activeTasks, that.activeTasks) &&
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/CompositeRestoreListener.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/CompositeRestoreListener.java
index 01ba457..4783734 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/CompositeRestoreListener.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/CompositeRestoreListener.java
@@ -102,7 +102,7 @@ public class CompositeRestoreListener implements BatchingStateRestoreCallback, S
                                                 + "through the delegated StateRestoreCallback instance");
     }
 
-    private BatchingStateRestoreCallback getBatchingRestoreCallback(StateRestoreCallback restoreCallback) {
+    private BatchingStateRestoreCallback getBatchingRestoreCallback(final StateRestoreCallback restoreCallback) {
         if (restoreCallback instanceof  BatchingStateRestoreCallback) {
             return (BatchingStateRestoreCallback) restoreCallback;
         }
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java
index 30599e2..56af6e5 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java
@@ -92,7 +92,7 @@ public class GlobalProcessorContextImpl extends AbstractProcessorContext {
      * @throws UnsupportedOperationException on every invocation
      */
     @Override
-    public Cancellable schedule(long interval, PunctuationType type, Punctuator callback) {
+    public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) {
         throw new UnsupportedOperationException("this should not happen: schedule() not supported in global processor context.");
     }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java
index 0ecae5e..250105a 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java
@@ -282,7 +282,7 @@ public class InternalTopologyBuilder {
 
         @Override
         Source describe() {
-            String sourceTopics;
+            final String sourceTopics;
 
             if (pattern == null) {
                 sourceTopics = topics.toString();
@@ -1242,7 +1242,7 @@ public class InternalTopologyBuilder {
         return description;
     }
 
-    private void describeGlobalStore(final TopologyDescription description, final Set<String> nodes, int id) {
+    private void describeGlobalStore(final TopologyDescription description, final Set<String> nodes, final int id) {
         final Iterator<String> it = nodes.iterator();
         while (it.hasNext()) {
             final String node = it.next();
@@ -1627,10 +1627,10 @@ public class InternalTopologyBuilder {
     }
 
     public static class TopicsInfo {
-        public Set<String> sinkTopics;
-        public Set<String> sourceTopics;
-        public Map<String, InternalTopicConfig> stateChangelogTopics;
-        public Map<String, InternalTopicConfig> repartitionSourceTopics;
+        public final Set<String> sinkTopics;
+        public final Set<String> sourceTopics;
+        public final Map<String, InternalTopicConfig> stateChangelogTopics;
+        public final Map<String, InternalTopicConfig> repartitionSourceTopics;
 
         TopicsInfo(final Set<String> sinkTopics,
                    final Set<String> sourceTopics,
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java
index eba46c5..8fcbbb4 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java
@@ -112,7 +112,7 @@ public class ProcessorTopology {
         return sourcesByTopic.keySet();
     }
 
-    public SourceNode source(String topic) {
+    public SourceNode source(final String topic) {
         return sourcesByTopic.get(topic);
     }
 
@@ -124,7 +124,7 @@ public class ProcessorTopology {
         return sinksByTopic.keySet();
     }
 
-    public SinkNode sink(String topic) {
+    public SinkNode sink(final String topic) {
         return sinksByTopic.get(topic);
     }
 
@@ -148,17 +148,17 @@ public class ProcessorTopology {
         return storeToChangelogTopic;
     }
 
-    boolean isRepartitionTopic(String topic) {
+    boolean isRepartitionTopic(final String topic) {
         return repartitionTopics.contains(topic);
     }
 
-    private String childrenToString(String indent, List<ProcessorNode<?, ?>> children) {
+    private String childrenToString(final String indent, final List<ProcessorNode<?, ?>> children) {
         if (children == null || children.isEmpty()) {
             return "";
         }
 
-        StringBuilder sb = new StringBuilder(indent + "\tchildren:\t[");
-        for (ProcessorNode child : children) {
+        final StringBuilder sb = new StringBuilder(indent + "\tchildren:\t[");
+        for (final ProcessorNode child : children) {
             sb.append(child.name());
             sb.append(", ");
         }
@@ -166,7 +166,7 @@ public class ProcessorTopology {
         sb.append("]\n");
 
         // recursively print children
-        for (ProcessorNode<?, ?> child : children) {
+        for (final ProcessorNode<?, ?> child : children) {
             sb.append(child.toString(indent)).append(childrenToString(indent, child.children()));
         }
         return sb.toString();
@@ -191,7 +191,7 @@ public class ProcessorTopology {
         final StringBuilder sb = new StringBuilder(indent + "ProcessorTopology:\n");
 
         // start from sources
-        for (SourceNode<?, ?> source : sourcesByTopic.values()) {
+        for (final SourceNode<?, ?> source : sourcesByTopic.values()) {
             sb.append(source.toString(indent + "\t")).append(childrenToString(indent + "\t", source.children()));
         }
         return sb.toString();
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java
index 354c602..eca9ae5 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java
@@ -26,7 +26,7 @@ public class PunctuationQueue {
 
     private final PriorityQueue<PunctuationSchedule> pq = new PriorityQueue<>();
 
-    public Cancellable schedule(PunctuationSchedule sched) {
+    public Cancellable schedule(final PunctuationSchedule sched) {
         synchronized (pq) {
             pq.add(sched);
         }
@@ -47,7 +47,7 @@ public class PunctuationQueue {
             boolean punctuated = false;
             PunctuationSchedule top = pq.peek();
             while (top != null && top.timestamp <= timestamp) {
-                PunctuationSchedule sched = top;
+                final PunctuationSchedule sched = top;
                 pq.poll();
 
                 if (!sched.isCancelled()) {
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java
index b222e4c..bd8a150 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java
@@ -86,7 +86,7 @@ public class PunctuationSchedule extends Stamped<ProcessorNode> {
     }
 
     @Override
-    public boolean equals(Object other) {
+    public boolean equals(final Object other) {
         return super.equals(other);
     }
 
@@ -98,7 +98,7 @@ public class PunctuationSchedule extends Stamped<ProcessorNode> {
     private static class RepointableCancellable implements Cancellable {
         private PunctuationSchedule schedule;
 
-        synchronized void setSchedule(PunctuationSchedule schedule) {
+        synchronized void setSchedule(final PunctuationSchedule schedule) {
             this.schedule = schedule;
         }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/QuickUnion.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/QuickUnion.java
index 136f1b4..3cd3e90 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/QuickUnion.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/QuickUnion.java
@@ -21,20 +21,20 @@ import java.util.NoSuchElementException;
 
 public class QuickUnion<T> {
 
-    private HashMap<T, T> ids = new HashMap<>();
+    private final HashMap<T, T> ids = new HashMap<>();
 
-    public void add(T id) {
+    public void add(final T id) {
         ids.put(id, id);
     }
 
-    public boolean exists(T id) {
+    public boolean exists(final T id) {
         return ids.containsKey(id);
     }
 
     /**
      * @throws NoSuchElementException if the parent of this node is null
      */
-    public T root(T id) {
+    public T root(final T id) {
         T current = id;
         T parent = ids.get(current);
 
@@ -43,7 +43,7 @@ public class QuickUnion<T> {
 
         while (!parent.equals(current)) {
             // do the path splitting
-            T grandparent = ids.get(parent);
+            final T grandparent = ids.get(parent);
             ids.put(current, grandparent);
 
             current = parent;
@@ -53,15 +53,15 @@ public class QuickUnion<T> {
     }
 
     @SuppressWarnings("unchecked")
-    public void unite(T id1, T... idList) {
-        for (T id2 : idList) {
+    public void unite(final T id1, final T... idList) {
+        for (final T id2 : idList) {
             unitePair(id1, id2);
         }
     }
 
-    private void unitePair(T id1, T id2) {
-        T root1 = root(id1);
-        T root2 = root(id2);
+    private void unitePair(final T id1, final T id2) {
+        final T root1 = root(id1);
+        final T root2 = root(id2);
 
         if (!root1.equals(root2))
             ids.put(root1, root2);
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Stamped.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Stamped.java
index 0d05265..ba90558 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Stamped.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Stamped.java
@@ -23,14 +23,14 @@ public class Stamped<V> implements Comparable {
     public final V value;
     public final long timestamp;
 
-    public Stamped(V value, long timestamp) {
+    public Stamped(final V value, final long timestamp) {
         this.value = value;
         this.timestamp = timestamp;
     }
 
     @Override
-    public int compareTo(Object other) {
-        long otherTimestamp = ((Stamped<?>) other).timestamp;
+    public int compareTo(final Object other) {
+        final long otherTimestamp = ((Stamped<?>) other).timestamp;
 
         if (timestamp < otherTimestamp) return -1;
         else if (timestamp > otherTimestamp) return 1;
@@ -38,11 +38,11 @@ public class Stamped<V> implements Comparable {
     }
 
     @Override
-    public boolean equals(Object other) {
+    public boolean equals(final Object other) {
 
         if (other == null || getClass() != other.getClass()) return false;
 
-        long otherTimestamp = ((Stamped<?>) other).timestamp;
+        final long otherTimestamp = ((Stamped<?>) other).timestamp;
         return Long.compare(timestamp, otherTimestamp) == 0;
     }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java
index aa9b79d..3c6df2a 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java
@@ -21,7 +21,7 @@ import org.apache.kafka.common.header.Headers;
 
 public class StampedRecord extends Stamped<ConsumerRecord<Object, Object>> {
 
-    public StampedRecord(ConsumerRecord<Object, Object> record, long timestamp) {
+    public StampedRecord(final ConsumerRecord<Object, Object> record, final long timestamp) {
         super(record, timestamp);
     }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java
index 4c06c39..18fe704 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java
@@ -185,7 +185,7 @@ class StandbyContextImpl extends AbstractProcessorContext implements RecordColle
      * @throws UnsupportedOperationException on every invocation
      */
     @Override
-    public Cancellable schedule(long interval, PunctuationType type, Punctuator callback) {
+    public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) {
         throw new UnsupportedOperationException("this should not happen: schedule() not supported in standby tasks.");
     }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java
index 7623c66..4f12384 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java
@@ -139,7 +139,7 @@ public class StateDirectory {
 
         try {
             lockFile = new File(directoryForTask(taskId), LOCK_FILE_NAME);
-        } catch (ProcessorStateException e) {
+        } catch (final ProcessorStateException e) {
             // directoryForTask could be throwing an exception if another thread
             // has concurrently deleted the directory
             return false;
@@ -149,7 +149,7 @@ public class StateDirectory {
 
         try {
             channel = getOrCreateFileChannel(taskId, lockFile.toPath());
-        } catch (NoSuchFileException e) {
+        } catch (final NoSuchFileException e) {
             // FileChannel.open(..) could throw NoSuchFileException when there is another thread
             // concurrently deleting the parent directory (i.e. the directory of the taskId) of the lock
             // file, in this case we will return immediately indicating locking failed.
@@ -175,7 +175,7 @@ public class StateDirectory {
         final FileChannel channel;
         try {
             channel = FileChannel.open(lockFile.toPath(), StandardOpenOption.CREATE, StandardOpenOption.WRITE);
-        } catch (NoSuchFileException e) {
+        } catch (final NoSuchFileException e) {
             // FileChannel.open(..) could throw NoSuchFileException when there is another thread
             // concurrently deleting the parent directory (i.e. the directory of the taskId) of the lock
             // file, in this case we will return immediately indicating locking failed.
@@ -337,7 +337,7 @@ public class StateDirectory {
     private FileLock tryLock(final FileChannel channel) throws IOException {
         try {
             return channel.tryLock();
-        } catch (OverlappingFileLockException e) {
+        } catch (final OverlappingFileLockException e) {
             return null;
         }
     }
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateRestorer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateRestorer.java
index 3bbf42e..096ed9d 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateRestorer.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateRestorer.java
@@ -75,7 +75,7 @@ public class StateRestorer {
         compositeRestoreListener.onRestoreEnd(partition, storeName, restoredNumRecords());
     }
 
-    void restoreBatchCompleted(long currentRestoredOffset, int numRestored) {
+    void restoreBatchCompleted(final long currentRestoredOffset, final int numRestored) {
         compositeRestoreListener.onBatchRestored(partition, storeName, currentRestoredOffset, numRestored);
     }
 
@@ -87,7 +87,7 @@ public class StateRestorer {
         return persistent;
     }
 
-    void setUserRestoreListener(StateRestoreListener userRestoreListener) {
+    void setUserRestoreListener(final StateRestoreListener userRestoreListener) {
         this.compositeRestoreListener.setUserRestoreListener(userRestoreListener);
     }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java
index ea306fb..27826a6 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java
@@ -104,7 +104,7 @@ public class StreamsMetadataState {
         }
 
         final ArrayList<StreamsMetadata> results = new ArrayList<>();
-        for (StreamsMetadata metadata : allMetadata) {
+        for (final StreamsMetadata metadata : allMetadata) {
             if (metadata.stateStoreNames().contains(storeName)) {
                 results.add(metadata);
             }
@@ -195,7 +195,7 @@ public class StreamsMetadataState {
             return myMetadata;
         }
 
-        SourceTopicsInfo sourceTopicsInfo = getSourceTopicsInfo(storeName);
+        final SourceTopicsInfo sourceTopicsInfo = getSourceTopicsInfo(storeName);
         if (sourceTopicsInfo == null) {
             return null;
         }
@@ -214,7 +214,7 @@ public class StreamsMetadataState {
     }
 
     private boolean hasPartitionsForAnyTopics(final List<String> topicNames, final Set<TopicPartition> partitionForHost) {
-        for (TopicPartition topicPartition : partitionForHost) {
+        for (final TopicPartition topicPartition : partitionForHost) {
             if (topicNames.contains(topicPartition.topic())) {
                 return true;
             }
@@ -228,11 +228,11 @@ public class StreamsMetadataState {
             return;
         }
         final Map<String, List<String>> stores = builder.stateStoreNameToSourceTopics();
-        for (Map.Entry<HostInfo, Set<TopicPartition>> entry : currentState.entrySet()) {
+        for (final Map.Entry<HostInfo, Set<TopicPartition>> entry : currentState.entrySet()) {
             final HostInfo key = entry.getKey();
             final Set<TopicPartition> partitionsForHost = new HashSet<>(entry.getValue());
             final Set<String> storesOnHost = new HashSet<>();
-            for (Map.Entry<String, List<String>> storeTopicEntry : stores.entrySet()) {
+            for (final Map.Entry<String, List<String>> storeTopicEntry : stores.entrySet()) {
                 final List<String> topicsForStore = storeTopicEntry.getValue();
                 if (hasPartitionsForAnyTopics(topicsForStore, partitionsForHost)) {
                     storesOnHost.add(storeTopicEntry.getKey());
@@ -254,11 +254,11 @@ public class StreamsMetadataState {
 
         final Integer partition = partitioner.partition(sourceTopicsInfo.topicWithMostPartitions, key, null, sourceTopicsInfo.maxPartitions);
         final Set<TopicPartition> matchingPartitions = new HashSet<>();
-        for (String sourceTopic : sourceTopicsInfo.sourceTopics) {
+        for (final String sourceTopic : sourceTopicsInfo.sourceTopics) {
             matchingPartitions.add(new TopicPartition(sourceTopic, partition));
         }
 
-        for (StreamsMetadata streamsMetadata : allMetadata) {
+        for (final StreamsMetadata streamsMetadata : allMetadata) {
             final Set<String> stateStoreNames = streamsMetadata.stateStoreNames();
             final Set<TopicPartition> topicPartitions = new HashSet<>(streamsMetadata.topicPartitions());
             topicPartitions.retainAll(matchingPartitions);
@@ -289,7 +289,7 @@ public class StreamsMetadataState {
 
         private SourceTopicsInfo(final List<String> sourceTopics) {
             this.sourceTopics = sourceTopics;
-            for (String topic : sourceTopics) {
+            for (final String topic : sourceTopics) {
                 final List<PartitionInfo> partitions = clusterMetadata.partitionsForTopic(topic);
                 if (partitions.size() > maxPartitions) {
                     maxPartitions = partitions.size();
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java
index a56fb28..493f56b 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java
@@ -64,7 +64,7 @@ public class StreamsPartitionAssignor implements PartitionAssignor, Configurable
     private final static int VERSION_THREE = 3;
     private final static int VERSION_FOUR = 4;
     private final static int EARLIEST_PROBEABLE_VERSION = VERSION_THREE;
-    protected Set<Integer> supportedVersions = new HashSet<>();
+    protected final Set<Integer> supportedVersions = new HashSet<>();
 
     private Logger log;
     private String logPrefix;
@@ -980,7 +980,7 @@ public class StreamsPartitionAssignor implements PartitionAssignor, Configurable
 
     static class CopartitionedTopicsValidator {
         private final String logPrefix;
-        private Logger log;
+        private final Logger log;
 
         CopartitionedTopicsValidator(final String logPrefix) {
             this.logPrefix = logPrefix;
@@ -997,7 +997,7 @@ public class StreamsPartitionAssignor implements PartitionAssignor, Configurable
                 if (!allRepartitionTopicsNumPartitions.containsKey(topic)) {
                     final Integer partitions = metadata.partitionCountForTopic(topic);
                     if (partitions == null) {
-                        String str = String.format("%sTopic not found: %s", logPrefix, topic);
+                        final String str = String.format("%sTopic not found: %s", logPrefix, topic);
                         log.error(str);
                         throw new IllegalStateException(str);
                     }
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java
index 9da2702..e251337 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java
@@ -323,7 +323,7 @@ public class TaskManager {
         active.updateRestored(restored);
 
         if (active.allTasksRunning()) {
-            Set<TopicPartition> assignment = consumer.assignment();
+            final Set<TopicPartition> assignment = consumer.assignment();
             log.trace("Resuming partitions {}", assignment);
             consumer.resume(assignment);
             assignStandbyPartitions();
@@ -373,7 +373,7 @@ public class TaskManager {
         this.assignedStandbyTasks = standbyTasks;
     }
 
-    public void updateSubscriptionsFromAssignment(List<TopicPartition> partitions) {
+    public void updateSubscriptionsFromAssignment(final List<TopicPartition> partitions) {
         if (builder().sourceTopicPattern() != null) {
             final Set<String> assignedTopics = new HashSet<>();
             for (final TopicPartition topicPartition : partitions) {
@@ -388,7 +388,7 @@ public class TaskManager {
         }
     }
 
-    public void updateSubscriptionsFromMetadata(Set<String> topics) {
+    public void updateSubscriptionsFromMetadata(final Set<String> topics) {
         if (builder().sourceTopicPattern() != null) {
             final Collection<String> existingTopics = builder().subscriptionUpdates().getUpdates();
             if (!existingTopics.equals(topics)) {
@@ -402,7 +402,7 @@ public class TaskManager {
      *                               or if the task producer got fenced (EOS)
      */
     int commitAll() {
-        int committed = active.commit();
+        final int committed = active.commit();
         return committed + standby.commit();
     }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/WrappedBatchingStateRestoreCallback.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/WrappedBatchingStateRestoreCallback.java
index 02814d1..b469b38 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/WrappedBatchingStateRestoreCallback.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/WrappedBatchingStateRestoreCallback.java
@@ -33,7 +33,7 @@ public class WrappedBatchingStateRestoreCallback implements BatchingStateRestore
 
     @Override
     public void restoreAll(final Collection<KeyValue<byte[], byte[]>> records) {
-        for (KeyValue<byte[], byte[]> record : records) {
+        for (final KeyValue<byte[], byte[]> record : records) {
             restore(record.key, record.value);
         }
     }
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java
index 1179ca0..8ad4036 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java
@@ -234,7 +234,7 @@ public class AssignmentInfo {
             final AssignmentInfo assignmentInfo;
 
             final int usedVersion = in.readInt();
-            int latestSupportedVersion;
+            final int latestSupportedVersion;
             switch (usedVersion) {
                 case 1:
                     assignmentInfo = new AssignmentInfo(usedVersion, UNKNOWN);
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java
index 8767d0f..7b33813 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java
@@ -109,7 +109,7 @@ public class StickyTaskAssignor<ID> implements TaskAssignor<ID, TaskId> {
         }
 
         // assign any remaining unassigned tasks
-        List<TaskId> sortedTasks = new ArrayList<>(unassigned);
+        final List<TaskId> sortedTasks = new ArrayList<>(unassigned);
         Collections.sort(sortedTasks);
         for (final TaskId taskId : sortedTasks) {
             allocateTaskWithClientCandidates(taskId, clients.keySet(), true);
@@ -139,7 +139,7 @@ public class StickyTaskAssignor<ID> implements TaskAssignor<ID, TaskId> {
     }
 
 
-    private ClientState findClient(final TaskId taskId, final Set<ID> clientsWithin, boolean active) {
+    private ClientState findClient(final TaskId taskId, final Set<ID> clientsWithin, final boolean active) {
 
         // optimize the case where there is only 1 id to search within.
         if (clientsWithin.size() == 1) {
@@ -168,7 +168,7 @@ public class StickyTaskAssignor<ID> implements TaskAssignor<ID, TaskId> {
     }
 
     private boolean hasClientsWithMoreAvailableCapacity(final ClientState client) {
-        for (ClientState clientState : clients.values()) {
+        for (final ClientState clientState : clients.values()) {
             if (clientState.hasMoreAvailableCapacityThan(client)) {
                 return true;
             }
@@ -245,7 +245,7 @@ public class StickyTaskAssignor<ID> implements TaskAssignor<ID, TaskId> {
 
     private int sumCapacity(final Collection<ClientState> values) {
         int capacity = 0;
-        for (ClientState client : values) {
+        for (final ClientState client : values) {
             capacity += client.capacity();
         }
         return capacity;
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java
index e99a5b3..d36acdc 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java
@@ -308,8 +308,8 @@ public class StreamsMetricsImpl implements StreamsMetrics {
     }
 
 
-    private String buildUniqueSensorName(String operationName, String taskName) {
-        String task = taskName == null ? "" : taskName + ".";
+    private String buildUniqueSensorName(final String operationName, final String taskName) {
+        final String task = taskName == null ? "" : taskName + ".";
         return threadName + "." + task + operationName;
     }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/QueryableStoreTypes.java b/streams/src/main/java/org/apache/kafka/streams/state/QueryableStoreTypes.java
index 4d07f9b..f7a9970 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/QueryableStoreTypes.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/QueryableStoreTypes.java
@@ -63,7 +63,7 @@ public class QueryableStoreTypes {
 
         private final Class matchTo;
 
-        QueryableStoreTypeMatcher(Class matchTo) {
+        QueryableStoreTypeMatcher(final Class matchTo) {
             this.matchTo = matchTo;
         }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java b/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java
index ec7803a..55e9fde 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java
@@ -145,7 +145,7 @@ public final class StateSerdes<K, V> {
      * @param rawKey  the key as raw bytes
      * @return        the key as typed object
      */
-    public K keyFrom(byte[] rawKey) {
+    public K keyFrom(final byte[] rawKey) {
         return keySerde.deserializer().deserialize(topic, rawKey);
     }
 
@@ -155,7 +155,7 @@ public final class StateSerdes<K, V> {
      * @param rawValue  the value as raw bytes
      * @return          the value as typed object
      */
-    public V valueFrom(byte[] rawValue) {
+    public V valueFrom(final byte[] rawValue) {
         return valueSerde.deserializer().deserialize(topic, rawValue);
     }
 
@@ -165,7 +165,7 @@ public final class StateSerdes<K, V> {
      * @param key  the key to be serialized
      * @return     the serialized key
      */
-    public byte[] rawKey(K key) {
+    public byte[] rawKey(final K key) {
         try {
             return keySerde.serializer().serialize(topic, key);
         } catch (final ClassCastException e) {
@@ -186,7 +186,7 @@ public final class StateSerdes<K, V> {
      * @param value  the value to be serialized
      * @return       the serialized value
      */
-    public byte[] rawValue(V value) {
+    public byte[] rawValue(final V value) {
         try {
             return valueSerde.serializer().serialize(topic, value);
         } catch (final ClassCastException e) {
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java
index 1a3b075..dfcd763 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java
@@ -98,7 +98,7 @@ abstract class AbstractMergedSortedCacheStoreIterator<K, KS, V, VS> implements K
         }
     }
 
-    private KeyValue<K, V> nextStoreValue(KS nextStoreKey) {
+    private KeyValue<K, V> nextStoreValue(final KS nextStoreKey) {
         final KeyValue<KS, VS> next = storeIterator.next();
 
         if (!next.key.equals(nextStoreKey)) {
@@ -108,7 +108,7 @@ abstract class AbstractMergedSortedCacheStoreIterator<K, KS, V, VS> implements K
         return deserializeStorePair(next);
     }
 
-    private KeyValue<K, V> nextCacheValue(Bytes nextCacheKey) {
+    private KeyValue<K, V> nextCacheValue(final Bytes nextCacheKey) {
         final KeyValue<Bytes, LRUCacheEntry> next = cacheIterator.next();
 
         if (!next.key.equals(nextCacheKey)) {
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
index da308a1..c016f64 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
@@ -47,7 +47,7 @@ class CachingKeyValueStore<K, V> extends WrappedStateStore.AbstractStateStore im
     private InternalProcessorContext context;
     private StateSerdes<K, V> serdes;
     private Thread streamThread;
-    private ReadWriteLock lock = new ReentrantReadWriteLock();
+    private final ReadWriteLock lock = new ReentrantReadWriteLock();
 
     CachingKeyValueStore(final KeyValueStore<Bytes, byte[]> underlying,
                          final Serde<K> keySerde,
@@ -79,7 +79,7 @@ class CachingKeyValueStore<K, V> extends WrappedStateStore.AbstractStateStore im
         cache.addDirtyEntryFlushListener(cacheName, new ThreadCache.DirtyEntryFlushListener() {
             @Override
             public void apply(final List<ThreadCache.DirtyEntry> entries) {
-                for (ThreadCache.DirtyEntry entry : entries) {
+                for (final ThreadCache.DirtyEntry entry : entries) {
                     putAndMaybeForward(entry, (InternalProcessorContext) context);
                 }
             }
@@ -148,7 +148,7 @@ class CachingKeyValueStore<K, V> extends WrappedStateStore.AbstractStateStore im
     public byte[] get(final Bytes key) {
         Objects.requireNonNull(key, "key cannot be null");
         validateStoreOpen();
-        Lock theLock;
+        final Lock theLock;
         if (Thread.currentThread().equals(streamThread)) {
             theLock = lock.writeLock();
         } else {
@@ -258,7 +258,7 @@ class CachingKeyValueStore<K, V> extends WrappedStateStore.AbstractStateStore im
         validateStoreOpen();
         lock.writeLock().lock();
         try {
-            for (KeyValue<Bytes, byte[]> entry : entries) {
+            for (final KeyValue<Bytes, byte[]> entry : entries) {
                 Objects.requireNonNull(entry.key, "key cannot be null");
                 put(entry.key, entry.value);
             }
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java
index c307f6d..2da5ab9 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java
@@ -83,7 +83,7 @@ class CachingSessionStore<K, AGG> extends WrappedStateStore.AbstractStateStore i
         cache.addDirtyEntryFlushListener(cacheName, new ThreadCache.DirtyEntryFlushListener() {
             @Override
             public void apply(final List<ThreadCache.DirtyEntry> entries) {
-                for (ThreadCache.DirtyEntry entry : entries) {
+                for (final ThreadCache.DirtyEntry entry : entries) {
                     putAndMaybeForward(entry, context);
                 }
             }
@@ -138,7 +138,7 @@ class CachingSessionStore<K, AGG> extends WrappedStateStore.AbstractStateStore i
     }
 
     @Override
-    public void put(final Windowed<Bytes> key, byte[] value) {
+    public void put(final Windowed<Bytes> key, final byte[] value) {
         validateStoreOpen();
         final Bytes binaryKey = Bytes.wrap(SessionKeySchema.toBinary(key));
         final LRUCacheEntry entry =
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java
index 07120df..4347811 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java
@@ -89,7 +89,7 @@ class CachingWindowStore<K, V> extends WrappedStateStore.AbstractStateStore impl
         cache.addDirtyEntryFlushListener(name, new ThreadCache.DirtyEntryFlushListener() {
             @Override
             public void apply(final List<ThreadCache.DirtyEntry> entries) {
-                for (ThreadCache.DirtyEntry entry : entries) {
+                for (final ThreadCache.DirtyEntry entry : entries) {
                     final byte[] binaryWindowKey = cacheFunction.key(entry.key()).get();
                     final long timestamp = WindowKeySchema.extractStoreTimestamp(binaryWindowKey);
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java
index 1fcf60e..1d8fb58 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java
@@ -47,7 +47,7 @@ public class ChangeLoggingKeyValueBytesStore extends WrappedStateStore.AbstractS
         if (inner instanceof MemoryLRUCache) {
             ((MemoryLRUCache<Bytes, byte[]>) inner).whenEldestRemoved(new MemoryLRUCache.EldestEntryRemovalListener<Bytes, byte[]>() {
                 @Override
-                public void apply(Bytes key, byte[] value) {
+                public void apply(final Bytes key, final byte[] value) {
                     // pass null to indicate removal
                     changeLogger.logChange(key, null);
                 }
@@ -78,7 +78,7 @@ public class ChangeLoggingKeyValueBytesStore extends WrappedStateStore.AbstractS
     @Override
     public void putAll(final List<KeyValue<Bytes, byte[]>> entries) {
         inner.putAll(entries);
-        for (KeyValue<Bytes, byte[]> entry : entries) {
+        for (final KeyValue<Bytes, byte[]> entry : entries) {
             changeLogger.logChange(entry.key, entry.value);
         }
     }
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java
index 89d7260..aa9cbe6 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java
@@ -57,7 +57,7 @@ class ChangeLoggingWindowBytesStore extends WrappedStateStore.AbstractStateStore
     }
 
     @Override
-    public KeyValueIterator<Windowed<Bytes>, byte[]> fetch(Bytes keyFrom, Bytes keyTo, long from, long to) {
+    public KeyValueIterator<Windowed<Bytes>, byte[]> fetch(final Bytes keyFrom, final Bytes keyTo, final long from, final long to) {
         return bytesStore.fetch(keyFrom, keyTo, from, to);
     }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStore.java
index 2c895ef..c790b89 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStore.java
@@ -50,13 +50,13 @@ public class CompositeReadOnlyKeyValueStore<K, V> implements ReadOnlyKeyValueSto
     public V get(final K key) {
         Objects.requireNonNull(key);
         final List<ReadOnlyKeyValueStore<K, V>> stores = storeProvider.stores(storeName, storeType);
-        for (ReadOnlyKeyValueStore<K, V> store : stores) {
+        for (final ReadOnlyKeyValueStore<K, V> store : stores) {
             try {
                 final V result = store.get(key);
                 if (result != null) {
                     return result;
                 }
-            } catch (InvalidStateStoreException e) {
+            } catch (final InvalidStateStoreException e) {
                 throw new InvalidStateStoreException("State store is not available anymore and may have been migrated to another instance; please re-discover its location from the state metadata.");
             }
 
@@ -73,7 +73,7 @@ public class CompositeReadOnlyKeyValueStore<K, V> implements ReadOnlyKeyValueSto
             public KeyValueIterator<K, V> apply(final ReadOnlyKeyValueStore<K, V> store) {
                 try {
                     return store.range(from, to);
-                } catch (InvalidStateStoreException e) {
+                } catch (final InvalidStateStoreException e) {
                     throw new InvalidStateStoreException("State store is not available anymore and may have been migrated to another instance; please re-discover its location from the state metadata.");
                 }
             }
@@ -89,7 +89,7 @@ public class CompositeReadOnlyKeyValueStore<K, V> implements ReadOnlyKeyValueSto
             public KeyValueIterator<K, V> apply(final ReadOnlyKeyValueStore<K, V> store) {
                 try {
                     return store.all();
-                } catch (InvalidStateStoreException e) {
+                } catch (final InvalidStateStoreException e) {
                     throw new InvalidStateStoreException("State store is not available anymore and may have been migrated to another instance; please re-discover its location from the state metadata.");
                 }
             }
@@ -102,7 +102,7 @@ public class CompositeReadOnlyKeyValueStore<K, V> implements ReadOnlyKeyValueSto
     public long approximateNumEntries() {
         final List<ReadOnlyKeyValueStore<K, V>> stores = storeProvider.stores(storeName, storeType);
         long total = 0;
-        for (ReadOnlyKeyValueStore<K, V> store : stores) {
+        for (final ReadOnlyKeyValueStore<K, V> store : stores) {
             total += store.approximateNumEntries();
             if (total < 0) {
                 return Long.MAX_VALUE;
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/FilteredCacheIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/FilteredCacheIterator.java
index 4486fda..a26f8cf 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/FilteredCacheIterator.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/FilteredCacheIterator.java
@@ -57,7 +57,7 @@ class FilteredCacheIterator implements PeekingKeyValueIterator<Bytes, LRUCacheEn
                 return cachedPair(cacheIterator.next());
             }
 
-            private KeyValue<Bytes, LRUCacheEntry> cachedPair(KeyValue<Bytes, LRUCacheEntry> next) {
+            private KeyValue<Bytes, LRUCacheEntry> cachedPair(final KeyValue<Bytes, LRUCacheEntry> next) {
                 return KeyValue.pair(cacheFunction.key(next.key), next.value);
             }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProvider.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProvider.java
index 29f2aee..0db69d0 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProvider.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProvider.java
@@ -27,7 +27,7 @@ import java.util.Map;
 public class GlobalStateStoreProvider implements StateStoreProvider {
     private final Map<String, StateStore> globalStateStores;
 
-    public GlobalStateStoreProvider(Map<String, StateStore> globalStateStores) {
+    public GlobalStateStoreProvider(final Map<String, StateStore> globalStateStores) {
         this.globalStateStores = globalStateStores;
     }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java
index c3fe070..4dccd6e 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java
@@ -35,7 +35,7 @@ public class InMemoryKeyValueLoggedStore<K, V> extends WrappedStateStore.Abstrac
 
     private StoreChangeLogger<K, V> changeLogger;
 
-    public InMemoryKeyValueLoggedStore(final KeyValueStore<K, V> inner, Serde<K> keySerde, Serde<V> valueSerde) {
+    public InMemoryKeyValueLoggedStore(final KeyValueStore<K, V> inner, final Serde<K> keySerde, final Serde<V> valueSerde) {
         super(inner);
         this.inner = inner;
         this.keySerde = keySerde;
@@ -44,11 +44,11 @@ public class InMemoryKeyValueLoggedStore<K, V> extends WrappedStateStore.Abstrac
 
     @Override
     @SuppressWarnings("unchecked")
-    public void init(ProcessorContext context, StateStore root) {
+    public void init(final ProcessorContext context, final StateStore root) {
         inner.init(context, root);
 
         // construct the serde
-        StateSerdes<K, V>  serdes = new StateSerdes<>(
+        final StateSerdes<K, V>  serdes = new StateSerdes<>(
             ProcessorStateManager.storeChangelogTopic(context.applicationId(), inner.name()),
             keySerde == null ? (Serde<K>) context.keySerde() : keySerde,
             valueSerde == null ? (Serde<V>) context.valueSerde() : valueSerde);
@@ -59,7 +59,7 @@ public class InMemoryKeyValueLoggedStore<K, V> extends WrappedStateStore.Abstrac
         if (inner instanceof MemoryLRUCache) {
             ((MemoryLRUCache<K, V>) inner).whenEldestRemoved(new MemoryNavigableLRUCache.EldestEntryRemovalListener<K, V>() {
                 @Override
-                public void apply(K key, V value) {
+                public void apply(final K key, final V value) {
                     removed(key);
                 }
             });
@@ -72,20 +72,20 @@ public class InMemoryKeyValueLoggedStore<K, V> extends WrappedStateStore.Abstrac
     }
 
     @Override
-    public V get(K key) {
+    public V get(final K key) {
         return this.inner.get(key);
     }
 
     @Override
-    public void put(K key, V value) {
+    public void put(final K key, final V value) {
         this.inner.put(key, value);
 
         changeLogger.logChange(key, value);
     }
 
     @Override
-    public V putIfAbsent(K key, V value) {
-        V originalValue = this.inner.putIfAbsent(key, value);
+    public V putIfAbsent(final K key, final V value) {
+        final V originalValue = this.inner.putIfAbsent(key, value);
         if (originalValue == null) {
             changeLogger.logChange(key, value);
         }
@@ -93,18 +93,18 @@ public class InMemoryKeyValueLoggedStore<K, V> extends WrappedStateStore.Abstrac
     }
 
     @Override
-    public void putAll(List<KeyValue<K, V>> entries) {
+    public void putAll(final List<KeyValue<K, V>> entries) {
         this.inner.putAll(entries);
 
-        for (KeyValue<K, V> entry : entries) {
-            K key = entry.key;
+        for (final KeyValue<K, V> entry : entries) {
+            final K key = entry.key;
             changeLogger.logChange(key, entry.value);
         }
     }
 
     @Override
-    public V delete(K key) {
-        V value = this.inner.delete(key);
+    public V delete(final K key) {
+        final V value = this.inner.delete(key);
 
         removed(key);
 
@@ -117,12 +117,12 @@ public class InMemoryKeyValueLoggedStore<K, V> extends WrappedStateStore.Abstrac
      *
      * @param key the key for the entry that the inner store removed
      */
-    protected void removed(K key) {
+    protected void removed(final K key) {
         changeLogger.logChange(key, null);
     }
 
     @Override
-    public KeyValueIterator<K, V> range(K from, K to) {
+    public KeyValueIterator<K, V> range(final K from, final K to) {
         return this.inner.range(from, to);
     }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java
index 1957aa4..582eb46 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java
@@ -73,8 +73,8 @@ public class MemoryLRUCache<K, V> implements KeyValueStore<K, V> {
             private static final long serialVersionUID = 1L;
 
             @Override
-            protected boolean removeEldestEntry(Map.Entry<K, V> eldest) {
-                boolean evict = super.size() > maxCacheSize;
+            protected boolean removeEldestEntry(final Map.Entry<K, V> eldest) {
+                final boolean evict = super.size() > maxCacheSize;
                 if (evict && !restoring && listener != null) {
                     listener.apply(eldest.getKey(), eldest.getValue());
                 }
@@ -100,7 +100,7 @@ public class MemoryLRUCache<K, V> implements KeyValueStore<K, V> {
 
     @Override
     @SuppressWarnings("unchecked")
-    public void init(ProcessorContext context, StateStore root) {
+    public void init(final ProcessorContext context, final StateStore root) {
         // construct the serde
         this.serdes = new StateSerdes<>(
             ProcessorStateManager.storeChangelogTopic(context.applicationId(), name),
@@ -110,7 +110,7 @@ public class MemoryLRUCache<K, V> implements KeyValueStore<K, V> {
         // register the store
         context.register(root, new StateRestoreCallback() {
             @Override
-            public void restore(byte[] key, byte[] value) {
+            public void restore(final byte[] key, final byte[] value) {
                 restoring = true;
                 // check value for null, to avoid  deserialization error.
                 if (value == null) {
@@ -153,7 +153,7 @@ public class MemoryLRUCache<K, V> implements KeyValueStore<K, V> {
     @Override
     public synchronized V putIfAbsent(final K key, final V value) {
         Objects.requireNonNull(key);
-        V originalValue = get(key);
+        final V originalValue = get(key);
         if (originalValue == null) {
             put(key, value);
         }
@@ -162,7 +162,7 @@ public class MemoryLRUCache<K, V> implements KeyValueStore<K, V> {
 
     @Override
     public void putAll(final List<KeyValue<K, V>> entries) {
-        for (KeyValue<K, V> entry : entries)
+        for (final KeyValue<K, V> entry : entries)
             put(entry.key, entry.value);
     }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryNavigableLRUCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryNavigableLRUCache.java
index 9500e8e..d7b7b11 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryNavigableLRUCache.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryNavigableLRUCache.java
@@ -27,12 +27,12 @@ import java.util.TreeMap;
 public class MemoryNavigableLRUCache<K, V> extends MemoryLRUCache<K, V> {
 
 
-    public MemoryNavigableLRUCache(String name, final int maxCacheSize, Serde<K> keySerde, Serde<V> valueSerde) {
+    public MemoryNavigableLRUCache(final String name, final int maxCacheSize, final Serde<K> keySerde, final Serde<V> valueSerde) {
         super(name, maxCacheSize, keySerde, valueSerde);
     }
 
     @Override
-    public KeyValueIterator<K, V> range(K from, K to) {
+    public KeyValueIterator<K, V> range(final K from, final K to) {
         final TreeMap<K, V> treeMap = toTreeMap();
         return new DelegatingPeekingKeyValueIterator<>(name(), new MemoryNavigableLRUCache.CacheIterator<>(treeMap.navigableKeySet().subSet(from, true, to, true).iterator(), treeMap));
     }
@@ -53,7 +53,7 @@ public class MemoryNavigableLRUCache<K, V> extends MemoryLRUCache<K, V> {
         private final Map<K, V> entries;
         private K lastKey;
 
-        public CacheIterator(Iterator<K> keys, Map<K, V> entries) {
+        public CacheIterator(final Iterator<K> keys, final Map<K, V> entries) {
             this.keys = keys;
             this.entries = entries;
         }
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIterator.java
index d5bb421..98b3f06 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIterator.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIterator.java
@@ -42,7 +42,7 @@ class MergedSortedCacheWindowStoreIterator extends AbstractMergedSortedCacheStor
 
     @Override
     Long deserializeCacheKey(final Bytes cacheKey) {
-        byte[] binaryKey = bytesFromCacheKey(cacheKey);
+        final byte[] binaryKey = bytesFromCacheKey(cacheKey);
         return WindowKeySchema.extractStoreTimestamp(binaryKey);
     }
 
@@ -58,7 +58,7 @@ class MergedSortedCacheWindowStoreIterator extends AbstractMergedSortedCacheStor
 
     @Override
     public int compare(final Bytes cacheKey, final Long storeKey) {
-        byte[] binaryKey = bytesFromCacheKey(cacheKey);
+        final byte[] binaryKey = bytesFromCacheKey(cacheKey);
 
         final Long cacheTimestamp = WindowKeySchema.extractStoreTimestamp(binaryKey);
         return cacheTimestamp.compareTo(storeKey);
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java
index 3e881ed..b285b65 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java
@@ -140,7 +140,7 @@ public class MeteredSessionStore<K, V> extends WrappedStateStore.AbstractStateSt
     @Override
     public void put(final Windowed<K> sessionKey, final V aggregate) {
         Objects.requireNonNull(sessionKey, "sessionKey can't be null");
-        long startNs = time.nanoseconds();
+        final long startNs = time.nanoseconds();
         try {
             final Bytes key = keyBytes(sessionKey.key());
             this.inner.put(new Windowed<>(key, sessionKey.window()), serdes.rawValue(aggregate));
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/OffsetCheckpoint.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/OffsetCheckpoint.java
index c8367b3..34e0947 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/OffsetCheckpoint.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/OffsetCheckpoint.java
@@ -74,7 +74,7 @@ public class OffsetCheckpoint {
             final File temp = new File(file.getAbsolutePath() + ".tmp");
 
             final FileOutputStream fileOutputStream = new FileOutputStream(temp);
-            try (BufferedWriter writer = new BufferedWriter(
+            try (final BufferedWriter writer = new BufferedWriter(
                     new OutputStreamWriter(fileOutputStream, StandardCharsets.UTF_8))) {
                 writeIntLine(writer, VERSION);
                 writeIntLine(writer, offsets.size());
@@ -121,7 +121,7 @@ public class OffsetCheckpoint {
      */
     public Map<TopicPartition, Long> read() throws IOException {
         synchronized (lock) {
-            try (BufferedReader reader = Files.newBufferedReader(file.toPath())) {
+            try (final BufferedReader reader = Files.newBufferedReader(file.toPath())) {
                 final int version = readInt(reader);
                 switch (version) {
                     case 0:
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/OrderedBytes.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/OrderedBytes.java
index c0d1c3b..cd6b6ad 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/OrderedBytes.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/OrderedBytes.java
@@ -29,9 +29,9 @@ class OrderedBytes {
      *
      * Assumes the minimum key length is one byte
      */
-    static Bytes upperRange(Bytes key, byte[] maxSuffix) {
+    static Bytes upperRange(final Bytes key, final byte[] maxSuffix) {
         final byte[] bytes = key.get();
-        ByteBuffer rangeEnd = ByteBuffer.allocate(bytes.length + maxSuffix.length);
+        final ByteBuffer rangeEnd = ByteBuffer.allocate(bytes.length + maxSuffix.length);
 
         int i = 0;
         while (i < bytes.length && (
@@ -44,14 +44,14 @@ class OrderedBytes {
         rangeEnd.put(maxSuffix);
         rangeEnd.flip();
 
-        byte[] res = new byte[rangeEnd.remaining()];
+        final byte[] res = new byte[rangeEnd.remaining()];
         ByteBuffer.wrap(res).put(rangeEnd);
         return Bytes.wrap(res);
     }
 
-    static Bytes lowerRange(Bytes key, byte[] minSuffix) {
+    static Bytes lowerRange(final Bytes key, final byte[] minSuffix) {
         final byte[] bytes = key.get();
-        ByteBuffer rangeStart = ByteBuffer.allocate(bytes.length + minSuffix.length);
+        final ByteBuffer rangeStart = ByteBuffer.allocate(bytes.length + minSuffix.length);
         // any key in the range would start at least with the given prefix to be
         // in the range, and have at least SUFFIX_SIZE number of trailing zero bytes.
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/QueryableStoreProvider.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/QueryableStoreProvider.java
index 971dcd3..4270d27 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/QueryableStoreProvider.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/QueryableStoreProvider.java
@@ -53,7 +53,7 @@ public class QueryableStoreProvider {
             return queryableStoreType.create(new WrappingStoreProvider(Collections.<StateStoreProvider>singletonList(globalStoreProvider)), storeName);
         }
         final List<T> allStores = new ArrayList<>();
-        for (StateStoreProvider storeProvider : storeProviders) {
+        for (final StateStoreProvider storeProvider : storeProviders) {
             allStores.addAll(storeProvider.stores(storeName, queryableStoreType));
         }
         if (allStores.isEmpty()) {
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStore.java
index 5f1ec37..6a9284c 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStore.java
@@ -69,7 +69,7 @@ class RocksDBSegmentedBytesStore implements SegmentedBytesStore {
     }
 
     @Override
-    public KeyValueIterator<Bytes, byte[]> fetch(final Bytes keyFrom, Bytes keyTo, final long from, final long to) {
+    public KeyValueIterator<Bytes, byte[]> fetch(final Bytes keyFrom, final Bytes keyTo, final long from, final long to) {
         final List<Segment> searchSpace = keySchema.segmentsToSearch(segments, from, to);
 
         final Bytes binaryFrom = keySchema.lowerRange(keyFrom, from);
@@ -134,7 +134,7 @@ class RocksDBSegmentedBytesStore implements SegmentedBytesStore {
     }
 
     @Override
-    public void init(ProcessorContext context, StateStore root) {
+    public void init(final ProcessorContext context, final StateStore root) {
         this.context = (InternalProcessorContext) context;
 
         keySchema.init(ProcessorStateManager.storeChangelogTopic(context.applicationId(), root.name()));
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java
index f902cba..5d9c045 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java
@@ -79,7 +79,7 @@ public class RocksDBSessionStore<K, AGG> extends WrappedStateStore.AbstractState
     }
 
     @Override
-    public KeyValueIterator<Windowed<K>, AGG> fetch(K from, K to) {
+    public KeyValueIterator<Windowed<K>, AGG> fetch(final K from, final K to) {
         return findSessions(from, to, 0, Long.MAX_VALUE);
     }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java
index cb00747..c0e8363 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java
@@ -96,11 +96,11 @@ public class RocksDBStore implements KeyValueStore<Bytes, byte[]> {
 
     protected volatile boolean open = false;
 
-    RocksDBStore(String name) {
+    RocksDBStore(final String name) {
         this(name, DB_FILE_DIR);
     }
 
-    RocksDBStore(String name, String parentDir) {
+    RocksDBStore(final String name, final String parentDir) {
         this.name = name;
         this.parentDir = parentDir;
     }
@@ -153,7 +153,7 @@ public class RocksDBStore implements KeyValueStore<Bytes, byte[]> {
 
         try {
             this.db = openDB(this.dbDir, this.options, TTL_SECONDS);
-        } catch (IOException e) {
+        } catch (final IOException e) {
             throw new ProcessorStateException(e);
         }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segment.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segment.java
index d107812..f95a104 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segment.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segment.java
@@ -25,7 +25,7 @@ import java.util.Objects;
 class Segment extends RocksDBStore implements Comparable<Segment> {
     public final long id;
 
-    Segment(String segmentName, String windowName, long id) {
+    Segment(final String segmentName, final String windowName, final long id) {
         super(segmentName, windowName);
         this.id = id;
     }
@@ -35,7 +35,7 @@ class Segment extends RocksDBStore implements Comparable<Segment> {
     }
 
     @Override
-    public int compareTo(Segment segment) {
+    public int compareTo(final Segment segment) {
         return Long.compare(id, segment.id);
     }
 
@@ -53,10 +53,10 @@ class Segment extends RocksDBStore implements Comparable<Segment> {
     }
 
     @Override
-    public boolean equals(Object obj) {
+    public boolean equals(final Object obj) {
         if (obj == null || getClass() != obj.getClass()) return false;
 
-        Segment segment = (Segment) obj;
+        final Segment segment = (Segment) obj;
         return Long.compare(id, segment.id) == 0;
     }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java
index 331ffdb..5b07812 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java
@@ -76,7 +76,7 @@ class SegmentIterator implements KeyValueIterator<Bytes, byte[]> {
                 } else {
                     currentIterator = currentSegment.range(from, to);
                 }
-            } catch (InvalidStateStoreException e) {
+            } catch (final InvalidStateStoreException e) {
                 // segment may have been closed so we ignore it.
             }
         }
@@ -87,7 +87,7 @@ class SegmentIterator implements KeyValueIterator<Bytes, byte[]> {
         boolean hasNext = false;
         try {
             hasNext = hasNextCondition.hasNext(currentIterator);
-        } catch (InvalidStateStoreException e) {
+        } catch (final InvalidStateStoreException e) {
             //already closed so ignore
         }
         return hasNext;
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedCacheFunction.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedCacheFunction.java
index 3ab7930..8f5768c 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedCacheFunction.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedCacheFunction.java
@@ -29,42 +29,42 @@ class SegmentedCacheFunction implements CacheFunction {
     private final KeySchema keySchema;
     private final long segmentInterval;
 
-    SegmentedCacheFunction(KeySchema keySchema, long segmentInterval) {
+    SegmentedCacheFunction(final KeySchema keySchema, final long segmentInterval) {
         this.keySchema = keySchema;
         this.segmentInterval = segmentInterval;
     }
 
     @Override
-    public Bytes key(Bytes cacheKey) {
+    public Bytes key(final Bytes cacheKey) {
         return Bytes.wrap(bytesFromCacheKey(cacheKey));
     }
 
     @Override
-    public Bytes cacheKey(Bytes key) {
+    public Bytes cacheKey(final Bytes key) {
         final byte[] keyBytes = key.get();
-        ByteBuffer buf = ByteBuffer.allocate(SEGMENT_ID_BYTES + keyBytes.length);
+        final ByteBuffer buf = ByteBuffer.allocate(SEGMENT_ID_BYTES + keyBytes.length);
         buf.putLong(segmentId(key)).put(keyBytes);
         return Bytes.wrap(buf.array());
     }
 
-    static byte[] bytesFromCacheKey(Bytes cacheKey) {
-        byte[] binaryKey = new byte[cacheKey.get().length - SEGMENT_ID_BYTES];
+    static byte[] bytesFromCacheKey(final Bytes cacheKey) {
+        final byte[] binaryKey = new byte[cacheKey.get().length - SEGMENT_ID_BYTES];
         System.arraycopy(cacheKey.get(), SEGMENT_ID_BYTES, binaryKey, 0, binaryKey.length);
         return binaryKey;
     }
     
-    public long segmentId(Bytes key) {
+    public long segmentId(final Bytes key) {
         return keySchema.segmentTimestamp(key) / segmentInterval;
     }
 
-    int compareSegmentedKeys(Bytes cacheKey, Bytes storeKey) {
-        long storeSegmentId = segmentId(storeKey);
-        long cacheSegmentId = ByteBuffer.wrap(cacheKey.get()).getLong();
+    int compareSegmentedKeys(final Bytes cacheKey, final Bytes storeKey) {
+        final long storeSegmentId = segmentId(storeKey);
+        final long cacheSegmentId = ByteBuffer.wrap(cacheKey.get()).getLong();
 
         final int segmentCompare = Long.compare(cacheSegmentId, storeSegmentId);
         if (segmentCompare == 0) {
-            byte[] cacheKeyBytes = cacheKey.get();
-            byte[] storeKeyBytes = storeKey.get();
+            final byte[] cacheKeyBytes = cacheKey.get();
+            final byte[] storeKeyBytes = storeKey.get();
             return Bytes.BYTES_LEXICO_COMPARATOR.compare(
                 cacheKeyBytes, SEGMENT_ID_BYTES, cacheKeyBytes.length - SEGMENT_ID_BYTES,
                 storeKeyBytes, 0, storeKeyBytes.length
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java
index 181d409..debef9a 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java
@@ -57,7 +57,7 @@ public class SessionKeySchema implements SegmentedBytesStore.KeySchema {
     }
 
     @Override
-    public Bytes upperRange(Bytes key, long to) {
+    public Bytes upperRange(final Bytes key, final long to) {
         final byte[] maxSuffix = ByteBuffer.allocate(SUFFIX_SIZE)
             .putLong(to)
             // start can at most be equal to end
@@ -67,7 +67,7 @@ public class SessionKeySchema implements SegmentedBytesStore.KeySchema {
     }
 
     @Override
-    public Bytes lowerRange(Bytes key, long from) {
+    public Bytes lowerRange(final Bytes key, final long from) {
         return OrderedBytes.lowerRange(key, MIN_SUFFIX);
     }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java
index a8a04c6..98555ad 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java
@@ -39,11 +39,11 @@ class StoreChangeLogger<K, V> {
     private final ProcessorContext context;
     private final RecordCollector collector;
 
-    StoreChangeLogger(String storeName, ProcessorContext context, StateSerdes<K, V> serialization) {
+    StoreChangeLogger(final String storeName, final ProcessorContext context, final StateSerdes<K, V> serialization) {
         this(storeName, context, context.taskId().partition, serialization);
     }
 
-    private StoreChangeLogger(String storeName, ProcessorContext context, int partition, StateSerdes<K, V> serialization) {
+    private StoreChangeLogger(final String storeName, final ProcessorContext context, final int partition, final StateSerdes<K, V> serialization) {
         this.topic = ProcessorStateManager.storeChangelogTopic(context.applicationId(), storeName);
         this.context = context;
         this.partition = partition;
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java
index 415c6a2..e097963 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java
@@ -49,7 +49,7 @@ public class StreamThreadStateStoreProvider implements StateStoreProvider {
                     streamThread.state() + ", not RUNNING");
         }
         final List<T> stores = new ArrayList<>();
-        for (Task streamTask : streamThread.tasks().values()) {
+        for (final Task streamTask : streamThread.tasks().values()) {
             final StateStore store = streamTask.getStore(storeName);
             if (store != null && queryableStoreType.accepts(store)) {
                 if (!store.isOpen()) {
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java
index 7ce03a1..27270e6 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java
@@ -50,7 +50,7 @@ public class ThreadCache {
         void apply(final List<DirtyEntry> dirty);
     }
 
-    public ThreadCache(final LogContext logContext, long maxCacheSizeBytes, final StreamsMetricsImpl metrics) {
+    public ThreadCache(final LogContext logContext, final long maxCacheSizeBytes, final StreamsMetricsImpl metrics) {
         this.maxCacheSizeBytes = maxCacheSizeBytes;
         this.metrics = metrics;
         this.log = logContext.logger(getClass());
@@ -89,7 +89,7 @@ public class ThreadCache {
      * @return
      */
     public static String taskIDfromCacheName(final String cacheName) {
-        String[] tokens = cacheName.split("-", 2);
+        final String[] tokens = cacheName.split("-", 2);
         return tokens[0];
     }
 
@@ -99,7 +99,7 @@ public class ThreadCache {
      * @return
      */
     public static String underlyingStoreNamefromCacheName(final String cacheName) {
-        String[] tokens = cacheName.split("-", 2);
+        final String[] tokens = cacheName.split("-", 2);
         return tokens[1];
     }
 
@@ -110,7 +110,7 @@ public class ThreadCache {
      * @param namespace
      * @param listener
      */
-    public void addDirtyEntryFlushListener(final String namespace, DirtyEntryFlushListener listener) {
+    public void addDirtyEntryFlushListener(final String namespace, final DirtyEntryFlushListener listener) {
         final NamedCache cache = getOrCreateCache(namespace);
         cache.setListener(listener);
     }
@@ -129,7 +129,7 @@ public class ThreadCache {
         }
     }
 
-    public LRUCacheEntry get(final String namespace, Bytes key) {
+    public LRUCacheEntry get(final String namespace, final Bytes key) {
         numGets++;
 
         if (key == null) {
@@ -143,7 +143,7 @@ public class ThreadCache {
         return cache.get(key);
     }
 
-    public void put(final String namespace, Bytes key, LRUCacheEntry value) {
+    public void put(final String namespace, final Bytes key, final LRUCacheEntry value) {
         numPuts++;
 
         final NamedCache cache = getOrCreateCache(namespace);
@@ -151,7 +151,7 @@ public class ThreadCache {
         maybeEvict(namespace);
     }
 
-    public LRUCacheEntry putIfAbsent(final String namespace, Bytes key, LRUCacheEntry value) {
+    public LRUCacheEntry putIfAbsent(final String namespace, final Bytes key, final LRUCacheEntry value) {
         final NamedCache cache = getOrCreateCache(namespace);
 
         final LRUCacheEntry result = cache.putIfAbsent(key, value);
@@ -164,7 +164,7 @@ public class ThreadCache {
     }
 
     public void putAll(final String namespace, final List<KeyValue<Bytes, LRUCacheEntry>> entries) {
-        for (KeyValue<Bytes, LRUCacheEntry> entry : entries) {
+        for (final KeyValue<Bytes, LRUCacheEntry> entry : entries) {
             put(namespace, entry.key, entry.value);
         }
     }
@@ -196,7 +196,7 @@ public class ThreadCache {
     
     public long size() {
         long size = 0;
-        for (NamedCache cache : caches.values()) {
+        for (final NamedCache cache : caches.values()) {
             size += cache.size();
             if (isOverflowing(size)) {
                 return Long.MAX_VALUE;
@@ -309,7 +309,7 @@ public class ThreadCache {
         }
 
         private void internalNext() {
-            Bytes cacheKey = keys.next();
+            final Bytes cacheKey = keys.next();
             final LRUCacheEntry entry = cache.get(cacheKey);
             if (entry == null) {
                 return;
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreIteratorWrapper.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreIteratorWrapper.java
index e83e6e6..1feab8f 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreIteratorWrapper.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreIteratorWrapper.java
@@ -50,7 +50,7 @@ class WindowStoreIteratorWrapper<K, V> {
         final StateSerdes<?, V> serdes;
 
         WrappedWindowStoreIterator(
-            KeyValueIterator<Bytes, byte[]> bytesIterator, StateSerdes<?, V> serdes) {
+            final KeyValueIterator<Bytes, byte[]> bytesIterator, final StateSerdes<?, V> serdes) {
             this.bytesIterator = bytesIterator;
             this.serdes = serdes;
         }
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappingStoreProvider.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappingStoreProvider.java
index 36d0173..6de39cc 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappingStoreProvider.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappingStoreProvider.java
@@ -42,9 +42,9 @@ public class WrappingStoreProvider implements StateStoreProvider {
      * @param <T>       The type of the Store, for example, {@link org.apache.kafka.streams.state.ReadOnlyKeyValueStore}
      * @return  a List of all the stores with the storeName and are accepted by {@link QueryableStoreType#accepts(StateStore)}
      */
-    public <T> List<T> stores(final String storeName, QueryableStoreType<T> type) {
+    public <T> List<T> stores(final String storeName, final QueryableStoreType<T> type) {
         final List<T> allStores = new ArrayList<>();
-        for (StateStoreProvider provider : storeProviders) {
+        for (final StateStoreProvider provider : storeProviders) {
             final List<T> stores =
                 provider.stores(storeName, type);
             allStores.addAll(stores);
diff --git a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
index 491334f..28e3d55 100644
--- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
@@ -125,11 +125,11 @@ public class KafkaStreamsTest {
     public void shouldCleanupResourcesOnCloseWithoutPreviousStart() throws Exception {
         final StreamsBuilder builder = new StreamsBuilder();
         builder.globalTable("anyTopic");
-        List<Node> nodes = Arrays.asList(new Node(0, "localhost", 8121));
-        Cluster cluster = new Cluster("mockClusterId", nodes,
-            Collections.<PartitionInfo>emptySet(), Collections.<String>emptySet(),
-            Collections.<String>emptySet(), nodes.get(0));
-        MockClientSupplier clientSupplier = new MockClientSupplier();
+        final List<Node> nodes = Arrays.asList(new Node(0, "localhost", 8121));
+        final Cluster cluster = new Cluster("mockClusterId", nodes,
+                                            Collections.<PartitionInfo>emptySet(), Collections.<String>emptySet(),
+                                            Collections.<String>emptySet(), nodes.get(0));
+        final MockClientSupplier clientSupplier = new MockClientSupplier();
         clientSupplier.setClusterForAdminClient(cluster);
         final KafkaStreams streams = new KafkaStreams(builder.build(), props, clientSupplier);
         streams.close();
@@ -143,7 +143,7 @@ public class KafkaStreamsTest {
         // Ensure that any created clients are closed
         assertTrue(clientSupplier.consumer.closed());
         assertTrue(clientSupplier.restoreConsumer.closed());
-        for (MockProducer p : clientSupplier.producers) {
+        for (final MockProducer p : clientSupplier.producers) {
             assertTrue(p.closed());
         }
     }
@@ -462,10 +462,10 @@ public class KafkaStreamsTest {
     @Test
     public void shouldReturnThreadMetadata() {
         streams.start();
-        Set<ThreadMetadata> threadMetadata = streams.localThreadsMetadata();
+        final Set<ThreadMetadata> threadMetadata = streams.localThreadsMetadata();
         assertNotNull(threadMetadata);
         assertEquals(2, threadMetadata.size());
-        for (ThreadMetadata metadata : threadMetadata) {
+        for (final ThreadMetadata metadata : threadMetadata) {
             assertTrue("#threadState() was: " + metadata.threadState() + "; expected either RUNNING, PARTITIONS_REVOKED, PARTITIONS_ASSIGNED, or CREATED",
                 Utils.mkList("RUNNING", "PARTITIONS_REVOKED", "PARTITIONS_ASSIGNED", "CREATED").contains(metadata.threadState()));
             assertEquals(0, metadata.standbyTasks().size());
diff --git a/streams/src/test/java/org/apache/kafka/streams/KeyValueTest.java b/streams/src/test/java/org/apache/kafka/streams/KeyValueTest.java
index ce60fef..7681068 100644
--- a/streams/src/test/java/org/apache/kafka/streams/KeyValueTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/KeyValueTest.java
@@ -25,8 +25,8 @@ public class KeyValueTest {
 
     @Test
     public void shouldHaveSaneEqualsAndHashCode() {
-        KeyValue<String, Long> kv = KeyValue.pair("key1", 1L);
-        KeyValue<String, Long> copyOfKV = KeyValue.pair(kv.key, kv.value);
+        final KeyValue<String, Long> kv = KeyValue.pair("key1", 1L);
+        final KeyValue<String, Long> copyOfKV = KeyValue.pair(kv.key, kv.value);
 
         // Reflexive
         assertTrue(kv.equals(kv));
@@ -38,7 +38,7 @@ public class KeyValueTest {
         assertTrue(copyOfKV.hashCode() == kv.hashCode());
 
         // Transitive
-        KeyValue<String, Long> copyOfCopyOfKV = KeyValue.pair(copyOfKV.key, copyOfKV.value);
+        final KeyValue<String, Long> copyOfCopyOfKV = KeyValue.pair(copyOfKV.key, copyOfKV.value);
         assertTrue(copyOfKV.equals(copyOfCopyOfKV));
         assertTrue(copyOfKV.hashCode() == copyOfCopyOfKV.hashCode());
         assertTrue(kv.equals(copyOfCopyOfKV));
@@ -48,23 +48,23 @@ public class KeyValueTest {
         assertFalse("must be false for null", kv.equals(null));
         assertFalse("must be false if key is non-null and other key is null", kv.equals(KeyValue.pair(null, kv.value)));
         assertFalse("must be false if value is non-null and other value is null", kv.equals(KeyValue.pair(kv.key, null)));
-        KeyValue<Long, Long> differentKeyType = KeyValue.pair(1L, kv.value);
+        final KeyValue<Long, Long> differentKeyType = KeyValue.pair(1L, kv.value);
         assertFalse("must be false for different key types", kv.equals(differentKeyType));
-        KeyValue<String, String> differentValueType = KeyValue.pair(kv.key, "anyString");
+        final KeyValue<String, String> differentValueType = KeyValue.pair(kv.key, "anyString");
         assertFalse("must be false for different value types", kv.equals(differentValueType));
-        KeyValue<Long, String> differentKeyValueTypes = KeyValue.pair(1L, "anyString");
+        final KeyValue<Long, String> differentKeyValueTypes = KeyValue.pair(1L, "anyString");
         assertFalse("must be false for different key and value types", kv.equals(differentKeyValueTypes));
         assertFalse("must be false for different types of objects", kv.equals(new Object()));
 
-        KeyValue<String, Long> differentKey = KeyValue.pair(kv.key + "suffix", kv.value);
+        final KeyValue<String, Long> differentKey = KeyValue.pair(kv.key + "suffix", kv.value);
         assertFalse("must be false if key is different", kv.equals(differentKey));
         assertFalse("must be false if key is different", differentKey.equals(kv));
 
-        KeyValue<String, Long> differentValue = KeyValue.pair(kv.key, kv.value + 1L);
+        final KeyValue<String, Long> differentValue = KeyValue.pair(kv.key, kv.value + 1L);
         assertFalse("must be false if value is different", kv.equals(differentValue));
         assertFalse("must be false if value is different", differentValue.equals(kv));
 
-        KeyValue<String, Long> differentKeyAndValue = KeyValue.pair(kv.key + "suffix", kv.value + 1L);
+        final KeyValue<String, Long> differentKeyAndValue = KeyValue.pair(kv.key + "suffix", kv.value + 1L);
         assertFalse("must be false if key and value are different", kv.equals(differentKeyAndValue));
         assertFalse("must be false if key and value are different", differentKeyAndValue.equals(kv));
     }
diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
index 9755334..1791672 100644
--- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
@@ -550,7 +550,7 @@ public class StreamsConfigTest {
         try {
             config.defaultKeySerde();
             fail("Test should throw a StreamsException");
-        } catch (StreamsException e) {
+        } catch (final StreamsException e) {
             assertEquals("Failed to configure key serde class org.apache.kafka.streams.StreamsConfigTest$MisconfiguredSerde", e.getMessage());
         }
     }
@@ -564,7 +564,7 @@ public class StreamsConfigTest {
         try {
             config.defaultValueSerde();
             fail("Test should throw a StreamsException");
-        } catch (StreamsException e) {
+        } catch (final StreamsException e) {
             assertEquals("Failed to configure value serde class org.apache.kafka.streams.StreamsConfigTest$MisconfiguredSerde", e.getMessage());
         }
     }
diff --git a/streams/src/test/java/org/apache/kafka/streams/TopologyWrapper.java b/streams/src/test/java/org/apache/kafka/streams/TopologyWrapper.java
index 60f0e6f..940ec0e 100644
--- a/streams/src/test/java/org/apache/kafka/streams/TopologyWrapper.java
+++ b/streams/src/test/java/org/apache/kafka/streams/TopologyWrapper.java
@@ -32,7 +32,7 @@ public class TopologyWrapper extends Topology {
         return internalTopologyBuilder;
     }
 
-    public void setApplicationId(String applicationId) {
+    public void setApplicationId(final String applicationId) {
         internalTopologyBuilder.setApplicationId(applicationId);
     }
 }
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java
index 3e29fc2..cddf590 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java
@@ -72,8 +72,8 @@ public abstract class AbstractJoinIntegrationTest {
 
     @Parameterized.Parameters(name = "caching enabled = {0}")
     public static Collection<Object[]> data() {
-        List<Object[]> values = new ArrayList<>();
-        for (boolean cacheEnabled : Arrays.asList(true, false))
+        final List<Object[]> values = new ArrayList<>();
+        for (final boolean cacheEnabled : Arrays.asList(true, false))
             values.add(new Object[] {cacheEnabled});
         return values;
     }
@@ -124,7 +124,7 @@ public abstract class AbstractJoinIntegrationTest {
 
     final boolean cacheEnabled;
 
-    AbstractJoinIntegrationTest(boolean cacheEnabled) {
+    AbstractJoinIntegrationTest(final boolean cacheEnabled) {
         this.cacheEnabled = cacheEnabled;
     }
 
@@ -206,7 +206,7 @@ public abstract class AbstractJoinIntegrationTest {
             for (final Input<String> singleInput : input) {
                 producer.send(new ProducerRecord<>(singleInput.topic, null, ++ts, singleInput.record.key, singleInput.record.value)).get();
 
-                List<String> expected = resultIterator.next();
+                final List<String> expected = resultIterator.next();
 
                 if (expected != null) {
                     checkResult(OUTPUT_TOPIC, expected);
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java
index 64b23cb..caa02db 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java
@@ -124,7 +124,7 @@ public abstract class AbstractResetIntegrationTest {
         commonClientConfig = new Properties();
         commonClientConfig.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers());
 
-        Map<String, Object> sslConfig = getClientSslConfig();
+        final Map<String, Object> sslConfig = getClientSslConfig();
         if (sslConfig != null) {
             commonClientConfig.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, sslConfig.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG));
             commonClientConfig.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, ((Password) sslConfig.get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG)).value());
@@ -176,7 +176,7 @@ public abstract class AbstractResetIntegrationTest {
         @Override
         public boolean conditionMet() {
             try {
-                ConsumerGroupDescription groupDescription = adminClient.describeConsumerGroups(Collections.singletonList(appID)).describedGroups().get(appID).get();
+                final ConsumerGroupDescription groupDescription = adminClient.describeConsumerGroups(Collections.singletonList(appID)).describedGroups().get(appID).get();
                 return groupDescription.members().isEmpty();
             } catch (final ExecutionException | InterruptedException e) {
                 return false;
@@ -205,18 +205,18 @@ public abstract class AbstractResetIntegrationTest {
     }
 
     private void add10InputElements() throws java.util.concurrent.ExecutionException, InterruptedException {
-        List<KeyValue<Long, String>> records = Arrays.asList(KeyValue.pair(0L, "aaa"),
-                KeyValue.pair(1L, "bbb"),
-                KeyValue.pair(0L, "ccc"),
-                KeyValue.pair(1L, "ddd"),
-                KeyValue.pair(0L, "eee"),
-                KeyValue.pair(1L, "fff"),
-                KeyValue.pair(0L, "ggg"),
-                KeyValue.pair(1L, "hhh"),
-                KeyValue.pair(0L, "iii"),
-                KeyValue.pair(1L, "jjj"));
-
-        for (KeyValue<Long, String> record : records) {
+        final List<KeyValue<Long, String>> records = Arrays.asList(KeyValue.pair(0L, "aaa"),
+                                                                   KeyValue.pair(1L, "bbb"),
+                                                                   KeyValue.pair(0L, "ccc"),
+                                                                   KeyValue.pair(1L, "ddd"),
+                                                                   KeyValue.pair(0L, "eee"),
+                                                                   KeyValue.pair(1L, "fff"),
+                                                                   KeyValue.pair(0L, "ggg"),
+                                                                   KeyValue.pair(1L, "hhh"),
+                                                                   KeyValue.pair(0L, "iii"),
+                                                                   KeyValue.pair(1L, "jjj"));
+
+        for (final KeyValue<Long, String> record : records) {
             mockTime.sleep(10);
             IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(INPUT_TOPIC, Collections.singleton(record), producerConfig, mockTime.milliseconds());
         }
@@ -332,7 +332,7 @@ public abstract class AbstractResetIntegrationTest {
 
         // insert bad record to make sure intermediate user topic gets seekToEnd()
         mockTime.sleep(1);
-        KeyValue<Long, String> badMessage = new KeyValue<>(-1L, "badRecord-ShouldBeSkipped");
+        final KeyValue<Long, String> badMessage = new KeyValue<>(-1L, "badRecord-ShouldBeSkipped");
         IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(
             INTERMEDIATE_USER_TOPIC,
             Collections.singleton(badMessage),
@@ -387,7 +387,7 @@ public abstract class AbstractResetIntegrationTest {
 
         // RESET
         final File resetFile = File.createTempFile("reset", ".csv");
-        try (BufferedWriter writer = new BufferedWriter(new FileWriter(resetFile))) {
+        try (final BufferedWriter writer = new BufferedWriter(new FileWriter(resetFile))) {
             writer.write(INPUT_TOPIC + ",0,1");
             writer.close();
         }
@@ -431,7 +431,7 @@ public abstract class AbstractResetIntegrationTest {
 
         // RESET
         final File resetFile = File.createTempFile("reset", ".csv");
-        try (BufferedWriter writer = new BufferedWriter(new FileWriter(resetFile))) {
+        try (final BufferedWriter writer = new BufferedWriter(new FileWriter(resetFile))) {
             writer.write(INPUT_TOPIC + ",0,1");
             writer.close();
         }
@@ -479,7 +479,7 @@ public abstract class AbstractResetIntegrationTest {
 
         // RESET
         final File resetFile = File.createTempFile("reset", ".csv");
-        try (BufferedWriter writer = new BufferedWriter(new FileWriter(resetFile))) {
+        try (final BufferedWriter writer = new BufferedWriter(new FileWriter(resetFile))) {
             writer.write(INPUT_TOPIC + ",0,1");
             writer.close();
         }
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/FineGrainedAutoResetIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/FineGrainedAutoResetIntegrationTest.java
index 44b42cf..08ab120 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/FineGrainedAutoResetIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/FineGrainedAutoResetIntegrationTest.java
@@ -139,7 +139,7 @@ public class FineGrainedAutoResetIntegrationTest {
     @Before
     public void setUp() throws IOException {
 
-        Properties props = new Properties();
+        final Properties props = new Properties();
         props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
         props.put(IntegrationTestUtils.INTERNAL_LEAVE_GROUP_ON_CLOSE, true);
 
@@ -276,10 +276,10 @@ public class FineGrainedAutoResetIntegrationTest {
 
     @Test
     public void shouldThrowStreamsExceptionNoResetSpecified() throws InterruptedException {
-        Properties props = new Properties();
+        final Properties props = new Properties();
         props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none");
 
-        Properties localConfig = StreamsTestUtils.getStreamsConfig(
+        final Properties localConfig = StreamsTestUtils.getStreamsConfig(
                 "testAutoOffsetWithNone",
                 CLUSTER.bootstrapServers(),
                 STRING_SERDE_CLASSNAME,
@@ -291,7 +291,7 @@ public class FineGrainedAutoResetIntegrationTest {
 
         exceptionStream.to(DEFAULT_OUTPUT_TOPIC, Produced.with(stringSerde, stringSerde));
 
-        KafkaStreams streams = new KafkaStreams(builder.build(), localConfig);
+        final KafkaStreams streams = new KafkaStreams(builder.build(), localConfig);
 
         final TestingUncaughtExceptionHandler uncaughtExceptionHandler = new TestingUncaughtExceptionHandler();
 
@@ -312,7 +312,7 @@ public class FineGrainedAutoResetIntegrationTest {
     private static final class TestingUncaughtExceptionHandler implements Thread.UncaughtExceptionHandler {
         boolean correctExceptionThrown = false;
         @Override
-        public void uncaughtException(Thread t, Throwable e) {
+        public void uncaughtException(final Thread t, final Throwable e) {
             assertThat(e.getClass().getSimpleName(), is("StreamsException"));
             assertThat(e.getCause().getClass().getSimpleName(), is("NoOffsetForPartitionException"));
             correctExceptionThrown = true;
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java
index f7c0e55..084519e 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java
@@ -248,13 +248,13 @@ public class GlobalKTableEOSIntegrationTest {
                 ReadOnlyKeyValueStore<Long, String> store = null;
                 try {
                     store = kafkaStreams.store(globalStore, QueryableStoreTypes.<Long, String>keyValueStore());
-                } catch (InvalidStateStoreException ex) {
+                } catch (final InvalidStateStoreException ex) {
                     return false;
                 }
-                Map<Long, String> result = new HashMap<>();
-                Iterator<KeyValue<Long, String>> it = store.all();
+                final Map<Long, String> result = new HashMap<>();
+                final Iterator<KeyValue<Long, String>> it = store.all();
                 while (it.hasNext()) {
-                    KeyValue<Long, String> kv = it.next();
+                    final KeyValue<Long, String> kv = it.next();
                     result.put(kv.key, kv.value);
                 }
                 return result.equals(expected);
@@ -282,13 +282,13 @@ public class GlobalKTableEOSIntegrationTest {
                 ReadOnlyKeyValueStore<Long, String> store = null;
                 try {
                     store = kafkaStreams.store(globalStore, QueryableStoreTypes.<Long, String>keyValueStore());
-                } catch (InvalidStateStoreException ex) {
+                } catch (final InvalidStateStoreException ex) {
                     return false;
                 }
-                Map<Long, String> result = new HashMap<>();
-                Iterator<KeyValue<Long, String>> it = store.all();
+                final Map<Long, String> result = new HashMap<>();
+                final Iterator<KeyValue<Long, String>> it = store.all();
                 while (it.hasNext()) {
-                    KeyValue<Long, String> kv = it.next();
+                    final KeyValue<Long, String> kv = it.next();
                     result.put(kv.key, kv.value);
                 }
                 return result.equals(expected);
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java
index 0bbea04..7106aa3 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java
@@ -103,10 +103,10 @@ public class GlobalThreadShutDownOrderTest {
 
         final Consumed<String, Long> stringLongConsumed = Consumed.with(Serdes.String(), Serdes.Long());
 
-        KeyValueStoreBuilder<String, Long> storeBuilder = new KeyValueStoreBuilder<>(Stores.persistentKeyValueStore(globalStore),
-                                                                                     Serdes.String(),
-                                                                                     Serdes.Long(),
-                                                                                     mockTime);
+        final KeyValueStoreBuilder<String, Long> storeBuilder = new KeyValueStoreBuilder<>(Stores.persistentKeyValueStore(globalStore),
+                                                                                           Serdes.String(),
+                                                                                           Serdes.Long(),
+                                                                                           mockTime);
 
         builder.addGlobalStore(storeBuilder,
                                globalStoreTopic,
@@ -150,7 +150,7 @@ public class GlobalThreadShutDownOrderTest {
 
         kafkaStreams.close(30, TimeUnit.SECONDS);
 
-        List<Long> expectedRetrievedValues = Arrays.asList(1L, 2L, 3L, 4L);
+        final List<Long> expectedRetrievedValues = Arrays.asList(1L, 2L, 3L, 4L);
         assertEquals(expectedRetrievedValues, retrievedValuesList);
     }
 
@@ -163,7 +163,7 @@ public class GlobalThreadShutDownOrderTest {
     }
 
 
-    private void populateTopics(String topicName) throws Exception {
+    private void populateTopics(final String topicName) throws Exception {
         IntegrationTestUtils.produceKeyValuesSynchronously(
             topicName,
             Arrays.asList(
@@ -205,8 +205,8 @@ public class GlobalThreadShutDownOrderTest {
 
         @Override
         public void close() {
-            List<String> keys = Arrays.asList("A", "B", "C", "D");
-            for (String key : keys) {
+            final List<String> keys = Arrays.asList("A", "B", "C", "D");
+            for (final String key : keys) {
                 // need to simulate thread slow in closing
                 Utils.sleep(1000);
                 retrievedValuesList.add(store.get(key));
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java
index e120999..5eadc6e 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java
@@ -186,8 +186,8 @@ public class InternalTopicIntegrationTest {
         //
         // Step 1: Configure and start a simple word count topology
         //
-        StreamsBuilder builder = new StreamsBuilder();
-        KStream<String, String> textLines = builder.stream(DEFAULT_INPUT_TOPIC);
+        final StreamsBuilder builder = new StreamsBuilder();
+        final KStream<String, String> textLines = builder.stream(DEFAULT_INPUT_TOPIC);
 
         final int durationMs = 2000;
 
@@ -201,7 +201,7 @@ public class InternalTopicIntegrationTest {
                 .windowedBy(TimeWindows.of(1000).until(2000))
                 .count(Materialized.<String, Long, WindowStore<org.apache.kafka.common.utils.Bytes, byte[]>>as("CountWindows"));
 
-        KafkaStreams streams = new KafkaStreams(builder.build(), streamsProp);
+        final KafkaStreams streams = new KafkaStreams(builder.build(), streamsProp);
         streams.start();
 
         //
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java
index 5681d7c..4593e59 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java
@@ -93,7 +93,7 @@ public class KStreamAggregationDedupIntegrationTest {
         builder = new StreamsBuilder();
         createTopics();
         streamsConfiguration = new Properties();
-        String applicationId = "kgrouped-stream-test-" +
+        final String applicationId = "kgrouped-stream-test-" +
             testNo;
         streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId);
         streamsConfiguration
@@ -104,7 +104,7 @@ public class KStreamAggregationDedupIntegrationTest {
         streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10 * 1024 * 1024L);
         streamsConfiguration.put(IntegrationTestUtils.INTERNAL_LEAVE_GROUP_ON_CLOSE, true);
 
-        KeyValueMapper<Integer, String, String> mapper = MockMapper.selectValueMapper();
+        final KeyValueMapper<Integer, String, String> mapper = MockMapper.selectValueMapper();
         stream = builder.stream(streamOneInput, Consumed.with(Serdes.Integer(), Serdes.String()));
         groupedStream = stream
             .groupBy(
@@ -113,7 +113,7 @@ public class KStreamAggregationDedupIntegrationTest {
 
         reducer = new Reducer<String>() {
             @Override
-            public String apply(String value1, String value2) {
+            public String apply(final String value1, final String value2) {
                 return value1 + ":" + value2;
             }
         };
@@ -140,14 +140,14 @@ public class KStreamAggregationDedupIntegrationTest {
 
         produceMessages(System.currentTimeMillis());
 
-        List<KeyValue<String, String>> results = receiveMessages(
+        final List<KeyValue<String, String>> results = receiveMessages(
             new StringDeserializer(),
             new StringDeserializer(),
             5);
 
         Collections.sort(results, new Comparator<KeyValue<String, String>>() {
             @Override
-            public int compare(KeyValue<String, String> o1, KeyValue<String, String> o2) {
+            public int compare(final KeyValue<String, String> o1, final KeyValue<String, String> o2) {
                 return KStreamAggregationDedupIntegrationTest.compare(o1, o2);
             }
         });
@@ -172,9 +172,9 @@ public class KStreamAggregationDedupIntegrationTest {
 
     @Test
     public void shouldReduceWindowed() throws Exception {
-        long firstBatchTimestamp = System.currentTimeMillis() - 1000;
+        final long firstBatchTimestamp = System.currentTimeMillis() - 1000;
         produceMessages(firstBatchTimestamp);
-        long secondBatchTimestamp = System.currentTimeMillis();
+        final long secondBatchTimestamp = System.currentTimeMillis();
         produceMessages(secondBatchTimestamp);
         produceMessages(secondBatchTimestamp);
 
@@ -183,7 +183,7 @@ public class KStreamAggregationDedupIntegrationTest {
             .reduce(reducer, Materialized.<String, String, WindowStore<Bytes, byte[]>>as("reduce-time-windows"))
             .toStream(new KeyValueMapper<Windowed<String>, String, String>() {
                 @Override
-                public String apply(Windowed<String> windowedKey, String value) {
+                public String apply(final Windowed<String> windowedKey, final String value) {
                     return windowedKey.key() + "@" + windowedKey.window().start();
                 }
             })
@@ -191,12 +191,12 @@ public class KStreamAggregationDedupIntegrationTest {
 
         startStreams();
 
-        List<KeyValue<String, String>> windowedOutput = receiveMessages(
+        final List<KeyValue<String, String>> windowedOutput = receiveMessages(
             new StringDeserializer(),
             new StringDeserializer(),
             10);
 
-        Comparator<KeyValue<String, String>>
+        final Comparator<KeyValue<String, String>>
             comparator =
             new Comparator<KeyValue<String, String>>() {
                 @Override
@@ -207,8 +207,8 @@ public class KStreamAggregationDedupIntegrationTest {
             };
 
         Collections.sort(windowedOutput, comparator);
-        long firstBatchWindow = firstBatchTimestamp / 500 * 500;
-        long secondBatchWindow = secondBatchTimestamp / 500 * 500;
+        final long firstBatchWindow = firstBatchTimestamp / 500 * 500;
+        final long secondBatchWindow = secondBatchTimestamp / 500 * 500;
 
         assertThat(windowedOutput, is(
             Arrays.asList(
@@ -267,7 +267,7 @@ public class KStreamAggregationDedupIntegrationTest {
     }
 
 
-    private void produceMessages(long timestamp) throws Exception {
+    private void produceMessages(final long timestamp) throws Exception {
         IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(
             streamOneInput,
             Arrays.asList(
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java
index a29332c..fecb8ea 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java
@@ -254,9 +254,8 @@ public class KStreamAggregationIntegrationTest {
 
         // check every message is contained in the expect result
         final String[] allRecords = resultFromConsoleConsumer.split("\n");
-        for (String record: allRecords) {
-            record = "KeyValue(" + record + ")";
-            assertTrue(expectResultString.contains(record));
+        for (final String record: allRecords) {
+            assertTrue(expectResultString.contains("KeyValue(" + record + ")"));
         }
     }
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java
index 829d3a7..dd78190 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java
@@ -227,7 +227,7 @@ public class KTableSourceTopicRestartIntegrationTest {
                                    final long endingOffset) {
             try {
                 produceKeyValues("d");
-            } catch (ExecutionException | InterruptedException e) {
+            } catch (final ExecutionException | InterruptedException e) {
                 throw new RuntimeException(e);
             }
         }
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java
index 9dfb6dd..29f077f 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java
@@ -81,7 +81,7 @@ public class PurgeRepartitionTopicIntegrationTest {
         @Override
         final public boolean conditionMet() {
             try {
-                Set<String> topics = adminClient.listTopics().names().get();
+                final Set<String> topics = adminClient.listTopics().names().get();
 
                 if (!topics.contains(REPARTITION_TOPIC)) {
                     return false;
@@ -91,8 +91,8 @@ public class PurgeRepartitionTopicIntegrationTest {
             }
 
             try {
-                ConfigResource resource = new ConfigResource(ConfigResource.Type.TOPIC, REPARTITION_TOPIC);
-                Config config = adminClient.describeConfigs(Collections.singleton(resource))
+                final ConfigResource resource = new ConfigResource(ConfigResource.Type.TOPIC, REPARTITION_TOPIC);
+                final Config config = adminClient.describeConfigs(Collections.singleton(resource))
                         .values().get(resource).get();
                 return config.get(TopicConfig.CLEANUP_POLICY_CONFIG).value().equals(TopicConfig.CLEANUP_POLICY_DELETE)
                         && config.get(TopicConfig.SEGMENT_MS_CONFIG).value().equals(purgeIntervalMs.toString())
@@ -111,7 +111,7 @@ public class PurgeRepartitionTopicIntegrationTest {
     private class RepartitionTopicVerified implements TestCondition {
         private final TopicSizeVerifier verifier;
 
-        RepartitionTopicVerified(TopicSizeVerifier verifier) {
+        RepartitionTopicVerified(final TopicSizeVerifier verifier) {
             this.verifier = verifier;
         }
 
@@ -145,11 +145,11 @@ public class PurgeRepartitionTopicIntegrationTest {
     @Before
     public void setup() {
         // create admin client for verification
-        Properties adminConfig = new Properties();
+        final Properties adminConfig = new Properties();
         adminConfig.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
         adminClient = AdminClient.create(adminConfig);
 
-        Properties streamsConfiguration = new Properties();
+        final Properties streamsConfiguration = new Properties();
         streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, APPLICATION_ID);
         streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
         streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
@@ -160,7 +160,7 @@ public class PurgeRepartitionTopicIntegrationTest {
         streamsConfiguration.put(StreamsConfig.producerPrefix(ProducerConfig.BATCH_SIZE_CONFIG), purgeSegmentBytes / 2);    // we cannot allow batch size larger than segment size
         streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, purgeIntervalMs);
 
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
         builder.stream(INPUT_TOPIC)
                .groupBy(MockMapper.selectKeyKeyValueMapper())
@@ -199,7 +199,7 @@ public class PurgeRepartitionTopicIntegrationTest {
         TestUtils.waitForCondition(
                 new RepartitionTopicVerified(new TopicSizeVerifier() {
                     @Override
-                    public boolean verify(long currentSize) {
+                    public boolean verify(final long currentSize) {
                         return currentSize > 0;
                     }
                 }),
@@ -211,7 +211,7 @@ public class PurgeRepartitionTopicIntegrationTest {
         TestUtils.waitForCondition(
                 new RepartitionTopicVerified(new TopicSizeVerifier() {
                     @Override
-                    public boolean verify(long currentSize) {
+                    public boolean verify(final long currentSize) {
                         return currentSize <= purgeSegmentBytes;
                     }
                 }),
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
index d035586..d3a7aee 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
@@ -397,7 +397,7 @@ public class RegexSourceIntegrationTest {
         streams = new KafkaStreams(builder.build(), streamsConfiguration);
         streams.setStateListener(new KafkaStreams.StateListener() {
             @Override
-            public void onChange(KafkaStreams.State newState, KafkaStreams.State oldState) {
+            public void onChange(final KafkaStreams.State newState, final KafkaStreams.State oldState) {
                 if (newState == KafkaStreams.State.ERROR)
                     expectError.set(true);
             }
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java
index ea9fbad..c1d07dc 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java
@@ -97,7 +97,7 @@ public class RestoreIntegrationTest {
     }
 
     private Properties props(final String applicationId) {
-        Properties streamsConfiguration = new Properties();
+        final Properties streamsConfiguration = new Properties();
         streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId);
         streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
         streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
@@ -397,7 +397,7 @@ public class RestoreIntegrationTest {
         consumer.assign(partitions);
         consumer.seekToEnd(partitions);
 
-        for (TopicPartition partition : partitions) {
+        for (final TopicPartition partition : partitions) {
             final long position = consumer.position(partition);
             consumer.seek(partition, position - limitDelta);
         }
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java
index 571dc05..38bc7a6 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java
@@ -42,7 +42,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest
     private KStream<Long, String> leftStream;
     private KStream<Long, String> rightStream;
 
-    public StreamStreamJoinIntegrationTest(boolean cacheEnabled) {
+    public StreamStreamJoinIntegrationTest(final boolean cacheEnabled) {
         super(cacheEnabled);
     }
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StreamTableJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StreamTableJoinIntegrationTest.java
index 61bbb8b..7503dd6 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/StreamTableJoinIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/StreamTableJoinIntegrationTest.java
@@ -46,7 +46,7 @@ public class StreamTableJoinIntegrationTest extends AbstractJoinIntegrationTest
     private KStream<Long, String> leftStream;
     private KTable<Long, String> rightTable;
 
-    public StreamTableJoinIntegrationTest(boolean cacheEnabled) {
+    public StreamTableJoinIntegrationTest(final boolean cacheEnabled) {
         super(cacheEnabled);
     }
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java
index 5fab666..73d1e3d 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java
@@ -44,7 +44,7 @@ public class TableTableJoinIntegrationTest extends AbstractJoinIntegrationTest {
     private KTable<Long, String> leftTable;
     private KTable<Long, String> rightTable;
 
-    public TableTableJoinIntegrationTest(boolean cacheEnabled) {
+    public TableTableJoinIntegrationTest(final boolean cacheEnabled) {
         super(cacheEnabled);
     }
 
@@ -80,7 +80,7 @@ public class TableTableJoinIntegrationTest extends AbstractJoinIntegrationTest {
         public void apply(final Long key, final String value) {
             numRecordsExpected++;
             if (expected.equals(value)) {
-                boolean ret = finalResultReached.compareAndSet(false, true);
+                final boolean ret = finalResultReached.compareAndSet(false, true);
 
                 if (!ret) {
                     // do nothing; it is possible that we will see multiple duplicates of final results due to KAFKA-4309
@@ -98,7 +98,7 @@ public class TableTableJoinIntegrationTest extends AbstractJoinIntegrationTest {
             leftTable.join(rightTable, valueJoiner, materialized).toStream().peek(new CountingPeek(false)).to(OUTPUT_TOPIC);
             runTest(expectedFinalJoinResult, storeName);
         } else {
-            List<List<String>> expectedResult = Arrays.asList(
+            final List<List<String>> expectedResult = Arrays.asList(
                     null,
                     null,
                     null,
@@ -129,7 +129,7 @@ public class TableTableJoinIntegrationTest extends AbstractJoinIntegrationTest {
             leftTable.leftJoin(rightTable, valueJoiner, materialized).toStream().peek(new CountingPeek(false)).to(OUTPUT_TOPIC);
             runTest(expectedFinalJoinResult, storeName);
         } else {
-            List<List<String>> expectedResult = Arrays.asList(
+            final List<List<String>> expectedResult = Arrays.asList(
                     null,
                     null,
                     Collections.singletonList("A-null"),
@@ -160,7 +160,7 @@ public class TableTableJoinIntegrationTest extends AbstractJoinIntegrationTest {
             leftTable.outerJoin(rightTable, valueJoiner, materialized).toStream().peek(new CountingPeek(false)).to(OUTPUT_TOPIC);
             runTest(expectedFinalJoinResult, storeName);
         } else {
-            List<List<String>> expectedResult = Arrays.asList(
+            final List<List<String>> expectedResult = Arrays.asList(
                     null,
                     null,
                     Collections.singletonList("A-null"),
@@ -197,7 +197,7 @@ public class TableTableJoinIntegrationTest extends AbstractJoinIntegrationTest {
         } else {
             // FIXME: the duplicate below for all the multi-joins
             //        are due to KAFKA-6443, should be updated once it is fixed.
-            List<List<String>> expectedResult = Arrays.asList(
+            final List<List<String>> expectedResult = Arrays.asList(
                     null,
                     null,
                     null,
@@ -235,7 +235,7 @@ public class TableTableJoinIntegrationTest extends AbstractJoinIntegrationTest {
                     .to(OUTPUT_TOPIC);
             runTest(expectedFinalMultiJoinResult, storeName);
         } else {
-            List<List<String>> expectedResult = Arrays.asList(
+            final List<List<String>> expectedResult = Arrays.asList(
                     null,
                     null,
                     null,
@@ -274,7 +274,7 @@ public class TableTableJoinIntegrationTest extends AbstractJoinIntegrationTest {
                     .to(OUTPUT_TOPIC);
             runTest(expectedFinalMultiJoinResult, storeName);
         } else {
-            List<List<String>> expectedResult = Arrays.asList(
+            final List<List<String>> expectedResult = Arrays.asList(
                     null,
                     null,
                     null,
@@ -312,7 +312,7 @@ public class TableTableJoinIntegrationTest extends AbstractJoinIntegrationTest {
                     .to(OUTPUT_TOPIC);
             runTest(expectedFinalMultiJoinResult, storeName);
         } else {
-            List<List<String>> expectedResult = Arrays.asList(
+            final List<List<String>> expectedResult = Arrays.asList(
                     null,
                     null,
                     null,
@@ -351,7 +351,7 @@ public class TableTableJoinIntegrationTest extends AbstractJoinIntegrationTest {
                     .to(OUTPUT_TOPIC);
             runTest(expectedFinalMultiJoinResult, storeName);
         } else {
-            List<List<String>> expectedResult = Arrays.asList(
+            final List<List<String>> expectedResult = Arrays.asList(
                     null,
                     null,
                     null,
@@ -390,7 +390,7 @@ public class TableTableJoinIntegrationTest extends AbstractJoinIntegrationTest {
                     .to(OUTPUT_TOPIC);
             runTest(expectedFinalMultiJoinResult, storeName);
         } else {
-            List<List<String>> expectedResult = Arrays.asList(
+            final List<List<String>> expectedResult = Arrays.asList(
                     null,
                     null,
                     null,
@@ -428,7 +428,7 @@ public class TableTableJoinIntegrationTest extends AbstractJoinIntegrationTest {
                     .to(OUTPUT_TOPIC);
             runTest(expectedFinalMultiJoinResult, storeName);
         } else {
-            List<List<String>> expectedResult = Arrays.asList(
+            final List<List<String>> expectedResult = Arrays.asList(
                     null,
                     null,
                     null,
@@ -467,7 +467,7 @@ public class TableTableJoinIntegrationTest extends AbstractJoinIntegrationTest {
                     .to(OUTPUT_TOPIC);
             runTest(expectedFinalMultiJoinResult, storeName);
         } else {
-            List<List<String>> expectedResult = Arrays.asList(
+            final List<List<String>> expectedResult = Arrays.asList(
                     null,
                     null,
                     null,
@@ -506,7 +506,7 @@ public class TableTableJoinIntegrationTest extends AbstractJoinIntegrationTest {
                     .to(OUTPUT_TOPIC);
             runTest(expectedFinalMultiJoinResult, storeName);
         } else {
-            List<List<String>> expectedResult = Arrays.asList(
+            final List<List<String>> expectedResult = Arrays.asList(
                     null,
                     null,
                     null,
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java
index dcacf68..6c2c464 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java
@@ -110,7 +110,7 @@ public class KafkaEmbedded {
      * You can use this to tell Kafka producers and consumers how to connect to this instance.
      */
     public String brokerList() {
-        Object listenerConfig = effectiveConfig.get(KafkaConfig$.MODULE$.InterBrokerListenerNameProp());
+        final Object listenerConfig = effectiveConfig.get(KafkaConfig$.MODULE$.InterBrokerListenerNameProp());
         return kafka.config().hostName() + ":" + kafka.boundPort(
             new ListenerName(listenerConfig != null ? listenerConfig.toString() : "PLAINTEXT"));
     }
@@ -134,7 +134,7 @@ public class KafkaEmbedded {
         log.debug("Removing log dir at {} ...", logDir);
         try {
             Utils.delete(logDir);
-        } catch (IOException e) {
+        } catch (final IOException e) {
             throw new RuntimeException(e);
         }
         tmpFolder.delete();
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java
index 7b22df1..ab973e8 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java
@@ -30,8 +30,8 @@ public class JoinWindowsTest {
 
     @Test
     public void shouldHaveSaneEqualsAndHashCode() {
-        JoinWindows w1 = JoinWindows.of(anySize);
-        JoinWindows w2 = JoinWindows.of(anySize);
+        final JoinWindows w1 = JoinWindows.of(anySize);
+        final JoinWindows w2 = JoinWindows.of(anySize);
 
         // Reflexive
         assertEquals(w1, w1);
@@ -42,8 +42,8 @@ public class JoinWindowsTest {
         assertEquals(w2, w1);
         assertEquals(w1.hashCode(), w2.hashCode());
 
-        JoinWindows w3 = JoinWindows.of(w2.afterMs).before(anyOtherSize);
-        JoinWindows w4 = JoinWindows.of(anyOtherSize).after(w2.afterMs);
+        final JoinWindows w3 = JoinWindows.of(w2.afterMs).before(anyOtherSize);
+        final JoinWindows w4 = JoinWindows.of(anyOtherSize).after(w2.afterMs);
         assertEquals(w3, w4);
         assertEquals(w4, w3);
         assertEquals(w3.hashCode(), w4.hashCode());
@@ -53,13 +53,13 @@ public class JoinWindowsTest {
         assertNotEquals("must be false for different window types", UnlimitedWindows.of(), w1);
         assertNotEquals("must be false for different types", new Object(), w1);
 
-        JoinWindows differentWindowSize = JoinWindows.of(w1.afterMs + 1);
+        final JoinWindows differentWindowSize = JoinWindows.of(w1.afterMs + 1);
         assertNotEquals("must be false when window sizes are different", differentWindowSize, w1);
 
-        JoinWindows differentWindowSize2 = JoinWindows.of(w1.afterMs).after(w1.afterMs + 1);
+        final JoinWindows differentWindowSize2 = JoinWindows.of(w1.afterMs).after(w1.afterMs + 1);
         assertNotEquals("must be false when window sizes are different", differentWindowSize2, w1);
 
-        JoinWindows differentWindowSize3 = JoinWindows.of(w1.afterMs).before(w1.beforeMs + 1);
+        final JoinWindows differentWindowSize3 = JoinWindows.of(w1.afterMs).before(w1.beforeMs + 1);
         assertNotEquals("must be false when window sizes are different", differentWindowSize3, w1);
     }
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedDeserializerTest.java
index ec3cc83..7a56909 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedDeserializerTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedDeserializerTest.java
@@ -43,7 +43,7 @@ public class SessionWindowedDeserializerTest {
     @Test
     public void testWindowedKeyDeserializerNoArgConstructors() {
         sessionWindowedDeserializer.configure(props, true);
-        Deserializer<?> inner = sessionWindowedDeserializer.innerDeserializer();
+        final Deserializer<?> inner = sessionWindowedDeserializer.innerDeserializer();
         assertNotNull("Inner deserializer should be not null", inner);
         assertTrue("Inner deserializer type should be StringDeserializer", inner instanceof StringDeserializer);
     }
@@ -51,7 +51,7 @@ public class SessionWindowedDeserializerTest {
     @Test
     public void testWindowedValueDeserializerNoArgConstructors() {
         sessionWindowedDeserializer.configure(props, false);
-        Deserializer<?> inner = sessionWindowedDeserializer.innerDeserializer();
+        final Deserializer<?> inner = sessionWindowedDeserializer.innerDeserializer();
         assertNotNull("Inner deserializer should be not null", inner);
         assertTrue("Inner deserializer type should be ByteArrayDeserializer", inner instanceof ByteArrayDeserializer);
     }
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedSerializerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedSerializerTest.java
index e7266db..f4e28a8 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedSerializerTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedSerializerTest.java
@@ -43,7 +43,7 @@ public class SessionWindowedSerializerTest {
     @Test
     public void testWindowedKeySerializerNoArgConstructors() {
         sessionWindowedSerializer.configure(props, true);
-        Serializer<?> inner = sessionWindowedSerializer.innerSerializer();
+        final Serializer<?> inner = sessionWindowedSerializer.innerSerializer();
         assertNotNull("Inner serializer should be not null", inner);
         assertTrue("Inner serializer type should be StringSerializer", inner instanceof StringSerializer);
     }
@@ -51,7 +51,7 @@ public class SessionWindowedSerializerTest {
     @Test
     public void testWindowedValueSerializerNoArgConstructors() {
         sessionWindowedSerializer.configure(props, false);
-        Serializer<?> inner = sessionWindowedSerializer.innerSerializer();
+        final Serializer<?> inner = sessionWindowedSerializer.innerSerializer();
         assertNotNull("Inner serializer should be not null", inner);
         assertTrue("Inner serializer type should be ByteArraySerializer", inner instanceof ByteArraySerializer);
     }
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedDeserializerTest.java
index 660a530..9eaa520 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedDeserializerTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedDeserializerTest.java
@@ -44,7 +44,7 @@ public class TimeWindowedDeserializerTest {
     @Test
     public void testWindowedKeyDeserializerNoArgConstructors() {
         timeWindowedDeserializer.configure(props, true);
-        Deserializer<?> inner = timeWindowedDeserializer.innerDeserializer();
+        final Deserializer<?> inner = timeWindowedDeserializer.innerDeserializer();
         assertNotNull("Inner deserializer should be not null", inner);
         assertTrue("Inner deserializer type should be StringDeserializer", inner instanceof StringDeserializer);
     }
@@ -52,7 +52,7 @@ public class TimeWindowedDeserializerTest {
     @Test
     public void testWindowedValueDeserializerNoArgConstructors() {
         timeWindowedDeserializer.configure(props, false);
-        Deserializer<?> inner = timeWindowedDeserializer.innerDeserializer();
+        final Deserializer<?> inner = timeWindowedDeserializer.innerDeserializer();
         assertNotNull("Inner deserializer should be not null", inner);
         assertTrue("Inner deserializer type should be ByteArrayDeserializer", inner instanceof ByteArrayDeserializer);
     }
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedSerializerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedSerializerTest.java
index cd019c8..03ee891 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedSerializerTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedSerializerTest.java
@@ -43,7 +43,7 @@ public class TimeWindowedSerializerTest {
     @Test
     public void testWindowedKeySerializerNoArgConstructors() {
         timeWindowedSerializer.configure(props, true);
-        Serializer<?> inner = timeWindowedSerializer.innerSerializer();
+        final Serializer<?> inner = timeWindowedSerializer.innerSerializer();
         assertNotNull("Inner serializer should be not null", inner);
         assertTrue("Inner serializer type should be StringSerializer", inner instanceof StringSerializer);
     }
@@ -51,7 +51,7 @@ public class TimeWindowedSerializerTest {
     @Test
     public void testWindowedValueSerializerNoArgConstructors() {
         timeWindowedSerializer.configure(props, false);
-        Serializer<?> inner = timeWindowedSerializer.innerSerializer();
+        final Serializer<?> inner = timeWindowedSerializer.innerSerializer();
         assertNotNull("Inner serializer should be not null", inner);
         assertTrue("Inner serializer type should be ByteArraySerializer", inner instanceof ByteArraySerializer);
     }
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java
index 390678f..a90c86f 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java
@@ -53,8 +53,8 @@ public class TimeWindowsTest {
 
     @Test
     public void shouldHaveSaneEqualsAndHashCode() {
-        TimeWindows w1 = TimeWindows.of(ANY_SIZE);
-        TimeWindows w2 = TimeWindows.of(w1.sizeMs);
+        final TimeWindows w1 = TimeWindows.of(ANY_SIZE);
+        final TimeWindows w2 = TimeWindows.of(w1.sizeMs);
 
         // Reflexive
         assertEquals(w1, w1);
@@ -66,7 +66,7 @@ public class TimeWindowsTest {
         assertEquals(w1.hashCode(), w2.hashCode());
 
         // Transitive
-        TimeWindows w3 = TimeWindows.of(w2.sizeMs);
+        final TimeWindows w3 = TimeWindows.of(w2.sizeMs);
         assertEquals(w2, w3);
         assertEquals(w1, w3);
         assertEquals(w1.hashCode(), w3.hashCode());
@@ -76,10 +76,10 @@ public class TimeWindowsTest {
         assertNotEquals("must be false for different window types", UnlimitedWindows.of(), w1);
         assertNotEquals("must be false for different types", new Object(), w1);
 
-        TimeWindows differentWindowSize = TimeWindows.of(w1.sizeMs + 1);
+        final TimeWindows differentWindowSize = TimeWindows.of(w1.sizeMs + 1);
         assertNotEquals("must be false when window sizes are different", differentWindowSize, w1);
 
-        TimeWindows differentAdvanceInterval = w1.advanceBy(w1.advanceMs - 1);
+        final TimeWindows differentAdvanceInterval = w1.advanceBy(w1.advanceMs - 1);
         assertNotEquals("must be false when advance intervals are different", differentAdvanceInterval, w1);
     }
 
@@ -139,8 +139,8 @@ public class TimeWindowsTest {
 
     @Test
     public void shouldComputeWindowsForHoppingWindows() {
-        TimeWindows windows = TimeWindows.of(12L).advanceBy(5L);
-        Map<Long, TimeWindow> matched = windows.windowsFor(21L);
+        final TimeWindows windows = TimeWindows.of(12L).advanceBy(5L);
+        final Map<Long, TimeWindow> matched = windows.windowsFor(21L);
         assertEquals(12L / 5L + 1, matched.size());
         assertEquals(new TimeWindow(10L, 22L), matched.get(10L));
         assertEquals(new TimeWindow(15L, 27L), matched.get(15L));
@@ -149,16 +149,16 @@ public class TimeWindowsTest {
 
     @Test
     public void shouldComputeWindowsForBarelyOverlappingHoppingWindows() {
-        TimeWindows windows = TimeWindows.of(6L).advanceBy(5L);
-        Map<Long, TimeWindow> matched = windows.windowsFor(7L);
+        final TimeWindows windows = TimeWindows.of(6L).advanceBy(5L);
+        final Map<Long, TimeWindow> matched = windows.windowsFor(7L);
         assertEquals(1, matched.size());
         assertEquals(new TimeWindow(5L, 11L), matched.get(5L));
     }
 
     @Test
     public void shouldComputeWindowsForTumblingWindows() {
-        TimeWindows windows = TimeWindows.of(12L);
-        Map<Long, TimeWindow> matched = windows.windowsFor(21L);
+        final TimeWindows windows = TimeWindows.of(12L);
+        final Map<Long, TimeWindow> matched = windows.windowsFor(21L);
         assertEquals(1, matched.size());
         assertEquals(new TimeWindow(12L, 24L), matched.get(12L));
     }
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java
index e1f530b..c5e1ce5 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java
@@ -52,26 +52,26 @@ public class UnlimitedWindowsTest {
 
     @Test
     public void shouldIncludeRecordsThatHappenedOnWindowStart() {
-        UnlimitedWindows w = UnlimitedWindows.of().startOn(anyStartTime);
-        Map<Long, UnlimitedWindow> matchedWindows = w.windowsFor(w.startMs);
+        final UnlimitedWindows w = UnlimitedWindows.of().startOn(anyStartTime);
+        final Map<Long, UnlimitedWindow> matchedWindows = w.windowsFor(w.startMs);
         assertEquals(1, matchedWindows.size());
         assertEquals(new UnlimitedWindow(anyStartTime), matchedWindows.get(anyStartTime));
     }
 
     @Test
     public void shouldIncludeRecordsThatHappenedAfterWindowStart() {
-        UnlimitedWindows w = UnlimitedWindows.of().startOn(anyStartTime);
-        long timestamp = w.startMs + 1;
-        Map<Long, UnlimitedWindow> matchedWindows = w.windowsFor(timestamp);
+        final UnlimitedWindows w = UnlimitedWindows.of().startOn(anyStartTime);
+        final long timestamp = w.startMs + 1;
+        final Map<Long, UnlimitedWindow> matchedWindows = w.windowsFor(timestamp);
         assertEquals(1, matchedWindows.size());
         assertEquals(new UnlimitedWindow(anyStartTime), matchedWindows.get(anyStartTime));
     }
 
     @Test
     public void shouldExcludeRecordsThatHappenedBeforeWindowStart() {
-        UnlimitedWindows w = UnlimitedWindows.of().startOn(anyStartTime);
-        long timestamp = w.startMs - 1;
-        Map<Long, UnlimitedWindow> matchedWindows = w.windowsFor(timestamp);
+        final UnlimitedWindows w = UnlimitedWindows.of().startOn(anyStartTime);
+        final long timestamp = w.startMs - 1;
+        final Map<Long, UnlimitedWindow> matchedWindows = w.windowsFor(timestamp);
         assertTrue(matchedWindows.isEmpty());
     }
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/WindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/WindowsTest.java
index 2e9246e..fc097ca 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/WindowsTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/WindowsTest.java
@@ -27,7 +27,7 @@ public class WindowsTest {
     private class TestWindows extends Windows {
 
         @Override
-        public Map windowsFor(long timestamp) {
+        public Map windowsFor(final long timestamp) {
             return null;
         }
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java
index b5aebf0..d98fd79 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java
@@ -78,7 +78,7 @@ public class AbstractStreamTest {
         final MockProcessorSupplier<Integer, String> supplier = new MockProcessorSupplier<>();
         final String topicName = "topic";
 
-        ExtendedKStream<Integer, String> stream = new ExtendedKStream<>(builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String())));
+        final ExtendedKStream<Integer, String> stream = new ExtendedKStream<>(builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String())));
 
         stream.randomFilter().process(supplier);
 
@@ -89,7 +89,7 @@ public class AbstractStreamTest {
 
         final ConsumerRecordFactory<Integer, String> recordFactory = new ConsumerRecordFactory<>(new IntegerSerializer(), new StringSerializer());
         final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props);
-        for (int expectedKey : expectedKeys) {
+        for (final int expectedKey : expectedKeys) {
             driver.pipeInput(recordFactory.create(topicName, expectedKey, "V" + expectedKey));
         }
 
@@ -103,10 +103,10 @@ public class AbstractStreamTest {
         }
 
         KStream<K, V> randomFilter() {
-            String name = builder.newProcessorName("RANDOM-FILTER-");
-            ProcessorGraphNode processorNode = new ProcessorGraphNode(name,
-                                                                      new ProcessorParameters<>(new ExtendedKStreamDummy<>(), name),
-                                                                      false);
+            final String name = builder.newProcessorName("RANDOM-FILTER-");
+            final ProcessorGraphNode processorNode = new ProcessorGraphNode(name,
+                                                                            new ProcessorParameters<>(new ExtendedKStreamDummy<>(), name),
+                                                                            false);
             builder.addGraphNode(this.streamsGraphNode, processorNode);
             return new KStreamImpl<>(builder, name, sourceNodes, false, processorNode);
         }
@@ -127,7 +127,7 @@ public class AbstractStreamTest {
 
         private class ExtendedKStreamDummyProcessor extends AbstractProcessor<K, V> {
             @Override
-            public void process(K key, V value) {
+            public void process(final K key, final V value) {
                 // flip a coin and filter
                 if (rand.nextBoolean())
                     context().forward(key, value);
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilderTest.java
index 269c2f6..c6c4996 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilderTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilderTest.java
@@ -238,11 +238,11 @@ public class InternalStreamsBuilderTest {
         stream2.leftJoin(globalTable2, kvMapper, MockValueJoiner.TOSTRING_JOINER);
 
         final Map<Integer, Set<String>> nodeGroups = builder.internalTopologyBuilder.nodeGroups();
-        for (Integer groupId : nodeGroups.keySet()) {
+        for (final Integer groupId : nodeGroups.keySet()) {
             final ProcessorTopology topology = builder.internalTopologyBuilder.build(groupId);
             final List<StateStore> stateStores = topology.globalStateStores();
             final Set<String> names = new HashSet<>();
-            for (StateStore stateStore : stateStores) {
+            for (final StateStore stateStore : stateStores) {
                 names.add(stateStore.name());
             }
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java
index 79e0b42..31863f2 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java
@@ -132,7 +132,7 @@ public class KGroupedTableImplTest {
         final KeyValueMapper<String, Number, KeyValue<String, Integer>> intProjection =
             new KeyValueMapper<String, Number, KeyValue<String, Integer>>() {
                 @Override
-                public KeyValue<String, Integer> apply(String key, Number value) {
+                public KeyValue<String, Integer> apply(final String key, final Number value) {
                     return KeyValue.pair(key, value.intValue());
                 }
             };
@@ -157,7 +157,7 @@ public class KGroupedTableImplTest {
         final KeyValueMapper<String, Number, KeyValue<String, Integer>> intProjection =
             new KeyValueMapper<String, Number, KeyValue<String, Integer>>() {
                 @Override
-                public KeyValue<String, Integer> apply(String key, Number value) {
+                public KeyValue<String, Integer> apply(final String key, final Number value) {
                     return KeyValue.pair(key, value.intValue());
                 }
             };
@@ -183,7 +183,7 @@ public class KGroupedTableImplTest {
         final KeyValueMapper<String, Number, KeyValue<String, Integer>> intProjection =
             new KeyValueMapper<String, Number, KeyValue<String, Integer>>() {
                 @Override
-                public KeyValue<String, Integer> apply(String key, Number value) {
+                public KeyValue<String, Integer> apply(final String key, final Number value) {
                     return KeyValue.pair(key, value.intValue());
                 }
             };
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java
index 7a7a61e..b7035aa 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java
@@ -46,29 +46,29 @@ public class KStreamBranchTest {
     public void testKStreamBranch() {
         final StreamsBuilder builder = new StreamsBuilder();
 
-        Predicate<Integer, String> isEven = new Predicate<Integer, String>() {
+        final Predicate<Integer, String> isEven = new Predicate<Integer, String>() {
             @Override
-            public boolean test(Integer key, String value) {
+            public boolean test(final Integer key, final String value) {
                 return (key % 2) == 0;
             }
         };
-        Predicate<Integer, String> isMultipleOfThree = new Predicate<Integer, String>() {
+        final Predicate<Integer, String> isMultipleOfThree = new Predicate<Integer, String>() {
             @Override
-            public boolean test(Integer key, String value) {
+            public boolean test(final Integer key, final String value) {
                 return (key % 3) == 0;
             }
         };
-        Predicate<Integer, String> isOdd = new Predicate<Integer, String>() {
+        final Predicate<Integer, String> isOdd = new Predicate<Integer, String>() {
             @Override
-            public boolean test(Integer key, String value) {
+            public boolean test(final Integer key, final String value) {
                 return (key % 2) != 0;
             }
         };
 
         final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6};
 
-        KStream<Integer, String> stream;
-        KStream<Integer, String>[] branches;
+        final KStream<Integer, String> stream;
+        final KStream<Integer, String>[] branches;
 
         stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String()));
         branches = stream.branch(isEven, isMultipleOfThree, isOdd);
@@ -81,7 +81,7 @@ public class KStreamBranchTest {
         }
 
         try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topicName, expectedKey, "V" + expectedKey));
             }
         }
@@ -94,16 +94,16 @@ public class KStreamBranchTest {
 
     @Test
     public void testTypeVariance() {
-        Predicate<Number, Object> positive = new Predicate<Number, Object>() {
+        final Predicate<Number, Object> positive = new Predicate<Number, Object>() {
             @Override
-            public boolean test(Number key, Object value) {
+            public boolean test(final Number key, final Object value) {
                 return key.doubleValue() > 0;
             }
         };
 
-        Predicate<Number, Object> negative = new Predicate<Number, Object>() {
+        final Predicate<Number, Object> negative = new Predicate<Number, Object>() {
             @Override
-            public boolean test(Number key, Object value) {
+            public boolean test(final Number key, final Object value) {
                 return key.doubleValue() < 0;
             }
         };
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java
index dad6721..6306d2e 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java
@@ -41,25 +41,25 @@ public class KStreamFilterTest {
 
     private final Predicate<Integer, String> isMultipleOfThree = new Predicate<Integer, String>() {
         @Override
-        public boolean test(Integer key, String value) {
+        public boolean test(final Integer key, final String value) {
             return (key % 3) == 0;
         }
     };
 
     @Test
     public void testFilter() {
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
         final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7};
 
-        KStream<Integer, String> stream;
-        MockProcessorSupplier<Integer, String> supplier;
+        final KStream<Integer, String> stream;
+        final MockProcessorSupplier<Integer, String> supplier;
 
         supplier = new MockProcessorSupplier<>();
         stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String()));
         stream.filter(isMultipleOfThree).process(supplier);
 
         try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topicName, expectedKey, "V" + expectedKey));
             }
         }
@@ -69,18 +69,18 @@ public class KStreamFilterTest {
 
     @Test
     public void testFilterNot() {
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
         final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7};
 
-        KStream<Integer, String> stream;
-        MockProcessorSupplier<Integer, String> supplier;
+        final KStream<Integer, String> stream;
+        final MockProcessorSupplier<Integer, String> supplier;
 
         supplier = new MockProcessorSupplier<>();
         stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String()));
         stream.filterNot(isMultipleOfThree).process(supplier);
 
         try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topicName, expectedKey, "V" + expectedKey));
             }
         }
@@ -90,9 +90,9 @@ public class KStreamFilterTest {
 
     @Test
     public void testTypeVariance() {
-        Predicate<Number, Object> numberKeyPredicate = new Predicate<Number, Object>() {
+        final Predicate<Number, Object> numberKeyPredicate = new Predicate<Number, Object>() {
             @Override
-            public boolean test(Number key, Object value) {
+            public boolean test(final Number key, final Object value) {
                 return false;
             }
         };
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java
index 0a16859..8daad99 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java
@@ -43,13 +43,13 @@ public class KStreamFlatMapTest {
 
     @Test
     public void testFlatMap() {
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
-        KeyValueMapper<Number, Object, Iterable<KeyValue<String, String>>> mapper =
+        final KeyValueMapper<Number, Object, Iterable<KeyValue<String, String>>> mapper =
             new KeyValueMapper<Number, Object, Iterable<KeyValue<String, String>>>() {
                 @Override
-                public Iterable<KeyValue<String, String>> apply(Number key, Object value) {
-                    ArrayList<KeyValue<String, String>> result = new ArrayList<>();
+                public Iterable<KeyValue<String, String>> apply(final Number key, final Object value) {
+                    final ArrayList<KeyValue<String, String>> result = new ArrayList<>();
                     for (int i = 0; i < key.intValue(); i++) {
                         result.add(KeyValue.pair(Integer.toString(key.intValue() * 10 + i), value.toString()));
                     }
@@ -59,22 +59,22 @@ public class KStreamFlatMapTest {
 
         final int[] expectedKeys = {0, 1, 2, 3};
 
-        KStream<Integer, String> stream;
-        MockProcessorSupplier<String, String> supplier;
+        final KStream<Integer, String> stream;
+        final MockProcessorSupplier<String, String> supplier;
 
         supplier = new MockProcessorSupplier<>();
         stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String()));
         stream.flatMap(mapper).process(supplier);
 
         try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topicName, expectedKey, "V" + expectedKey));
             }
         }
 
         assertEquals(6, supplier.theCapturedProcessor().processed.size());
 
-        String[] expected = {"10:V1", "20:V2", "21:V2", "30:V3", "31:V3", "32:V3"};
+        final String[] expected = {"10:V1", "20:V2", "21:V2", "30:V3", "31:V3", "32:V3"};
 
         for (int i = 0; i < expected.length; i++) {
             assertEquals(expected[i], supplier.theCapturedProcessor().processed.get(i));
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java
index c37daf1..aca0911 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java
@@ -42,13 +42,13 @@ public class KStreamFlatMapValuesTest {
 
     @Test
     public void testFlatMapValues() {
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
-        ValueMapper<Number, Iterable<String>> mapper =
+        final ValueMapper<Number, Iterable<String>> mapper =
             new ValueMapper<Number, Iterable<String>>() {
                 @Override
-                public Iterable<String> apply(Number value) {
-                    ArrayList<String> result = new ArrayList<String>();
+                public Iterable<String> apply(final Number value) {
+                    final ArrayList<String> result = new ArrayList<String>();
                     result.add("v" + value);
                     result.add("V" + value);
                     return result;
@@ -68,7 +68,7 @@ public class KStreamFlatMapValuesTest {
             }
         }
 
-        String[] expected = {"0:v0", "0:V0", "1:v1", "1:V1", "2:v2", "2:V2", "3:v3", "3:V3"};
+        final String[] expected = {"0:v0", "0:V0", "1:v1", "1:V1", "2:v2", "2:V2", "3:v3", "3:V3"};
 
         assertArrayEquals(expected, supplier.theCapturedProcessor().processed.toArray());
     }
@@ -76,13 +76,13 @@ public class KStreamFlatMapValuesTest {
 
     @Test
     public void testFlatMapValuesWithKeys() {
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
-        ValueMapperWithKey<Integer, Number, Iterable<String>> mapper =
+        final ValueMapperWithKey<Integer, Number, Iterable<String>> mapper =
                 new ValueMapperWithKey<Integer, Number, Iterable<String>>() {
             @Override
             public Iterable<String> apply(final Integer readOnlyKey, final Number value) {
-                ArrayList<String> result = new ArrayList<>();
+                final ArrayList<String> result = new ArrayList<>();
                 result.add("v" + value);
                 result.add("k" + readOnlyKey);
                 return result;
@@ -103,7 +103,7 @@ public class KStreamFlatMapValuesTest {
             }
         }
 
-        String[] expected = {"0:v0", "0:k0", "1:v1", "1:k1", "2:v2", "2:k2", "3:v3", "3:k3"};
+        final String[] expected = {"0:v0", "0:k0", "1:v1", "1:k1", "2:v2", "2:k2", "3:v3", "3:k3"};
 
         assertArrayEquals(expected, supplier.theCapturedProcessor().processed.toArray());
     }
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamForeachTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamForeachTest.java
index 4bfdd39..fbcd6db 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamForeachTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamForeachTest.java
@@ -46,14 +46,14 @@ public class KStreamForeachTest {
     @Test
     public void testForeach() {
         // Given
-        List<KeyValue<Integer, String>> inputRecords = Arrays.asList(
+        final List<KeyValue<Integer, String>> inputRecords = Arrays.asList(
             new KeyValue<>(0, "zero"),
             new KeyValue<>(1, "one"),
             new KeyValue<>(2, "two"),
             new KeyValue<>(3, "three")
         );
 
-        List<KeyValue<Integer, String>> expectedRecords = Arrays.asList(
+        final List<KeyValue<Integer, String>> expectedRecords = Arrays.asList(
             new KeyValue<>(0, "ZERO"),
             new KeyValue<>(2, "ONE"),
             new KeyValue<>(4, "TWO"),
@@ -61,39 +61,39 @@ public class KStreamForeachTest {
         );
 
         final List<KeyValue<Integer, String>> actualRecords = new ArrayList<>();
-        ForeachAction<Integer, String> action =
+        final ForeachAction<Integer, String> action =
             new ForeachAction<Integer, String>() {
                 @Override
-                public void apply(Integer key, String value) {
+                public void apply(final Integer key, final String value) {
                     actualRecords.add(new KeyValue<>(key * 2, value.toUpperCase(Locale.ROOT)));
                 }
             };
 
         // When
-        StreamsBuilder builder = new StreamsBuilder();
-        KStream<Integer, String> stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String()));
+        final StreamsBuilder builder = new StreamsBuilder();
+        final KStream<Integer, String> stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String()));
         stream.foreach(action);
 
         // Then
         try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
-            for (KeyValue<Integer, String> record : inputRecords) {
+            for (final KeyValue<Integer, String> record : inputRecords) {
                 driver.pipeInput(recordFactory.create(topicName, record.key, record.value));
             }
         }
 
         assertEquals(expectedRecords.size(), actualRecords.size());
         for (int i = 0; i < expectedRecords.size(); i++) {
-            KeyValue<Integer, String> expectedRecord = expectedRecords.get(i);
-            KeyValue<Integer, String> actualRecord = actualRecords.get(i);
+            final KeyValue<Integer, String> expectedRecord = expectedRecords.get(i);
+            final KeyValue<Integer, String> actualRecord = actualRecords.get(i);
             assertEquals(expectedRecord, actualRecord);
         }
     }
 
     @Test
     public void testTypeVariance() {
-        ForeachAction<Number, Object> consume = new ForeachAction<Number, Object>() {
+        final ForeachAction<Number, Object> consume = new ForeachAction<Number, Object>() {
             @Override
-            public void apply(Number key, Object value) {}
+            public void apply(final Number key, final Object value) {}
         };
 
         new StreamsBuilder()
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java
index 7aed8e1..c41b19b 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java
@@ -85,62 +85,62 @@ public class KStreamImplTest {
     public void testNumProcesses() {
         final StreamsBuilder builder = new StreamsBuilder();
 
-        KStream<String, String> source1 = builder.stream(Arrays.asList("topic-1", "topic-2"), stringConsumed);
+        final KStream<String, String> source1 = builder.stream(Arrays.asList("topic-1", "topic-2"), stringConsumed);
 
-        KStream<String, String> source2 = builder.stream(Arrays.asList("topic-3", "topic-4"), stringConsumed);
+        final KStream<String, String> source2 = builder.stream(Arrays.asList("topic-3", "topic-4"), stringConsumed);
 
-        KStream<String, String> stream1 =
+        final KStream<String, String> stream1 =
             source1.filter(new Predicate<String, String>() {
                 @Override
-                public boolean test(String key, String value) {
+                public boolean test(final String key, final String value) {
                     return true;
                 }
             }).filterNot(new Predicate<String, String>() {
                 @Override
-                public boolean test(String key, String value) {
+                public boolean test(final String key, final String value) {
                     return false;
                 }
             });
 
-        KStream<String, Integer> stream2 = stream1.mapValues(new ValueMapper<String, Integer>() {
+        final KStream<String, Integer> stream2 = stream1.mapValues(new ValueMapper<String, Integer>() {
             @Override
-            public Integer apply(String value) {
+            public Integer apply(final String value) {
                 return new Integer(value);
             }
         });
 
-        KStream<String, Integer> stream3 = source2.flatMapValues(new ValueMapper<String, Iterable<Integer>>() {
+        final KStream<String, Integer> stream3 = source2.flatMapValues(new ValueMapper<String, Iterable<Integer>>() {
             @Override
-            public Iterable<Integer> apply(String value) {
+            public Iterable<Integer> apply(final String value) {
                 return Collections.singletonList(new Integer(value));
             }
         });
 
-        KStream<String, Integer>[] streams2 = stream2.branch(
+        final KStream<String, Integer>[] streams2 = stream2.branch(
                 new Predicate<String, Integer>() {
                     @Override
-                    public boolean test(String key, Integer value) {
+                    public boolean test(final String key, final Integer value) {
                         return (value % 2) == 0;
                     }
                 },
                 new Predicate<String, Integer>() {
                     @Override
-                    public boolean test(String key, Integer value) {
+                    public boolean test(final String key, final Integer value) {
                         return true;
                     }
                 }
         );
 
-        KStream<String, Integer>[] streams3 = stream3.branch(
+        final KStream<String, Integer>[] streams3 = stream3.branch(
                 new Predicate<String, Integer>() {
                     @Override
-                    public boolean test(String key, Integer value) {
+                    public boolean test(final String key, final Integer value) {
                         return (value % 2) == 0;
                     }
                 },
                 new Predicate<String, Integer>() {
                     @Override
-                    public boolean test(String key, Integer value) {
+                    public boolean test(final String key, final Integer value) {
                         return true;
                     }
                 }
@@ -148,16 +148,16 @@ public class KStreamImplTest {
 
         final int anyWindowSize = 1;
         final Joined<String, Integer, Integer> joined = Joined.with(Serdes.String(), Serdes.Integer(), Serdes.Integer());
-        KStream<String, Integer> stream4 = streams2[0].join(streams3[0], new ValueJoiner<Integer, Integer, Integer>() {
+        final KStream<String, Integer> stream4 = streams2[0].join(streams3[0], new ValueJoiner<Integer, Integer, Integer>() {
             @Override
-            public Integer apply(Integer value1, Integer value2) {
+            public Integer apply(final Integer value1, final Integer value2) {
                 return value1 + value2;
             }
         }, JoinWindows.of(anyWindowSize), joined);
 
         streams2[1].join(streams3[1], new ValueJoiner<Integer, Integer, Integer>() {
             @Override
-            public Integer apply(Integer value1, Integer value2) {
+            public Integer apply(final Integer value1, final Integer value2) {
                 return value1 + value2;
             }
         }, JoinWindows.of(anyWindowSize), joined);
@@ -182,13 +182,13 @@ public class KStreamImplTest {
     @Test
     public void shouldUseRecordMetadataTimestampExtractorWithThrough() {
         final StreamsBuilder builder = new StreamsBuilder();
-        KStream<String, String> stream1 = builder.stream(Arrays.asList("topic-1", "topic-2"), stringConsumed);
-        KStream<String, String> stream2 = builder.stream(Arrays.asList("topic-3", "topic-4"), stringConsumed);
+        final KStream<String, String> stream1 = builder.stream(Arrays.asList("topic-1", "topic-2"), stringConsumed);
+        final KStream<String, String> stream2 = builder.stream(Arrays.asList("topic-3", "topic-4"), stringConsumed);
 
         stream1.to("topic-5");
         stream2.through("topic-6");
 
-        ProcessorTopology processorTopology = TopologyWrapper.getInternalTopologyBuilder(builder.build()).setApplicationId("X").build(null);
+        final ProcessorTopology processorTopology = TopologyWrapper.getInternalTopologyBuilder(builder.build()).setApplicationId("X").build(null);
         assertThat(processorTopology.source("topic-6").getTimestampExtractor(), instanceOf(FailOnInvalidTimestamp.class));
         assertEquals(processorTopology.source("topic-4").getTimestampExtractor(), null);
         assertEquals(processorTopology.source("topic-3").getTimestampExtractor(), null);
@@ -238,7 +238,7 @@ public class KStreamImplTest {
             driver.pipeInput(recordFactory.create(input, "a", "v2"));
             driver.pipeInput(recordFactory.create(input, "b", "v1"));
         }
-        List<MockProcessor<String, String>> mockProcessors = processorSupplier.capturedProcessors(2);
+        final List<MockProcessor<String, String>> mockProcessors = processorSupplier.capturedProcessors(2);
         assertThat(mockProcessors.get(0).processed, equalTo(Utils.mkList("a:v1", "a:v2")));
         assertThat(mockProcessors.get(1).processed, equalTo(Collections.singletonList("b:v1")));
     }
@@ -252,7 +252,7 @@ public class KStreamImplTest {
         final KStream<String, String> stream = kStream
                         .map(new KeyValueMapper<String, String, KeyValue<? extends String, ? extends String>>() {
                             @Override
-                            public KeyValue<? extends String, ? extends String> apply(String key, String value) {
+                            public KeyValue<? extends String, ? extends String> apply(final String key, final String value) {
                                 return KeyValue.pair(value, value);
                             }
                         });
@@ -268,7 +268,7 @@ public class KStreamImplTest {
 
         final SourceNode originalSourceNode = topology.source("topic-1");
 
-        for (SourceNode sourceNode: topology.sources()) {
+        for (final SourceNode sourceNode: topology.sources()) {
             if (sourceNode.name().equals(originalSourceNode.name())) {
                 assertEquals(sourceNode.getTimestampExtractor(), null);
             } else {
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java
index 2ee81eb..963b681 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java
@@ -144,7 +144,7 @@ public class KStreamKStreamJoinTest {
             // --> w1 = { 0:X0, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 }
             //     w2 = { 0:Y0, 1:Y1 }
 
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "X" + expectedKey));
             }
 
@@ -156,7 +156,7 @@ public class KStreamKStreamJoinTest {
             // --> w1 = { 0:X0, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 }
             //     w2 = { 0:Y0, 1:Y1, 0:YY0, 1:YY1, 2:YY2, 3:YY3 }
 
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey));
             }
 
@@ -168,7 +168,7 @@ public class KStreamKStreamJoinTest {
             // --> w1 = { 0:X0, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3,  0:XX0, 1:XX1, 2:XX2, 3:XX3 }
             //     w2 = { 0:Y0, 1:Y1, 0:YY0, 1:YY1, 2:YY2, 3:YY3 }
 
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey));
             }
 
@@ -246,7 +246,7 @@ public class KStreamKStreamJoinTest {
             // --> w1 = { 0:X0, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 }
             //     w2 = { 0:Y0, 1:Y1 }
 
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "X" + expectedKey));
             }
 
@@ -258,7 +258,7 @@ public class KStreamKStreamJoinTest {
             // --> w1 = { 0:X0, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 }
             //     w2 = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 }
 
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey));
             }
 
@@ -270,7 +270,7 @@ public class KStreamKStreamJoinTest {
             // --> w1 = { 0:X0, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3,  0:XX0, 1:XX1, 2:XX2, 3:XX3 }
             //     w2 = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 }
 
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey));
             }
 
@@ -355,35 +355,35 @@ public class KStreamKStreamJoinTest {
             // w1 = { 0:X0, 1:X1, 2:X2, 3:X3 }
 
             time += 100L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("2:X2+YY2", "3:X3+YY3");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("3:X3+YY3");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
@@ -392,35 +392,35 @@ public class KStreamKStreamJoinTest {
             // go back to the time before expiration
 
             time = 1000L - 100L - 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult();
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("0:X0+YY0");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1");
 
             time += 1;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2");
 
             time += 1;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
@@ -438,35 +438,35 @@ public class KStreamKStreamJoinTest {
             // w2 = { 0:Y0, 1:Y1, 2:Y2, 3:Y3 }
 
             time = 2000L + 100L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("2:XX2+Y2", "3:XX3+Y3");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("3:XX3+Y3");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey, time));
             }
 
@@ -475,35 +475,35 @@ public class KStreamKStreamJoinTest {
             // go back to the time before expiration
 
             time = 2000L - 100L - 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult();
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("0:XX0+Y0");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey, time));
             }
 
@@ -550,70 +550,70 @@ public class KStreamKStreamJoinTest {
             processor.checkAndClearProcessResult();
 
             time = 1000L - 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult();
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("0:X0+YY0");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
 
             time = 1000 + 100L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("2:X2+YY2", "3:X3+YY3");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("3:X3+YY3");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
@@ -659,70 +659,70 @@ public class KStreamKStreamJoinTest {
             processor.checkAndClearProcessResult();
 
             time = 1000L - 100L - 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult();
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("0:X0+YY0");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
 
             time = 1000L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("2:X2+YY2", "3:X3+YY3");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
             processor.checkAndClearProcessResult("3:X3+YY3");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey, time));
             }
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java
index 56f3758..338b96a 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java
@@ -118,7 +118,7 @@ public class KStreamKStreamLeftJoinTest {
             // --> w1 = { 0:X0, 1:X1, 0:X0, 1:X1, 2:X2 }
             // --> w2 = { 0:Y0, 1:Y1, 0:YY0, 1:YY1, 2:YY2, 3:YY3}
 
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic2, expectedKey, "YY" + expectedKey));
             }
             processor.checkAndClearProcessResult("0:X0+YY0", "0:X0+YY0", "1:X1+YY1", "1:X1+YY1", "2:X2+YY2");
@@ -129,7 +129,7 @@ public class KStreamKStreamLeftJoinTest {
             // --> w1 = { 0:X0, 1:X1, 0:X0, 1:X1, 2:X2, 0:XX0, 1:XX1, 2:XX2, 3:XX3 }
             // --> w2 = { 0:Y0, 1:Y1, 0:YY0, 1:YY1, 2:YY2, 3:YY3}
 
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey));
             }
             processor.checkAndClearProcessResult("0:XX0+Y0", "0:XX0+YY0", "1:XX1+Y1", "1:XX1+YY1", "2:XX2+YY2", "3:XX3+YY3");
@@ -206,31 +206,31 @@ public class KStreamKStreamLeftJoinTest {
             // --> w2 = { 0:Y0, 1:Y1, 2:Y2, 3:Y3 }
 
             time = 1000L + 100L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey, time));
             }
             processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey, time));
             }
             processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey, time));
             }
             processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+Y2", "3:XX3+Y3");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey, time));
             }
             processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+null", "3:XX3+Y3");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey, time));
             }
             processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+null", "3:XX3+null");
@@ -238,31 +238,31 @@ public class KStreamKStreamLeftJoinTest {
             // go back to the time before expiration
 
             time = 1000L - 100L - 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey, time));
             }
             processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+null", "3:XX3+null");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey, time));
             }
             processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+null", "2:XX2+null", "3:XX3+null");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey, time));
             }
             processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+null", "3:XX3+null");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey, time));
             }
             processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+null");
 
             time += 1L;
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topic1, expectedKey, "XX" + expectedKey, time));
             }
             processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3");
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java
index 8f25265..e1851c1 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java
@@ -42,31 +42,31 @@ public class KStreamMapTest {
 
     @Test
     public void testMap() {
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
-        KeyValueMapper<Integer, String, KeyValue<String, Integer>> mapper =
+        final KeyValueMapper<Integer, String, KeyValue<String, Integer>> mapper =
             new KeyValueMapper<Integer, String, KeyValue<String, Integer>>() {
                 @Override
-                public KeyValue<String, Integer> apply(Integer key, String value) {
+                public KeyValue<String, Integer> apply(final Integer key, final String value) {
                     return KeyValue.pair(value, key);
                 }
             };
 
         final int[] expectedKeys = new int[]{0, 1, 2, 3};
 
-        MockProcessorSupplier<String, Integer> supplier = new MockProcessorSupplier<>();
-        KStream<Integer, String> stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String()));
+        final MockProcessorSupplier<String, Integer> supplier = new MockProcessorSupplier<>();
+        final KStream<Integer, String> stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String()));
         stream.map(mapper).process(supplier);
 
         try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topicName, expectedKey, "V" + expectedKey));
             }
         }
 
         assertEquals(4, supplier.theCapturedProcessor().processed.size());
 
-        String[] expected = new String[]{"V0:0", "V1:1", "V2:2", "V3:3"};
+        final String[] expected = new String[]{"V0:0", "V1:1", "V2:2", "V3:3"};
 
         for (int i = 0; i < expected.length; i++) {
             assertEquals(expected[i], supplier.theCapturedProcessor().processed.get(i));
@@ -75,9 +75,9 @@ public class KStreamMapTest {
 
     @Test
     public void testTypeVariance() {
-        KeyValueMapper<Number, Object, KeyValue<Number, String>> stringify = new KeyValueMapper<Number, Object, KeyValue<Number, String>>() {
+        final KeyValueMapper<Number, Object, KeyValue<Number, String>> stringify = new KeyValueMapper<Number, Object, KeyValue<Number, String>>() {
             @Override
-            public KeyValue<Number, String> apply(Number key, Object value) {
+            public KeyValue<Number, String> apply(final Number key, final Object value) {
                 return KeyValue.pair(key, key + ":" + value);
             }
         };
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java
index c8b267e..8de8a81 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java
@@ -43,36 +43,36 @@ public class KStreamMapValuesTest {
 
     @Test
     public void testFlatMapValues() {
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
-        ValueMapper<CharSequence, Integer> mapper =
+        final ValueMapper<CharSequence, Integer> mapper =
             new ValueMapper<CharSequence, Integer>() {
                 @Override
-                public Integer apply(CharSequence value) {
+                public Integer apply(final CharSequence value) {
                     return value.length();
                 }
             };
 
         final int[] expectedKeys = {1, 10, 100, 1000};
 
-        KStream<Integer, String> stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String()));
+        final KStream<Integer, String> stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String()));
         stream.mapValues(mapper).process(supplier);
 
         try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topicName, expectedKey, Integer.toString(expectedKey)));
             }
         }
-        String[] expected = {"1:1", "10:2", "100:3", "1000:4"};
+        final String[] expected = {"1:1", "10:2", "100:3", "1000:4"};
 
         assertArrayEquals(expected, supplier.theCapturedProcessor().processed.toArray());
     }
 
     @Test
     public void testMapValuesWithKeys() {
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
-        ValueMapperWithKey<Integer, CharSequence, Integer> mapper =
+        final ValueMapperWithKey<Integer, CharSequence, Integer> mapper =
                 new ValueMapperWithKey<Integer, CharSequence, Integer>() {
             @Override
             public Integer apply(final Integer readOnlyKey, final CharSequence value) {
@@ -82,15 +82,15 @@ public class KStreamMapValuesTest {
 
         final int[] expectedKeys = {1, 10, 100, 1000};
 
-        KStream<Integer, String> stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String()));
+        final KStream<Integer, String> stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.String()));
         stream.mapValues(mapper).process(supplier);
 
         try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topicName, expectedKey, Integer.toString(expectedKey)));
             }
         }
-        String[] expected = {"1:2", "10:12", "100:103", "1000:1004"};
+        final String[] expected = {"1:2", "10:12", "100:103", "1000:1004"};
 
         assertArrayEquals(expected, supplier.theCapturedProcessor().processed.toArray());
     }
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamPeekTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamPeekTest.java
index aa9f0f0..3e4012e 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamPeekTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamPeekTest.java
@@ -69,7 +69,7 @@ public class KStreamPeekTest {
         try {
             stream.peek(null);
             fail("expected null action to throw NPE");
-        } catch (NullPointerException expected) {
+        } catch (final NullPointerException expected) {
             // do nothing
         }
     }
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamPrintTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamPrintTest.java
index 3ba88e7..ced57b2 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamPrintTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamPrintTest.java
@@ -53,7 +53,7 @@ public class KStreamPrintTest {
         kStreamPrint = new KStreamPrint<>(new PrintForeachAction<>(byteOutStream, mapper, "test-stream"));
 
         printProcessor = kStreamPrint.get();
-        ProcessorContext processorContext = EasyMock.createNiceMock(ProcessorContext.class);
+        final ProcessorContext processorContext = EasyMock.createNiceMock(ProcessorContext.class);
         EasyMock.replay(processorContext);
 
         printProcessor.init(processorContext);
@@ -85,7 +85,7 @@ public class KStreamPrintTest {
     @SuppressWarnings("unchecked")
     private <K, V> void doTest(final List<KeyValue<K, V>> inputRecords, final String[] expectedResult) {
 
-        for (KeyValue<K, V> record: inputRecords) {
+        for (final KeyValue<K, V> record: inputRecords) {
             printProcessor.process(record.key, record.value);
         }
         printProcessor.close();
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java
index 9472ec7..62f7677 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java
@@ -45,7 +45,7 @@ public class KStreamSelectKeyTest {
 
     @Test
     public void testSelectKey() {
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
         final Map<Number, String> keyMap = new HashMap<>();
         keyMap.put(1, "ONE");
@@ -53,9 +53,9 @@ public class KStreamSelectKeyTest {
         keyMap.put(3, "THREE");
 
 
-        KeyValueMapper<Object, Number, String> selector = new KeyValueMapper<Object, Number, String>() {
+        final KeyValueMapper<Object, Number, String> selector = new KeyValueMapper<Object, Number, String>() {
             @Override
-            public String apply(Object key, Number value) {
+            public String apply(final Object key, final Number value) {
                 return keyMap.get(value);
             }
         };
@@ -63,14 +63,14 @@ public class KStreamSelectKeyTest {
         final String[] expected = new String[]{"ONE:1", "TWO:2", "THREE:3"};
         final int[] expectedValues = new int[]{1, 2, 3};
 
-        KStream<String, Integer>  stream = builder.stream(topicName, Consumed.with(Serdes.String(), Serdes.Integer()));
+        final KStream<String, Integer>  stream = builder.stream(topicName, Consumed.with(Serdes.String(), Serdes.Integer()));
 
-        MockProcessorSupplier<String, Integer> supplier = new MockProcessorSupplier<>();
+        final MockProcessorSupplier<String, Integer> supplier = new MockProcessorSupplier<>();
 
         stream.selectKey(selector).process(supplier);
 
         try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
-            for (int expectedValue : expectedValues) {
+            for (final int expectedValue : expectedValues) {
                 driver.pipeInput(recordFactory.create(expectedValue));
             }
         }
@@ -85,9 +85,9 @@ public class KStreamSelectKeyTest {
 
     @Test
     public void testTypeVariance() {
-        ForeachAction<Number, Object> consume = new ForeachAction<Number, Object>() {
+        final ForeachAction<Number, Object> consume = new ForeachAction<Number, Object>() {
             @Override
-            public void apply(Number key, Object value) {}
+            public void apply(final Number key, final Object value) {}
         };
 
         new StreamsBuilder()
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java
index aec6098..9a61874 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java
@@ -51,7 +51,7 @@ public class KStreamTransformTest {
 
     @Test
     public void testTransform() {
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
         final TransformerSupplier<Number, Number, KeyValue<Integer, Integer>> transformerSupplier = new TransformerSupplier<Number, Number, KeyValue<Integer, Integer>>() {
             public Transformer<Number, Number, KeyValue<Integer, Integer>> get() {
@@ -76,12 +76,12 @@ public class KStreamTransformTest {
 
         final int[] expectedKeys = {1, 10, 100, 1000};
 
-        MockProcessorSupplier<Integer, Integer> processor = new MockProcessorSupplier<>();
-        KStream<Integer, Integer> stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.Integer()));
+        final MockProcessorSupplier<Integer, Integer> processor = new MockProcessorSupplier<>();
+        final KStream<Integer, Integer> stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.Integer()));
         stream.transform(transformerSupplier).process(processor);
 
         kstreamDriver.setUp(builder);
-        for (int expectedKey : expectedKeys) {
+        for (final int expectedKey : expectedKeys) {
             kstreamDriver.process(topicName, expectedKey, expectedKey * 10);
         }
 
@@ -93,7 +93,7 @@ public class KStreamTransformTest {
 
         //String[] expected = {"2:10", "20:110", "200:1110", "2000:11110", "-1:2", "-1:3"};
 
-        String[] expected = {"2:10", "20:110", "200:1110", "2000:11110"};
+        final String[] expected = {"2:10", "20:110", "200:1110", "2000:11110"};
 
         for (int i = 0; i < expected.length; i++) {
             assertEquals(expected[i], processor.theCapturedProcessor().processed.get(i));
@@ -102,9 +102,9 @@ public class KStreamTransformTest {
 
     @Test
     public void testTransformWithNewDriverAndPunctuator() {
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
-        TransformerSupplier<Number, Number, KeyValue<Integer, Integer>> transformerSupplier = new TransformerSupplier<Number, Number, KeyValue<Integer, Integer>>() {
+        final TransformerSupplier<Number, Number, KeyValue<Integer, Integer>> transformerSupplier = new TransformerSupplier<Number, Number, KeyValue<Integer, Integer>>() {
             public Transformer<Number, Number, KeyValue<Integer, Integer>> get() {
                 return new Transformer<Number, Number, KeyValue<Integer, Integer>>() {
 
@@ -114,7 +114,7 @@ public class KStreamTransformTest {
                     public void init(final ProcessorContext context) {
                         context.schedule(1, PunctuationType.WALL_CLOCK_TIME, new Punctuator() {
                             @Override
-                            public void punctuate(long timestamp) {
+                            public void punctuate(final long timestamp) {
                                 context.forward(-1, (int) timestamp);
                             }
                         });
@@ -135,12 +135,12 @@ public class KStreamTransformTest {
 
         final int[] expectedKeys = {1, 10, 100, 1000};
 
-        MockProcessorSupplier<Integer, Integer> processor = new MockProcessorSupplier<>();
-        KStream<Integer, Integer> stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.Integer()));
+        final MockProcessorSupplier<Integer, Integer> processor = new MockProcessorSupplier<>();
+        final KStream<Integer, Integer> stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.Integer()));
         stream.transform(transformerSupplier).process(processor);
 
         try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props, 0L)) {
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topicName, expectedKey, expectedKey * 10, 0L));
             }
 
@@ -152,7 +152,7 @@ public class KStreamTransformTest {
 
         assertEquals(6, processor.theCapturedProcessor().processed.size());
 
-        String[] expected = {"2:10", "20:110", "200:1110", "2000:11110", "-1:2", "-1:3"};
+        final String[] expected = {"2:10", "20:110", "200:1110", "2000:11110", "-1:2", "-1:3"};
 
         for (int i = 0; i < expected.length; i++) {
             assertEquals(expected[i], processor.theCapturedProcessor().processed.get(i));
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java
index eac2585..c399650 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java
@@ -57,9 +57,9 @@ public class KStreamTransformValuesTest {
 
     @Test
     public void testTransform() {
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
-        ValueTransformerSupplier<Number, Integer> valueTransformerSupplier =
+        final ValueTransformerSupplier<Number, Integer> valueTransformerSupplier =
             new ValueTransformerSupplier<Number, Integer>() {
                 public ValueTransformer<Number, Integer> get() {
                     return new ValueTransformer<Number, Integer>() {
@@ -67,11 +67,11 @@ public class KStreamTransformValuesTest {
                         private int total = 0;
 
                         @Override
-                        public void init(ProcessorContext context) {
+                        public void init(final ProcessorContext context) {
                         }
 
                         @Override
-                        public Integer transform(Number value) {
+                        public Integer transform(final Number value) {
                             total += value.intValue();
                             return total;
                         }
@@ -85,25 +85,25 @@ public class KStreamTransformValuesTest {
 
         final int[] expectedKeys = {1, 10, 100, 1000};
 
-        KStream<Integer, Integer> stream;
+        final KStream<Integer, Integer> stream;
         stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.Integer()));
         stream.transformValues(valueTransformerSupplier).process(supplier);
 
         try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props, 0L)) {
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topicName, expectedKey, expectedKey * 10, 0L));
             }
         }
-        String[] expected = {"1:10", "10:110", "100:1110", "1000:11110"};
+        final String[] expected = {"1:10", "10:110", "100:1110", "1000:11110"};
 
         assertArrayEquals(expected, supplier.theCapturedProcessor().processed.toArray());
     }
 
     @Test
     public void testTransformWithKey() {
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
-        ValueTransformerWithKeySupplier<Integer, Number, Integer> valueTransformerSupplier =
+        final ValueTransformerWithKeySupplier<Integer, Number, Integer> valueTransformerSupplier =
                 new ValueTransformerWithKeySupplier<Integer, Number, Integer>() {
             public ValueTransformerWithKey<Integer, Number, Integer> get() {
                 return new ValueTransformerWithKey<Integer, Number, Integer>() {
@@ -128,16 +128,16 @@ public class KStreamTransformValuesTest {
 
         final int[] expectedKeys = {1, 10, 100, 1000};
 
-        KStream<Integer, Integer> stream;
+        final KStream<Integer, Integer> stream;
         stream = builder.stream(topicName, Consumed.with(Serdes.Integer(), Serdes.Integer()));
         stream.transformValues(valueTransformerSupplier).process(supplier);
 
         try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props, 0L)) {
-            for (int expectedKey : expectedKeys) {
+            for (final int expectedKey : expectedKeys) {
                 driver.pipeInput(recordFactory.create(topicName, expectedKey, expectedKey * 10, 0L));
             }
         }
-        String[] expected = {"1:11", "10:121", "100:1221", "1000:12221"};
+        final String[] expected = {"1:11", "10:121", "100:1221", "1000:12221"};
 
         assertArrayEquals(expected, supplier.theCapturedProcessor().processed.toArray());
     }
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java
index d32aa07..b75b239 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java
@@ -20,11 +20,11 @@ import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.utils.Bytes;
 import org.apache.kafka.common.utils.Utils;
-import org.apache.kafka.streams.kstream.Consumed;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.StreamsBuilder;
 import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
 import org.apache.kafka.streams.kstream.Aggregator;
+import org.apache.kafka.streams.kstream.Consumed;
 import org.apache.kafka.streams.kstream.ForeachAction;
 import org.apache.kafka.streams.kstream.Initializer;
 import org.apache.kafka.streams.kstream.KTable;
@@ -77,9 +77,9 @@ public class KTableAggregateTest {
         final String topic1 = "topic1";
 
 
-        KTable<String, String> table1 = builder.table(topic1, consumed);
-        KTable<String, String> table2 = table1.groupBy(MockMapper.<String, String>noOpKeyValueMapper(),
-                                                       stringSerialzied
+        final KTable<String, String> table1 = builder.table(topic1, consumed);
+        final KTable<String, String> table2 = table1.groupBy(MockMapper.<String, String>noOpKeyValueMapper(),
+                                                             stringSerialzied
         ).aggregate(MockInitializer.STRING_INIT,
                 MockAggregator.TOSTRING_ADDER,
                 MockAggregator.TOSTRING_REMOVER,
@@ -123,9 +123,9 @@ public class KTableAggregateTest {
         final StreamsBuilder builder = new StreamsBuilder();
         final String topic1 = "topic1";
 
-        KTable<String, String> table1 = builder.table(topic1, consumed);
-        KTable<String, String> table2 = table1.groupBy(MockMapper.<String, String>noOpKeyValueMapper(),
-                                                       stringSerialzied
+        final KTable<String, String> table1 = builder.table(topic1, consumed);
+        final KTable<String, String> table2 = table1.groupBy(MockMapper.<String, String>noOpKeyValueMapper(),
+                                                             stringSerialzied
         ).aggregate(MockInitializer.STRING_INIT,
             MockAggregator.TOSTRING_ADDER,
             MockAggregator.TOSTRING_REMOVER,
@@ -149,21 +149,22 @@ public class KTableAggregateTest {
         final StreamsBuilder builder = new StreamsBuilder();
         final String topic1 = "topic1";
 
-        KTable<String, String> table1 = builder.table(topic1, consumed);
-        KTable<String, String> table2 = table1.groupBy(new KeyValueMapper<String, String, KeyValue<String, String>>() {
-            @Override
-                public KeyValue<String, String> apply(String key, String value) {
-                switch (key) {
-                    case "null":
-                        return KeyValue.pair(null, value);
-                    case "NULL":
-                        return null;
-                    default:
-                        return KeyValue.pair(value, value);
-                }
+        final KTable<String, String> table1 = builder.table(topic1, consumed);
+        final KTable<String, String> table2 = table1.groupBy(
+            new KeyValueMapper<String, String, KeyValue<String, String>>() {
+                @Override
+                public KeyValue<String, String> apply(final String key, final String value) {
+                    switch (key) {
+                        case "null":
+                            return KeyValue.pair(null, value);
+                        case "NULL":
+                            return null;
+                        default:
+                            return KeyValue.pair(value, value);
+                    }
                 }
             },
-                stringSerialzied
+            stringSerialzied
         )
                 .aggregate(MockInitializer.STRING_INIT,
                 MockAggregator.TOSTRING_ADDER,
@@ -195,7 +196,7 @@ public class KTableAggregateTest {
                 "1:0+1",
                 "1:0+1-1",
                 "1:0+1-1+1",
-                "2:0+2", 
+                "2:0+2",
                   //noop
                 "2:0+2-2", "4:0+4",
                   //noop
@@ -286,7 +287,7 @@ public class KTableAggregateTest {
             "green:2"
             ), proc.processed);
     }
-    
+
     @Test
     public void testRemoveOldBeforeAddNew() {
         final StreamsBuilder builder = new StreamsBuilder();
@@ -297,7 +298,7 @@ public class KTableAggregateTest {
                 .groupBy(new KeyValueMapper<String, String, KeyValue<String, String>>() {
 
                     @Override
-                    public KeyValue<String, String> apply(String key, String value) {
+                    public KeyValue<String, String> apply(final String key, final String value) {
                         return KeyValue.pair(String.valueOf(key.charAt(0)), String.valueOf(key.charAt(1)));
                     }
                 }, stringSerialzied)
@@ -308,15 +309,15 @@ public class KTableAggregateTest {
                         return "";
                     }
                 }, new Aggregator<String, String, String>() {
-                    
+
                     @Override
-                    public String apply(String aggKey, String value, String aggregate) {
+                    public String apply(final String aggKey, final String value, final String aggregate) {
                         return aggregate + value;
-                    } 
+                    }
                 }, new Aggregator<String, String, String>() {
 
                     @Override
-                    public String apply(String key, String value, String aggregate) {
+                    public String apply(final String key, final String value, final String aggregate) {
                         return aggregate.replaceAll(value, "");
                     }
                 }, Materialized.<String, String, KeyValueStore<Bytes, byte[]>>as("someStore").withValueSerde(Serdes.String()))
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java
index 2cf192b..2995b23 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java
@@ -55,7 +55,7 @@ public class KTableFilterTest {
                               final KTable<String, Integer> table2,
                               final KTable<String, Integer> table3,
                               final String topic) {
-        MockProcessorSupplier<String, Integer> supplier = new MockProcessorSupplier<>();
+        final MockProcessorSupplier<String, Integer> supplier = new MockProcessorSupplier<>();
         table2.toStream().process(supplier);
         table3.toStream().process(supplier);
 
@@ -80,17 +80,17 @@ public class KTableFilterTest {
 
         final String topic1 = "topic1";
 
-        KTable<String, Integer> table1 = builder.table(topic1, consumed);
+        final KTable<String, Integer> table1 = builder.table(topic1, consumed);
 
-        KTable<String, Integer> table2 = table1.filter(new Predicate<String, Integer>() {
+        final KTable<String, Integer> table2 = table1.filter(new Predicate<String, Integer>() {
             @Override
-            public boolean test(String key, Integer value) {
+            public boolean test(final String key, final Integer value) {
                 return (value % 2) == 0;
             }
         });
-        KTable<String, Integer> table3 = table1.filterNot(new Predicate<String, Integer>() {
+        final KTable<String, Integer> table3 = table1.filterNot(new Predicate<String, Integer>() {
             @Override
-            public boolean test(String key, Integer value) {
+            public boolean test(final String key, final Integer value) {
                 return (value % 2) == 0;
             }
         });
@@ -104,17 +104,17 @@ public class KTableFilterTest {
 
         final String topic1 = "topic1";
 
-        KTable<String, Integer> table1 = builder.table(topic1, consumed);
+        final KTable<String, Integer> table1 = builder.table(topic1, consumed);
 
-        KTable<String, Integer> table2 = table1.filter(new Predicate<String, Integer>() {
+        final KTable<String, Integer> table2 = table1.filter(new Predicate<String, Integer>() {
             @Override
-            public boolean test(String key, Integer value) {
+            public boolean test(final String key, final Integer value) {
                 return (value % 2) == 0;
             }
         }, Materialized.<String, Integer, KeyValueStore<Bytes, byte[]>>as("anyStoreNameFilter"));
-        KTable<String, Integer> table3 = table1.filterNot(new Predicate<String, Integer>() {
+        final KTable<String, Integer> table3 = table1.filterNot(new Predicate<String, Integer>() {
             @Override
-            public boolean test(String key, Integer value) {
+            public boolean test(final String key, final Integer value) {
                 return (value % 2) == 0;
             }
         });
@@ -132,8 +132,8 @@ public class KTableFilterTest {
 
         final Topology topology = builder.build();
 
-        KTableValueGetterSupplier<String, Integer> getterSupplier2 = table2.valueGetterSupplier();
-        KTableValueGetterSupplier<String, Integer> getterSupplier3 = table3.valueGetterSupplier();
+        final KTableValueGetterSupplier<String, Integer> getterSupplier2 = table2.valueGetterSupplier();
+        final KTableValueGetterSupplier<String, Integer> getterSupplier3 = table3.valueGetterSupplier();
 
         final InternalTopologyBuilder topologyBuilder = TopologyWrapper.getInternalTopologyBuilder(topology);
         topologyBuilder.connectProcessorAndStateStores(table2.name, getterSupplier2.storeNames());
@@ -141,8 +141,8 @@ public class KTableFilterTest {
 
         try (final TopologyTestDriverWrapper driver = new TopologyTestDriverWrapper(topology, props)) {
 
-            KTableValueGetter<String, Integer> getter2 = getterSupplier2.get();
-            KTableValueGetter<String, Integer> getter3 = getterSupplier3.get();
+            final KTableValueGetter<String, Integer> getter2 = getterSupplier2.get();
+            final KTableValueGetter<String, Integer> getter3 = getterSupplier3.get();
 
             getter2.init(driver.setCurrentNodeForProcessorContext(table2.name));
             getter3.init(driver.setCurrentNodeForProcessorContext(table3.name));
@@ -195,23 +195,23 @@ public class KTableFilterTest {
 
     @Test
     public void testValueGetter() {
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
-        String topic1 = "topic1";
+        final String topic1 = "topic1";
 
-        KTableImpl<String, Integer, Integer> table1 =
+        final KTableImpl<String, Integer, Integer> table1 =
                 (KTableImpl<String, Integer, Integer>) builder.table(topic1, consumed);
-        KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
+        final KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
                 new Predicate<String, Integer>() {
                     @Override
-                    public boolean test(String key, Integer value) {
+                    public boolean test(final String key, final Integer value) {
                         return (value % 2) == 0;
                     }
                 });
-        KTableImpl<String, Integer, Integer> table3 = (KTableImpl<String, Integer, Integer>) table1.filterNot(
+        final KTableImpl<String, Integer, Integer> table3 = (KTableImpl<String, Integer, Integer>) table1.filterNot(
                 new Predicate<String, Integer>() {
                     @Override
-                    public boolean test(String key, Integer value) {
+                    public boolean test(final String key, final Integer value) {
                         return (value % 2) == 0;
                     }
                 });
@@ -221,23 +221,23 @@ public class KTableFilterTest {
 
     @Test
     public void testQueryableValueGetter() {
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
-        String topic1 = "topic1";
+        final String topic1 = "topic1";
 
-        KTableImpl<String, Integer, Integer> table1 =
+        final KTableImpl<String, Integer, Integer> table1 =
             (KTableImpl<String, Integer, Integer>) builder.table(topic1, consumed);
-        KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
+        final KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
             new Predicate<String, Integer>() {
                 @Override
-                public boolean test(String key, Integer value) {
+                public boolean test(final String key, final Integer value) {
                     return (value % 2) == 0;
                 }
             }, Materialized.<String, Integer, KeyValueStore<Bytes, byte[]>>as("anyStoreNameFilter"));
-        KTableImpl<String, Integer, Integer> table3 = (KTableImpl<String, Integer, Integer>) table1.filterNot(
+        final KTableImpl<String, Integer, Integer> table3 = (KTableImpl<String, Integer, Integer>) table1.filterNot(
             new Predicate<String, Integer>() {
                 @Override
-                public boolean test(String key, Integer value) {
+                public boolean test(final String key, final Integer value) {
                     return (value % 2) == 0;
                 }
             });
@@ -252,7 +252,7 @@ public class KTableFilterTest {
                                           final KTableImpl<String, Integer, Integer> table1,
                                           final KTableImpl<String, Integer, Integer> table2,
                                           final String topic1) {
-        MockProcessorSupplier<String, Integer> supplier = new MockProcessorSupplier<>();
+        final MockProcessorSupplier<String, Integer> supplier = new MockProcessorSupplier<>();
 
         builder.build().addProcessor("proc1", supplier, table1.name);
         builder.build().addProcessor("proc2", supplier, table2.name);
@@ -290,16 +290,16 @@ public class KTableFilterTest {
 
     @Test
     public void testNotSendingOldValue() {
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
-        String topic1 = "topic1";
+        final String topic1 = "topic1";
 
-        KTableImpl<String, Integer, Integer> table1 =
+        final KTableImpl<String, Integer, Integer> table1 =
                 (KTableImpl<String, Integer, Integer>) builder.table(topic1, consumed);
-        KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
+        final KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
                 new Predicate<String, Integer>() {
                     @Override
-                    public boolean test(String key, Integer value) {
+                    public boolean test(final String key, final Integer value) {
                         return (value % 2) == 0;
                     }
                 });
@@ -309,16 +309,16 @@ public class KTableFilterTest {
 
     @Test
     public void testQueryableNotSendingOldValue() {
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
-        String topic1 = "topic1";
+        final String topic1 = "topic1";
 
-        KTableImpl<String, Integer, Integer> table1 =
+        final KTableImpl<String, Integer, Integer> table1 =
             (KTableImpl<String, Integer, Integer>) builder.table(topic1, consumed);
-        KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
+        final KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
             new Predicate<String, Integer>() {
                 @Override
-                public boolean test(String key, Integer value) {
+                public boolean test(final String key, final Integer value) {
                     return (value % 2) == 0;
                 }
             }, Materialized.<String, Integer, KeyValueStore<Bytes, byte[]>>as("anyStoreNameFilter"));
@@ -370,16 +370,16 @@ public class KTableFilterTest {
 
     @Test
     public void testSendingOldValue() {
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
-        String topic1 = "topic1";
+        final String topic1 = "topic1";
 
-        KTableImpl<String, Integer, Integer> table1 =
+        final KTableImpl<String, Integer, Integer> table1 =
                 (KTableImpl<String, Integer, Integer>) builder.table(topic1, consumed);
-        KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
+        final KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
                 new Predicate<String, Integer>() {
                     @Override
-                    public boolean test(String key, Integer value) {
+                    public boolean test(final String key, final Integer value) {
                         return (value % 2) == 0;
                     }
                 });
@@ -389,16 +389,16 @@ public class KTableFilterTest {
 
     @Test
     public void testQueryableSendingOldValue() {
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
-        String topic1 = "topic1";
+        final String topic1 = "topic1";
 
-        KTableImpl<String, Integer, Integer> table1 =
+        final KTableImpl<String, Integer, Integer> table1 =
             (KTableImpl<String, Integer, Integer>) builder.table(topic1, consumed);
-        KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
+        final KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
             new Predicate<String, Integer>() {
                 @Override
-                public boolean test(String key, Integer value) {
+                public boolean test(final String key, final Integer value) {
                     return (value % 2) == 0;
                 }
             }, Materialized.<String, Integer, KeyValueStore<Bytes, byte[]>>as("anyStoreNameFilter"));
@@ -432,17 +432,17 @@ public class KTableFilterTest {
     @Test
     public void testSkipNullOnMaterialization() {
         // Do not explicitly set enableSendingOldValues. Let a further downstream stateful operator trigger it instead.
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
-        String topic1 = "topic1";
+        final String topic1 = "topic1";
 
         final Consumed<String, String> consumed = Consumed.with(Serdes.String(), Serdes.String());
-        KTableImpl<String, String, String> table1 =
+        final KTableImpl<String, String, String> table1 =
             (KTableImpl<String, String, String>) builder.table(topic1, consumed);
-        KTableImpl<String, String, String> table2 = (KTableImpl<String, String, String>) table1.filter(
+        final KTableImpl<String, String, String> table2 = (KTableImpl<String, String, String>) table1.filter(
             new Predicate<String, String>() {
                 @Override
-                public boolean test(String key, String value) {
+                public boolean test(final String key, final String value) {
                     return value.equalsIgnoreCase("accept");
                 }
             }).groupBy(MockMapper.<String, String>noOpKeyValueMapper())
@@ -454,17 +454,17 @@ public class KTableFilterTest {
     @Test
     public void testQueryableSkipNullOnMaterialization() {
         // Do not explicitly set enableSendingOldValues. Let a further downstream stateful operator trigger it instead.
-        StreamsBuilder builder = new StreamsBuilder();
+        final StreamsBuilder builder = new StreamsBuilder();
 
-        String topic1 = "topic1";
+        final String topic1 = "topic1";
 
         final Consumed<String, String> consumed = Consumed.with(Serdes.String(), Serdes.String());
-        KTableImpl<String, String, String> table1 =
+        final KTableImpl<String, String, String> table1 =
             (KTableImpl<String, String, String>) builder.table(topic1, consumed);
-        KTableImpl<String, String, String> table2 = (KTableImpl<String, String, String>) table1.filter(
+        final KTableImpl<String, String, String> table2 = (KTableImpl<String, String, String>) table1.filter(
             new Predicate<String, String>() {
                 @Override
-                public boolean test(String key, String value) {
+                public boolean test(final String key, final String value) {
                     return value.equalsIgnoreCase("accept");
                 }
             }, Materialized.<String, String, KeyValueStore<Bytes, byte[]>>as("anyStoreNameFilter")).groupBy(MockMapper.<String, String>noOpKeyValueMapper())
@@ -475,9 +475,9 @@ public class KTableFilterTest {
 
     @Test
     public void testTypeVariance() {
-        Predicate<Number, Object> numberKeyPredicate = new Predicate<Number, Object>() {
+        final Predicate<Number, Object> numberKeyPredicate = new Predicate<Number, Object>() {
             @Override
-            public boolean test(Number key, Object value) {
+            public boolean test(final Number key, final Object value) {
                 return false;
             }
         };
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java
index 016cde2..60d7426 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java
@@ -80,8 +80,8 @@ public class KTableImplTest {
     public void testKTable() {
         final StreamsBuilder builder = new StreamsBuilder();
 
-        String topic1 = "topic1";
-        String topic2 = "topic2";
+        final String topic1 = "topic1";
+        final String topic2 = "topic2";
 
         final KTable<String, String> table1 = builder.table(topic1, consumed);
 
@@ -90,7 +90,7 @@ public class KTableImplTest {
 
         final KTable<String, Integer> table2 = table1.mapValues(new ValueMapper<String, Integer>() {
             @Override
-            public Integer apply(String value) {
+            public Integer apply(final String value) {
                 return new Integer(value);
             }
         });
@@ -99,7 +99,7 @@ public class KTableImplTest {
 
         final KTable<String, Integer> table3 = table2.filter(new Predicate<String, Integer>() {
             @Override
-            public boolean test(String key, Integer value) {
+            public boolean test(final String key, final Integer value) {
                 return (value % 2) == 0;
             }
         });
@@ -137,14 +137,14 @@ public class KTableImplTest {
         final KTableImpl<String, String, Integer> table2 = (KTableImpl<String, String, Integer>) table1.mapValues(
                 new ValueMapper<String, Integer>() {
                     @Override
-                    public Integer apply(String value) {
+                    public Integer apply(final String value) {
                         return new Integer(value);
                     }
                 });
         final KTableImpl<String, Integer, Integer> table3 = (KTableImpl<String, Integer, Integer>) table2.filter(
                 new Predicate<String, Integer>() {
                     @Override
-                    public boolean test(String key, Integer value) {
+                    public boolean test(final String key, final Integer value) {
                         return (value % 2) == 0;
                     }
                 });
@@ -271,14 +271,14 @@ public class KTableImplTest {
         final KTableImpl<String, String, Integer> table1Mapped = (KTableImpl<String, String, Integer>) table1.mapValues(
                 new ValueMapper<String, Integer>() {
                     @Override
-                    public Integer apply(String value) {
+                    public Integer apply(final String value) {
                         return new Integer(value);
                     }
                 });
         table1Mapped.filter(
                 new Predicate<String, Integer>() {
                     @Override
-                    public boolean test(String key, Integer value) {
+                    public boolean test(final String key, final Integer value) {
                         return (value % 2) == 0;
                     }
                 });
@@ -303,21 +303,21 @@ public class KTableImplTest {
         final KTableImpl<String, String, Integer> table1Mapped = (KTableImpl<String, String, Integer>) table1.mapValues(
                 new ValueMapper<String, Integer>() {
                     @Override
-                    public Integer apply(String value) {
+                    public Integer apply(final String value) {
                         return new Integer(value);
                     }
                 });
         final KTableImpl<String, Integer, Integer> table1MappedFiltered = (KTableImpl<String, Integer, Integer>) table1Mapped.filter(
                 new Predicate<String, Integer>() {
                     @Override
-                    public boolean test(String key, Integer value) {
+                    public boolean test(final String key, final Integer value) {
                         return (value % 2) == 0;
                     }
                 });
         table2.join(table1MappedFiltered,
                 new ValueJoiner<String, Integer, String>() {
                     @Override
-                    public String apply(String v1, Integer v2) {
+                    public String apply(final String v1, final Integer v2) {
                         return v1 + v2;
                     }
                 });
@@ -371,8 +371,8 @@ public class KTableImplTest {
             assertTopologyContainsProcessor(topology, "KSTREAM-SINK-0000000007");
             assertTopologyContainsProcessor(topology, "KSTREAM-SOURCE-0000000008");
 
-            Field valSerializerField = ((SinkNode) driver.getProcessor("KSTREAM-SINK-0000000003")).getClass().getDeclaredField("valSerializer");
-            Field valDeserializerField = ((SourceNode) driver.getProcessor("KSTREAM-SOURCE-0000000004")).getClass().getDeclaredField("valDeserializer");
+            final Field valSerializerField = ((SinkNode) driver.getProcessor("KSTREAM-SINK-0000000003")).getClass().getDeclaredField("valSerializer");
+            final Field valDeserializerField = ((SourceNode) driver.getProcessor("KSTREAM-SOURCE-0000000004")).getClass().getDeclaredField("valDeserializer");
             valSerializerField.setAccessible(true);
             valDeserializerField.setAccessible(true);
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java
index 081c6a0..bf5660a 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java
@@ -46,29 +46,29 @@ public class KTableMapKeysTest {
     public void testMapKeysConvertingToStream() {
         final StreamsBuilder builder = new StreamsBuilder();
 
-        String topic1 = "topic_map_keys";
+        final String topic1 = "topic_map_keys";
 
-        KTable<Integer, String> table1 = builder.table(topic1, Consumed.with(Serdes.Integer(), Serdes.String()));
+        final KTable<Integer, String> table1 = builder.table(topic1, Consumed.with(Serdes.Integer(), Serdes.String()));
 
         final Map<Integer, String> keyMap = new HashMap<>();
         keyMap.put(1, "ONE");
         keyMap.put(2, "TWO");
         keyMap.put(3, "THREE");
 
-        KeyValueMapper<Integer, String, String> keyMapper = new KeyValueMapper<Integer, String, String>() {
+        final KeyValueMapper<Integer, String, String> keyMapper = new KeyValueMapper<Integer, String, String>() {
             @Override
-            public  String apply(Integer key, String value) {
+            public  String apply(final Integer key, final String value) {
                 return keyMap.get(key);
             }
         };
 
-        KStream<String, String> convertedStream = table1.toStream(keyMapper);
+        final KStream<String, String> convertedStream = table1.toStream(keyMapper);
 
         final String[] expected = new String[]{"ONE:V_ONE", "TWO:V_TWO", "THREE:V_THREE"};
         final int[] originalKeys = new int[]{1, 2, 3};
         final String[] values = new String[]{"V_ONE", "V_TWO", "V_THREE"};
 
-        MockProcessorSupplier<String, String> supplier = new MockProcessorSupplier<>();
+        final MockProcessorSupplier<String, String> supplier = new MockProcessorSupplier<>();
 
         convertedStream.process(supplier);
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java
index a01d5cb..ddfd5a5 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java
@@ -72,7 +72,7 @@ public class KTableMapValuesTest {
         final KTable<String, String> table1 = builder.table(topic1, consumed);
         final KTable<String, Integer> table2 = table1.mapValues(new ValueMapper<CharSequence, Integer>() {
             @Override
-            public Integer apply(CharSequence value) {
+            public Integer apply(final CharSequence value) {
                 return value.charAt(0) - 48;
             }
         });
@@ -92,7 +92,7 @@ public class KTableMapValuesTest {
         final KTable<String, String> table1 = builder.table(topic1, consumed);
         final KTable<String, Integer> table2 = table1.mapValues(new ValueMapper<CharSequence, Integer>() {
             @Override
-            public Integer apply(CharSequence value) {
+            public Integer apply(final CharSequence value) {
                 return value.charAt(0) - 48;
             }
         }, Materialized.<String, Integer, KeyValueStore<Bytes, byte[]>>as("anyName").withValueSerde(Serdes.Integer()));
@@ -124,10 +124,10 @@ public class KTableMapValuesTest {
         topologyBuilder.connectProcessorAndStateStores(table4.name, getterSupplier4.storeNames());
 
         try (final TopologyTestDriverWrapper driver = new TopologyTestDriverWrapper(builder.build(), props)) {
-            KTableValueGetter<String, String> getter1 = getterSupplier1.get();
-            KTableValueGetter<String, Integer> getter2 = getterSupplier2.get();
-            KTableValueGetter<String, Integer> getter3 = getterSupplier3.get();
-            KTableValueGetter<String, String> getter4 = getterSupplier4.get();
+            final KTableValueGetter<String, String> getter1 = getterSupplier1.get();
+            final KTableValueGetter<String, Integer> getter2 = getterSupplier2.get();
+            final KTableValueGetter<String, Integer> getter3 = getterSupplier3.get();
+            final KTableValueGetter<String, String> getter4 = getterSupplier4.get();
 
             getter1.init(driver.setCurrentNodeForProcessorContext(table1.name));
             getter2.init(driver.setCurrentNodeForProcessorContext(table2.name));
@@ -223,14 +223,14 @@ public class KTableMapValuesTest {
         final KTableImpl<String, String, Integer> table2 = (KTableImpl<String, String, Integer>) table1.mapValues(
                 new ValueMapper<String, Integer>() {
                     @Override
-                    public Integer apply(String value) {
+                    public Integer apply(final String value) {
                         return new Integer(value);
                     }
                 });
         final KTableImpl<String, Integer, Integer> table3 = (KTableImpl<String, Integer, Integer>) table2.filter(
                 new Predicate<String, Integer>() {
                     @Override
-                    public boolean test(String key, Integer value) {
+                    public boolean test(final String key, final Integer value) {
                         return (value % 2) == 0;
                     }
                 });
@@ -254,14 +254,14 @@ public class KTableMapValuesTest {
         final KTableImpl<String, String, Integer> table2 = (KTableImpl<String, String, Integer>) table1.mapValues(
             new ValueMapper<String, Integer>() {
                 @Override
-                public Integer apply(String value) {
+                public Integer apply(final String value) {
                     return new Integer(value);
                 }
             }, Materialized.<String, Integer, KeyValueStore<Bytes, byte[]>>as(storeName2).withValueSerde(Serdes.Integer()));
         final KTableImpl<String, Integer, Integer> table3 = (KTableImpl<String, Integer, Integer>) table2.filter(
             new Predicate<String, Integer>() {
                 @Override
-                public boolean test(String key, Integer value) {
+                public boolean test(final String key, final Integer value) {
                     return (value % 2) == 0;
                 }
             }, Materialized.<String, Integer, KeyValueStore<Bytes, byte[]>>as(storeName3).withValueSerde(Serdes.Integer()));
@@ -282,7 +282,7 @@ public class KTableMapValuesTest {
         final KTableImpl<String, String, Integer> table2 = (KTableImpl<String, String, Integer>) table1.mapValues(
                 new ValueMapper<String, Integer>() {
                     @Override
-                    public Integer apply(String value) {
+                    public Integer apply(final String value) {
                         return new Integer(value);
                     }
                 });
@@ -330,7 +330,7 @@ public class KTableMapValuesTest {
         final KTableImpl<String, String, Integer> table2 = (KTableImpl<String, String, Integer>) table1.mapValues(
                 new ValueMapper<String, Integer>() {
                     @Override
-                    public Integer apply(String value) {
+                    public Integer apply(final String value) {
                         return new Integer(value);
                     }
                 });
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java
index be20c86..dbcce90 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java
@@ -413,7 +413,7 @@ public class KTableTransformValuesTest {
     private static KeyValueMapper<String, Integer, KeyValue<String, Integer>> toForceSendingOfOldValues() {
         return new KeyValueMapper<String, Integer, KeyValue<String, Integer>>() {
             @Override
-            public KeyValue<String, Integer> apply(String key, Integer value) {
+            public KeyValue<String, Integer> apply(final String key, final Integer value) {
                 return new KeyValue<>(key, value);
             }
         };
@@ -422,7 +422,7 @@ public class KTableTransformValuesTest {
     private static ValueMapper<Integer, String> mapBackToStrings() {
         return new ValueMapper<Integer, String>() {
             @Override
-            public String apply(Integer value) {
+            public String apply(final Integer value) {
                 return value.toString();
             }
         };
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java
index c41ae15..adebf16 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java
@@ -60,19 +60,19 @@ public class WindowedStreamPartitionerTest {
         final WindowedStreamPartitioner<Integer, String> streamPartitioner = new WindowedStreamPartitioner<>(timeWindowedSerializer);
 
         for (int k = 0; k < 10; k++) {
-            Integer key = rand.nextInt();
-            byte[] keyBytes = intSerializer.serialize(topicName, key);
+            final Integer key = rand.nextInt();
+            final byte[] keyBytes = intSerializer.serialize(topicName, key);
 
-            String value = key.toString();
-            byte[] valueBytes = stringSerializer.serialize(topicName, value);
+            final String value = key.toString();
+            final byte[] valueBytes = stringSerializer.serialize(topicName, value);
 
-            Integer expected = defaultPartitioner.partition("topic", key, keyBytes, value, valueBytes, cluster);
+            final Integer expected = defaultPartitioner.partition("topic", key, keyBytes, value, valueBytes, cluster);
 
             for (int w = 1; w < 10; w++) {
-                TimeWindow window = new TimeWindow(10 * w, 20 * w);
+                final TimeWindow window = new TimeWindow(10 * w, 20 * w);
 
-                Windowed<Integer> windowedKey = new Windowed<>(key, window);
-                Integer actual = streamPartitioner.partition(topicName, windowedKey, value, infos.size());
+                final Windowed<Integer> windowedKey = new Windowed<>(key, window);
+                final Integer actual = streamPartitioner.partition(topicName, windowedKey, value, infos.size());
 
                 assertEquals(expected, actual);
             }
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java
index 5012704..2bddf41 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java
@@ -120,7 +120,7 @@ public class StreamsGraphTest {
         // build step one
         assertEquals(expectedJoinedTopology, builder.build().describe().toString());
 
-        KStream<String, String> filteredJoinStream = joinedStream.filter((k, v) -> v.equals("foo"));
+        final KStream<String, String> filteredJoinStream = joinedStream.filter((k, v) -> v.equals("foo"));
         // build step two
         assertEquals(expectedJoinedFilteredTopology, builder.build().describe().toString());
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java
index 9301e5f..0fa0583 100644
--- a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java
+++ b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java
@@ -212,17 +212,17 @@ public class SimpleBenchmark {
         }
     }
 
-    public static void main(String[] args) throws IOException {
+    public static void main(final String[] args) throws IOException {
         if (args.length < 5) {
             System.err.println("Not enough parameters are provided; expecting propFileName, testName, numRecords, keySkew, valueSize");
             System.exit(1);
         }
 
-        String propFileName = args[0];
-        String testName = args[1].toLowerCase(Locale.ROOT);
-        int numRecords = Integer.parseInt(args[2]);
-        double keySkew = Double.parseDouble(args[3]); // 0d means even distribution
-        int valueSize = Integer.parseInt(args[4]);
+        final String propFileName = args[0];
+        final String testName = args[1].toLowerCase(Locale.ROOT);
+        final int numRecords = Integer.parseInt(args[2]);
+        final double keySkew = Double.parseDouble(args[3]); // 0d means even distribution
+        final int valueSize = Integer.parseInt(args[4]);
 
         final Properties props = Utils.loadProps(propFileName);
         final String kafka = props.getProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG);
@@ -265,7 +265,7 @@ public class SimpleBenchmark {
     }
 
     private Properties setProduceConsumeProperties(final String clientId) {
-        Properties clientProps = new Properties();
+        final Properties clientProps = new Properties();
         clientProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, props.getProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG));
         clientProps.put(ProducerConfig.CLIENT_ID_CONFIG, clientId);
         // the socket buffer needs to be large, especially when running in AWS with
@@ -605,7 +605,7 @@ public class SimpleBenchmark {
      * Measure the performance of a KTable-KTable left join. The setup is such that each
      * KTable record joins to exactly one element in the other KTable
      */
-    private void tableTableJoin(String kTableTopic1, String kTableTopic2) {
+    private void tableTableJoin(final String kTableTopic1, final String kTableTopic2) {
         final CountDownLatch latch = new CountDownLatch(1);
 
         // setup join
@@ -674,7 +674,7 @@ public class SimpleBenchmark {
         final KafkaStreams streamsClient = new KafkaStreams(builder.build(), props);
         streamsClient.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
             @Override
-            public void uncaughtException(Thread t, Throwable e) {
+            public void uncaughtException(final Thread t, final Throwable e) {
                 System.out.println("FATAL: An unexpected exception is encountered on thread " + t + ": " + e);
 
                 streamsClient.close(30, TimeUnit.SECONDS);
@@ -684,19 +684,19 @@ public class SimpleBenchmark {
         return streamsClient;
     }
     
-    private double megabytesPerSec(long time, long processedBytes) {
+    private double megabytesPerSec(final long time, final long processedBytes) {
         return  (processedBytes / 1024.0 / 1024.0) / (time / 1000.0);
     }
 
-    private double recordsPerSec(long time, int numRecords) {
+    private double recordsPerSec(final long time, final int numRecords) {
         return numRecords / (time / 1000.0);
     }
 
-    private List<TopicPartition> getAllPartitions(KafkaConsumer<?, ?> consumer, String... topics) {
-        ArrayList<TopicPartition> partitions = new ArrayList<>();
+    private List<TopicPartition> getAllPartitions(final KafkaConsumer<?, ?> consumer, final String... topics) {
+        final ArrayList<TopicPartition> partitions = new ArrayList<>();
 
-        for (String topic : topics) {
-            for (PartitionInfo info : consumer.partitionsFor(topic)) {
+        for (final String topic : topics) {
+            for (final PartitionInfo info : consumer.partitionsFor(topic)) {
                 partitions.add(new TopicPartition(info.topic(), info.partition()));
             }
         }
diff --git a/streams/src/test/java/org/apache/kafka/streams/perf/YahooBenchmark.java b/streams/src/test/java/org/apache/kafka/streams/perf/YahooBenchmark.java
index 2104221..6e09ad1 100644
--- a/streams/src/test/java/org/apache/kafka/streams/perf/YahooBenchmark.java
+++ b/streams/src/test/java/org/apache/kafka/streams/perf/YahooBenchmark.java
@@ -17,7 +17,6 @@
 package org.apache.kafka.streams.perf;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
-
 import org.apache.kafka.clients.producer.KafkaProducer;
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.clients.producer.ProducerRecord;
@@ -28,10 +27,10 @@ import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.serialization.Serializer;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.kafka.common.utils.Bytes;
-import org.apache.kafka.streams.kstream.Consumed;
 import org.apache.kafka.streams.KafkaStreams;
 import org.apache.kafka.streams.StreamsBuilder;
 import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.kstream.Consumed;
 import org.apache.kafka.streams.kstream.ForeachAction;
 import org.apache.kafka.streams.kstream.Joined;
 import org.apache.kafka.streams.kstream.KStream;
@@ -41,7 +40,6 @@ import org.apache.kafka.streams.kstream.Materialized;
 import org.apache.kafka.streams.kstream.Predicate;
 import org.apache.kafka.streams.kstream.Serialized;
 import org.apache.kafka.streams.kstream.TimeWindows;
-import org.apache.kafka.streams.kstream.ValueJoiner;
 import org.apache.kafka.streams.kstream.ValueMapper;
 import org.apache.kafka.streams.state.WindowStore;
 
@@ -103,13 +101,13 @@ public class YahooBenchmark {
         // initialize topics
         System.out.println("Initializing topic " + topic);
 
-        Properties props = new Properties();
+        final Properties props = new Properties();
         props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, parent.props.get(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG));
         props.put(ProducerConfig.CLIENT_ID_CONFIG, clientId);
         props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
         props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
 
-        try (KafkaProducer<String, String> producer = new KafkaProducer<>(props)) {
+        try (final KafkaProducer<String, String> producer = new KafkaProducer<>(props)) {
             for (int c = 0; c < numCampaigns; c++) {
                 final String campaignID = UUID.randomUUID().toString();
                 for (int a = 0; a < adsPerCampaign; a++) {
@@ -143,7 +141,7 @@ public class YahooBenchmark {
 
         final long startTime = System.currentTimeMillis();
 
-        try (KafkaProducer<String, byte[]> producer = new KafkaProducer<>(props)) {
+        try (final KafkaProducer<String, byte[]> producer = new KafkaProducer<>(props)) {
             final ProjectedEvent event = new ProjectedEvent();
             final Map<String, Object> serdeProps = new HashMap<>();
             final Serializer<ProjectedEvent> projectedEventSerializer = new JsonPOJOSerializer<>();
@@ -193,17 +191,17 @@ public class YahooBenchmark {
         }
 
         @Override
-        public void configure(Map<String, ?> props, boolean isKey) {
+        public void configure(final Map<String, ?> props, final boolean isKey) {
         }
 
         @Override
-        public byte[] serialize(String topic, T data) {
+        public byte[] serialize(final String topic, final T data) {
             if (data == null)
                 return null;
 
             try {
                 return objectMapper.writeValueAsBytes(data);
-            } catch (Exception e) {
+            } catch (final Exception e) {
                 throw new SerializationException("Error serializing JSON message", e);
             }
         }
@@ -228,19 +226,19 @@ public class YahooBenchmark {
 
         @SuppressWarnings("unchecked")
         @Override
-        public void configure(Map<String, ?> props, boolean isKey) {
+        public void configure(final Map<String, ?> props, final boolean isKey) {
             tClass = (Class<T>) props.get("JsonPOJOClass");
         }
 
         @Override
-        public T deserialize(String topic, byte[] bytes) {
+        public T deserialize(final String topic, final byte[] bytes) {
             if (bytes == null)
                 return null;
 
-            T data;
+            final T data;
             try {
                 data = objectMapper.readValue(bytes, tClass);
-            } catch (Exception e) {
+            } catch (final Exception e) {
                 throw new SerializationException(e);
             }
 
@@ -256,7 +254,7 @@ public class YahooBenchmark {
 
     private KafkaStreams createYahooBenchmarkStreams(final Properties streamConfig, final String campaignsTopic, final String eventsTopic,
                                                      final CountDownLatch latch, final int numRecords) {
-        Map<String, Object> serdeProps = new HashMap<>();
+        final Map<String, Object> serdeProps = new HashMap<>();
         final Serializer<ProjectedEvent> projectedEventSerializer = new JsonPOJOSerializer<>();
         serdeProps.put("JsonPOJOClass", ProjectedEvent.class);
         projectedEventSerializer.configure(serdeProps, false);
@@ -271,11 +269,11 @@ public class YahooBenchmark {
         final KTable<String, String> kCampaigns = builder.table(campaignsTopic, Consumed.with(Serdes.String(), Serdes.String()));
 
 
-        KStream<String, ProjectedEvent> filteredEvents = kEvents
+        final KStream<String, ProjectedEvent> filteredEvents = kEvents
             // use peek to quick when last element is processed
             .peek(new ForeachAction<String, ProjectedEvent>() {
                 @Override
-                public void apply(String key, ProjectedEvent value) {
+                public void apply(final String key, final ProjectedEvent value) {
                     parent.processedRecords++;
                     if (parent.processedRecords % 1000000 == 0) {
                         System.out.println("Processed " + parent.processedRecords);
@@ -295,8 +293,8 @@ public class YahooBenchmark {
             // select just a few of the columns
             .mapValues(new ValueMapper<ProjectedEvent, ProjectedEvent>() {
                 @Override
-                public ProjectedEvent apply(ProjectedEvent value) {
-                    ProjectedEvent event = new ProjectedEvent();
+                public ProjectedEvent apply(final ProjectedEvent value) {
+                    final ProjectedEvent event = new ProjectedEvent();
                     event.adID = value.adID;
                     event.eventTime = value.eventTime;
                     event.eventType = value.eventType;
@@ -305,11 +303,11 @@ public class YahooBenchmark {
             });
 
         // deserialize the add ID and campaign ID from the stored value in Kafka
-        KTable<String, CampaignAd> deserCampaigns = kCampaigns.mapValues(new ValueMapper<String, CampaignAd>() {
+        final KTable<String, CampaignAd> deserCampaigns = kCampaigns.mapValues(new ValueMapper<String, CampaignAd>() {
             @Override
-            public CampaignAd apply(String value) {
-                String[] parts = value.split(":");
-                CampaignAd cAdd = new CampaignAd();
+            public CampaignAd apply(final String value) {
+                final String[] parts = value.split(":");
+                final CampaignAd cAdd = new CampaignAd();
                 cAdd.adID = parts[0];
                 cAdd.campaignID = parts[1];
                 return cAdd;
@@ -317,20 +315,18 @@ public class YahooBenchmark {
         });
 
         // join the events with the campaigns
-        KStream<String, String> joined = filteredEvents.join(deserCampaigns,
-            new ValueJoiner<ProjectedEvent, CampaignAd, String>() {
-                @Override
-                public String apply(ProjectedEvent value1, CampaignAd value2) {
-                    return value2.campaignID;
-                }
-            }, Joined.<String, ProjectedEvent, CampaignAd>with(Serdes.String(), Serdes.serdeFrom(projectedEventSerializer, projectedEventDeserializer), null));
+        final KStream<String, String> joined = filteredEvents.join(
+            deserCampaigns,
+            (value1, value2) -> value2.campaignID,
+            Joined.with(Serdes.String(), Serdes.serdeFrom(projectedEventSerializer, projectedEventDeserializer), null)
+        );
 
 
         // key by campaign rather than by ad as original
-        KStream<String, String> keyedByCampaign = joined
+        final KStream<String, String> keyedByCampaign = joined
             .selectKey(new KeyValueMapper<String, String, String>() {
                 @Override
-                public String apply(String key, String value) {
+                public String apply(final String key, final String value) {
                     return value;
                 }
             });
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/TimestampExtractorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/TimestampExtractorTest.java
index 8fd9556..bd235be 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/TimestampExtractorTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/TimestampExtractorTest.java
@@ -24,7 +24,7 @@ import static org.hamcrest.MatcherAssert.assertThat;
 
 class TimestampExtractorTest {
 
-    void testExtractMetadataTimestamp(TimestampExtractor extractor) {
+    void testExtractMetadataTimestamp(final TimestampExtractor extractor) {
         final long metadataTimestamp = 42;
 
         final long timestamp = extractor.extract(
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/WallclockTimestampExtractorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/WallclockTimestampExtractorTest.java
index 12313b7..ac4f4e7 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/WallclockTimestampExtractorTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/WallclockTimestampExtractorTest.java
@@ -41,22 +41,22 @@ public class WallclockTimestampExtractorTest {
         private final long before;
         private final long after;
 
-        public InBetween(long before, long after) {
+        public InBetween(final long before, final long after) {
             this.before = before;
             this.after = after;
         }
 
         @Override
-        public boolean matches(Object item) {
+        public boolean matches(final Object item) {
             final long timestamp = (Long) item;
             return before <= timestamp && timestamp <= after;
         }
 
         @Override
-        public void describeMismatch(Object item, Description mismatchDescription) {}
+        public void describeMismatch(final Object item, final Description mismatchDescription) {}
 
         @Override
-        public void describeTo(Description description) {}
+        public void describeTo(final Description description) {}
     }
 
 }
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java
index f25f38f..54a927c 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java
@@ -62,7 +62,7 @@ public class AbstractProcessorContextTest {
         try {
             context.register(stateStore, null);
             fail("should throw illegal state exception when context already initialized");
-        } catch (IllegalStateException e) {
+        } catch (final IllegalStateException e) {
             // pass
         }
     }
@@ -198,7 +198,7 @@ public class AbstractProcessorContextTest {
         }
 
         @Override
-        public Cancellable schedule(long interval, PunctuationType type, Punctuator callback) {
+        public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) {
             return null;
         }
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AssignedStreamsTasksTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AssignedStreamsTasksTest.java
index 7efe653..a3c47b5 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AssignedStreamsTasksTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AssignedStreamsTasksTest.java
@@ -90,7 +90,7 @@ public class AssignedStreamsTasksTest {
 
         assignedTasks.initializeNewTasks();
 
-        Collection<StreamTask> restoring = assignedTasks.restoringTasks();
+        final Collection<StreamTask> restoring = assignedTasks.restoringTasks();
         assertThat(restoring.size(), equalTo(1));
         assertSame(restoring.iterator().next(), t1);
     }
@@ -293,7 +293,7 @@ public class AssignedStreamsTasksTest {
         try {
             assignedTasks.commit();
             fail("Should have thrown exception");
-        } catch (Exception e) {
+        } catch (final Exception e) {
             // ok
         }
         assertThat(assignedTasks.runningTaskIds(), equalTo(Collections.singleton(taskId1)));
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/CompositeRestoreListenerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/CompositeRestoreListenerTest.java
index e2ea668..ef2e6f7 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/CompositeRestoreListenerTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/CompositeRestoreListenerTest.java
@@ -156,28 +156,28 @@ public class CompositeRestoreListenerTest {
         compositeRestoreListener.restore(key, value);
     }
 
-    private void assertStateRestoreListenerOnStartNotification(MockStateRestoreListener restoreListener) {
+    private void assertStateRestoreListenerOnStartNotification(final MockStateRestoreListener restoreListener) {
         assertTrue(restoreListener.storeNameCalledStates.containsKey(RESTORE_START));
         assertThat(restoreListener.restoreTopicPartition, is(topicPartition));
         assertThat(restoreListener.restoreStartOffset, is(startOffset));
         assertThat(restoreListener.restoreEndOffset, is(endOffset));
     }
 
-    private void assertStateRestoreListenerOnBatchCompleteNotification(MockStateRestoreListener restoreListener) {
+    private void assertStateRestoreListenerOnBatchCompleteNotification(final MockStateRestoreListener restoreListener) {
         assertTrue(restoreListener.storeNameCalledStates.containsKey(RESTORE_BATCH));
         assertThat(restoreListener.restoreTopicPartition, is(topicPartition));
         assertThat(restoreListener.restoredBatchOffset, is(batchOffset));
         assertThat(restoreListener.numBatchRestored, is(numberRestored));
     }
 
-    private void assertStateRestoreOnEndNotification(MockStateRestoreListener restoreListener) {
+    private void assertStateRestoreOnEndNotification(final MockStateRestoreListener restoreListener) {
         assertTrue(restoreListener.storeNameCalledStates.containsKey(RESTORE_END));
         assertThat(restoreListener.restoreTopicPartition, is(topicPartition));
         assertThat(restoreListener.totalNumRestored, is(numberRestored));
     }
 
 
-    private void setUpCompositeRestoreListener(StateRestoreCallback stateRestoreCallback) {
+    private void setUpCompositeRestoreListener(final StateRestoreCallback stateRestoreCallback) {
         compositeRestoreListener = new CompositeRestoreListener(stateRestoreCallback);
         compositeRestoreListener.setUserRestoreListener(reportingStoreListener);
     }
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java
index 3374851..32ac23e 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java
@@ -349,7 +349,7 @@ public class GlobalStateManagerImplTest {
         try {
             stateManager.register(store1, null);
             fail("should have thrown due to null callback");
-        } catch (IllegalArgumentException e) {
+        } catch (final IllegalArgumentException e) {
             //pass
         }
     }
@@ -404,7 +404,7 @@ public class GlobalStateManagerImplTest {
 
         try {
             stateManager.close(Collections.<TopicPartition, Long>emptyMap());
-        } catch (ProcessorStateException e) {
+        } catch (final ProcessorStateException e) {
             // expected
         }
         assertFalse(store.isOpen());
@@ -416,7 +416,7 @@ public class GlobalStateManagerImplTest {
         writeCorruptCheckpoint();
         try {
             stateManager.initialize();
-        } catch (StreamsException e) {
+        } catch (final StreamsException e) {
             // expected
         }
         final StateDirectory stateDir = new StateDirectory(streamsConfig, new MockTime());
@@ -535,7 +535,7 @@ public class GlobalStateManagerImplTest {
         final AtomicInteger numberOfCalls = new AtomicInteger(0);
         consumer = new MockConsumer<byte[], byte[]>(OffsetResetStrategy.EARLIEST) {
             @Override
-            public synchronized Map<TopicPartition, Long> endOffsets(Collection<org.apache.kafka.common.TopicPartition> partitions) {
+            public synchronized Map<TopicPartition, Long> endOffsets(final Collection<org.apache.kafka.common.TopicPartition> partitions) {
                 numberOfCalls.incrementAndGet();
                 throw new TimeoutException();
             }
@@ -568,7 +568,7 @@ public class GlobalStateManagerImplTest {
         final AtomicInteger numberOfCalls = new AtomicInteger(0);
         consumer = new MockConsumer<byte[], byte[]>(OffsetResetStrategy.EARLIEST) {
             @Override
-            public synchronized List<PartitionInfo> partitionsFor(String topic) {
+            public synchronized List<PartitionInfo> partitionsFor(final String topic) {
                 numberOfCalls.incrementAndGet();
                 throw new TimeoutException();
             }
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java
index f3e9299..95c6943 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java
@@ -122,7 +122,7 @@ public class GlobalStreamThreadTest {
         try {
             globalStreamThread.start();
             fail("Should have thrown StreamsException if start up failed");
-        } catch (StreamsException e) {
+        } catch (final StreamsException e) {
             // ok
         }
         assertFalse(globalStreamThread.stillRunning());
@@ -150,7 +150,7 @@ public class GlobalStreamThreadTest {
         try {
             globalStreamThread.start();
             fail("Should have thrown StreamsException if start up failed");
-        } catch (StreamsException e) {
+        } catch (final StreamsException e) {
             assertThat(e.getCause(), instanceOf(RuntimeException.class));
             assertThat(e.getCause().getMessage(), equalTo("KABOOM!"));
         }
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java
index c50d358..344dc05 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java
@@ -122,9 +122,9 @@ public class InternalTopicManagerTest {
             }
         }), mockAdminClient.describeTopics(Collections.singleton(topic3)).values().get(topic3).get());
 
-        ConfigResource resource = new ConfigResource(ConfigResource.Type.TOPIC, topic);
-        ConfigResource resource2 = new ConfigResource(ConfigResource.Type.TOPIC, topic2);
-        ConfigResource resource3 = new ConfigResource(ConfigResource.Type.TOPIC, topic3);
+        final ConfigResource resource = new ConfigResource(ConfigResource.Type.TOPIC, topic);
+        final ConfigResource resource2 = new ConfigResource(ConfigResource.Type.TOPIC, topic2);
+        final ConfigResource resource3 = new ConfigResource(ConfigResource.Type.TOPIC, topic3);
 
         assertEquals(new ConfigEntry(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE), mockAdminClient.describeConfigs(Collections.singleton(resource)).values().get(resource).get().get(TopicConfig.CLEANUP_POLICY_CONFIG));
         assertEquals(new ConfigEntry(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT), mockAdminClient.describeConfigs(Collections.singleton(resource2)).values().get(resource2).get().get(TopicConfig.CLEANUP_POLICY_CONFIG));
@@ -150,7 +150,7 @@ public class InternalTopicManagerTest {
             internalTopicConfig.setNumberOfPartitions(1);
             internalTopicManager.makeReady(Collections.singletonMap(topic, internalTopicConfig));
             fail("Should have thrown StreamsException");
-        } catch (StreamsException expected) { /* pass */ }
+        } catch (final StreamsException expected) { /* pass */ }
     }
 
     @Test
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java
index d88d3b5..3495e80 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java
@@ -130,7 +130,7 @@ public class ProcessorTopologyTest {
 
     @Test
     public void testDrivingSimpleTopology() {
-        int partition = 10;
+        final int partition = 10;
         driver = new TopologyTestDriver(createSimpleTopology(partition), props);
         driver.pipeInput(recordFactory.create(INPUT_TOPIC_1, "key1", "value1"));
         assertNextOutputRecord(OUTPUT_TOPIC_1, "key1", "value1", partition);
@@ -196,7 +196,7 @@ public class ProcessorTopologyTest {
 
     @Test
     public void testDrivingStatefulTopology() {
-        String storeName = "entries";
+        final String storeName = "entries";
         driver = new TopologyTestDriver(createStatefulTopology(storeName), props);
         driver.pipeInput(recordFactory.create(INPUT_TOPIC_1, "key1", "value1"));
         driver.pipeInput(recordFactory.create(INPUT_TOPIC_1, "key2", "value2"));
@@ -498,7 +498,7 @@ public class ProcessorTopologyTest {
                 .addSink("sink-2", OUTPUT_TOPIC_2, "source-2");
     }
 
-    private Topology createSimpleMultiSourceTopology(int partition) {
+    private Topology createSimpleMultiSourceTopology(final int partition) {
         return topology.addSource("source-1", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC_1)
                 .addProcessor("processor-1", define(new ForwardingProcessor()), "source-1")
                 .addSink("sink-1", OUTPUT_TOPIC_1, constantPartitioner(partition), "processor-1")
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java
index ee0d5a3..6e4f229 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java
@@ -44,9 +44,9 @@ public class PunctuationQueueTest {
 
         queue.schedule(sched);
 
-        ProcessorNodePunctuator processorNodePunctuator = new ProcessorNodePunctuator() {
+        final ProcessorNodePunctuator processorNodePunctuator = new ProcessorNodePunctuator() {
             @Override
-            public void punctuate(ProcessorNode node, long time, PunctuationType type, Punctuator punctuator) {
+            public void punctuate(final ProcessorNode node, final long time, final PunctuationType type, final Punctuator punctuator) {
                 punctuator.punctuate(time);
             }
         };
@@ -83,9 +83,9 @@ public class PunctuationQueueTest {
 
         queue.schedule(sched);
 
-        ProcessorNodePunctuator processorNodePunctuator = new ProcessorNodePunctuator() {
+        final ProcessorNodePunctuator processorNodePunctuator = new ProcessorNodePunctuator() {
             @Override
-            public void punctuate(ProcessorNode node, long time, PunctuationType type, Punctuator punctuator) {
+            public void punctuate(final ProcessorNode node, final long time, final PunctuationType type, final Punctuator punctuator) {
                 punctuator.punctuate(time);
             }
         };
@@ -122,9 +122,9 @@ public class PunctuationQueueTest {
 
         final Cancellable cancellable = queue.schedule(sched);
 
-        ProcessorNodePunctuator processorNodePunctuator = new ProcessorNodePunctuator() {
+        final ProcessorNodePunctuator processorNodePunctuator = new ProcessorNodePunctuator() {
             @Override
-            public void punctuate(ProcessorNode node, long time, PunctuationType type, Punctuator punctuator) {
+            public void punctuate(final ProcessorNode node, final long time, final PunctuationType type, final Punctuator punctuator) {
                 punctuator.punctuate(time);
                 // simulate scheduler cancelled from within punctuator
                 cancellable.cancel();
@@ -144,10 +144,10 @@ public class PunctuationQueueTest {
     private static class TestProcessor extends AbstractProcessor<String, String> {
 
         @Override
-        public void init(ProcessorContext context) {}
+        public void init(final ProcessorContext context) {}
 
         @Override
-        public void process(String key, String value) {}
+        public void process(final String key, final String value) {}
 
         @Override
         public void close() {}
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/QuickUnionTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/QuickUnionTest.java
index e6cca87..f5c15ea 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/QuickUnionTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/QuickUnionTest.java
@@ -28,13 +28,13 @@ public class QuickUnionTest {
 
     @Test
     public void testUnite() {
-        QuickUnion<Long> qu = new QuickUnion<>();
+        final QuickUnion<Long> qu = new QuickUnion<>();
 
-        long[] ids = {
+        final long[] ids = {
             1L, 2L, 3L, 4L, 5L
         };
 
-        for (long id : ids) {
+        for (final long id : ids) {
             qu.add(id);
         }
 
@@ -65,13 +65,13 @@ public class QuickUnionTest {
 
     @Test
     public void testUniteMany() {
-        QuickUnion<Long> qu = new QuickUnion<>();
+        final QuickUnion<Long> qu = new QuickUnion<>();
 
-        long[] ids = {
+        final long[] ids = {
             1L, 2L, 3L, 4L, 5L
         };
 
-        for (long id : ids) {
+        for (final long id : ids) {
             qu.add(id);
         }
 
@@ -85,9 +85,9 @@ public class QuickUnionTest {
         assertNotEquals(qu.root(1L), qu.root(5L));
     }
 
-    private Set<Long> roots(QuickUnion<Long> qu, long... ids) {
-        HashSet<Long> roots = new HashSet<>();
-        for (long id : ids) {
+    private Set<Long> roots(final QuickUnion<Long> qu, final long... ids) {
+        final HashSet<Long> roots = new HashSet<>();
+        for (final long id : ids) {
             roots.add(qu.root(id));
         }
         return roots;
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SingleGroupPartitionGrouperStub.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SingleGroupPartitionGrouperStub.java
index 00c77af..2378ea7 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SingleGroupPartitionGrouperStub.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SingleGroupPartitionGrouperStub.java
@@ -33,14 +33,14 @@ public class SingleGroupPartitionGrouperStub implements PartitionGrouper {
     private PartitionGrouper defaultPartitionGrouper = new DefaultPartitionGrouper();
 
     @Override
-    public Map<TaskId, Set<TopicPartition>> partitionGroups(Map<Integer, Set<String>> topicGroups, Cluster metadata) {
-        Map<Integer, Set<String>> includedTopicGroups = new HashMap<>();
+    public Map<TaskId, Set<TopicPartition>> partitionGroups(final Map<Integer, Set<String>> topicGroups, final Cluster metadata) {
+        final Map<Integer, Set<String>> includedTopicGroups = new HashMap<>();
 
-        for (Map.Entry<Integer, Set<String>> entry : topicGroups.entrySet()) {
+        for (final Map.Entry<Integer, Set<String>> entry : topicGroups.entrySet()) {
             includedTopicGroups.put(entry.getKey(), entry.getValue());
             break; // arbitrarily use the first entry only
         }
-        Map<TaskId, Set<TopicPartition>> result = defaultPartitionGrouper.partitionGroups(includedTopicGroups, metadata);
+        final Map<TaskId, Set<TopicPartition>> result = defaultPartitionGrouper.partitionGroups(includedTopicGroups, metadata);
         return result;
     }
 }
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
index 6c2eb4a..6056b2d 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
@@ -862,7 +862,7 @@ public class StreamThreadTest {
         restoreConsumer.updateEndOffsets(Collections.singletonMap(partition2, 10L));
         restoreConsumer.updateBeginningOffsets(Collections.singletonMap(partition2, 0L));
         // let the store1 be restored from 0 to 10; store2 be restored from 5 (checkpointed) to 10
-        OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(stateDirectory.directoryForTask(task3), CHECKPOINT_FILE_NAME));
+        final OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(stateDirectory.directoryForTask(task3), CHECKPOINT_FILE_NAME));
         checkpoint.write(Collections.singletonMap(partition2, 5L));
 
         for (long i = 0L; i < 10L; i++) {
@@ -1302,14 +1302,14 @@ public class StreamThreadTest {
                 testMetricName,
                 new Measurable() {
                     @Override
-                    public double measure(MetricConfig config, long now) {
+                    public double measure(final MetricConfig config, final long now) {
                         return 0;
                     }
                 },
                 null,
                 new MockTime());
         producer.setMockMetrics(testMetricName, testMetric);
-        Map<MetricName, Metric> producerMetrics = thread.producerMetrics();
+        final Map<MetricName, Metric> producerMetrics = thread.producerMetrics();
         assertEquals(testMetricName, producerMetrics.get(testMetricName).metricName());
     }
 
@@ -1317,7 +1317,7 @@ public class StreamThreadTest {
     public void adminClientMetricsVerification() {
         final Node broker1 = new Node(0, "dummyHost-1", 1234);
         final Node broker2 = new Node(1, "dummyHost-2", 1234);
-        List<Node> cluster = Arrays.asList(broker1, broker2);
+        final List<Node> cluster = Arrays.asList(broker1, broker2);
 
         final MockAdminClient adminClient = new MockAdminClient(cluster, broker1, null);
 
@@ -1346,7 +1346,7 @@ public class StreamThreadTest {
                 testMetricName,
                 new Measurable() {
                     @Override
-                    public double measure(MetricConfig config, long now) {
+                    public double measure(final MetricConfig config, final long now) {
                         return 0;
                     }
                 },
@@ -1359,7 +1359,7 @@ public class StreamThreadTest {
         EasyMock.replay(taskManager, consumer);
 
         adminClient.setMockMetrics(testMetricName, testMetric);
-        Map<MetricName, Metric> adminClientMetrics = thread.adminClientMetrics();
+        final Map<MetricName, Metric> adminClientMetrics = thread.adminClientMetrics();
         assertEquals(testMetricName, adminClientMetrics.get(testMetricName).metricName());
     }
 }
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java
index fce5342..0cf147c 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java
@@ -145,7 +145,7 @@ public class StreamsMetadataStateTest {
         final StreamsMetadata three = new StreamsMetadata(hostThree, Utils.mkSet(globalTable, "table-three"),
                 Collections.singleton(topic3P0));
 
-        Collection<StreamsMetadata> actual = discovery.getAllMetadata();
+        final Collection<StreamsMetadata> actual = discovery.getAllMetadata();
         assertEquals(3, actual.size());
         assertTrue("expected " + actual + " to contain " + one, actual.contains(one));
         assertTrue("expected " + actual + " to contain " + two, actual.contains(two));
@@ -223,7 +223,7 @@ public class StreamsMetadataStateTest {
         final StreamsMetadata expected = new StreamsMetadata(hostTwo, Utils.mkSet(globalTable, "table-two", "table-three", "merged-table"),
                 Utils.mkSet(topic2P0, tp4));
 
-        StreamsMetadata actual = discovery.getMetadataWithKey("table-three", "the-key", partitioner);
+        final StreamsMetadata actual = discovery.getMetadataWithKey("table-three", "the-key", partitioner);
         assertEquals(expected, actual);
     }
 
@@ -287,7 +287,7 @@ public class StreamsMetadataStateTest {
     public void shouldHaveGlobalStoreInAllMetadata() {
         final Collection<StreamsMetadata> metadata = discovery.getAllMetadataForStore(globalTable);
         assertEquals(3, metadata.size());
-        for (StreamsMetadata streamsMetadata : metadata) {
+        for (final StreamsMetadata streamsMetadata : metadata) {
             assertTrue(streamsMetadata.stateStoreNames().contains(globalTable));
         }
     }
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java
index e8ef7ea..508f2ee 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java
@@ -212,11 +212,11 @@ public class TaskManagerTest {
 
     @Test
     public void shouldReturnCachedTaskIdsFromDirectory() throws IOException {
-        File[] taskFolders = Utils.mkList(testFolder.newFolder("0_1"),
-                testFolder.newFolder("0_2"),
-                testFolder.newFolder("0_3"),
-                testFolder.newFolder("1_1"),
-                testFolder.newFolder("dummy")).toArray(new File[0]);
+        final File[] taskFolders = Utils.mkList(testFolder.newFolder("0_1"),
+                                                testFolder.newFolder("0_2"),
+                                                testFolder.newFolder("0_3"),
+                                                testFolder.newFolder("1_1"),
+                                                testFolder.newFolder("dummy")).toArray(new File[0]);
 
         assertTrue((new File(taskFolders[0], ProcessorStateManager.CHECKPOINT_FILE_NAME)).createNewFile());
         assertTrue((new File(taskFolders[1], ProcessorStateManager.CHECKPOINT_FILE_NAME)).createNewFile());
@@ -227,7 +227,7 @@ public class TaskManagerTest {
 
         EasyMock.replay(activeTaskCreator, stateDirectory);
 
-        Set<TaskId> tasks = taskManager.cachedTasksIds();
+        final Set<TaskId> tasks = taskManager.cachedTasksIds();
 
         EasyMock.verify(activeTaskCreator, stateDirectory);
 
@@ -375,7 +375,7 @@ public class TaskManagerTest {
         try {
             taskManager.suspendTasksAndState();
             fail("Should have thrown streams exception");
-        } catch (StreamsException e) {
+        } catch (final StreamsException e) {
             // expected
         }
         verify(restoreConsumer, active, standby);
@@ -518,7 +518,7 @@ public class TaskManagerTest {
         try {
             taskManager.commitAll();
             fail("should have thrown first exception");
-        } catch (Exception e) {
+        } catch (final Exception e) {
             // ok
         }
         verify(active, standby);
@@ -532,7 +532,7 @@ public class TaskManagerTest {
         try {
             taskManager.commitAll();
             fail("should have thrown exception");
-        } catch (Exception e) {
+        } catch (final Exception e) {
             // ok
         }
         verify(standby);
@@ -618,7 +618,7 @@ public class TaskManagerTest {
     @Test
     public void shouldNotResumeConsumptionUntilAllStoresRestored() {
         EasyMock.expect(active.allTasksRunning()).andReturn(false);
-        Consumer<byte[], byte[]> consumer = (Consumer<byte[], byte[]>) EasyMock.createStrictMock(Consumer.class);
+        final Consumer<byte[], byte[]> consumer = (Consumer<byte[], byte[]>) EasyMock.createStrictMock(Consumer.class);
         taskManager.setConsumer(consumer);
         EasyMock.replay(active, consumer);
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/WrappedBatchingStateRestoreCallbackTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/WrappedBatchingStateRestoreCallbackTest.java
index d04d1d5..c602ee1 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/WrappedBatchingStateRestoreCallbackTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/WrappedBatchingStateRestoreCallbackTest.java
@@ -42,7 +42,7 @@ public class WrappedBatchingStateRestoreCallbackTest {
     public void shouldRestoreSinglePutsFromArray() {
         wrappedBatchingStateRestoreCallback.restoreAll(records);
         assertThat(mockRestoreCallback.restored, is(records));
-        KeyValue<byte[], byte[]> record = mockRestoreCallback.restored.get(0);
+        final KeyValue<byte[], byte[]> record = mockRestoreCallback.restored.get(0);
         assertThat(record.key, is(key));
         assertThat(record.value, is(value));
     }
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java b/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java
index 05016bc..8b20b0b 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java
@@ -93,7 +93,7 @@ public class NoOpWindowStore implements ReadOnlyWindowStore, StateStore {
     }
 
     @Override
-    public WindowStoreIterator<KeyValue> fetch(Object from, Object to, long timeFrom, long timeTo) {
+    public WindowStoreIterator<KeyValue> fetch(final Object from, final Object to, final long timeFrom, final long timeTo) {
         return EMPTY_WINDOW_STORE_ITERATOR;
     }
     
@@ -103,7 +103,7 @@ public class NoOpWindowStore implements ReadOnlyWindowStore, StateStore {
     }
     
     @Override
-    public WindowStoreIterator<KeyValue> fetchAll(long timeFrom, long timeTo) {
+    public WindowStoreIterator<KeyValue> fetchAll(final long timeFrom, final long timeTo) {
         return EMPTY_WINDOW_STORE_ITERATOR;
     }
 }
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/StateSerdesTest.java b/streams/src/test/java/org/apache/kafka/streams/state/StateSerdesTest.java
index 714ce18..88b6997 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/StateSerdesTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/StateSerdesTest.java
@@ -89,17 +89,17 @@ public class StateSerdesTest {
 
     @Test(expected = StreamsException.class)
     public void shouldThrowIfIncompatibleSerdeForValue() throws ClassNotFoundException {
-        Class myClass = Class.forName("java.lang.String");
-        StateSerdes<Object, Object> stateSerdes = new StateSerdes<Object, Object>("anyName", Serdes.serdeFrom(myClass), Serdes.serdeFrom(myClass));
-        Integer myInt = 123;
+        final Class myClass = Class.forName("java.lang.String");
+        final StateSerdes<Object, Object> stateSerdes = new StateSerdes<Object, Object>("anyName", Serdes.serdeFrom(myClass), Serdes.serdeFrom(myClass));
+        final Integer myInt = 123;
         stateSerdes.rawValue(myInt);
     }
 
     @Test(expected = StreamsException.class)
     public void shouldThrowIfIncompatibleSerdeForKey() throws ClassNotFoundException {
-        Class myClass = Class.forName("java.lang.String");
-        StateSerdes<Object, Object> stateSerdes = new StateSerdes<Object, Object>("anyName", Serdes.serdeFrom(myClass), Serdes.serdeFrom(myClass));
-        Integer myInt = 123;
+        final Class myClass = Class.forName("java.lang.String");
+        final StateSerdes<Object, Object> stateSerdes = new StateSerdes<Object, Object>("anyName", Serdes.serdeFrom(myClass), Serdes.serdeFrom(myClass));
+        final Integer myInt = 123;
         stateSerdes.rawKey(myInt);
     }
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java
index c4536df..44ff3a2 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java
@@ -69,7 +69,7 @@ public abstract class AbstractKeyValueStoreTest {
     private static Map<Integer, String> getContents(final KeyValueIterator<Integer, String> iter) {
         final HashMap<Integer, String> result = new HashMap<>();
         while (iter.hasNext()) {
-            KeyValue<Integer, String> entry = iter.next();
+            final KeyValue<Integer, String> entry = iter.next();
             result.put(entry.key, entry.value);
         }
         return result;
@@ -349,7 +349,7 @@ public abstract class AbstractKeyValueStoreTest {
 
     @Test
     public void shouldPutAll() {
-        List<KeyValue<Integer, String>> entries = new ArrayList<>();
+        final List<KeyValue<Integer, String>> entries = new ArrayList<>();
         entries.add(new KeyValue<>(1, "one"));
         entries.add(new KeyValue<>(2, "two"));
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java
index 2f6aac7..ae6bded 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java
@@ -124,7 +124,7 @@ public class CachingKeyValueStoreTest extends AbstractKeyValueStoreTest {
 
     @Test
     public void shouldFlushEvictedItemsIntoUnderlyingStore() throws IOException {
-        int added = addItemsToCache();
+        final int added = addItemsToCache();
         // all dirty entries should have been flushed
         assertEquals(added, underlyingStore.approximateNumEntries());
         assertEquals(added, store.approximateNumEntries());
@@ -133,7 +133,7 @@ public class CachingKeyValueStoreTest extends AbstractKeyValueStoreTest {
 
     @Test
     public void shouldForwardDirtyItemToListenerWhenEvicted() throws IOException {
-        int numRecords = addItemsToCache();
+        final int numRecords = addItemsToCache();
         assertEquals(numRecords, cacheFlushListener.forwarded.size());
     }
 
@@ -168,7 +168,7 @@ public class CachingKeyValueStoreTest extends AbstractKeyValueStoreTest {
 
     @Test
     public void shouldIterateAllStoredItems() throws IOException {
-        int items = addItemsToCache();
+        final int items = addItemsToCache();
         final KeyValueIterator<Bytes, byte[]> all = store.all();
         final List<Bytes> results = new ArrayList<>();
         while (all.hasNext()) {
@@ -179,7 +179,7 @@ public class CachingKeyValueStoreTest extends AbstractKeyValueStoreTest {
 
     @Test
     public void shouldIterateOverRange() throws IOException {
-        int items = addItemsToCache();
+        final int items = addItemsToCache();
         final KeyValueIterator<Bytes, byte[]> range = store.range(bytesKey(String.valueOf(0)), bytesKey(String.valueOf(items)));
         final List<Bytes> results = new ArrayList<>();
         while (range.hasNext()) {
@@ -269,12 +269,12 @@ public class CachingKeyValueStoreTest extends AbstractKeyValueStoreTest {
 
     @Test
     public void shouldThrowNullPointerExceptionOnPutAllWithNullKey() {
-        List<KeyValue<Bytes, byte[]>> entries = new ArrayList<>();
+        final List<KeyValue<Bytes, byte[]>> entries = new ArrayList<>();
         entries.add(new KeyValue<Bytes, byte[]>(null, bytesValue("a")));
         try {
             store.putAll(entries);
             fail("Should have thrown NullPointerException while putAll null key");
-        } catch (NullPointerException e) { }
+        } catch (final NullPointerException e) { }
     }
 
     @Test
@@ -288,7 +288,7 @@ public class CachingKeyValueStoreTest extends AbstractKeyValueStoreTest {
 
     @Test
     public void shouldPutAll() {
-        List<KeyValue<Bytes, byte[]>> entries = new ArrayList<>();
+        final List<KeyValue<Bytes, byte[]>> entries = new ArrayList<>();
         entries.add(new KeyValue<>(bytesKey("a"), bytesValue("1")));
         entries.add(new KeyValue<>(bytesKey("b"), bytesValue("2")));
         store.putAll(entries);
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java
index 7f5a08e..117fd8f 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java
@@ -56,13 +56,13 @@ public class ChangeLoggingKeyValueBytesStoreTest {
         final NoOpRecordCollector collector = new NoOpRecordCollector() {
             @Override
             public <K, V> void send(final String topic,
-                                    K key,
-                                    V value,
-                                    Headers headers,
-                                    Integer partition,
-                                    Long timestamp,
-                                    Serializer<K> keySerializer,
-                                    Serializer<V> valueSerializer) {
+                                    final K key,
+                                    final V value,
+                                    final Headers headers,
+                                    final Integer partition,
+                                    final Long timestamp,
+                                    final Serializer<K> keySerializer,
+                                    final Serializer<V> valueSerializer) {
                 sent.put(key, value);
             }
         };
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStoreTest.java
index edcaa05..e8b1e55 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStoreTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStoreTest.java
@@ -48,13 +48,13 @@ public class ChangeLoggingSessionBytesStoreTest {
     private final NoOpRecordCollector collector = new NoOpRecordCollector() {
         @Override
         public <K, V> void send(final String topic,
-                                K key,
-                                V value,
-                                Headers headers,
-                                Integer partition,
-                                Long timestamp,
-                                Serializer<K> keySerializer,
-                                Serializer<V> valueSerializer) {
+                                final K key,
+                                final V value,
+                                final Headers headers,
+                                final Integer partition,
+                                final Long timestamp,
+                                final Serializer<K> keySerializer,
+                                final Serializer<V> valueSerializer) {
             sent.put(key, value);
         }
     };
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java
index e56887e..ec81b93 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java
@@ -46,13 +46,13 @@ public class ChangeLoggingWindowBytesStoreTest {
     private final NoOpRecordCollector collector = new NoOpRecordCollector() {
         @Override
         public <K, V> void send(final String topic,
-                                K key,
-                                V value,
-                                Headers headers,
-                                Integer partition,
-                                Long timestamp,
-                                Serializer<K> keySerializer,
-                                Serializer<V> valueSerializer) {
+                                final K key,
+                                final V value,
+                                final Headers headers,
+                                final Integer partition,
+                                final Long timestamp,
+                                final Serializer<K> keySerializer,
+                                final Serializer<V> valueSerializer) {
             sent.put(key, value);
         }
     };
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java
index 317ffc9..7ca0edb 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java
@@ -116,43 +116,43 @@ public class CompositeReadOnlyKeyValueStoreTest {
     @Test
     public void shouldThrowNoSuchElementExceptionWhileNext() {
         stubOneUnderlying.put("a", "1");
-        KeyValueIterator<String, String> keyValueIterator = theStore.range("a", "b");
+        final KeyValueIterator<String, String> keyValueIterator = theStore.range("a", "b");
         keyValueIterator.next();
         try {
             keyValueIterator.next();
             fail("Should have thrown NoSuchElementException with next()");
-        } catch (NoSuchElementException e) { }
+        } catch (final NoSuchElementException e) { }
     }
 
     @Test
     public void shouldThrowNoSuchElementExceptionWhilePeekNext() {
         stubOneUnderlying.put("a", "1");
-        KeyValueIterator<String, String> keyValueIterator = theStore.range("a", "b");
+        final KeyValueIterator<String, String> keyValueIterator = theStore.range("a", "b");
         keyValueIterator.next();
         try {
             keyValueIterator.peekNextKey();
             fail("Should have thrown NoSuchElementException with peekNextKey()");
-        } catch (NoSuchElementException e) { }
+        } catch (final NoSuchElementException e) { }
     }
 
     @Test
     public void shouldThrowUnsupportedOperationExceptionWhileRemove() {
-        KeyValueIterator<String, String> keyValueIterator = theStore.all();
+        final KeyValueIterator<String, String> keyValueIterator = theStore.all();
         try {
             keyValueIterator.remove();
             fail("Should have thrown UnsupportedOperationException");
-        } catch (UnsupportedOperationException e) { }
+        } catch (final UnsupportedOperationException e) { }
     }
 
     @Test
     public void shouldThrowUnsupportedOperationExceptionWhileRange() {
         stubOneUnderlying.put("a", "1");
         stubOneUnderlying.put("b", "1");
-        KeyValueIterator<String, String> keyValueIterator = theStore.range("a", "b");
+        final KeyValueIterator<String, String> keyValueIterator = theStore.range("a", "b");
         try {
             keyValueIterator.remove();
             fail("Should have thrown UnsupportedOperationException");
-        } catch (UnsupportedOperationException e) { }
+        } catch (final UnsupportedOperationException e) { }
     }
 
     @Test
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java
index cfb65ee..00b440f 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java
@@ -121,7 +121,7 @@ public class CompositeReadOnlySessionStoreTest {
         try {
             sessionStore.fetch("key");
             fail("Should have thrown InvalidStateStoreException with session store");
-        } catch (InvalidStateStoreException e) { }
+        } catch (final InvalidStateStoreException e) { }
     }
 
     @Test(expected = NullPointerException.class)
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java
index a241510..66f318a 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java
@@ -132,7 +132,7 @@ public class CompositeReadOnlyWindowStoreTest {
         try {
             store.fetch("key", 1, 10);
             Assert.fail("InvalidStateStoreException was expected");
-        } catch (InvalidStateStoreException e) {
+        } catch (final InvalidStateStoreException e) {
             Assert.assertEquals("State store is not available anymore and may have been migrated to another instance; " +
                     "please re-discover its location from the state metadata.", e.getMessage());
         }
@@ -173,7 +173,7 @@ public class CompositeReadOnlyWindowStoreTest {
         stubProviderTwo.addStore(storeName, secondUnderlying);
         underlyingWindowStore.put("a", "a", 0L);
         secondUnderlying.put("b", "b", 10L);
-        List<KeyValue<Windowed<String>, String>> results = StreamsTestUtils.toList(windowStore.fetch("a", "b", 0, 10));
+        final List<KeyValue<Windowed<String>, String>> results = StreamsTestUtils.toList(windowStore.fetch("a", "b", 0, 10));
         assertThat(results, equalTo(Arrays.asList(
                 KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"),
                 KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"))));
@@ -199,7 +199,7 @@ public class CompositeReadOnlyWindowStoreTest {
         stubProviderTwo.addStore(storeName, secondUnderlying);
         underlyingWindowStore.put("a", "a", 0L);
         secondUnderlying.put("b", "b", 10L);
-        List<KeyValue<Windowed<String>, String>> results = StreamsTestUtils.toList(windowStore.all());
+        final List<KeyValue<Windowed<String>, String>> results = StreamsTestUtils.toList(windowStore.all());
         assertThat(results, equalTo(Arrays.asList(
                 KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"),
                 KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"))));
@@ -212,7 +212,7 @@ public class CompositeReadOnlyWindowStoreTest {
         stubProviderTwo.addStore(storeName, secondUnderlying);
         underlyingWindowStore.put("a", "a", 0L);
         secondUnderlying.put("b", "b", 10L);
-        List<KeyValue<Windowed<String>, String>> results = StreamsTestUtils.toList(windowStore.fetchAll(0, 10));
+        final List<KeyValue<Windowed<String>, String>> results = StreamsTestUtils.toList(windowStore.fetchAll(0, 10));
         assertThat(results, equalTo(Arrays.asList(
                 KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"),
                 KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"))));
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java
index 3c180aa..8b6fc95 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java
@@ -59,7 +59,7 @@ public class DelegatingPeekingKeyValueIteratorTest {
     @Test
     public void shouldPeekAndIterate() {
         final String[] kvs = {"a", "b", "c", "d", "e", "f"};
-        for (String kv : kvs) {
+        for (final String kv : kvs) {
             store.put(kv, kv);
         }
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/FilteredCacheIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/FilteredCacheIteratorTest.java
index 2cb9502..08b6b23 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/FilteredCacheIteratorTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/FilteredCacheIteratorTest.java
@@ -36,12 +36,12 @@ public class FilteredCacheIteratorTest {
 
     private static final CacheFunction IDENTITY_FUNCTION = new CacheFunction() {
         @Override
-        public Bytes key(Bytes cacheKey) {
+        public Bytes key(final Bytes cacheKey) {
             return cacheKey;
         }
 
         @Override
-        public Bytes cacheKey(Bytes key) {
+        public Bytes cacheKey(final Bytes key) {
             return key;
         }
     };
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreTest.java
index 2d39ae7..91eda9f 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreTest.java
@@ -61,7 +61,7 @@ public class InMemoryLRUCacheStoreTest extends AbstractKeyValueStoreTest {
 
         assertThat(store.approximateNumEntries(), equalTo(3L));
 
-        for (KeyValue<Integer, String> kvPair : kvPairs) {
+        for (final KeyValue<Integer, String> kvPair : kvPairs) {
             assertThat(store.get(kvPair.key), equalTo(kvPair.value));
         }
     }
@@ -83,7 +83,7 @@ public class InMemoryLRUCacheStoreTest extends AbstractKeyValueStoreTest {
 
         assertThat(store.approximateNumEntries(), equalTo(3L));
         
-        for (KeyValue<Integer, String> kvPair : updatedKvPairs) {
+        for (final KeyValue<Integer, String> kvPair : updatedKvPairs) {
             assertThat(store.get(kvPair.key), equalTo(kvPair.value));
         }
     }
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueBytesStoreIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueBytesStoreIteratorTest.java
index edc0b94..d7f164c 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueBytesStoreIteratorTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueBytesStoreIteratorTest.java
@@ -57,7 +57,7 @@ public class MergedSortedCacheKeyValueBytesStoreIteratorTest {
         final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.range(namespace, from, to);
 
         final MergedSortedCacheKeyValueBytesStoreIterator iterator = new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, storeIterator);
-        byte[][] values = new byte[8][];
+        final byte[][] values = new byte[8][];
         int index = 0;
         int bytesIndex = 2;
         while (iterator.hasNext()) {
@@ -101,8 +101,8 @@ public class MergedSortedCacheKeyValueBytesStoreIteratorTest {
     @Test
     public void shouldNotHaveNextIfAllCachedItemsDeleted() throws Exception {
         final byte[][] bytes = {{0}, {1}, {2}};
-        for (byte[] aByte : bytes) {
-            Bytes aBytes = Bytes.wrap(aByte);
+        for (final byte[] aByte : bytes) {
+            final Bytes aBytes = Bytes.wrap(aByte);
             store.put(aBytes, aByte);
             cache.put(namespace, aBytes, new LRUCacheEntry(null));
         }
@@ -112,7 +112,7 @@ public class MergedSortedCacheKeyValueBytesStoreIteratorTest {
     @Test
     public void shouldNotHaveNextIfOnlyCacheItemsAndAllDeleted() throws Exception {
         final byte[][] bytes = {{0}, {1}, {2}};
-        for (byte[] aByte : bytes) {
+        for (final byte[] aByte : bytes) {
             cache.put(namespace, Bytes.wrap(aByte), new LRUCacheEntry(null));
         }
         assertFalse(createIterator().hasNext());
@@ -121,8 +121,8 @@ public class MergedSortedCacheKeyValueBytesStoreIteratorTest {
     @Test
     public void shouldSkipAllDeletedFromCache() throws Exception {
         final byte[][] bytes = {{0}, {1}, {2}, {3}, {4}, {5}, {6}, {7}, {8}, {9}, {10}, {11}};
-        for (byte[] aByte : bytes) {
-            Bytes aBytes = Bytes.wrap(aByte);
+        for (final byte[] aByte : bytes) {
+            final Bytes aBytes = Bytes.wrap(aByte);
             store.put(aBytes, aByte);
             cache.put(namespace, aBytes, new LRUCacheEntry(aByte));
         }
@@ -148,7 +148,7 @@ public class MergedSortedCacheKeyValueBytesStoreIteratorTest {
     public void shouldPeekNextKey() throws Exception {
         final KeyValueStore<Bytes, byte[]> kv = new InMemoryKeyValueStore<>("one", Serdes.Bytes(), Serdes.ByteArray());
         final ThreadCache cache = new ThreadCache(new LogContext("testCache "), 1000000L, new MockStreamsMetrics(new Metrics()));
-        byte[][] bytes = {{0}, {1}, {2}, {3}, {4}, {5}, {6}, {7}, {8}, {9}, {10}};
+        final byte[][] bytes = {{0}, {1}, {2}, {3}, {4}, {5}, {6}, {7}, {8}, {9}, {10}};
         for (int i = 0; i < bytes.length - 1; i += 2) {
             kv.put(Bytes.wrap(bytes[i]), bytes[i]);
             cache.put(namespace, Bytes.wrap(bytes[i + 1]), new LRUCacheEntry(bytes[i + 1]));
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedSessionStoreIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedSessionStoreIteratorTest.java
index ea31a04..26438bf 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedSessionStoreIteratorTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedSessionStoreIteratorTest.java
@@ -35,7 +35,7 @@ public class MergedSortedCacheWrappedSessionStoreIteratorTest {
 
     private static final SegmentedCacheFunction SINGLE_SEGMENT_CACHE_FUNCTION = new SegmentedCacheFunction(null, -1) {
         @Override
-        public long segmentId(Bytes key) {
+        public long segmentId(final Bytes key) {
             return 0;
         }
     };
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreIteratorTest.java
index 9135136..77d97ba 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreIteratorTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreIteratorTest.java
@@ -39,7 +39,7 @@ public class MergedSortedCacheWrappedWindowStoreIteratorTest {
 
     private static final SegmentedCacheFunction SINGLE_SEGMENT_CACHE_FUNCTION = new SegmentedCacheFunction(null, -1) {
         @Override
-        public long segmentId(Bytes key) {
+        public long segmentId(final Bytes key) {
             return 0;
         }
     };
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreKeyValueIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreKeyValueIteratorTest.java
index 1e5f62c..1e926e4 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreKeyValueIteratorTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreKeyValueIteratorTest.java
@@ -38,7 +38,7 @@ import static org.junit.Assert.assertTrue;
 public class MergedSortedCacheWrappedWindowStoreKeyValueIteratorTest {
     private static final SegmentedCacheFunction SINGLE_SEGMENT_CACHE_FUNCTION = new SegmentedCacheFunction(null, -1) {
         @Override
-        public long segmentId(Bytes key) {
+        public long segmentId(final Bytes key) {
             return 0;
         }
     };
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/NamedCacheTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/NamedCacheTest.java
index 3f78be6..0fdbdf7 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/NamedCacheTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/NamedCacheTest.java
@@ -60,18 +60,18 @@ public class NamedCacheTest {
 
     @Test
     public void shouldKeepTrackOfMostRecentlyAndLeastRecentlyUsed() throws IOException {
-        List<KeyValue<String, String>> toInsert = Arrays.asList(
+        final List<KeyValue<String, String>> toInsert = Arrays.asList(
                 new KeyValue<>("K1", "V1"),
                 new KeyValue<>("K2", "V2"),
                 new KeyValue<>("K3", "V3"),
                 new KeyValue<>("K4", "V4"),
                 new KeyValue<>("K5", "V5"));
         for (int i = 0; i < toInsert.size(); i++) {
-            byte[] key = toInsert.get(i).key.getBytes();
-            byte[] value = toInsert.get(i).value.getBytes();
+            final byte[] key = toInsert.get(i).key.getBytes();
+            final byte[] value = toInsert.get(i).value.getBytes();
             cache.put(Bytes.wrap(key), new LRUCacheEntry(value, null, true, 1, 1, 1, ""));
-            LRUCacheEntry head = cache.first();
-            LRUCacheEntry tail = cache.last();
+            final LRUCacheEntry head = cache.first();
+            final LRUCacheEntry tail = cache.last();
             assertEquals(new String(head.value()), toInsert.get(i).value);
             assertEquals(new String(tail.value()), toInsert.get(0).value);
             assertEquals(cache.flushes(), 0);
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/OffsetCheckpointTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/OffsetCheckpointTest.java
index 54cd3df..cb2e549 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/OffsetCheckpointTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/OffsetCheckpointTest.java
@@ -39,7 +39,7 @@ public class OffsetCheckpointTest {
         final OffsetCheckpoint checkpoint = new OffsetCheckpoint(f);
 
         try {
-            Map<TopicPartition, Long> offsets = new HashMap<>();
+            final Map<TopicPartition, Long> offsets = new HashMap<>();
             offsets.put(new TopicPartition(topic, 0), 0L);
             offsets.put(new TopicPartition(topic, 1), 1L);
             offsets.put(new TopicPartition(topic, 2), 2L);
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java
index 6d911a3..4818d09 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java
@@ -44,7 +44,7 @@ public class ReadOnlyWindowStoreStub<K, V> implements ReadOnlyWindowStore<K, V>,
     private final Map<Long, NavigableMap<K, V>> data = new HashMap<>();
     private boolean open  = true;
 
-    public ReadOnlyWindowStoreStub(long windowSize) {
+    public ReadOnlyWindowStoreStub(final long windowSize) {
         this.windowSize = windowSize;
     }
 
@@ -79,10 +79,10 @@ public class ReadOnlyWindowStoreStub<K, V> implements ReadOnlyWindowStore<K, V>,
             throw new InvalidStateStoreException("Store is not open");
         }
         final List<KeyValue<Windowed<K>, V>> results = new ArrayList<>();
-        for (long now : data.keySet()) {
+        for (final long now : data.keySet()) {
             final NavigableMap<K, V> kvMap = data.get(now);
             if (kvMap != null) {
-                for (Entry<K, V> entry : kvMap.entrySet()) {
+                for (final Entry<K, V> entry : kvMap.entrySet()) {
                     results.add(new KeyValue<>(new Windowed<>(entry.getKey(), new TimeWindow(now, now + windowSize)), entry.getValue()));
                 }
             }
@@ -119,16 +119,16 @@ public class ReadOnlyWindowStoreStub<K, V> implements ReadOnlyWindowStore<K, V>,
     }
     
     @Override
-    public KeyValueIterator<Windowed<K>, V> fetchAll(long timeFrom, long timeTo) {
+    public KeyValueIterator<Windowed<K>, V> fetchAll(final long timeFrom, final long timeTo) {
         if (!open) {
             throw new InvalidStateStoreException("Store is not open");
         }
         final List<KeyValue<Windowed<K>, V>> results = new ArrayList<>();
-        for (long now : data.keySet()) {
+        for (final long now : data.keySet()) {
             if (!(now >= timeFrom && now <= timeTo)) continue;
             final NavigableMap<K, V> kvMap = data.get(now);
             if (kvMap != null) {
-                for (Entry<K, V> entry : kvMap.entrySet()) {
+                for (final Entry<K, V> entry : kvMap.entrySet()) {
                     results.add(new KeyValue<>(new Windowed<>(entry.getKey(), new TimeWindow(now, now + windowSize)), entry.getValue()));
                 }
             }
@@ -165,7 +165,7 @@ public class ReadOnlyWindowStoreStub<K, V> implements ReadOnlyWindowStore<K, V>,
     }
 
     @Override
-    public KeyValueIterator<Windowed<K>, V> fetch(K from, K to, long timeFrom, long timeTo) {
+    public KeyValueIterator<Windowed<K>, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) {
         if (!open) {
             throw new InvalidStateStoreException("Store is not open");
         }
@@ -173,7 +173,7 @@ public class ReadOnlyWindowStoreStub<K, V> implements ReadOnlyWindowStore<K, V>,
         for (long now = timeFrom; now <= timeTo; now++) {
             final NavigableMap<K, V> kvMap = data.get(now);
             if (kvMap != null) {
-                for (Entry<K, V> entry : kvMap.subMap(from, true, to, true).entrySet()) {
+                for (final Entry<K, V> entry : kvMap.subMap(from, true, to, true).entrySet()) {
                     results.add(new KeyValue<>(new Windowed<>(entry.getKey(), new TimeWindow(now, now + windowSize)), entry.getValue()));
                 }
             }
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreTest.java
index b44d369..cf4b90d 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreTest.java
@@ -96,7 +96,7 @@ public class RocksDBSessionStoreTest {
                                                                                     KeyValue.pair(new Windowed<>("a", new SessionWindow(10, 10)), 2L),
                                                                                     KeyValue.pair(new Windowed<>("a", new SessionWindow(100, 100)), 3L),
                                                                                     KeyValue.pair(new Windowed<>("a", new SessionWindow(1000, 1000)), 4L));
-        for (KeyValue<Windowed<String>, Long> kv : expected) {
+        for (final KeyValue<Windowed<String>, Long> kv : expected) {
             sessionStore.put(kv.key, kv.value);
         }
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java
index 63d877a..113e3e1 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java
@@ -82,7 +82,7 @@ public class RocksDBStoreTest {
     public void shouldRespectBulkloadOptionsDuringInit() {
         rocksDBStore.init(context, rocksDBStore);
 
-        StateRestoreListener restoreListener = context.getRestoreListener(rocksDBStore.name());
+        final StateRestoreListener restoreListener = context.getRestoreListener(rocksDBStore.name());
 
         restoreListener.onRestoreStart(null, rocksDBStore.name(), 0L, 0L);
 
@@ -131,21 +131,21 @@ public class RocksDBStoreTest {
     @Test
     public void shouldThrowProcessorStateExceptionOnOpeningReadOnlyDir() {
         final File tmpDir = TestUtils.tempDirectory();
-        InternalMockProcessorContext tmpContext = new InternalMockProcessorContext(tmpDir, new StreamsConfig(StreamsTestUtils.minimalStreamsConfig()));
+        final InternalMockProcessorContext tmpContext = new InternalMockProcessorContext(tmpDir, new StreamsConfig(StreamsTestUtils.minimalStreamsConfig()));
 
         assertTrue(tmpDir.setReadOnly());
 
         try {
             rocksDBStore.openDB(tmpContext);
             fail("Should have thrown ProcessorStateException");
-        } catch (ProcessorStateException e) {
+        } catch (final ProcessorStateException e) {
             // this is good, do nothing
         }
     }
 
     @Test
     public void shouldPutAll() {
-        List<KeyValue<Bytes, byte[]>> entries = new ArrayList<>();
+        final List<KeyValue<Bytes, byte[]>> entries = new ArrayList<>();
         entries.add(new KeyValue<>(
             new Bytes(stringSerializer.serialize(null, "1")),
             stringSerializer.serialize(null, "a")));
@@ -180,7 +180,7 @@ public class RocksDBStoreTest {
     @Test
     public void shouldTogglePrepareForBulkloadSetting() {
         rocksDBStore.init(context, rocksDBStore);
-        RocksDBStore.RocksDBBatchingRestoreCallback restoreListener =
+        final RocksDBStore.RocksDBBatchingRestoreCallback restoreListener =
             (RocksDBStore.RocksDBBatchingRestoreCallback) rocksDBStore.batchingStateRestoreCallback;
 
         restoreListener.onRestoreStart(null, null, 0, 0);
@@ -197,7 +197,7 @@ public class RocksDBStoreTest {
         rocksDBStore.init(context, rocksDBStore);
         context.restore(rocksDBStore.name(), entries);
 
-        RocksDBStore.RocksDBBatchingRestoreCallback restoreListener =
+        final RocksDBStore.RocksDBBatchingRestoreCallback restoreListener =
             (RocksDBStore.RocksDBBatchingRestoreCallback) rocksDBStore.batchingStateRestoreCallback;
 
         restoreListener.onRestoreStart(null, null, 0, 0);
@@ -353,7 +353,7 @@ public class RocksDBStoreTest {
         try {
             rocksDBStore.put(null, stringSerializer.serialize(null, "someVal"));
             fail("Should have thrown NullPointerException on null put()");
-        } catch (NullPointerException e) {
+        } catch (final NullPointerException e) {
             // this is good
         }
     }
@@ -364,7 +364,7 @@ public class RocksDBStoreTest {
         try {
             rocksDBStore.put(null, stringSerializer.serialize(null, "someVal"));
             fail("Should have thrown NullPointerException on null put()");
-        } catch (NullPointerException e) {
+        } catch (final NullPointerException e) {
             // this is good
         }
     }
@@ -375,7 +375,7 @@ public class RocksDBStoreTest {
         try {
             rocksDBStore.get(null);
             fail("Should have thrown NullPointerException on null get()");
-        } catch (NullPointerException e) {
+        } catch (final NullPointerException e) {
             // this is good
         }
     }
@@ -386,7 +386,7 @@ public class RocksDBStoreTest {
         try {
             rocksDBStore.delete(null);
             fail("Should have thrown NullPointerException on deleting null key");
-        } catch (NullPointerException e) {
+        } catch (final NullPointerException e) {
             // this is good
         }
     }
@@ -397,7 +397,7 @@ public class RocksDBStoreTest {
         try {
             rocksDBStore.range(null, new Bytes(stringSerializer.serialize(null, "2")));
             fail("Should have thrown NullPointerException on deleting null key");
-        } catch (NullPointerException e) {
+        } catch (final NullPointerException e) {
             // this is good
         }
     }
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentedCacheFunctionTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentedCacheFunctionTest.java
index 29cdf11..1f6a747 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentedCacheFunctionTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentedCacheFunctionTest.java
@@ -58,7 +58,7 @@ public class SegmentedCacheFunctionTest {
 
         assertThat(buffer.getLong(), equalTo(segmentId));
 
-        byte[] actualKey = new byte[buffer.remaining()];
+        final byte[] actualKey = new byte[buffer.remaining()];
         buffer.get(actualKey);
         assertThat(Bytes.wrap(actualKey), equalTo(THE_KEY));
     }
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/SessionKeySchemaTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/SessionKeySchemaTest.java
index 6852721..6b51687 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/SessionKeySchemaTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/SessionKeySchemaTest.java
@@ -90,7 +90,7 @@ public class SessionKeySchemaTest {
     
     @Test
     public void testUpperBoundWithLargeTimestamps() {
-        Bytes upper = sessionKeySchema.upperRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), Long.MAX_VALUE);
+        final Bytes upper = sessionKeySchema.upperRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), Long.MAX_VALUE);
 
         assertThat(
             "shorter key with max timestamp should be in range",
@@ -121,7 +121,7 @@ public class SessionKeySchemaTest {
 
     @Test
     public void testUpperBoundWithKeyBytesLargerThanFirstTimestampByte() {
-        Bytes upper = sessionKeySchema.upperRange(Bytes.wrap(new byte[]{0xA, (byte) 0x8F, (byte) 0x9F}), Long.MAX_VALUE);
+        final Bytes upper = sessionKeySchema.upperRange(Bytes.wrap(new byte[]{0xA, (byte) 0x8F, (byte) 0x9F}), Long.MAX_VALUE);
 
         assertThat(
             "shorter key with max timestamp should be in range",
@@ -141,7 +141,7 @@ public class SessionKeySchemaTest {
 
     @Test
     public void testUpperBoundWithZeroTimestamp() {
-        Bytes upper = sessionKeySchema.upperRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), 0);
+        final Bytes upper = sessionKeySchema.upperRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), 0);
 
         assertThat(upper, equalTo(Bytes.wrap(SessionKeySchema.toBinary(
             new Windowed<>(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), new SessionWindow(0, 0))))
@@ -150,13 +150,13 @@ public class SessionKeySchemaTest {
 
     @Test
     public void testLowerBoundWithZeroTimestamp() {
-        Bytes lower = sessionKeySchema.lowerRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), 0);
+        final Bytes lower = sessionKeySchema.lowerRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), 0);
         assertThat(lower, equalTo(Bytes.wrap(SessionKeySchema.toBinary(new Windowed<>(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), new SessionWindow(0, 0))))));
     }
 
     @Test
     public void testLowerBoundMatchesTrailingZeros() {
-        Bytes lower = sessionKeySchema.lowerRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), Long.MAX_VALUE);
+        final Bytes lower = sessionKeySchema.lowerRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), Long.MAX_VALUE);
 
         assertThat(
             "appending zeros to key should still be in range",
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java
index 3fa9367..4f1e181 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java
@@ -25,8 +25,8 @@ import org.apache.kafka.streams.processor.internals.MockStreamsMetrics;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.util.Arrays;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.NoSuchElementException;
@@ -45,26 +45,26 @@ public class ThreadCacheTest {
 
     @Test
     public void basicPutGet() throws IOException {
-        List<KeyValue<String, String>> toInsert = Arrays.asList(
+        final List<KeyValue<String, String>> toInsert = Arrays.asList(
                 new KeyValue<>("K1", "V1"),
                 new KeyValue<>("K2", "V2"),
                 new KeyValue<>("K3", "V3"),
                 new KeyValue<>("K4", "V4"),
                 new KeyValue<>("K5", "V5"));
         final KeyValue<String, String> kv = toInsert.get(0);
-        ThreadCache cache = new ThreadCache(logContext,
-                toInsert.size() * memoryCacheEntrySize(kv.key.getBytes(), kv.value.getBytes(), ""),
-            new MockStreamsMetrics(new Metrics()));
+        final ThreadCache cache = new ThreadCache(logContext,
+                                                  toInsert.size() * memoryCacheEntrySize(kv.key.getBytes(), kv.value.getBytes(), ""),
+                                                  new MockStreamsMetrics(new Metrics()));
 
-        for (KeyValue<String, String> kvToInsert : toInsert) {
-            Bytes key = Bytes.wrap(kvToInsert.key.getBytes());
-            byte[] value = kvToInsert.value.getBytes();
+        for (final KeyValue<String, String> kvToInsert : toInsert) {
+            final Bytes key = Bytes.wrap(kvToInsert.key.getBytes());
+            final byte[] value = kvToInsert.value.getBytes();
             cache.put(namespace, key, new LRUCacheEntry(value, null, true, 1L, 1L, 1, ""));
         }
 
-        for (KeyValue<String, String> kvToInsert : toInsert) {
-            Bytes key = Bytes.wrap(kvToInsert.key.getBytes());
-            LRUCacheEntry entry = cache.get(namespace, key);
+        for (final KeyValue<String, String> kvToInsert : toInsert) {
+            final Bytes key = Bytes.wrap(kvToInsert.key.getBytes());
+            final LRUCacheEntry entry = cache.get(namespace, key);
             assertEquals(entry.isDirty(), true);
             assertEquals(new String(entry.value()), kvToInsert.value);
         }
@@ -74,28 +74,31 @@ public class ThreadCacheTest {
         assertEquals(cache.flushes(), 0);
     }
 
-    private void checkOverheads(double entryFactor, double systemFactor, long desiredCacheSize, int keySizeBytes,
-                            int valueSizeBytes) {
-        Runtime runtime = Runtime.getRuntime();
-        long numElements = desiredCacheSize / memoryCacheEntrySize(new byte[keySizeBytes], new byte[valueSizeBytes], "");
+    private void checkOverheads(final double entryFactor,
+                                final double systemFactor,
+                                final long desiredCacheSize,
+                                final int keySizeBytes,
+                                final int valueSizeBytes) {
+        final Runtime runtime = Runtime.getRuntime();
+        final long numElements = desiredCacheSize / memoryCacheEntrySize(new byte[keySizeBytes], new byte[valueSizeBytes], "");
 
         System.gc();
-        long prevRuntimeMemory = runtime.totalMemory() - runtime.freeMemory();
+        final long prevRuntimeMemory = runtime.totalMemory() - runtime.freeMemory();
 
-        ThreadCache cache = new ThreadCache(logContext, desiredCacheSize, new MockStreamsMetrics(new Metrics()));
-        long size = cache.sizeBytes();
+        final ThreadCache cache = new ThreadCache(logContext, desiredCacheSize, new MockStreamsMetrics(new Metrics()));
+        final long size = cache.sizeBytes();
         assertEquals(size, 0);
         for (int i = 0; i < numElements; i++) {
-            String keyStr = "K" + i;
-            Bytes key = Bytes.wrap(keyStr.getBytes());
-            byte[] value = new byte[valueSizeBytes];
+            final String keyStr = "K" + i;
+            final Bytes key = Bytes.wrap(keyStr.getBytes());
+            final byte[] value = new byte[valueSizeBytes];
             cache.put(namespace, key, new LRUCacheEntry(value, null, true, 1L, 1L, 1, ""));
         }
 
 
         System.gc();
-        double ceiling = desiredCacheSize + desiredCacheSize * entryFactor;
-        long usedRuntimeMemory = runtime.totalMemory() - runtime.freeMemory() - prevRuntimeMemory;
+        final double ceiling = desiredCacheSize + desiredCacheSize * entryFactor;
+        final long usedRuntimeMemory = runtime.totalMemory() - runtime.freeMemory() - prevRuntimeMemory;
         assertTrue((double) cache.sizeBytes() <= ceiling);
 
         assertTrue("Used memory size " + usedRuntimeMemory + " greater than expected " + cache.sizeBytes() * systemFactor,
@@ -104,30 +107,30 @@ public class ThreadCacheTest {
 
     @Test
     public void cacheOverheadsSmallValues() {
-        Runtime runtime = Runtime.getRuntime();
-        double factor = 0.05;
-        double systemFactor = 3; // if I ask for a cache size of 10 MB, accept an overhead of 3x, i.e., 30 MBs might be allocated
-        long desiredCacheSize = Math.min(100 * 1024 * 1024L, runtime.maxMemory());
-        int keySizeBytes = 8;
-        int valueSizeBytes = 100;
+        final Runtime runtime = Runtime.getRuntime();
+        final double factor = 0.05;
+        final double systemFactor = 3; // if I ask for a cache size of 10 MB, accept an overhead of 3x, i.e., 30 MBs might be allocated
+        final long desiredCacheSize = Math.min(100 * 1024 * 1024L, runtime.maxMemory());
+        final int keySizeBytes = 8;
+        final int valueSizeBytes = 100;
 
         checkOverheads(factor, systemFactor, desiredCacheSize, keySizeBytes, valueSizeBytes);
     }
 
     @Test
     public void cacheOverheadsLargeValues() {
-        Runtime runtime = Runtime.getRuntime();
-        double factor = 0.05;
-        double systemFactor = 2; // if I ask for a cache size of 10 MB, accept an overhead of 2x, i.e., 20 MBs might be allocated
-        long desiredCacheSize = Math.min(100 * 1024 * 1024L, runtime.maxMemory());
-        int keySizeBytes = 8;
-        int valueSizeBytes = 1000;
+        final Runtime runtime = Runtime.getRuntime();
+        final double factor = 0.05;
+        final double systemFactor = 2; // if I ask for a cache size of 10 MB, accept an overhead of 2x, i.e., 20 MBs might be allocated
+        final long desiredCacheSize = Math.min(100 * 1024 * 1024L, runtime.maxMemory());
+        final int keySizeBytes = 8;
+        final int valueSizeBytes = 1000;
 
         checkOverheads(factor, systemFactor, desiredCacheSize, keySizeBytes, valueSizeBytes);
     }
 
 
-    static int memoryCacheEntrySize(byte[] key, byte[] value, final String topic) {
+    static int memoryCacheEntrySize(final byte[] key, final byte[] value, final String topic) {
         return key.length +
                 value.length +
                 1 + // isDirty
@@ -145,38 +148,38 @@ public class ThreadCacheTest {
     @Test
     public void evict() throws IOException {
         final List<KeyValue<String, String>> received = new ArrayList<>();
-        List<KeyValue<String, String>> expected = Collections.singletonList(
+        final List<KeyValue<String, String>> expected = Collections.singletonList(
                 new KeyValue<>("K1", "V1"));
 
-        List<KeyValue<String, String>> toInsert = Arrays.asList(
+        final List<KeyValue<String, String>> toInsert = Arrays.asList(
                 new KeyValue<>("K1", "V1"),
                 new KeyValue<>("K2", "V2"),
                 new KeyValue<>("K3", "V3"),
                 new KeyValue<>("K4", "V4"),
                 new KeyValue<>("K5", "V5"));
         final KeyValue<String, String> kv = toInsert.get(0);
-        ThreadCache cache = new ThreadCache(logContext,
-                memoryCacheEntrySize(kv.key.getBytes(), kv.value.getBytes(), ""),
-            new MockStreamsMetrics(new Metrics()));
+        final ThreadCache cache = new ThreadCache(logContext,
+                                                  memoryCacheEntrySize(kv.key.getBytes(), kv.value.getBytes(), ""),
+                                                  new MockStreamsMetrics(new Metrics()));
         cache.addDirtyEntryFlushListener(namespace, new ThreadCache.DirtyEntryFlushListener() {
             @Override
             public void apply(final List<ThreadCache.DirtyEntry> dirty) {
-                for (ThreadCache.DirtyEntry dirtyEntry : dirty) {
+                for (final ThreadCache.DirtyEntry dirtyEntry : dirty) {
                     received.add(new KeyValue<>(dirtyEntry.key().toString(), new String(dirtyEntry.newValue())));
                 }
             }
 
         });
 
-        for (KeyValue<String, String> kvToInsert : toInsert) {
+        for (final KeyValue<String, String> kvToInsert : toInsert) {
             final Bytes key = Bytes.wrap(kvToInsert.key.getBytes());
             final byte[] value = kvToInsert.value.getBytes();
             cache.put(namespace, key, new LRUCacheEntry(value, null, true, 1, 1, 1, ""));
         }
 
         for (int i = 0; i < expected.size(); i++) {
-            KeyValue<String, String> expectedRecord = expected.get(i);
-            KeyValue<String, String> actualRecord = received.get(i);
+            final KeyValue<String, String> expectedRecord = expected.get(i);
+            final KeyValue<String, String> actualRecord = received.get(i);
             assertEquals(expectedRecord, actualRecord);
         }
         assertEquals(cache.evicts(), 4);
@@ -282,7 +285,7 @@ public class ThreadCacheTest {
         final ThreadCache.MemoryLRUCacheBytesIterator iterator = cache.range(namespace, Bytes.wrap(new byte[]{1}), Bytes.wrap(new byte[]{4}));
         int bytesIndex = 1;
         while (iterator.hasNext()) {
-            Bytes peekedKey = iterator.peekNextKey();
+            final Bytes peekedKey = iterator.peekNextKey();
             final KeyValue<Bytes, LRUCacheEntry> next = iterator.next();
             assertArrayEquals(bytes[bytesIndex], peekedKey.get());
             assertArrayEquals(bytes[bytesIndex], next.key.get());
@@ -301,7 +304,7 @@ public class ThreadCacheTest {
 
             }
         });
-        byte[][] bytes = {{0}, {1}, {2}, {3}, {4}, {5}, {6}, {7}, {8}, {9}};
+        final byte[][] bytes = {{0}, {1}, {2}, {3}, {4}, {5}, {6}, {7}, {8}, {9}};
         for (int i = 0; i < 5; i++) {
             cache.put(namespace, Bytes.wrap(bytes[i]), dirtyEntry(bytes[i]));
         }
@@ -321,13 +324,13 @@ public class ThreadCacheTest {
         cache.addDirtyEntryFlushListener(namespace1, new ThreadCache.DirtyEntryFlushListener() {
             @Override
             public void apply(final List<ThreadCache.DirtyEntry> dirty) {
-                for (ThreadCache.DirtyEntry dirtyEntry : dirty) {
+                for (final ThreadCache.DirtyEntry dirtyEntry : dirty) {
                     received.add(dirtyEntry.key().get());
                 }
             }
         });
         final List<byte[]> expected = Arrays.asList(new byte[]{0}, new byte[]{1}, new byte[]{2});
-        for (byte[] bytes : expected) {
+        for (final byte[] bytes : expected) {
             cache.put(namespace1, Bytes.wrap(bytes), dirtyEntry(bytes));
         }
         cache.put(namespace2, Bytes.wrap(new byte[]{4}), dirtyEntry(new byte[]{4}));
@@ -343,13 +346,13 @@ public class ThreadCacheTest {
         cache.addDirtyEntryFlushListener(namespace1, new ThreadCache.DirtyEntryFlushListener() {
             @Override
             public void apply(final List<ThreadCache.DirtyEntry> dirty) {
-                for (ThreadCache.DirtyEntry dirtyEntry : dirty) {
+                for (final ThreadCache.DirtyEntry dirtyEntry : dirty) {
                     received.add(dirtyEntry.key().get());
                 }
             }
         });
         final List<byte[]> toInsert =  Arrays.asList(new byte[]{0}, new byte[]{1}, new byte[]{2});
-        for (byte[] bytes : toInsert) {
+        for (final byte[] bytes : toInsert) {
             cache.put(namespace1, Bytes.wrap(bytes), cleanEntry(bytes));
         }
         cache.put(namespace2, Bytes.wrap(new byte[]{4}), cleanEntry(new byte[]{4}));
@@ -489,7 +492,7 @@ public class ThreadCacheTest {
         threadCache.put(namespace, Bytes.wrap(new byte[]{1}), dirtyEntry(new byte[1]));
         // Put a large item such that when the eldest item is removed
         // cache sizeInBytes() > maxCacheSizeBytes
-        int remaining = (int) (maxCacheSizeInBytes - threadCache.sizeBytes());
+        final int remaining = (int) (maxCacheSizeInBytes - threadCache.sizeBytes());
         threadCache.put(namespace, Bytes.wrap(new byte[]{2}), dirtyEntry(new byte[remaining + 100]));
     }
 
@@ -514,7 +517,7 @@ public class ThreadCacheTest {
     @Test
     public void shouldCalculateSizeInBytes() {
         final ThreadCache cache = new ThreadCache(logContext, 100000, new MockStreamsMetrics(new Metrics()));
-        NamedCache.LRUNode node = new NamedCache.LRUNode(Bytes.wrap(new byte[]{1}), dirtyEntry(new byte[]{0}));
+        final NamedCache.LRUNode node = new NamedCache.LRUNode(Bytes.wrap(new byte[]{1}), dirtyEntry(new byte[]{0}));
         cache.put(namespace1, Bytes.wrap(new byte[]{1}), cleanEntry(new byte[]{0}));
         assertEquals(cache.sizeBytes(), node.size());
     }
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowKeySchemaTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowKeySchemaTest.java
index 63214a1..9f617f2 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowKeySchemaTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowKeySchemaTest.java
@@ -81,7 +81,7 @@ public class WindowKeySchemaTest {
 
     @Test
     public void testUpperBoundWithLargeTimestamps() {
-        Bytes upper = windowKeySchema.upperRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), Long.MAX_VALUE);
+        final Bytes upper = windowKeySchema.upperRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), Long.MAX_VALUE);
 
         assertThat(
             "shorter key with max timestamp should be in range",
@@ -110,7 +110,7 @@ public class WindowKeySchemaTest {
 
     @Test
     public void testUpperBoundWithKeyBytesLargerThanFirstTimestampByte() {
-        Bytes upper = windowKeySchema.upperRange(Bytes.wrap(new byte[]{0xA, (byte) 0x8F, (byte) 0x9F}), Long.MAX_VALUE);
+        final Bytes upper = windowKeySchema.upperRange(Bytes.wrap(new byte[]{0xA, (byte) 0x8F, (byte) 0x9F}), Long.MAX_VALUE);
 
         assertThat(
             "shorter key with max timestamp should be in range",
@@ -129,7 +129,7 @@ public class WindowKeySchemaTest {
 
     @Test
     public void testUpperBoundWithKeyBytesLargerAndSmallerThanFirstTimestampByte() {
-        Bytes upper = windowKeySchema.upperRange(Bytes.wrap(new byte[]{0xC, 0xC, 0x9}), 0x0AffffffffffffffL);
+        final Bytes upper = windowKeySchema.upperRange(Bytes.wrap(new byte[]{0xC, 0xC, 0x9}), 0x0AffffffffffffffL);
 
         assertThat(
             "shorter key with max timestamp should be in range",
@@ -147,25 +147,25 @@ public class WindowKeySchemaTest {
 
     @Test
     public void testUpperBoundWithZeroTimestamp() {
-        Bytes upper = windowKeySchema.upperRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), 0);
+        final Bytes upper = windowKeySchema.upperRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), 0);
         assertThat(upper, equalTo(WindowKeySchema.toStoreKeyBinary(new byte[]{0xA, 0xB, 0xC}, 0, Integer.MAX_VALUE)));
     }
 
     @Test
     public void testLowerBoundWithZeroTimestamp() {
-        Bytes lower = windowKeySchema.lowerRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), 0);
+        final Bytes lower = windowKeySchema.lowerRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), 0);
         assertThat(lower, equalTo(WindowKeySchema.toStoreKeyBinary(new byte[]{0xA, 0xB, 0xC}, 0, 0)));
     }
 
     @Test
     public void testLowerBoundWithMonZeroTimestamp() {
-        Bytes lower = windowKeySchema.lowerRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), 42);
+        final Bytes lower = windowKeySchema.lowerRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), 42);
         assertThat(lower, equalTo(WindowKeySchema.toStoreKeyBinary(new byte[]{0xA, 0xB, 0xC}, 0, 0)));
     }
 
     @Test
     public void testLowerBoundMatchesTrailingZeros() {
-        Bytes lower = windowKeySchema.lowerRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), Long.MAX_VALUE - 1);
+        final Bytes lower = windowKeySchema.lowerRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), Long.MAX_VALUE - 1);
 
         assertThat(
             "appending zeros to key should still be in range",
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/WrappingStoreProviderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/WrappingStoreProviderTest.java
index 06c14ee..b864732 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/WrappingStoreProviderTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/WrappingStoreProviderTest.java
@@ -61,7 +61,7 @@ public class WrappingStoreProviderTest {
 
     @Test
     public void shouldFindKeyValueStores() {
-        List<ReadOnlyKeyValueStore<String, String>> results =
+        final List<ReadOnlyKeyValueStore<String, String>> results =
                 wrappingStoreProvider.stores("kv", QueryableStoreTypes.<String, String>keyValueStore());
         assertEquals(2, results.size());
     }
diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java
index 767c9f1..01333db 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java
@@ -80,7 +80,7 @@ public class BrokerCompatibilityTest {
         streamsProperties.put(StreamsConfig.consumerPrefix(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), timeout);
         streamsProperties.put(StreamsConfig.consumerPrefix(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG), timeout);
         streamsProperties.put(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG, timeout + 1);
-        Serde<String> stringSerde = Serdes.String();
+        final Serde<String> stringSerde = Serdes.String();
 
 
         final StreamsBuilder builder = new StreamsBuilder();
@@ -89,7 +89,7 @@ public class BrokerCompatibilityTest {
             .toStream()
             .mapValues(new ValueMapper<Long, String>() {
                 @Override
-                public String apply(Long value) {
+                public String apply(final Long value) {
                     return value.toString();
                 }
             })
diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java b/streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java
index 6069298..1f8238e 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java
@@ -89,7 +89,7 @@ public class EosTestClient extends SmokeTestUtil {
                 });
                 streams.setStateListener(new KafkaStreams.StateListener() {
                     @Override
-                    public void onChange(KafkaStreams.State newState, KafkaStreams.State oldState) {
+                    public void onChange(final KafkaStreams.State newState, final KafkaStreams.State oldState) {
                         // don't remove this -- it's required test output
                         System.out.println(System.currentTimeMillis());
                         System.out.println("StateChange: " + oldState + " -> " + newState);
diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/ShutdownDeadlockTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/ShutdownDeadlockTest.java
index 01731f3..0bbb889 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tests/ShutdownDeadlockTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tests/ShutdownDeadlockTest.java
@@ -84,7 +84,7 @@ public class ShutdownDeadlockTest {
         synchronized (this) {
             try {
                 wait();
-            } catch (InterruptedException e) {
+            } catch (final InterruptedException e) {
                 // ignored
             }
         }
diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java
index 79dfb30..51dc05c 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java
@@ -89,7 +89,7 @@ public class SmokeTestClient extends SmokeTestUtil {
         }
         try {
             thread.join();
-        } catch (Exception ex) {
+        } catch (final Exception ex) {
             // do not remove these printouts since they are needed for health scripts
             System.out.println("SMOKE-TEST-CLIENT-EXCEPTION");
             // ignore
diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java
index a504333..7087298 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java
@@ -57,7 +57,7 @@ public class SmokeTestDriver extends SmokeTestUtil {
         private final int[] values;
         private int index;
 
-        ValueList(int min, int max) {
+        ValueList(final int min, final int max) {
             this.key = min + "-" + max;
 
             this.values = new int[max - min + 1];
@@ -78,19 +78,19 @@ public class SmokeTestDriver extends SmokeTestUtil {
     }
 
     // This main() is not used by the system test. It is intended to be used for local debugging.
-    public static void main(String[] args) throws InterruptedException {
+    public static void main(final String[] args) throws InterruptedException {
         final String kafka = "localhost:9092";
         final File stateDir = TestUtils.tempDirectory();
 
         final int numKeys = 20;
         final int maxRecordsPerKey = 1000;
 
-        Thread driver = new Thread() {
+        final Thread driver = new Thread() {
             public void run() {
                 try {
-                    Map<String, Set<Integer>> allData = generate(kafka, numKeys, maxRecordsPerKey);
+                    final Map<String, Set<Integer>> allData = generate(kafka, numKeys, maxRecordsPerKey);
                     verify(kafka, allData, maxRecordsPerKey);
-                } catch (Exception ex) {
+                } catch (final Exception ex) {
                     ex.printStackTrace();
                 }
             }
@@ -99,13 +99,13 @@ public class SmokeTestDriver extends SmokeTestUtil {
         final Properties props = new Properties();
         props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka);
         props.put(StreamsConfig.STATE_DIR_CONFIG, createDir(stateDir, "1").getAbsolutePath());
-        SmokeTestClient streams1 = new SmokeTestClient(props);
+        final SmokeTestClient streams1 = new SmokeTestClient(props);
         props.put(StreamsConfig.STATE_DIR_CONFIG, createDir(stateDir, "2").getAbsolutePath());
-        SmokeTestClient streams2 = new SmokeTestClient(props);
+        final SmokeTestClient streams2 = new SmokeTestClient(props);
         props.put(StreamsConfig.STATE_DIR_CONFIG, createDir(stateDir, "3").getAbsolutePath());
-        SmokeTestClient streams3 = new SmokeTestClient(props);
+        final SmokeTestClient streams3 = new SmokeTestClient(props);
         props.put(StreamsConfig.STATE_DIR_CONFIG, createDir(stateDir, "4").getAbsolutePath());
-        SmokeTestClient streams4 = new SmokeTestClient(props);
+        final SmokeTestClient streams4 = new SmokeTestClient(props);
 
         System.out.println("starting the driver");
         driver.start();
@@ -177,7 +177,7 @@ public class SmokeTestDriver extends SmokeTestUtil {
         while (remaining > 0) {
             final int index = autoTerminate ? rand.nextInt(remaining) : rand.nextInt(numKeys);
             final String key = data[index].key;
-            int value = data[index].next();
+            final int value = data[index].next();
 
             if (autoTerminate && value < 0) {
                 remaining--;
@@ -241,53 +241,53 @@ public class SmokeTestDriver extends SmokeTestUtil {
         }
     }
 
-    private static void shuffle(int[] data, int windowSize) {
-        Random rand = new Random();
+    private static void shuffle(final int[] data, final int windowSize) {
+        final Random rand = new Random();
         for (int i = 0; i < data.length; i++) {
             // we shuffle data within windowSize
-            int j = rand.nextInt(Math.min(data.length - i, windowSize)) + i;
+            final int j = rand.nextInt(Math.min(data.length - i, windowSize)) + i;
 
             // swap
-            int tmp = data[i];
+            final int tmp = data[i];
             data[i] = data[j];
             data[j] = tmp;
         }
     }
 
-    public static void verify(String kafka, Map<String, Set<Integer>> allData, int maxRecordsPerKey) {
-        Properties props = new Properties();
+    public static void verify(final String kafka, final Map<String, Set<Integer>> allData, final int maxRecordsPerKey) {
+        final Properties props = new Properties();
         props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier");
         props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka);
         props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
         props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
 
-        KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(props);
-        List<TopicPartition> partitions = getAllPartitions(consumer, "echo", "max", "min", "dif", "sum", "cnt", "avg", "wcnt", "tagg");
+        final KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(props);
+        final List<TopicPartition> partitions = getAllPartitions(consumer, "echo", "max", "min", "dif", "sum", "cnt", "avg", "wcnt", "tagg");
         consumer.assign(partitions);
         consumer.seekToBeginning(partitions);
 
         final int recordsGenerated = allData.size() * maxRecordsPerKey;
         int recordsProcessed = 0;
 
-        HashMap<String, Integer> max = new HashMap<>();
-        HashMap<String, Integer> min = new HashMap<>();
-        HashMap<String, Integer> dif = new HashMap<>();
-        HashMap<String, Long> sum = new HashMap<>();
-        HashMap<String, Long> cnt = new HashMap<>();
-        HashMap<String, Double> avg = new HashMap<>();
-        HashMap<String, Long> wcnt = new HashMap<>();
-        HashMap<String, Long> tagg = new HashMap<>();
-
-        HashSet<String> keys = new HashSet<>();
-        HashMap<String, Set<Integer>> received = new HashMap<>();
-        for (String key : allData.keySet()) {
+        final HashMap<String, Integer> max = new HashMap<>();
+        final HashMap<String, Integer> min = new HashMap<>();
+        final HashMap<String, Integer> dif = new HashMap<>();
+        final HashMap<String, Long> sum = new HashMap<>();
+        final HashMap<String, Long> cnt = new HashMap<>();
+        final HashMap<String, Double> avg = new HashMap<>();
+        final HashMap<String, Long> wcnt = new HashMap<>();
+        final HashMap<String, Long> tagg = new HashMap<>();
+
+        final HashSet<String> keys = new HashSet<>();
+        final HashMap<String, Set<Integer>> received = new HashMap<>();
+        for (final String key : allData.keySet()) {
             keys.add(key);
             received.put(key, new HashSet<Integer>());
         }
         int retry = 0;
         final long start = System.currentTimeMillis();
         while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(6)) {
-            ConsumerRecords<byte[], byte[]> records = consumer.poll(Duration.ofMillis(500));
+            final ConsumerRecords<byte[], byte[]> records = consumer.poll(Duration.ofMillis(500));
             if (records.isEmpty() && recordsProcessed >= recordsGenerated) {
                 if (verifyMin(min, allData, false)
                     && verifyMax(max, allData, false)
@@ -302,11 +302,11 @@ public class SmokeTestDriver extends SmokeTestUtil {
                     break;
                 }
             } else {
-                for (ConsumerRecord<byte[], byte[]> record : records) {
-                    String key = stringSerde.deserializer().deserialize("", record.key());
+                for (final ConsumerRecord<byte[], byte[]> record : records) {
+                    final String key = stringSerde.deserializer().deserialize("", record.key());
                     switch (record.topic()) {
                         case "echo":
-                            Integer value = intSerde.deserializer().deserialize("", record.value());
+                            final Integer value = intSerde.deserializer().deserialize("", record.value());
                             recordsProcessed++;
                             if (recordsProcessed % 100 == 0) {
                                 System.out.println("Echo records processed = " + recordsProcessed);
@@ -365,7 +365,7 @@ public class SmokeTestDriver extends SmokeTestUtil {
             System.out.println("ALL-RECORDS-DELIVERED");
         } else {
             int missedCount = 0;
-            for (Map.Entry<String, Set<Integer>> entry : allData.entrySet()) {
+            for (final Map.Entry<String, Set<Integer>> entry : allData.entrySet()) {
                 missedCount += received.get(entry.getKey()).size();
             }
             System.out.println("missedRecords=" + missedCount);
@@ -382,7 +382,7 @@ public class SmokeTestDriver extends SmokeTestUtil {
         System.out.println(success ? "SUCCESS" : "FAILURE");
     }
 
-    private static boolean verifyMin(Map<String, Integer> map, Map<String, Set<Integer>> allData, final boolean print) {
+    private static boolean verifyMin(final Map<String, Integer> map, final Map<String, Set<Integer>> allData, final boolean print) {
         if (map.isEmpty()) {
             if (print) {
                 System.out.println("min is empty");
@@ -399,8 +399,8 @@ public class SmokeTestDriver extends SmokeTestUtil {
                 }
                 return false;
             }
-            for (Map.Entry<String, Integer> entry : map.entrySet()) {
-                int expected = getMin(entry.getKey());
+            for (final Map.Entry<String, Integer> entry : map.entrySet()) {
+                final int expected = getMin(entry.getKey());
                 if (expected != entry.getValue()) {
                     if (print) {
                         System.out.println("fail: key=" + entry.getKey() + " min=" + entry.getValue() + " expected=" + expected);
@@ -412,7 +412,7 @@ public class SmokeTestDriver extends SmokeTestUtil {
         return true;
     }
 
-    private static boolean verifyMax(Map<String, Integer> map, Map<String, Set<Integer>> allData, final boolean print) {
+    private static boolean verifyMax(final Map<String, Integer> map, final Map<String, Set<Integer>> allData, final boolean print) {
         if (map.isEmpty()) {
             if (print) {
                 System.out.println("max is empty");
@@ -429,8 +429,8 @@ public class SmokeTestDriver extends SmokeTestUtil {
                 }
                 return false;
             }
-            for (Map.Entry<String, Integer> entry : map.entrySet()) {
-                int expected = getMax(entry.getKey());
+            for (final Map.Entry<String, Integer> entry : map.entrySet()) {
+                final int expected = getMax(entry.getKey());
                 if (expected != entry.getValue()) {
                     if (print) {
                         System.out.println("fail: key=" + entry.getKey() + " max=" + entry.getValue() + " expected=" + expected);
@@ -442,7 +442,7 @@ public class SmokeTestDriver extends SmokeTestUtil {
         return true;
     }
 
-    private static boolean verifyDif(Map<String, Integer> map, Map<String, Set<Integer>> allData, final boolean print) {
+    private static boolean verifyDif(final Map<String, Integer> map, final Map<String, Set<Integer>> allData, final boolean print) {
         if (map.isEmpty()) {
             if (print) {
                 System.out.println("dif is empty");
@@ -459,10 +459,10 @@ public class SmokeTestDriver extends SmokeTestUtil {
                 }
                 return false;
             }
-            for (Map.Entry<String, Integer> entry : map.entrySet()) {
-                int min = getMin(entry.getKey());
-                int max = getMax(entry.getKey());
-                int expected = max - min;
+            for (final Map.Entry<String, Integer> entry : map.entrySet()) {
+                final int min = getMin(entry.getKey());
+                final int max = getMax(entry.getKey());
+                final int expected = max - min;
                 if (entry.getValue() == null || expected != entry.getValue()) {
                     if (print) {
                         System.out.println("fail: key=" + entry.getKey() + " dif=" + entry.getValue() + " expected=" + expected);
@@ -474,7 +474,7 @@ public class SmokeTestDriver extends SmokeTestUtil {
         return true;
     }
 
-    private static boolean verifyCnt(Map<String, Long> map, Map<String, Set<Integer>> allData, final boolean print) {
+    private static boolean verifyCnt(final Map<String, Long> map, final Map<String, Set<Integer>> allData, final boolean print) {
         if (map.isEmpty()) {
             if (print) {
                 System.out.println("cnt is empty");
@@ -491,10 +491,10 @@ public class SmokeTestDriver extends SmokeTestUtil {
                 }
                 return false;
             }
-            for (Map.Entry<String, Long> entry : map.entrySet()) {
-                int min = getMin(entry.getKey());
-                int max = getMax(entry.getKey());
-                long expected = (max - min) + 1L;
+            for (final Map.Entry<String, Long> entry : map.entrySet()) {
+                final int min = getMin(entry.getKey());
+                final int max = getMax(entry.getKey());
+                final long expected = (max - min) + 1L;
                 if (expected != entry.getValue()) {
                     if (print) {
                         System.out.println("fail: key=" + entry.getKey() + " cnt=" + entry.getValue() + " expected=" + expected);
@@ -506,7 +506,7 @@ public class SmokeTestDriver extends SmokeTestUtil {
         return true;
     }
 
-    private static boolean verifySum(Map<String, Long> map, Map<String, Set<Integer>> allData, final boolean print) {
+    private static boolean verifySum(final Map<String, Long> map, final Map<String, Set<Integer>> allData, final boolean print) {
         if (map.isEmpty()) {
             if (print) {
                 System.out.println("sum is empty");
@@ -523,10 +523,10 @@ public class SmokeTestDriver extends SmokeTestUtil {
                 }
                 return false;
             }
-            for (Map.Entry<String, Long> entry : map.entrySet()) {
-                int min = getMin(entry.getKey());
-                int max = getMax(entry.getKey());
-                long expected = ((long) min + (long) max) * (max - min + 1L) / 2L;
+            for (final Map.Entry<String, Long> entry : map.entrySet()) {
+                final int min = getMin(entry.getKey());
+                final int max = getMax(entry.getKey());
+                final long expected = ((long) min + (long) max) * (max - min + 1L) / 2L;
                 if (expected != entry.getValue()) {
                     if (print) {
                         System.out.println("fail: key=" + entry.getKey() + " sum=" + entry.getValue() + " expected=" + expected);
@@ -538,7 +538,7 @@ public class SmokeTestDriver extends SmokeTestUtil {
         return true;
     }
 
-    private static boolean verifyAvg(Map<String, Double> map, Map<String, Set<Integer>> allData, final boolean print) {
+    private static boolean verifyAvg(final Map<String, Double> map, final Map<String, Set<Integer>> allData, final boolean print) {
         if (map.isEmpty()) {
             if (print) {
                 System.out.println("avg is empty");
@@ -555,10 +555,10 @@ public class SmokeTestDriver extends SmokeTestUtil {
                 }
                 return false;
             }
-            for (Map.Entry<String, Double> entry : map.entrySet()) {
-                int min = getMin(entry.getKey());
-                int max = getMax(entry.getKey());
-                double expected = ((long) min + (long) max) / 2.0;
+            for (final Map.Entry<String, Double> entry : map.entrySet()) {
+                final int min = getMin(entry.getKey());
+                final int max = getMax(entry.getKey());
+                final double expected = ((long) min + (long) max) / 2.0;
 
                 if (entry.getValue() == null || expected != entry.getValue()) {
                     if (print) {
@@ -572,7 +572,7 @@ public class SmokeTestDriver extends SmokeTestUtil {
     }
 
 
-    private static boolean verifyTAgg(Map<String, Long> map, Map<String, Set<Integer>> allData, final boolean print) {
+    private static boolean verifyTAgg(final Map<String, Long> map, final Map<String, Set<Integer>> allData, final boolean print) {
         if (map.isEmpty()) {
             if (print) {
                 System.out.println("tagg is empty");
@@ -584,11 +584,11 @@ public class SmokeTestDriver extends SmokeTestUtil {
             }
 
             // generate expected answer
-            Map<String, Long> expected = new HashMap<>();
-            for (String key : allData.keySet()) {
-                int min = getMin(key);
-                int max = getMax(key);
-                String cnt = Long.toString(max - min + 1L);
+            final Map<String, Long> expected = new HashMap<>();
+            for (final String key : allData.keySet()) {
+                final int min = getMin(key);
+                final int max = getMax(key);
+                final String cnt = Long.toString(max - min + 1L);
 
                 if (expected.containsKey(cnt)) {
                     expected.put(cnt, expected.get(cnt) + 1L);
@@ -598,8 +598,8 @@ public class SmokeTestDriver extends SmokeTestUtil {
             }
 
             // check the result
-            for (Map.Entry<String, Long> entry : map.entrySet()) {
-                String key = entry.getKey();
+            for (final Map.Entry<String, Long> entry : map.entrySet()) {
+                final String key = entry.getKey();
                 Long expectedCount = expected.remove(key);
                 if (expectedCount == null)
                     expectedCount = 0L;
@@ -616,31 +616,31 @@ public class SmokeTestDriver extends SmokeTestUtil {
         return true;
     }
 
-    private static int getMin(String key) {
+    private static int getMin(final String key) {
         return Integer.parseInt(key.split("-")[0]);
     }
 
-    private static int getMax(String key) {
+    private static int getMax(final String key) {
         return Integer.parseInt(key.split("-")[1]);
     }
 
-    private static int getMinFromWKey(String key) {
+    private static int getMinFromWKey(final String key) {
         return getMin(key.split("@")[0]);
     }
 
-    private static int getMaxFromWKey(String key) {
+    private static int getMaxFromWKey(final String key) {
         return getMax(key.split("@")[0]);
     }
 
-    private static long getStartFromWKey(String key) {
+    private static long getStartFromWKey(final String key) {
         return Long.parseLong(key.split("@")[1]);
     }
 
-    private static List<TopicPartition> getAllPartitions(KafkaConsumer<?, ?> consumer, String... topics) {
-        ArrayList<TopicPartition> partitions = new ArrayList<>();
+    private static List<TopicPartition> getAllPartitions(final KafkaConsumer<?, ?> consumer, final String... topics) {
+        final ArrayList<TopicPartition> partitions = new ArrayList<>();
 
-        for (String topic : topics) {
-            for (PartitionInfo info : consumer.partitionsFor(topic)) {
+        for (final String topic : topics) {
+            for (final PartitionInfo info : consumer.partitionsFor(topic)) {
                 partitions.add(new TopicPartition(info.topic(), info.partition()));
             }
         }
diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsBrokerDownResilienceTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsBrokerDownResilienceTest.java
index 8a3e75d..eb19688 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsBrokerDownResilienceTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsBrokerDownResilienceTest.java
@@ -44,7 +44,7 @@ public class StreamsBrokerDownResilienceTest {
 
     private static final String SINK_TOPIC = "streamsResilienceSink";
 
-    public static void main(String[] args) throws IOException {
+    public static void main(final String[] args) throws IOException {
         if (args.length < 2) {
             System.err.println("StreamsBrokerDownResilienceTest are expecting two parameters: propFile, additionalConfigs; but only see " + args.length + " parameter");
             System.exit(1);
@@ -72,7 +72,7 @@ public class StreamsBrokerDownResilienceTest {
         // it is expected that max.poll.interval, retries, request.timeout and max.block.ms set
         // streams_broker_down_resilience_test and passed as args
         if (additionalConfigs != null && !additionalConfigs.equalsIgnoreCase("none")) {
-            Map<String, String> updated = updatedConfigs(additionalConfigs);
+            final Map<String, String> updated = updatedConfigs(additionalConfigs);
             System.out.println("Updating configs with " + updated);
             streamsProperties.putAll(updated);
         }
@@ -94,7 +94,7 @@ public class StreamsBrokerDownResilienceTest {
             .peek(new ForeachAction<String, String>() {
                 int messagesProcessed = 0;
                 @Override
-                public void apply(String key, String value) {
+                public void apply(final String key, final String value) {
                     System.out.println("received key " + key + " and value " + value);
                     messagesProcessed++;
                     System.out.println("processed" + messagesProcessed + "messages");
@@ -127,7 +127,7 @@ public class StreamsBrokerDownResilienceTest {
 
     }
 
-    private static boolean confirmCorrectConfigs(Properties properties) {
+    private static boolean confirmCorrectConfigs(final Properties properties) {
         return properties.containsKey(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG)) &&
                properties.containsKey(StreamsConfig.producerPrefix(ProducerConfig.RETRIES_CONFIG)) &&
                properties.containsKey(StreamsConfig.producerPrefix(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG)) &&
@@ -141,11 +141,11 @@ public class StreamsBrokerDownResilienceTest {
      * @param formattedConfigs the formatted config string
      * @return HashMap with keys and values inserted
      */
-    private static Map<String, String> updatedConfigs(String formattedConfigs) {
-        String[] parts = formattedConfigs.split(",");
-        Map<String, String> updatedConfigs = new HashMap<>();
-        for (String part : parts) {
-            String[] keyValue = part.split("=");
+    private static Map<String, String> updatedConfigs(final String formattedConfigs) {
+        final String[] parts = formattedConfigs.split(",");
+        final Map<String, String> updatedConfigs = new HashMap<>();
+        for (final String part : parts) {
+            final String[] keyValue = part.split("=");
             updatedConfigs.put(keyValue[KEY], keyValue[VALUE]);
         }
         return updatedConfigs;
diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java
index 8c80780..f34471b 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java
@@ -66,7 +66,7 @@ public class StreamsSmokeTest {
                 if (disableAutoTerminate) {
                     SmokeTestDriver.generate(kafka, numKeys, maxRecordsPerKey, false);
                 } else {
-                    Map<String, Set<Integer>> allData = SmokeTestDriver.generate(kafka, numKeys, maxRecordsPerKey);
+                    final Map<String, Set<Integer>> allData = SmokeTestDriver.generate(kafka, numKeys, maxRecordsPerKey);
                     SmokeTestDriver.verify(kafka, allData, maxRecordsPerKey);
                 }
                 break;
diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java
index 690768f..db160fe 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java
@@ -110,9 +110,9 @@ public class StreamsStandByReplicaTest {
 
         final Serde<String> stringSerde = Serdes.String();
 
-        KStream<String, String> inputStream = builder.stream(sourceTopic, Consumed.with(stringSerde, stringSerde));
+        final KStream<String, String> inputStream = builder.stream(sourceTopic, Consumed.with(stringSerde, stringSerde));
 
-        ValueMapper<Long, String> countMapper = new ValueMapper<Long, String>() {
+        final ValueMapper<Long, String> countMapper = new ValueMapper<Long, String>() {
             @Override
             public String apply(final Long value) {
                 return value.toString();
diff --git a/streams/src/test/java/org/apache/kafka/streams/tools/StreamsResetterTest.java b/streams/src/test/java/org/apache/kafka/streams/tools/StreamsResetterTest.java
index 33cf1fa..32d616f 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tools/StreamsResetterTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tools/StreamsResetterTest.java
@@ -233,17 +233,17 @@ public class StreamsResetterTest {
 
     @Test
     public void shouldDeleteTopic() throws InterruptedException, ExecutionException {
-        Cluster cluster = createCluster(1);
-        try (MockAdminClient adminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) {
-            TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.<Node>emptyList());
+        final Cluster cluster = createCluster(1);
+        try (final MockAdminClient adminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) {
+            final TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.<Node>emptyList());
             adminClient.addTopic(false, TOPIC, Collections.singletonList(topicPartitionInfo), null);
             streamsResetter.doDelete(Collections.singletonList(TOPIC), adminClient);
             assertEquals(Collections.emptySet(), adminClient.listTopics().names().get());
         }
     }
 
-    private Cluster createCluster(int numNodes) {
-        HashMap<Integer, Node> nodes = new HashMap<>();
+    private Cluster createCluster(final int numNodes) {
+        final HashMap<Integer, Node> nodes = new HashMap<>();
         for (int i = 0; i < numNodes; ++i) {
             nodes.put(i, new Node(i, "localhost", 8121 + i));
         }
diff --git a/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java b/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java
index ae0b923..a97a91e 100644
--- a/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java
+++ b/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java
@@ -41,7 +41,7 @@ public class GlobalStateManagerStub implements GlobalStateManager {
     }
 
     @Override
-    public void setGlobalProcessorContext(InternalProcessorContext processorContext) {}
+    public void setGlobalProcessorContext(final InternalProcessorContext processorContext) {}
 
     @Override
     public Set<String> initialize() {
diff --git a/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java
index ca6d5a7..e3a9ce7 100644
--- a/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java
+++ b/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java
@@ -206,7 +206,7 @@ public class InternalMockProcessorContext extends AbstractProcessorContext imple
     }
 
     @Override
-    public Cancellable schedule(long interval, PunctuationType type, Punctuator callback) {
+    public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) {
         throw new UnsupportedOperationException("schedule() not supported.");
     }
 
@@ -316,8 +316,8 @@ public class InternalMockProcessorContext extends AbstractProcessorContext imple
 
         restoreListener.onRestoreStart(null, storeName, 0L, 0L);
 
-        List<KeyValue<byte[], byte[]>> records = new ArrayList<>();
-        for (KeyValue<byte[], byte[]> keyValue : changeLog) {
+        final List<KeyValue<byte[], byte[]>> records = new ArrayList<>();
+        for (final KeyValue<byte[], byte[]> keyValue : changeLog) {
             records.add(keyValue);
         }
 
@@ -326,7 +326,7 @@ public class InternalMockProcessorContext extends AbstractProcessorContext imple
         restoreListener.onRestoreEnd(null, storeName, 0L);
     }
 
-    private StateRestoreListener getStateRestoreListener(StateRestoreCallback restoreCallback) {
+    private StateRestoreListener getStateRestoreListener(final StateRestoreCallback restoreCallback) {
         if (restoreCallback instanceof StateRestoreListener) {
             return (StateRestoreListener) restoreCallback;
         }
@@ -334,7 +334,7 @@ public class InternalMockProcessorContext extends AbstractProcessorContext imple
         return CompositeRestoreListener.NO_OP_STATE_RESTORE_LISTENER;
     }
 
-    private BatchingStateRestoreCallback getBatchingRestoreCallback(StateRestoreCallback restoreCallback) {
+    private BatchingStateRestoreCallback getBatchingRestoreCallback(final StateRestoreCallback restoreCallback) {
         if (restoreCallback instanceof BatchingStateRestoreCallback) {
             return (BatchingStateRestoreCallback) restoreCallback;
         }
diff --git a/streams/src/test/java/org/apache/kafka/test/KTableValueGetterStub.java b/streams/src/test/java/org/apache/kafka/test/KTableValueGetterStub.java
index f739755..75f6db0 100644
--- a/streams/src/test/java/org/apache/kafka/test/KTableValueGetterStub.java
+++ b/streams/src/test/java/org/apache/kafka/test/KTableValueGetterStub.java
@@ -36,7 +36,7 @@ public class KTableValueGetterStub<K, V> implements KTableValueGetter<K, V> {
         return data.get(key);
     }
 
-    public void put(final K key, V value) {
+    public void put(final K key, final V value) {
         data.put(key, value);
     }
 
diff --git a/streams/src/test/java/org/apache/kafka/test/MockAggregator.java b/streams/src/test/java/org/apache/kafka/test/MockAggregator.java
index eabb315..829e011 100644
--- a/streams/src/test/java/org/apache/kafka/test/MockAggregator.java
+++ b/streams/src/test/java/org/apache/kafka/test/MockAggregator.java
@@ -26,7 +26,7 @@ public class MockAggregator {
     public static <K, V> Aggregator<K, V, String> toStringInstance(final String sep) {
         return new Aggregator<K, V, String>() {
             @Override
-            public String apply(K aggKey, V value, String aggregate) {
+            public String apply(final K aggKey, final V value, final String aggregate) {
                 return aggregate + sep + value;
             }
         };
diff --git a/streams/src/test/java/org/apache/kafka/test/MockInternalTopicManager.java b/streams/src/test/java/org/apache/kafka/test/MockInternalTopicManager.java
index ea27045..8ab50b8 100644
--- a/streams/src/test/java/org/apache/kafka/test/MockInternalTopicManager.java
+++ b/streams/src/test/java/org/apache/kafka/test/MockInternalTopicManager.java
@@ -62,7 +62,7 @@ public class MockInternalTopicManager extends InternalTopicManager {
     @Override
     protected Map<String, Integer> getNumPartitions(final Set<String> topics) {
         final Map<String, Integer> partitions = new HashMap<>();
-        for (String topic : topics) {
+        for (final String topic : topics) {
             partitions.put(topic, restoreConsumer.partitionsFor(topic) == null ?  null : restoreConsumer.partitionsFor(topic).size());
         }
 
diff --git a/streams/src/test/java/org/apache/kafka/test/MockMapper.java b/streams/src/test/java/org/apache/kafka/test/MockMapper.java
index 5184199..c3d9e93 100644
--- a/streams/src/test/java/org/apache/kafka/test/MockMapper.java
+++ b/streams/src/test/java/org/apache/kafka/test/MockMapper.java
@@ -26,35 +26,35 @@ public class MockMapper {
 
     private static class NoOpKeyValueMapper<K, V> implements KeyValueMapper<K, V, KeyValue<K, V>> {
         @Override
-        public KeyValue<K, V> apply(K key, V value) {
+        public KeyValue<K, V> apply(final K key, final V value) {
             return KeyValue.pair(key, value);
         }
     }
 
     private static class NoOpFlatKeyValueMapper<K, V> implements KeyValueMapper<K, V, Iterable<KeyValue<K, V>>> {
         @Override
-        public Iterable<KeyValue<K, V>> apply(K key, V value) {
+        public Iterable<KeyValue<K, V>> apply(final K key, final V value) {
             return Collections.singletonList(KeyValue.pair(key, value));
         }
     }
 
     private static class SelectValueKeyValueMapper<K, V> implements KeyValueMapper<K, V, KeyValue<V, V>> {
         @Override
-        public KeyValue<V, V> apply(K key, V value) {
+        public KeyValue<V, V> apply(final K key, final V value) {
             return KeyValue.pair(value, value);
         }
     }
 
     private static class SelectValueMapper<K, V> implements KeyValueMapper<K, V, V> {
         @Override
-        public V apply(K key, V value) {
+        public V apply(final K key, final V value) {
             return value;
         }
     }
 
     private static class SelectKeyMapper<K, V> implements KeyValueMapper<K, V, K> {
         @Override
-        public K apply(K key, V value) {
+        public K apply(final K key, final V value) {
             return key;
         }
     }
diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorNode.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorNode.java
index 094cb03..718e5af 100644
--- a/streams/src/test/java/org/apache/kafka/test/MockProcessorNode.java
+++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorNode.java
@@ -33,11 +33,11 @@ public class MockProcessorNode<K, V> extends ProcessorNode<K, V> {
     public boolean closed;
     public boolean initialized;
 
-    public MockProcessorNode(long scheduleInterval) {
+    public MockProcessorNode(final long scheduleInterval) {
         this(scheduleInterval, PunctuationType.STREAM_TIME);
     }
 
-    public MockProcessorNode(long scheduleInterval, PunctuationType punctuationType) {
+    public MockProcessorNode(final long scheduleInterval, final PunctuationType punctuationType) {
         this(new MockProcessor<K, V>(punctuationType, scheduleInterval));
     }
 
@@ -58,7 +58,7 @@ public class MockProcessorNode<K, V> extends ProcessorNode<K, V> {
     }
 
     @Override
-    public void process(K key, V value) {
+    public void process(final K key, final V value) {
         processor().process(key, value);
     }
 
diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java
index aec47a4..0bd22d4 100644
--- a/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java
+++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java
@@ -35,11 +35,11 @@ public class MockProcessorSupplier<K, V> implements ProcessorSupplier<K, V> {
         this(-1L);
     }
 
-    public MockProcessorSupplier(long scheduleInterval) {
+    public MockProcessorSupplier(final long scheduleInterval) {
         this(scheduleInterval, PunctuationType.STREAM_TIME);
     }
 
-    public MockProcessorSupplier(long scheduleInterval, PunctuationType punctuationType) {
+    public MockProcessorSupplier(final long scheduleInterval, final PunctuationType punctuationType) {
         this.scheduleInterval = scheduleInterval;
         this.punctuationType = punctuationType;
     }
diff --git a/streams/src/test/java/org/apache/kafka/test/MockReducer.java b/streams/src/test/java/org/apache/kafka/test/MockReducer.java
index 4db05d3..0ecb4a1 100644
--- a/streams/src/test/java/org/apache/kafka/test/MockReducer.java
+++ b/streams/src/test/java/org/apache/kafka/test/MockReducer.java
@@ -23,7 +23,7 @@ public class MockReducer {
     private static class StringAdd implements Reducer<String> {
 
         @Override
-        public String apply(String value1, String value2) {
+        public String apply(final String value1, final String value2) {
             return value1 + "+" + value2;
         }
     }
@@ -31,7 +31,7 @@ public class MockReducer {
     private static class StringRemove implements Reducer<String> {
 
         @Override
-        public String apply(String value1, String value2) {
+        public String apply(final String value1, final String value2) {
             return value1 + "-" + value2;
         }
     }
diff --git a/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java b/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java
index 78b4ff8..bc918ea 100644
--- a/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java
+++ b/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java
@@ -57,7 +57,7 @@ public class MockRestoreConsumer<K, V> extends MockConsumer<byte[], byte[]> {
     }
 
     // buffer a record (we cannot use addRecord because we need to add records before assigning a partition)
-    public void bufferRecord(ConsumerRecord<K, V> record) {
+    public void bufferRecord(final ConsumerRecord<K, V> record) {
         recordBuffer.add(
             new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), record.timestamp(),
                                  record.timestampType(), 0L, 0, 0,
@@ -69,8 +69,8 @@ public class MockRestoreConsumer<K, V> extends MockConsumer<byte[], byte[]> {
     }
 
     @Override
-    public synchronized void assign(Collection<TopicPartition> partitions) {
-        int numPartitions = partitions.size();
+    public synchronized void assign(final Collection<TopicPartition> partitions) {
+        final int numPartitions = partitions.size();
         if (numPartitions > 1)
             throw new IllegalArgumentException("RestoreConsumer: more than one partition specified");
 
@@ -90,16 +90,16 @@ public class MockRestoreConsumer<K, V> extends MockConsumer<byte[], byte[]> {
     @Override
     public ConsumerRecords<byte[], byte[]> poll(final Duration timeout) {
         // add buffered records to MockConsumer
-        for (ConsumerRecord<byte[], byte[]> record : recordBuffer) {
+        for (final ConsumerRecord<byte[], byte[]> record : recordBuffer) {
             super.addRecord(record);
         }
         recordBuffer.clear();
 
-        ConsumerRecords<byte[], byte[]> records = super.poll(timeout);
+        final ConsumerRecords<byte[], byte[]> records = super.poll(timeout);
 
         // set the current offset
-        Iterable<ConsumerRecord<byte[], byte[]>> partitionRecords = records.records(assignedPartition);
-        for (ConsumerRecord<byte[], byte[]> record : partitionRecords) {
+        final Iterable<ConsumerRecord<byte[], byte[]>> partitionRecords = records.records(assignedPartition);
+        for (final ConsumerRecord<byte[], byte[]> record : partitionRecords) {
             currentOffset = record.offset();
         }
 
@@ -107,7 +107,7 @@ public class MockRestoreConsumer<K, V> extends MockConsumer<byte[], byte[]> {
     }
 
     @Override
-    public synchronized long position(TopicPartition partition) {
+    public synchronized long position(final TopicPartition partition) {
         if (!partition.equals(assignedPartition))
             throw new IllegalStateException("RestoreConsumer: unassigned partition");
 
@@ -115,7 +115,7 @@ public class MockRestoreConsumer<K, V> extends MockConsumer<byte[], byte[]> {
     }
 
     @Override
-    public synchronized void seek(TopicPartition partition, long offset) {
+    public synchronized void seek(final TopicPartition partition, final long offset) {
         if (offset < 0)
             throw new IllegalArgumentException("RestoreConsumer: offset should not be negative");
 
@@ -128,11 +128,11 @@ public class MockRestoreConsumer<K, V> extends MockConsumer<byte[], byte[]> {
     }
 
     @Override
-    public synchronized void seekToBeginning(Collection<TopicPartition> partitions) {
+    public synchronized void seekToBeginning(final Collection<TopicPartition> partitions) {
         if (partitions.size() != 1)
             throw new IllegalStateException("RestoreConsumer: other than one partition specified");
 
-        for (TopicPartition partition : partitions) {
+        for (final TopicPartition partition : partitions) {
             if (!partition.equals(assignedPartition))
                 throw new IllegalStateException("RestoreConsumer: seek-to-end not on the assigned partition");
         }
@@ -146,7 +146,7 @@ public class MockRestoreConsumer<K, V> extends MockConsumer<byte[], byte[]> {
         if (partitions.size() != 1)
             throw new IllegalStateException("RestoreConsumer: other than one partition specified");
 
-        for (TopicPartition partition : partitions) {
+        for (final TopicPartition partition : partitions) {
             if (!partition.equals(assignedPartition))
                 throw new IllegalStateException("RestoreConsumer: seek-to-end not on the assigned partition");
         }
diff --git a/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java b/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java
index 1644daa..9cae17b 100644
--- a/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java
+++ b/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java
@@ -36,12 +36,12 @@ public class MockSourceNode<K, V> extends SourceNode<K, V> {
     public boolean initialized;
     public boolean closed;
 
-    public MockSourceNode(String[] topics, Deserializer<K> keyDeserializer, Deserializer<V> valDeserializer) {
+    public MockSourceNode(final String[] topics, final Deserializer<K> keyDeserializer, final Deserializer<V> valDeserializer) {
         super(NAME + INDEX.getAndIncrement(), Arrays.asList(topics), keyDeserializer, valDeserializer);
     }
 
     @Override
-    public void process(K key, V value) {
+    public void process(final K key, final V value) {
         this.numReceived++;
         this.keys.add(key);
         this.values.add(value);
diff --git a/streams/src/test/java/org/apache/kafka/test/MockStateRestoreListener.java b/streams/src/test/java/org/apache/kafka/test/MockStateRestoreListener.java
index e7b328c..106299c 100644
--- a/streams/src/test/java/org/apache/kafka/test/MockStateRestoreListener.java
+++ b/streams/src/test/java/org/apache/kafka/test/MockStateRestoreListener.java
@@ -43,7 +43,7 @@ public class MockStateRestoreListener extends AbstractNotifyingRestoreCallback {
     public static final String RESTORE_END = "restore_end";
 
     @Override
-    public void restore(byte[] key, byte[] value) {
+    public void restore(final byte[] key, final byte[] value) {
         restored.add(KeyValue.pair(key, value));
     }
 
diff --git a/streams/src/test/java/org/apache/kafka/test/MockValueJoiner.java b/streams/src/test/java/org/apache/kafka/test/MockValueJoiner.java
index 9005ee0..b1842ff 100644
--- a/streams/src/test/java/org/apache/kafka/test/MockValueJoiner.java
+++ b/streams/src/test/java/org/apache/kafka/test/MockValueJoiner.java
@@ -25,7 +25,7 @@ public class MockValueJoiner {
     public static <V1, V2> ValueJoiner<V1, V2, String> instance(final String separator) {
         return new ValueJoiner<V1, V2, String>() {
             @Override
-            public String apply(V1 value1, V2 value2) {
+            public String apply(final V1 value1, final V2 value2) {
                 return value1 + separator + value2;
             }
         };
diff --git a/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java
index 521a50e..42ee8fb 100644
--- a/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java
+++ b/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java
@@ -52,7 +52,7 @@ public class NoOpProcessorContext extends AbstractProcessorContext {
         return null;
     }
 
-    @Override public Cancellable schedule(long interval, PunctuationType type, Punctuator callback) {
+    @Override public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) {
         return null;
     }
 
diff --git a/streams/src/test/java/org/apache/kafka/test/ReadOnlySessionStoreStub.java b/streams/src/test/java/org/apache/kafka/test/ReadOnlySessionStoreStub.java
index 0d196fe..3fa23d2 100644
--- a/streams/src/test/java/org/apache/kafka/test/ReadOnlySessionStoreStub.java
+++ b/streams/src/test/java/org/apache/kafka/test/ReadOnlySessionStoreStub.java
@@ -54,7 +54,7 @@ public class ReadOnlySessionStoreStub<K, V> implements ReadOnlySessionStore<K, V
     }
 
     @Override
-    public KeyValueIterator<Windowed<K>, V> fetch(K from, K to) {
+    public KeyValueIterator<Windowed<K>, V> fetch(final K from, final K to) {
         if (!open) {
             throw new InvalidStateStoreException("not open");
         }
diff --git a/streams/src/test/java/org/apache/kafka/test/SegmentedBytesStoreStub.java b/streams/src/test/java/org/apache/kafka/test/SegmentedBytesStoreStub.java
index 4709ef3..17c5e68 100644
--- a/streams/src/test/java/org/apache/kafka/test/SegmentedBytesStoreStub.java
+++ b/streams/src/test/java/org/apache/kafka/test/SegmentedBytesStoreStub.java
@@ -54,7 +54,7 @@ public class SegmentedBytesStoreStub implements SegmentedBytesStore {
     }
 
     @Override
-    public KeyValueIterator<Bytes, byte[]> fetch(Bytes keyFrom, Bytes keyTo, long from, long to) {
+    public KeyValueIterator<Bytes, byte[]> fetch(final Bytes keyFrom, final Bytes keyTo, final long from, final long to) {
         fetchCalled = true;
         return new KeyValueIteratorStub<>(Collections.<KeyValue<Bytes, byte[]>>emptyIterator());
     }
@@ -66,7 +66,7 @@ public class SegmentedBytesStoreStub implements SegmentedBytesStore {
     }
     
     @Override
-    public KeyValueIterator<Bytes, byte[]> fetchAll(long timeFrom, long timeTo) {
+    public KeyValueIterator<Bytes, byte[]> fetchAll(final long timeFrom, final long timeTo) {
         fetchCalled = true;
         return new KeyValueIteratorStub<>(Collections.<KeyValue<Bytes, byte[]>>emptyIterator());
     }


Mime
View raw message