Merge branch 'trunk' into 0.10.0
diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh
index f45d8d4..e7f8d2e 100755
--- a/bin/kafka-run-class.sh
+++ b/bin/kafka-run-class.sh
@@ -20,6 +20,24 @@
   exit 1
 fi
 
+if [ -z "$INCLUDE_TEST_JARS" ]; then
+  INCLUDE_TEST_JARS=false
+fi
+
+# Exclude jars not necessary for running commands.
+regex="(-(test|src|scaladoc|javadoc)\.jar|jar.asc)$"
+should_include_file() {
+  if [ "$INCLUDE_TEST_JARS" = true ]; then
+    return 0
+  fi
+  file=$1
+  if [ -z "$(echo "$file" | egrep "$regex")" ] ; then
+    return 0
+  else
+    return 1
+  fi
+}
+
 base_dir=$(dirname $0)/..
 
 if [ -z "$SCALA_VERSION" ]; then
@@ -34,27 +52,39 @@
 shopt -s nullglob
 for dir in $base_dir/core/build/dependant-libs-${SCALA_VERSION}*;
 do
-  CLASSPATH=$CLASSPATH:$dir/*
+  if [ -z $CLASSPATH ] ; then
+    CLASSPATH=$dir/*
+  else
+    CLASSPATH=$CLASSPATH:$dir/*
+  fi
 done
 
-for file in $base_dir/examples/build/libs//kafka-examples*.jar;
+for file in $base_dir/examples/build/libs/kafka-examples*.jar;
 do
-  CLASSPATH=$CLASSPATH:$file
+  if should_include_file "$file"; then
+    CLASSPATH=$CLASSPATH:$file
+  fi
 done
 
 for file in $base_dir/clients/build/libs/kafka-clients*.jar;
 do
-  CLASSPATH=$CLASSPATH:$file
+  if should_include_file "$file"; then
+    CLASSPATH=$CLASSPATH:$file
+  fi
 done
 
 for file in $base_dir/streams/build/libs/kafka-streams*.jar;
 do
-  CLASSPATH=$CLASSPATH:$file
+  if should_include_file "$file"; then
+    CLASSPATH=$CLASSPATH:$file
+  fi
 done
 
 for file in $base_dir/streams/examples/build/libs/kafka-streams-examples*.jar;
 do
-  CLASSPATH=$CLASSPATH:$file
+  if should_include_file "$file"; then
+    CLASSPATH=$CLASSPATH:$file
+  fi
 done
 
 for file in $base_dir/streams/build/dependant-libs-${SCALA_VERSION}/rocksdb*.jar;
@@ -64,7 +94,9 @@
 
 for file in $base_dir/tools/build/libs/kafka-tools*.jar;
 do
-  CLASSPATH=$CLASSPATH:$file
+  if should_include_file "$file"; then
+    CLASSPATH=$CLASSPATH:$file
+  fi
 done
 
 for dir in $base_dir/tools/build/dependant-libs-${SCALA_VERSION}*;
@@ -76,7 +108,9 @@
 do
   for file in $base_dir/connect/${cc_pkg}/build/libs/connect-${cc_pkg}*.jar;
   do
-    CLASSPATH=$CLASSPATH:$file
+    if should_include_file "$file"; then
+      CLASSPATH=$CLASSPATH:$file
+    fi
   done
   if [ -d "$base_dir/connect/${cc_pkg}/build/dependant-libs" ] ; then
     CLASSPATH=$CLASSPATH:$base_dir/connect/${cc_pkg}/build/dependant-libs/*
@@ -84,11 +118,18 @@
 done
 
 # classpath addition for release
-CLASSPATH=$CLASSPATH:$base_dir/libs/*
+for file in $base_dir/libs;
+do
+  if should_include_file "$file"; then
+    CLASSPATH=$CLASSPATH:$file
+  fi
+done
 
 for file in $base_dir/core/build/libs/kafka_${SCALA_BINARY_VERSION}*.jar;
 do
-  CLASSPATH=$CLASSPATH:$file
+  if should_include_file "$file"; then
+    CLASSPATH=$CLASSPATH:$file
+  fi
 done
 shopt -u nullglob
 
@@ -141,8 +182,6 @@
         JAVA_DEBUG_OPTS="$DEFAULT_JAVA_DEBUG_OPTS"
     fi
 
-
-
     echo "Enabling Java debug options: $JAVA_DEBUG_OPTS"
     KAFKA_OPTS="$JAVA_DEBUG_OPTS $KAFKA_OPTS"
 fi
diff --git a/build.gradle b/build.gradle
index da0152b..06c41d5 100644
--- a/build.gradle
+++ b/build.gradle
@@ -59,7 +59,7 @@
 }
 
 ext {
-  gradleVersion = "2.11"
+  gradleVersion = "2.13"
   buildVersionFileName = "kafka-version.properties"
 
   userMaxForks = project.hasProperty('maxParallelForks') ? maxParallelForks.toInteger() : null
@@ -669,6 +669,8 @@
     compile libs.jacksonDatabind // this dependency should be removed after KIP-4
 
     testCompile project(':clients').sourceSets.test.output
+    testCompile project(':core')
+    testCompile project(':core').sourceSets.test.output
     testCompile libs.junit
   }
 
diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml
index fb2042b..3adc446 100644
--- a/checkstyle/checkstyle.xml
+++ b/checkstyle/checkstyle.xml
@@ -84,5 +84,12 @@
     <module name="MethodParamPad"/>
     <module name="ParenPad"/>
     <module name="TypecastParenPad"/>
+
+    <!-- locale-sensitive methods should specify locale -->
+    <module name="Regexp">
+      <property name="format" value="\.to(Lower|Upper)Case\(\)"/>
+      <property name="illegalPattern" value="true"/>
+      <property name="ignoreComments" value="true"/>
+    </module>
   </module>
 </module>
diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index 051c8d1..7a45515 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -69,6 +69,13 @@
       <allow pkg="org.apache.kafka.common.annotation" />
       <allow pkg="org.apache.kafka.common.network" />
       <allow pkg="org.apache.kafka.common.config" />
+      <subpackage name="authenticator">
+        <allow pkg="org.apache.kafka.common.protocol" />
+        <allow pkg="org.apache.kafka.common.protocol.types" />
+        <allow pkg="org.apache.kafka.common.requests" />
+        <allow pkg="org.apache.kafka.common.errors" />
+        <allow pkg="org.apache.kafka.clients" />
+      </subpackage>
     </subpackage>
 
     <subpackage name="protocol">
@@ -131,6 +138,17 @@
       <allow pkg="com.fasterxml.jackson.databind" />
       <allow pkg="org.apache.kafka.connect.json" />
     </subpackage>
+    
+    <subpackage name="integration">
+      <allow pkg="kafka.admin" />
+      <allow pkg="kafka.server" />
+      <allow pkg="kafka.utils" />
+      <allow pkg="kafka.zk" />
+      <allow pkg="kafka.log" />
+      <allow pkg="scala" />
+      <allow pkg="scala.collection" />
+      <allow pkg="org.I0Itec.zkclient" />
+    </subpackage>
 
     <subpackage name="state">
       <allow pkg="org.rocksdb" />
@@ -203,6 +221,7 @@
 
     <subpackage name="util">
       <allow pkg="org.apache.kafka.connect" />
+      <allow pkg="org.reflections.vfs" />
       <!-- for annotations to avoid code duplication -->
       <allow pkg="com.fasterxml.jackson.annotation" />
     </subpackage>
diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
index 0201257..ad9c5d0 100644
--- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
+++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
@@ -25,6 +25,7 @@
 import org.apache.kafka.common.protocol.SecurityProtocol;
 import org.apache.kafka.common.network.ChannelBuilder;
 import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.SaslConfigs;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -76,7 +77,8 @@
         SecurityProtocol securityProtocol = SecurityProtocol.forName((String) configs.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG));
         if (!SecurityProtocol.nonTestingValues().contains(securityProtocol))
             throw new ConfigException("Invalid SecurityProtocol " + securityProtocol);
-        return ChannelBuilders.create(securityProtocol, Mode.CLIENT, LoginType.CLIENT, configs);
+        String clientSaslMechanism = (String) configs.get(SaslConfigs.SASL_MECHANISM);
+        return ChannelBuilders.create(securityProtocol, Mode.CLIENT, LoginType.CLIENT, configs, clientSaslMechanism, true);
     }
 
 }
diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java
index 298e1d8c..2f1fe93 100644
--- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java
+++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java
@@ -50,7 +50,7 @@
     public static final String RECONNECT_BACKOFF_MS_DOC = "The amount of time to wait before attempting to reconnect to a given host. This avoids repeatedly connecting to a host in a tight loop. This backoff applies to all requests sent by the consumer to the broker.";
 
     public static final String RETRY_BACKOFF_MS_CONFIG = "retry.backoff.ms";
-    public static final String RETRY_BACKOFF_MS_DOC = "The amount of time to wait before attempting to retry a failed fetch request to a given topic partition. This avoids repeated fetching-and-failing in a tight loop.";
+    public static final String RETRY_BACKOFF_MS_DOC = "The amount of time to wait before attempting to retry a failed request to a given topic partition. This avoids repeatedly sending requests in a tight loop under some failure scenarios.";
 
     public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = "metrics.sample.window.ms";
     public static final String METRICS_SAMPLE_WINDOW_MS_DOC = "The window of time a metrics sample is computed over.";
diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java
index 73a9f33..322ae0f 100644
--- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java
+++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java
@@ -108,6 +108,14 @@
     }
 
     /**
+     * Check whether an update has been explicitly requested.
+     * @return true if an update was requested, false otherwise
+     */
+    public synchronized boolean updateRequested() {
+        return this.needUpdate;
+    }
+
+    /**
      * Wait for metadata update until the current version is larger than the last version we know of
      */
     public synchronized void awaitUpdate(final int lastVersion, final long maxWaitMs) throws InterruptedException {
@@ -203,10 +211,10 @@
 
     /**
      * Set state to indicate if metadata for all topics in Kafka cluster is required or not.
-     * @param needMetadaForAllTopics boolean indicating need for metadata of all topics in cluster.
+     * @param needMetadataForAllTopics boolean indicating need for metadata of all topics in cluster.
      */
-    public synchronized void needMetadataForAllTopics(boolean needMetadaForAllTopics) {
-        this.needMetadataForAllTopics = needMetadaForAllTopics;
+    public synchronized void needMetadataForAllTopics(boolean needMetadataForAllTopics) {
+        this.needMetadataForAllTopics = needMetadataForAllTopics;
     }
 
     /**
diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
index d2eaace..b134631 100644
--- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
@@ -33,12 +33,11 @@
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
-import java.util.Set;
 
 /**
  * A network client for asynchronous request/response network i/o. This is an internal class used to implement the
@@ -52,7 +51,7 @@
 
     /* the selector used to perform network i/o */
     private final Selectable selector;
-    
+
     private final MetadataUpdater metadataUpdater;
 
     private final Random randOffset;
@@ -77,7 +76,7 @@
 
     /* max time in ms for the producer to wait for acknowledgement from server*/
     private final int requestTimeoutMs;
-    
+
     private final Time time;
 
     public NetworkClient(Selectable selector,
@@ -113,7 +112,7 @@
                           int maxInFlightRequestsPerConnection,
                           long reconnectBackoffMs,
                           int socketSendBuffer,
-                          int socketReceiveBuffer, 
+                          int socketReceiveBuffer,
                           int requestTimeoutMs,
                           Time time) {
 
@@ -369,10 +368,20 @@
                 found = node;
             }
         }
-        
+
         return found;
     }
 
+    public static Struct parseResponse(ByteBuffer responseBuffer, RequestHeader requestHeader) {
+        ResponseHeader responseHeader = ResponseHeader.parse(responseBuffer);
+        // Always expect the response version id to be the same as the request version id
+        short apiKey = requestHeader.apiKey();
+        short apiVer = requestHeader.apiVersion();
+        Struct responseBody = ProtoUtils.responseSchema(apiKey, apiVer).read(responseBuffer);
+        correlate(requestHeader, responseHeader);
+        return responseBody;
+    }
+
     /**
      * Post process disconnection of a node
      *
@@ -437,12 +446,7 @@
         for (NetworkReceive receive : this.selector.completedReceives()) {
             String source = receive.source();
             ClientRequest req = inFlightRequests.completeNext(source);
-            ResponseHeader header = ResponseHeader.parse(receive.payload());
-            // Always expect the response version id to be the same as the request version id
-            short apiKey = req.request().header().apiKey();
-            short apiVer = req.request().header().apiVersion();
-            Struct body = ProtoUtils.responseSchema(apiKey, apiVer).read(receive.payload());
-            correlate(req.request().header(), header);
+            Struct body = parseResponse(receive.payload(), req.request().header());
             if (!metadataUpdater.maybeHandleCompletedReceive(req, now, body))
                 responses.add(new ClientResponse(req, now, false, body));
         }
@@ -477,7 +481,7 @@
     /**
      * Validate that the response corresponds to the request we expect or else explode
      */
-    private void correlate(RequestHeader requestHeader, ResponseHeader responseHeader) {
+    private static void correlate(RequestHeader requestHeader, ResponseHeader responseHeader) {
         if (requestHeader.correlationId() != responseHeader.correlationId())
             throw new IllegalStateException("Correlation id for response (" + responseHeader.correlationId()
                     + ") does not match request (" + requestHeader.correlationId() + ")");
@@ -540,7 +544,7 @@
             // if there is no node available to connect, back off refreshing metadata
             long metadataTimeout = Math.max(Math.max(timeToNextMetadataUpdate, timeToNextReconnectAttempt),
                     waitForMetadataFetch);
- 
+
             if (metadataTimeout == 0) {
                 // Beware that the behavior of this method and the computation of timeouts for poll() are
                 // highly dependent on the behavior of leastLoadedNode.
@@ -608,8 +612,7 @@
         /**
          * Create a metadata request for the given topics
          */
-        private ClientRequest request(long now, String node, Set<String> topics) {
-            MetadataRequest metadata = new MetadataRequest(new ArrayList<>(topics));
+        private ClientRequest request(long now, String node, MetadataRequest metadata) {
             RequestSend send = new RequestSend(node, nextRequestHeader(ApiKeys.METADATA), metadata.toStruct());
             return new ClientRequest(now, true, send, null, true);
         }
@@ -627,11 +630,15 @@
             String nodeConnectionId = node.idString();
 
             if (canSendRequest(nodeConnectionId)) {
-                Set<String> topics = metadata.needMetadataForAllTopics() ? new HashSet<String>() : metadata.topics();
                 this.metadataFetchInProgress = true;
-                ClientRequest metadataRequest = request(now, nodeConnectionId, topics);
+                MetadataRequest metadataRequest;
+                if (metadata.needMetadataForAllTopics())
+                    metadataRequest = MetadataRequest.allTopics();
+                else
+                    metadataRequest = new MetadataRequest(new ArrayList<>(metadata.topics()));
+                ClientRequest clientRequest = request(now, nodeConnectionId, metadataRequest);
                 log.debug("Sending metadata request {} to node {}", metadataRequest, node.id());
-                doSend(metadataRequest, now);
+                doSend(clientRequest, now);
             } else if (connectionStates.canConnect(nodeConnectionId, now)) {
                 // we don't have a connection to this node right now, make one
                 log.debug("Initialize connection to node {} for sending metadata request", node.id());
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
index 5576431..7290a38 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
@@ -35,6 +35,7 @@
 import org.apache.kafka.common.metrics.MetricsReporter;
 import org.apache.kafka.common.network.ChannelBuilder;
 import org.apache.kafka.common.network.Selector;
+import org.apache.kafka.common.requests.MetadataRequest;
 import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.kafka.common.utils.AppInfoParser;
 import org.apache.kafka.common.utils.SystemTime;
@@ -51,6 +52,7 @@
 import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
@@ -74,7 +76,7 @@
  * <h3>Offsets and Consumer Position</h3>
  * Kafka maintains a numerical offset for each record in a partition. This offset acts as a kind of unique identifier of
  * a record within that partition, and also denotes the position of the consumer in the partition. That is, a consumer
- * which has position 5 has consumed records with offsets 0 through 4 and will next receive record with offset 5. There
+ * which has position 5 has consumed records with offsets 0 through 4 and will next receive the record with offset 5. There
  * are actually two notions of position relevant to the user of the consumer.
  * <p>
  * The {@link #position(TopicPartition) position} of the consumer gives the offset of the next record that will be given
@@ -93,22 +95,23 @@
  *
  * <h3>Consumer Groups and Topic Subscriptions</h3>
  *
- * Kafka uses the concept of <i>consumer groups</i> to allow a pool of processes to divide up the work of consuming and
+ * Kafka uses the concept of <i>consumer groups</i> to allow a pool of processes to divide the work of consuming and
  * processing records. These processes can either be running on the same machine or, as is more likely, they can be
- * distributed over many machines to provide additional scalability and fault tolerance for processing.
+ * distributed over many machines to provide scalability and fault tolerance for processing.
  * <p>
- * Each Kafka consumer is able to configure a consumer group that it belongs to, and can dynamically set the
- * list of topics it wants to subscribe to through {@link #subscribe(Collection, ConsumerRebalanceListener)},
- * or subscribe to all topics matching certain pattern through {@link #subscribe(Pattern, ConsumerRebalanceListener)}.
- * Kafka will deliver each message in the
- * subscribed topics to one process in each consumer group. This is achieved by balancing the partitions in the topic
- * over the consumer processes in each group. So if there is a topic with four partitions, and a consumer group with two
- * processes, each process would consume from two partitions. This group membership is maintained dynamically: if a
- * process fails the partitions assigned to it will be reassigned to other processes in the same group, and if a new
- * process joins the group, partitions will be moved from existing consumers to this new process.
+ * Each Kafka consumer is able to configure a consumer group that it belongs to, and can dynamically set the list
+ * of topics it wants to subscribe to through one of the {@link #subscribe(Collection, ConsumerRebalanceListener) subscribe}
+ * APIs. Kafka will deliver each message in the subscribed topics to one process in each consumer group.
+ * This is achieved by balancing the partitions between all members in the consumer group so that each partition is
+ * assigned to exactly one consumer in the group. So if there is a topic with four partitions, and a consumer group with two
+ * processes, each process would consume from two partitions.
  * <p>
- * So if two processes subscribe to a topic both specifying different groups they will each get all the records in that
- * topic; if they both specify the same group they will each get about half the records.
+ * Membership in a consumer group is maintained dynamically: if a process fails, the partitions assigned to it will
+ * be reassigned to other consumers in the same group. Similarly, if a new consumer joins the group, partitions will be moved
+ * from existing consumers to the new one. This is known as <i>rebalancing</i> the group and is discussed in more
+ * detail <a href="#failuredetection">below</a>. Note that the same process is also used when new partitions are added
+ * to one of the subscribed topics: the group automatically detects the new partitions and rebalances the group so
+ * that every new partition is assigned to one of the members.
  * <p>
  * Conceptually you can think of a consumer group as being a single logical subscriber that happens to be made up of
  * multiple processes. As a multi-subscriber system, Kafka naturally supports having any number of consumer groups for a
@@ -129,6 +132,48 @@
  * (similar to the older "simple" consumer) using {@link #assign(Collection)}. In this case, dynamic partition
  * assignment and consumer group coordination will be disabled.
  *
+ * <h3><a name="failuredetection">Detecting Consumer Failures</a></h3>
+ *
+ * After subscribing to a set of topics, the consumer will automatically join the group when {@link #poll(long)} is
+ * invoked. The poll API is designed to ensure consumer liveness. As long as you continue to call poll, the consumer
+ * will stay in the group and continue to receive messages from the partitions it was assigned. Underneath the covers,
+ * the poll API sends periodic heartbeats to the server; when you stop calling poll (perhaps because an exception was thrown),
+ * then no heartbeats will be sent. If a period of the configured <i>session timeout</i> elapses before the server
+ * has received a heartbeat, then the consumer will be kicked out of the group and its partitions will be reassigned.
+ * This is designed to prevent situations where the consumer has failed, yet continues to hold onto the partitions
+ * it was assigned (thus preventing active consumers in the group from taking them). To stay in the group, you
+ * have to prove you are still alive by calling poll.
+ * <p>
+ * The implication of this design is that message processing time in the poll loop must be bounded so that
+ * heartbeats can be sent before expiration of the session timeout. What typically happens when processing time
+ * exceeds the session timeout is that the consumer won't be able to commit offsets for any of the processed records.
+ * For example, this is indicated by a {@link CommitFailedException} thrown from {@link #commitSync()}. This
+ * guarantees that only active members of the group are allowed to commit offsets. If the consumer
+ * has been kicked out of the group, then its partitions will have been assigned to another member, which will be
+ * committing its own offsets as it handles new records. This gives offset commits an isolation guarantee.
+ * <p>
+ * The consumer provides two configuration settings to control this behavior:
+ * <ol>
+ *     <li><code>session.timeout.ms</code>: By increasing the session timeout, you can give the consumer more
+ *     time to handle a batch of records returned from {@link #poll(long)}. The only drawback is that it
+ *     will take longer for the server to detect hard consumer failures, which can cause a delay before
+ *     a rebalance can be completed. However, clean shutdown with {@link #close()} is not impacted since
+ *     the consumer will send an explicit message to the server to leave the group and cause an immediate
+ *     rebalance.</li>
+ *     <li><code>max.poll.records</code>: Processing time in the poll loop is typically proportional to the number
+ *     of records processed, so it's natural to want to set a limit on the number of records handled at once.
+ *     This setting provides that. By default, there is essentially no limit.</li>
+ * </ol>
+ * <p>
+ * For use cases where message processing time varies unpredictably, neither of these options may be viable.
+ * The recommended way to handle these cases is to move message processing to another thread, which allows
+ * the consumer to continue sending heartbeats while the processor is still working. Some care must be taken
+ * to ensure that committed offsets do not get ahead of the actual position. Typically, you must disable automatic
+ * commits and manually commit processed offsets for records only after the thread has finished handling them
+ * (depending on the delivery semantics you need). Note also that you will generally need to {@link #pause(Collection)}
+ * the partition so that no new records are received from poll until after thread has finished handling those
+ * previously returned.
+ *
  * <h3>Usage Examples</h3>
  * The consumer APIs offer flexibility to cover a variety of consumption use cases. Here are some examples to
  * demonstrate how to use them.
@@ -586,7 +631,7 @@
                     config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG), time);
             this.client = new ConsumerNetworkClient(netClient, metadata, time, retryBackoffMs,
                     config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG));
-            OffsetResetStrategy offsetResetStrategy = OffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase());
+            OffsetResetStrategy offsetResetStrategy = OffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase(Locale.ROOT));
             this.subscriptions = new SubscriptionState(offsetResetStrategy);
             List<PartitionAssignor> assignors = config.getConfiguredInstances(
                     ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG,
@@ -1189,7 +1234,7 @@
             if (parts != null)
                 return parts;
 
-            Map<String, List<PartitionInfo>> topicMetadata = fetcher.getTopicMetadata(Collections.singletonList(topic), requestTimeoutMs);
+            Map<String, List<PartitionInfo>> topicMetadata = fetcher.getTopicMetadata(new MetadataRequest(Collections.singletonList(topic)), requestTimeoutMs);
             return topicMetadata.get(topic);
         } finally {
             release();
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
index 496a114..15185d7 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
@@ -410,10 +410,10 @@
         if (coordinatorUnknown())
             return RequestFuture.coordinatorNotAvailable();
         return client.send(coordinator, ApiKeys.SYNC_GROUP, request)
-                .compose(new SyncGroupRequestHandler());
+                .compose(new SyncGroupResponseHandler());
     }
 
-    private class SyncGroupRequestHandler extends CoordinatorResponseHandler<SyncGroupResponse, ByteBuffer> {
+    private class SyncGroupResponseHandler extends CoordinatorResponseHandler<SyncGroupResponse, ByteBuffer> {
 
         @Override
         public SyncGroupResponse parse(ClientResponse response) {
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
index 86b60d0..887f47c 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
@@ -63,7 +63,6 @@
 
     private final List<PartitionAssignor> assignors;
     private final org.apache.kafka.clients.Metadata metadata;
-    private final MetadataSnapshot metadataSnapshot;
     private final ConsumerCoordinatorMetrics sensors;
     private final SubscriptionState subscriptions;
     private final OffsetCommitCallback defaultOffsetCommitCallback;
@@ -72,6 +71,9 @@
     private final ConsumerInterceptors<?, ?> interceptors;
     private final boolean excludeInternalTopics;
 
+    private MetadataSnapshot metadataSnapshot;
+    private MetadataSnapshot assignmentSnapshot;
+
     /**
      * Initialize the coordination manager.
      */
@@ -102,7 +104,7 @@
         this.metadata = metadata;
 
         this.metadata.requestUpdate();
-        this.metadataSnapshot = new MetadataSnapshot();
+        this.metadataSnapshot = new MetadataSnapshot(subscriptions, metadata.fetch());
         this.subscriptions = subscriptions;
         this.defaultOffsetCommitCallback = defaultOffsetCommitCallback;
         this.autoCommitEnabled = autoCommitEnabled;
@@ -159,8 +161,14 @@
                 }
 
                 // check if there are any changes to the metadata which should trigger a rebalance
-                if (metadataSnapshot.update(subscriptions, cluster) && subscriptions.partitionsAutoAssigned())
-                    subscriptions.needReassignment();
+                if (subscriptions.partitionsAutoAssigned()) {
+                    MetadataSnapshot snapshot = new MetadataSnapshot(subscriptions, cluster);
+                    if (!snapshot.equals(metadataSnapshot)) {
+                        metadataSnapshot = snapshot;
+                        subscriptions.needReassignment();
+                    }
+                }
+
             }
         });
     }
@@ -178,6 +186,13 @@
                                   String memberId,
                                   String assignmentStrategy,
                                   ByteBuffer assignmentBuffer) {
+        // if we were the assignor, then we need to make sure that there have been no metadata updates
+        // since the rebalance begin. Otherwise, we won't rebalance again until the next metadata change
+        if (assignmentSnapshot != null && !assignmentSnapshot.equals(metadataSnapshot)) {
+            subscriptions.needReassignment();
+            return;
+        }
+
         PartitionAssignor assignor = lookupAssignor(assignmentStrategy);
         if (assignor == null)
             throw new IllegalStateException("Coordinator selected invalid assignment protocol: " + assignmentStrategy);
@@ -231,7 +246,11 @@
         // which ensures that all metadata changes will eventually be seen
         this.subscriptions.groupSubscribe(allSubscribedTopics);
         metadata.setTopics(this.subscriptions.groupSubscription());
+
+        // update metadata (if needed) and keep track of the metadata used for assignment so that
+        // we can check after rebalance completion whether anything has changed
         client.ensureFreshMetadata();
+        assignmentSnapshot = metadataSnapshot;
 
         log.debug("Performing assignment for group {} using strategy {} with subscriptions {}",
                 groupId, assignor.name(), subscriptions);
@@ -267,6 +286,7 @@
                     listener.getClass().getName(), groupId, e);
         }
 
+        assignmentSnapshot = null;
         subscriptions.needReassignment();
     }
 
@@ -669,19 +689,26 @@
     }
 
     private static class MetadataSnapshot {
-        private Map<String, Integer> partitionsPerTopic = new HashMap<>();
+        private final Map<String, Integer> partitionsPerTopic;
 
-        public boolean update(SubscriptionState subscription, Cluster cluster) {
+        public MetadataSnapshot(SubscriptionState subscription, Cluster cluster) {
             Map<String, Integer> partitionsPerTopic = new HashMap<>();
             for (String topic : subscription.groupSubscription())
                 partitionsPerTopic.put(topic, cluster.partitionCountForTopic(topic));
+            this.partitionsPerTopic = partitionsPerTopic;
+        }
 
-            if (!partitionsPerTopic.equals(this.partitionsPerTopic)) {
-                this.partitionsPerTopic = partitionsPerTopic;
-                return true;
-            }
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            MetadataSnapshot that = (MetadataSnapshot) o;
+            return partitionsPerTopic != null ? partitionsPerTopic.equals(that.partitionsPerTopic) : that.partitionsPerTopic == null;
+        }
 
-            return false;
+        @Override
+        public int hashCode() {
+            return partitionsPerTopic != null ? partitionsPerTopic.hashCode() : 0;
         }
     }
 
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java
index 4119954..d4c2656 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java
@@ -138,7 +138,7 @@
      * until it has completed).
      */
     public void ensureFreshMetadata() {
-        if (this.metadata.timeToNextUpdate(time.milliseconds()) == 0)
+        if (this.metadata.updateRequested() || this.metadata.timeToNextUpdate(time.milliseconds()) == 0)
             awaitMetadataUpdate();
     }
 
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
index 9a26551..f6d3387 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
@@ -61,6 +61,7 @@
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
 
@@ -182,25 +183,26 @@
      * @return The map of topics with their partition information
      */
     public Map<String, List<PartitionInfo>> getAllTopicMetadata(long timeout) {
-        return getTopicMetadata(null, timeout);
+        return getTopicMetadata(MetadataRequest.allTopics(), timeout);
     }
 
     /**
      * Get metadata for all topics present in Kafka cluster
      *
-     * @param topics The list of topics to fetch or null to fetch all
+     * @param request The MetadataRequest to send
      * @param timeout time for which getting topic metadata is attempted
      * @return The map of topics with their partition information
      */
-    public Map<String, List<PartitionInfo>> getTopicMetadata(List<String> topics, long timeout) {
-        if (topics != null && topics.isEmpty())
+    public Map<String, List<PartitionInfo>> getTopicMetadata(MetadataRequest request, long timeout) {
+        // Save the round trip if no topics are requested.
+        if (!request.isAllTopics() && request.topics().isEmpty())
             return Collections.emptyMap();
 
         long start = time.milliseconds();
         long remaining = timeout;
 
         do {
-            RequestFuture<ClientResponse> future = sendMetadataRequest(topics);
+            RequestFuture<ClientResponse> future = sendMetadataRequest(request);
             client.poll(future, remaining);
 
             if (future.failed() && !future.isRetriable())
@@ -265,14 +267,12 @@
      * Send Metadata Request to least loaded node in Kafka cluster asynchronously
      * @return A future that indicates result of sent metadata request
      */
-    private RequestFuture<ClientResponse> sendMetadataRequest(List<String> topics) {
-        if (topics == null)
-            topics = Collections.emptyList();
+    private RequestFuture<ClientResponse> sendMetadataRequest(MetadataRequest request) {
         final Node node = client.leastLoadedNode();
         if (node == null)
             return RequestFuture.noBrokersAvailable();
         else
-            return client.send(node, ApiKeys.METADATA, new MetadataRequest(topics));
+            return client.send(node, ApiKeys.METADATA, request);
     }
 
     /**
@@ -291,7 +291,7 @@
         else
             throw new NoOffsetForPartitionException(partition);
 
-        log.debug("Resetting offset for partition {} to {} offset.", partition, strategy.name().toLowerCase());
+        log.debug("Resetting offset for partition {} to {} offset.", partition, strategy.name().toLowerCase(Locale.ROOT));
         long offset = listOffset(partition, timestamp);
 
         // we might lose the assignment while fetching the offset, so check it is still active
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
index d60e28e..5b05272 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
@@ -73,7 +73,7 @@
  * <pre>
  * {@code
  * Properties props = new Properties();
- * props.put("bootstrap.servers", "localhost:4242");
+ * props.put("bootstrap.servers", "localhost:9092");
  * props.put("acks", "all");
  * props.put("retries", 0);
  * props.put("batch.size", 16384);
@@ -338,7 +338,7 @@
 
     private static int parseAcks(String acksString) {
         try {
-            return acksString.trim().toLowerCase().equals("all") ? -1 : Integer.parseInt(acksString.trim());
+            return acksString.trim().equalsIgnoreCase("all") ? -1 : Integer.parseInt(acksString.trim());
         } catch (NumberFormatException e) {
             throw new ConfigException("Invalid configuration value for 'acks': " + acksString);
         }
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java
index 988da16..00a4374 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java
@@ -17,6 +17,7 @@
 package org.apache.kafka.clients.producer;
 
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.record.Record;
 
 /**
  * The metadata for a record that has been acknowledged by the server
@@ -51,6 +52,11 @@
         this.topicPartition = topicPartition;
     }
 
+    @Deprecated
+    public RecordMetadata(TopicPartition topicPartition, long baseOffset, long relativeOffset) {
+        this(topicPartition, baseOffset, relativeOffset, Record.NO_TIMESTAMP, -1, -1, -1);
+    }
+
     public RecordMetadata(TopicPartition topicPartition, long baseOffset, long relativeOffset,
                           long timestamp, long checksum, int serializedKeySize, int serializedValueSize) {
         // ignore the relativeOffset if the base offset is -1,
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
index 7f5b16f..1766609 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
@@ -167,7 +167,7 @@
         appendsInProgress.incrementAndGet();
         try {
             // check if we have an in-progress batch
-            Deque<RecordBatch> dq = dequeFor(tp);
+            Deque<RecordBatch> dq = getOrCreateDeque(tp);
             synchronized (dq) {
                 if (closed)
                     throw new IllegalStateException("Cannot send after the producer is closed.");
@@ -213,7 +213,7 @@
      * Abort the batches that have been sitting in RecordAccumulator for more than the configured requestTimeout
      * due to metadata being unavailable
      */
-    public List<RecordBatch> abortExpiredBatches(int requestTimeout, Cluster cluster, long now) {
+    public List<RecordBatch> abortExpiredBatches(int requestTimeout, long now) {
         List<RecordBatch> expiredBatches = new ArrayList<RecordBatch>();
         int count = 0;
         for (Map.Entry<TopicPartition, Deque<RecordBatch>> entry : this.batches.entrySet()) {
@@ -259,7 +259,7 @@
         batch.lastAttemptMs = now;
         batch.lastAppendTime = now;
         batch.setRetry();
-        Deque<RecordBatch> deque = dequeFor(batch.topicPartition);
+        Deque<RecordBatch> deque = getOrCreateDeque(batch.topicPartition);
         synchronized (deque) {
             deque.addFirst(batch);
         }
@@ -369,7 +369,7 @@
                 TopicPartition tp = new TopicPartition(part.topic(), part.partition());
                 // Only proceed if the partition has no in-flight batches.
                 if (!muted.contains(tp)) {
-                    Deque<RecordBatch> deque = dequeFor(new TopicPartition(part.topic(), part.partition()));
+                    Deque<RecordBatch> deque = getDeque(new TopicPartition(part.topic(), part.partition()));
                     if (deque != null) {
                         synchronized (deque) {
                             RecordBatch first = deque.peekFirst();
@@ -401,10 +401,14 @@
         return batches;
     }
 
+    private Deque<RecordBatch> getDeque(TopicPartition tp) {
+        return batches.get(tp);
+    }
+
     /**
      * Get the deque for the given topic-partition, creating it if necessary.
      */
-    private Deque<RecordBatch> dequeFor(TopicPartition tp) {
+    private Deque<RecordBatch> getOrCreateDeque(TopicPartition tp) {
         Deque<RecordBatch> d = this.batches.get(tp);
         if (d != null)
             return d;
@@ -467,7 +471,7 @@
             abortBatches();
         } while (appendsInProgress());
         // After this point, no thread will append any messages because they will see the close
-        // flag set. We need to do the last abort after no thread was appending in case the there was a new
+        // flag set. We need to do the last abort after no thread was appending in case there was a new
         // batch appended by the last appending thread.
         abortBatches();
         this.batches.clear();
@@ -478,7 +482,7 @@
      */
     private void abortBatches() {
         for (RecordBatch batch : incomplete.all()) {
-            Deque<RecordBatch> dq = dequeFor(batch.topicPartition);
+            Deque<RecordBatch> dq = getDeque(batch.topicPartition);
             // Close the batch before aborting
             synchronized (dq) {
                 batch.records.close();
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
index db8918c..29077b6 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
@@ -202,7 +202,7 @@
             }
         }
 
-        List<RecordBatch> expiredBatches = this.accumulator.abortExpiredBatches(this.requestTimeout, cluster, now);
+        List<RecordBatch> expiredBatches = this.accumulator.abortExpiredBatches(this.requestTimeout, now);
         // update sensors
         for (RecordBatch expiredBatch : expiredBatches)
             this.sensors.recordErrors(expiredBatch.topicPartition.topic(), expiredBatch.recordCount);
diff --git a/clients/src/main/java/org/apache/kafka/common/Node.java b/clients/src/main/java/org/apache/kafka/common/Node.java
index 6c3fd0b..f569ddd 100644
--- a/clients/src/main/java/org/apache/kafka/common/Node.java
+++ b/clients/src/main/java/org/apache/kafka/common/Node.java
@@ -3,9 +3,9 @@
  * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
  * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
  * License. You may obtain a copy of the License at
- * 
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
  * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
  * specific language governing permissions and limitations under the License.
@@ -23,13 +23,19 @@
     private final String idString;
     private final String host;
     private final int port;
+    private final String rack;
 
     public Node(int id, String host, int port) {
+        this(id, host, port, null);
+    }
+
+    public Node(int id, String host, int port, String rack) {
         super();
         this.id = id;
         this.idString = Integer.toString(id);
         this.host = host;
         this.port = port;
+        this.rack = rack;
     }
 
     public static Node noNode() {
@@ -74,6 +80,20 @@
         return port;
     }
 
+    /**
+     * True if this node has a defined rack
+     */
+    public boolean hasRack() {
+        return rack != null;
+    }
+
+    /**
+     * The rack for this node
+     */
+    public String rack() {
+        return rack;
+    }
+
     @Override
     public int hashCode() {
         final int prime = 31;
@@ -81,6 +101,7 @@
         result = prime * result + ((host == null) ? 0 : host.hashCode());
         result = prime * result + id;
         result = prime * result + port;
+        result = prime * result + ((rack == null) ? 0 : rack.hashCode());
         return result;
     }
 
@@ -102,12 +123,17 @@
             return false;
         if (port != other.port)
             return false;
+        if (rack == null) {
+            if (other.rack != null)
+                return false;
+        } else if (!rack.equals(other.rack))
+            return false;
         return true;
     }
 
     @Override
     public String toString() {
-        return host + ":" + port + " (id: " + idString + ")";
+        return host + ":" + port + " (id: " + idString + " rack: " + rack + ")";
     }
 
 }
diff --git a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
index f833d7e..8e36f40 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
@@ -137,7 +137,8 @@
         Map<String, String> copy = new RecordingMap<>();
         for (Map.Entry<String, ?> entry : originals.entrySet()) {
             if (!(entry.getValue() instanceof String))
-                throw new ClassCastException("Non-string value found in original settings");
+                throw new ClassCastException("Non-string value found in original settings for key " + entry.getKey() +
+                        ": " + (entry.getValue() == null ? null : entry.getValue().getClass().getName()));
             copy.put(entry.getKey(), (String) entry.getValue());
         }
         return copy;
diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
index 881cb0b..3a562ce 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
@@ -24,6 +24,7 @@
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
 
@@ -538,19 +539,18 @@
         }
     }
 
+    @SuppressWarnings("unchecked")
     private void validate(String name, Map<String, Object> parsed, Map<String, ConfigValue> configs) {
         if (!configKeys.containsKey(name)) {
             return;
         }
         ConfigKey key = configKeys.get(name);
         ConfigValue config = configs.get(name);
-        Object value = parsed.get(name);
         List<Object> recommendedValues;
         if (key.recommender != null) {
             try {
                 recommendedValues = key.recommender.validValues(name, parsed);
                 List<Object> originalRecommendedValues = config.recommendedValues();
-
                 if (!originalRecommendedValues.isEmpty()) {
                     Set<Object> originalRecommendedValueSet = new HashSet<>(originalRecommendedValues);
                     Iterator<Object> it = recommendedValues.iterator();
@@ -562,9 +562,6 @@
                     }
                 }
                 config.recommendedValues(recommendedValues);
-                if (value != null && !recommendedValues.isEmpty() && !recommendedValues.contains(value)) {
-                    config.addErrorMessage("Invalid value for configuration " + key.name);
-                }
                 config.visible(key.recommender.visible(name, parsed));
             } catch (ConfigException e) {
                 config.addErrorMessage(e.getMessage());
@@ -676,6 +673,35 @@
         }
     }
 
+    public static String convertToString(Object parsedValue, Type type) {
+        if (parsedValue == null) {
+            return null;
+        }
+
+        if (type == null) {
+            return parsedValue.toString();
+        }
+
+        switch (type) {
+            case BOOLEAN:
+            case SHORT:
+            case INT:
+            case LONG:
+            case DOUBLE:
+            case STRING:
+            case PASSWORD:
+                return parsedValue.toString();
+            case LIST:
+                List<?> valueList = (List<?>) parsedValue;
+                return Utils.join(valueList, ",");
+            case CLASS:
+                Class<?> clazz = (Class<?>) parsedValue;
+                return clazz.getCanonicalName();
+            default:
+                throw new IllegalStateException("Unknown type.");
+        }
+    }
+
     /**
      * The config types
      */
@@ -866,7 +892,7 @@
             b.append(def.documentation);
             b.append("</td>");
             b.append("<td>");
-            b.append(def.type.toString().toLowerCase());
+            b.append(def.type.toString().toLowerCase(Locale.ROOT));
             b.append("</td>");
             b.append("<td>");
             if (def.hasDefault()) {
@@ -883,7 +909,7 @@
             b.append(def.validator != null ? def.validator.toString() : "");
             b.append("</td>");
             b.append("<td>");
-            b.append(def.importance.toString().toLowerCase());
+            b.append(def.importance.toString().toLowerCase(Locale.ROOT));
             b.append("</td>");
             b.append("</tr>\n");
         }
@@ -912,7 +938,7 @@
                 b.append("\n\n");
             }
             b.append("  * Type: ");
-            b.append(def.type.toString().toLowerCase());
+            b.append(def.type.toString().toLowerCase(Locale.ROOT));
             b.append("\n");
             if (def.defaultValue != null) {
                 b.append("  * Default: ");
@@ -926,7 +952,7 @@
                 b.append("\n");
             }
             b.append("  * Importance: ");
-            b.append(def.importance.toString().toLowerCase());
+            b.append(def.importance.toString().toLowerCase(Locale.ROOT));
             b.append("\n\n");
         }
         return b.toString();
diff --git a/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java
index d61838f..d3aa0d6 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java
@@ -20,6 +20,17 @@
     /*
      * NOTE: DO NOT CHANGE EITHER CONFIG NAMES AS THESE ARE PART OF THE PUBLIC API AND CHANGE WILL BREAK USER CODE.
      */
+    /** SASL mechanism configuration - standard mechanism names are listed <a href="http://www.iana.org/assignments/sasl-mechanisms/sasl-mechanisms.xhtml">here</a>. */
+    public static final String SASL_MECHANISM = "sasl.mechanism";
+    public static final String SASL_MECHANISM_DOC = "SASL mechanism used for client connections. This may be any mechanism for which a security provider is available. GSSAPI is the default mechanism.";
+    public static final String GSSAPI_MECHANISM = "GSSAPI";
+    public static final String DEFAULT_SASL_MECHANISM = GSSAPI_MECHANISM;
+
+    public static final String SASL_ENABLED_MECHANISMS = "sasl.enabled.mechanisms";
+    public static final String SASL_ENABLED_MECHANISMS_DOC = "The list of SASL mechanisms enabled in the Kafka server. "
+        + "The list may contain any mechanism for which a security provider is available. "
+        + "Only GSSAPI is enabled by default.";
+    public static final List<String> DEFAULT_SASL_ENABLED_MECHANISMS = Collections.singletonList(GSSAPI_MECHANISM);
 
     public static final String SASL_KERBEROS_SERVICE_NAME = "sasl.kerberos.service.name";
     public static final String SASL_KERBEROS_SERVICE_NAME_DOC = "The Kerberos principal name that Kafka runs as. "
@@ -54,7 +65,7 @@
                 .define(SaslConfigs.SASL_KERBEROS_KINIT_CMD, ConfigDef.Type.STRING, SaslConfigs.DEFAULT_KERBEROS_KINIT_CMD, ConfigDef.Importance.LOW, SaslConfigs.SASL_KERBEROS_KINIT_CMD_DOC)
                 .define(SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR, ConfigDef.Type.DOUBLE, SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_WINDOW_FACTOR, ConfigDef.Importance.LOW, SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_DOC)
                 .define(SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER, ConfigDef.Type.DOUBLE, SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_JITTER, ConfigDef.Importance.LOW, SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER_DOC)
-                .define(SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN, ConfigDef.Type.LONG, SaslConfigs.DEFAULT_KERBEROS_MIN_TIME_BEFORE_RELOGIN, ConfigDef.Importance.LOW, SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_DOC);
+                .define(SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN, ConfigDef.Type.LONG, SaslConfigs.DEFAULT_KERBEROS_MIN_TIME_BEFORE_RELOGIN, ConfigDef.Importance.LOW, SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_DOC)
+                .define(SaslConfigs.SASL_MECHANISM, ConfigDef.Type.STRING, SaslConfigs.DEFAULT_SASL_MECHANISM, ConfigDef.Importance.MEDIUM, SaslConfigs.SASL_MECHANISM_DOC);
     }
-
 }
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/AuthenticationException.java b/clients/src/main/java/org/apache/kafka/common/errors/AuthenticationException.java
new file mode 100644
index 0000000..7b60e11
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/errors/AuthenticationException.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.errors;
+
+public class AuthenticationException extends ApiException {
+
+    private static final long serialVersionUID = 1L;
+
+    public AuthenticationException(String message) {
+        super(message);
+    }
+
+    public AuthenticationException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/IllegalSaslStateException.java b/clients/src/main/java/org/apache/kafka/common/errors/IllegalSaslStateException.java
new file mode 100644
index 0000000..7fd008c
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/errors/IllegalSaslStateException.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.errors;
+
+public class IllegalSaslStateException extends AuthenticationException {
+
+    private static final long serialVersionUID = 1L;
+
+    public IllegalSaslStateException(String message) {
+        super(message);
+    }
+
+    public IllegalSaslStateException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedSaslMechanismException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedSaslMechanismException.java
new file mode 100644
index 0000000..289a09f
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedSaslMechanismException.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.errors;
+
+public class UnsupportedSaslMechanismException extends AuthenticationException {
+
+    private static final long serialVersionUID = 1L;
+
+    public UnsupportedSaslMechanismException(String message) {
+        super(message);
+    }
+
+    public UnsupportedSaslMechanismException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedVersionException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedVersionException.java
new file mode 100644
index 0000000..3679be4
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedVersionException.java
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.errors;
+
+public class UnsupportedVersionException extends ApiException {
+    private static final long serialVersionUID = 1L;
+
+    public UnsupportedVersionException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public UnsupportedVersionException(String message) {
+        super(message);
+    }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java
index 9dfc457b..971b7b6 100644
--- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java
@@ -13,6 +13,7 @@
 package org.apache.kafka.common.metrics.stats;
 
 import java.util.List;
+import java.util.Locale;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.kafka.common.metrics.MeasurableStat;
@@ -48,7 +49,7 @@
     }
 
     public String unitName() {
-        return unit.name().substring(0, unit.name().length() - 2).toLowerCase();
+        return unit.name().substring(0, unit.name().length() - 2).toLowerCase(Locale.ROOT);
     }
 
     @Override
diff --git a/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java b/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java
index 669f269..2d6ba8a 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java
@@ -31,10 +31,18 @@
      *             it is ignored otherwise
      * @param loginType the loginType, it must be non-null if `securityProtocol` is SASL_*; it is ignored otherwise
      * @param configs client/server configs
+     * @param clientSaslMechanism SASL mechanism if mode is CLIENT, ignored otherwise
+     * @param saslHandshakeRequestEnable flag to enable Sasl handshake requests; disabled only for SASL
+     *             inter-broker connections with inter-broker protocol version < 0.10
      * @return the configured `ChannelBuilder`
      * @throws IllegalArgumentException if `mode` invariants described above is not maintained
      */
-    public static ChannelBuilder create(SecurityProtocol securityProtocol, Mode mode, LoginType loginType, Map<String, ?> configs) {
+    public static ChannelBuilder create(SecurityProtocol securityProtocol,
+                                        Mode mode,
+                                        LoginType loginType,
+                                        Map<String, ?> configs,
+                                        String clientSaslMechanism,
+                                        boolean saslHandshakeRequestEnable) {
         ChannelBuilder channelBuilder;
 
         switch (securityProtocol) {
@@ -47,7 +55,9 @@
                 requireNonNullMode(mode, securityProtocol);
                 if (loginType == null)
                     throw new IllegalArgumentException("`loginType` must be non-null if `securityProtocol` is `" + securityProtocol + "`");
-                channelBuilder = new SaslChannelBuilder(mode, loginType, securityProtocol);
+                if (mode == Mode.CLIENT && clientSaslMechanism == null)
+                    throw new IllegalArgumentException("`clientSaslMechanism` must be non-null in client mode if `securityProtocol` is `" + securityProtocol + "`");
+                channelBuilder = new SaslChannelBuilder(mode, loginType, securityProtocol, clientSaslMechanism, saslHandshakeRequestEnable);
                 break;
             case PLAINTEXT:
             case TRACE:
diff --git a/clients/src/main/java/org/apache/kafka/common/network/LoginType.java b/clients/src/main/java/org/apache/kafka/common/network/LoginType.java
index 9216cb0..a3a2b27 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/LoginType.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/LoginType.java
@@ -20,7 +20,7 @@
 
 /**
  * The type of the login context, it should be SERVER for the broker and CLIENT for the clients (i.e. consumer and
- * producer). It provides the the login context name which defines the section of the JAAS configuration file to be used
+ * producer). It provides the login context name which defines the section of the JAAS configuration file to be used
  * for login.
  */
 public enum LoginType {
diff --git a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java
index 0cd5bfe..5c907ed 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java
@@ -21,13 +21,12 @@
 import org.apache.kafka.common.config.SaslConfigs;
 import org.apache.kafka.common.security.JaasUtils;
 import org.apache.kafka.common.security.kerberos.KerberosShortNamer;
-import org.apache.kafka.common.security.kerberos.LoginManager;
+import org.apache.kafka.common.security.authenticator.LoginManager;
 import org.apache.kafka.common.security.authenticator.SaslClientAuthenticator;
 import org.apache.kafka.common.security.authenticator.SaslServerAuthenticator;
 import org.apache.kafka.common.security.ssl.SslFactory;
 import org.apache.kafka.common.protocol.SecurityProtocol;
 import org.apache.kafka.common.KafkaException;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -35,36 +34,48 @@
     private static final Logger log = LoggerFactory.getLogger(SaslChannelBuilder.class);
 
     private final SecurityProtocol securityProtocol;
+    private final String clientSaslMechanism;
     private final Mode mode;
     private final LoginType loginType;
+    private final boolean handshakeRequestEnable;
 
     private LoginManager loginManager;
     private SslFactory sslFactory;
     private Map<String, ?> configs;
     private KerberosShortNamer kerberosShortNamer;
 
-    public SaslChannelBuilder(Mode mode, LoginType loginType, SecurityProtocol securityProtocol) {
+    public SaslChannelBuilder(Mode mode, LoginType loginType, SecurityProtocol securityProtocol, String clientSaslMechanism, boolean handshakeRequestEnable) {
         this.mode = mode;
         this.loginType = loginType;
         this.securityProtocol = securityProtocol;
+        this.handshakeRequestEnable = handshakeRequestEnable;
+        this.clientSaslMechanism = clientSaslMechanism;
     }
 
     public void configure(Map<String, ?> configs) throws KafkaException {
         try {
             this.configs = configs;
-            this.loginManager = LoginManager.acquireLoginManager(loginType, configs);
-
-            String defaultRealm;
-            try {
-                defaultRealm = JaasUtils.defaultRealm();
-            } catch (Exception ke) {
-                defaultRealm = "";
+            boolean hasKerberos;
+            if (mode == Mode.SERVER) {
+                List<String> enabledMechanisms = (List<String>) this.configs.get(SaslConfigs.SASL_ENABLED_MECHANISMS);
+                hasKerberos = enabledMechanisms == null || enabledMechanisms.contains(SaslConfigs.GSSAPI_MECHANISM);
+            } else {
+                hasKerberos = clientSaslMechanism.equals(SaslConfigs.GSSAPI_MECHANISM);
             }
 
-            @SuppressWarnings("unchecked")
-            List<String> principalToLocalRules = (List<String>) configs.get(SaslConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES);
-            if (principalToLocalRules != null)
-                kerberosShortNamer = KerberosShortNamer.fromUnparsedRules(defaultRealm, principalToLocalRules);
+            if (hasKerberos) {
+                String defaultRealm;
+                try {
+                    defaultRealm = JaasUtils.defaultKerberosRealm();
+                } catch (Exception ke) {
+                    defaultRealm = "";
+                }
+                @SuppressWarnings("unchecked")
+                List<String> principalToLocalRules = (List<String>) configs.get(SaslConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES);
+                if (principalToLocalRules != null)
+                    kerberosShortNamer = KerberosShortNamer.fromUnparsedRules(defaultRealm, principalToLocalRules);
+            }
+            this.loginManager = LoginManager.acquireLoginManager(loginType, hasKerberos, configs);
 
             if (this.securityProtocol == SecurityProtocol.SASL_SSL) {
                 // Disable SSL client authentication as we are using SASL authentication
@@ -82,10 +93,11 @@
             TransportLayer transportLayer = buildTransportLayer(id, key, socketChannel);
             Authenticator authenticator;
             if (mode == Mode.SERVER)
-                authenticator = new SaslServerAuthenticator(id, loginManager.subject(), kerberosShortNamer, maxReceiveSize);
+                authenticator = new SaslServerAuthenticator(id, loginManager.subject(), kerberosShortNamer,
+                        socketChannel.socket().getLocalAddress().getHostName(), maxReceiveSize);
             else
                 authenticator = new SaslClientAuthenticator(id, loginManager.subject(), loginManager.serviceName(),
-                        socketChannel.socket().getInetAddress().getHostName());
+                        socketChannel.socket().getInetAddress().getHostName(), clientSaslMechanism, handshakeRequestEnable);
             // Both authenticators don't use `PrincipalBuilder`, so we pass `null` for now. Reconsider if this changes.
             authenticator.configure(transportLayer, null, this.configs);
             return new KafkaChannel(id, transportLayer, authenticator, maxReceiveSize);
@@ -96,7 +108,8 @@
     }
 
     public void close()  {
-        this.loginManager.release();
+        if (this.loginManager != null)
+            this.loginManager.release();
     }
 
     protected TransportLayer buildTransportLayer(String id, SelectionKey key, SocketChannel socketChannel) throws IOException {
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
index e8fd3d3..aeb0b45 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
@@ -36,7 +36,9 @@
     LEAVE_GROUP(13, "LeaveGroup"),
     SYNC_GROUP(14, "SyncGroup"),
     DESCRIBE_GROUPS(15, "DescribeGroups"),
-    LIST_GROUPS(16, "ListGroups");
+    LIST_GROUPS(16, "ListGroups"),
+    SASL_HANDSHAKE(17, "SaslHandshake"),
+    API_VERSIONS(18, "ApiVersions");
 
     private static final ApiKeys[] ID_TO_TYPE;
     private static final int MIN_API_KEY = 0;
@@ -96,4 +98,4 @@
         System.out.println(toHtml());
     }
 
-}
+}
\ No newline at end of file
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
index 0f33516..64a709e 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
@@ -28,6 +28,7 @@
 import org.apache.kafka.common.errors.GroupCoordinatorNotAvailableException;
 import org.apache.kafka.common.errors.GroupLoadInProgressException;
 import org.apache.kafka.common.errors.IllegalGenerationException;
+import org.apache.kafka.common.errors.IllegalSaslStateException;
 import org.apache.kafka.common.errors.InconsistentGroupProtocolException;
 import org.apache.kafka.common.errors.InvalidCommitOffsetSizeException;
 import org.apache.kafka.common.errors.InvalidFetchSizeException;
@@ -49,8 +50,10 @@
 import org.apache.kafka.common.errors.RecordTooLargeException;
 import org.apache.kafka.common.errors.ReplicaNotAvailableException;
 import org.apache.kafka.common.errors.RetriableException;
+import org.apache.kafka.common.errors.UnsupportedSaslMechanismException;
 import org.apache.kafka.common.errors.TimeoutException;
 import org.apache.kafka.common.errors.TopicAuthorizationException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
 import org.apache.kafka.common.errors.UnknownMemberIdException;
 import org.apache.kafka.common.errors.UnknownServerException;
 import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
@@ -130,7 +133,13 @@
     CLUSTER_AUTHORIZATION_FAILED(31,
             new ClusterAuthorizationException("Cluster authorization failed.")),
     INVALID_TIMESTAMP(32,
-            new InvalidTimestampException("The timestamp of the message is out of acceptable range."));
+            new InvalidTimestampException("The timestamp of the message is out of acceptable range.")),
+    UNSUPPORTED_SASL_MECHANISM(33,
+            new UnsupportedSaslMechanismException("The broker does not support the requested SASL mechanism.")),
+    ILLEGAL_SASL_STATE(34,
+            new IllegalSaslStateException("Request is not valid given the current SASL state.")),
+    UNSUPPORTED_VERSION(35,
+            new UnsupportedVersionException("The version of API is not supported."));
 
     private static final Logger log = LoggerFactory.getLogger(Errors.class);
 
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
index 475a4f2..99cdbf9 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
@@ -26,6 +26,7 @@
 import java.util.Map;
 import java.util.Set;
 
+import static org.apache.kafka.common.protocol.types.Type.BOOLEAN;
 import static org.apache.kafka.common.protocol.types.Type.BYTES;
 import static org.apache.kafka.common.protocol.types.Type.INT16;
 import static org.apache.kafka.common.protocol.types.Type.INT32;
@@ -56,10 +57,13 @@
                                                                           new ArrayOf(STRING),
                                                                           "An array of topics to fetch metadata for. If no topics are specified fetch metadata for all topics."));
 
-    public static final Schema BROKER = new Schema(new Field("node_id", INT32, "The broker id."),
+    public static final Schema METADATA_REQUEST_V1 = new Schema(new Field("topics",
+                                                                          ArrayOf.nullable(STRING),
+                                                                          "An array of topics to fetch metadata for. If the topics array is null fetch metadata for all topics."));
+
+    public static final Schema METADATA_BROKER_V0 = new Schema(new Field("node_id", INT32, "The broker id."),
                                                    new Field("host", STRING, "The hostname of the broker."),
-                                                   new Field("port",
-                                                             INT32,
+                                                   new Field("port", INT32,
                                                              "The port on which the broker accepts requests."));
 
     public static final Schema PARTITION_METADATA_V0 = new Schema(new Field("partition_error_code",
@@ -87,13 +91,34 @@
                                                                         "Metadata for each partition of the topic."));
 
     public static final Schema METADATA_RESPONSE_V0 = new Schema(new Field("brokers",
-                                                                           new ArrayOf(BROKER),
+                                                                           new ArrayOf(METADATA_BROKER_V0),
                                                                            "Host and port information for all brokers."),
                                                                  new Field("topic_metadata",
                                                                            new ArrayOf(TOPIC_METADATA_V0)));
 
-    public static final Schema[] METADATA_REQUEST = new Schema[] {METADATA_REQUEST_V0};
-    public static final Schema[] METADATA_RESPONSE = new Schema[] {METADATA_RESPONSE_V0};
+    public static final Schema METADATA_BROKER_V1 = new Schema(new Field("node_id", INT32, "The broker id."),
+                                                      new Field("host", STRING, "The hostname of the broker."),
+                                                      new Field("port", INT32,
+                                                        "The port on which the broker accepts requests."),
+                                                      new Field("rack", NULLABLE_STRING, "The rack of the broker."));
+
+    public static final Schema PARTITION_METADATA_V1 = PARTITION_METADATA_V0;
+
+    public static final Schema TOPIC_METADATA_V1 = new Schema(new Field("topic_error_code", INT16, "The error code for the given topic."),
+                                                              new Field("topic", STRING, "The name of the topic"),
+                                                              new Field("is_internal", BOOLEAN,
+                                                                  "Indicates if the topic is considered a Kafka internal topic"),
+                                                              new Field("partition_metadata", new ArrayOf(PARTITION_METADATA_V1),
+                                                                "Metadata for each partition of the topic."));
+
+    public static final Schema METADATA_RESPONSE_V1 = new Schema(new Field("brokers", new ArrayOf(METADATA_BROKER_V1),
+                                                                    "Host and port information for all brokers."),
+                                                                 new Field("controller_id", INT32,
+                                                                     "The broker id of the controller broker."),
+                                                                 new Field("topic_metadata", new ArrayOf(TOPIC_METADATA_V1)));
+
+    public static final Schema[] METADATA_REQUEST = new Schema[] {METADATA_REQUEST_V0, METADATA_REQUEST_V1};
+    public static final Schema[] METADATA_RESPONSE = new Schema[] {METADATA_RESPONSE_V0, METADATA_RESPONSE_V1};
 
     /* Produce api */
 
@@ -478,7 +503,7 @@
                                                                                                  "The current state of the group (one of: Dead, Stable, AwaitingSync, or PreparingRebalance, or empty if there is no active group)"),
                                                                                        new Field("protocol_type",
                                                                                                  STRING,
-                                                                                                 "The current group protocol type (will be empty if the there is no active group)"),
+                                                                                                 "The current group protocol type (will be empty if there is no active group)"),
                                                                                        new Field("protocol",
                                                                                                  STRING,
                                                                                                  "The current group protocol (only provided if the group is Stable)"),
@@ -496,9 +521,14 @@
                                                                                    STRING,
                                                                                    "The unique group id."));
 
+    public static final Schema GROUP_COORDINATOR_BROKER_V0 = new Schema(new Field("node_id", INT32, "The broker id."),
+                                                                        new Field("host", STRING, "The hostname of the broker."),
+                                                                        new Field("port", INT32,
+                                                                            "The port on which the broker accepts requests."));
+
     public static final Schema GROUP_COORDINATOR_RESPONSE_V0 = new Schema(new Field("error_code", INT16),
                                                                           new Field("coordinator",
-                                                                                    BROKER,
+                                                                                    GROUP_COORDINATOR_BROKER_V0,
                                                                                     "Host and port information for the coordinator for a consumer group."));
 
     public static final Schema[] GROUP_COORDINATOR_REQUEST = new Schema[] {GROUP_COORDINATOR_REQUEST_V0};
@@ -718,10 +748,35 @@
     public static final Schema[] UPDATE_METADATA_REQUEST = new Schema[] {UPDATE_METADATA_REQUEST_V0, UPDATE_METADATA_REQUEST_V1, UPDATE_METADATA_REQUEST_V2};
     public static final Schema[] UPDATE_METADATA_RESPONSE = new Schema[] {UPDATE_METADATA_RESPONSE_V0, UPDATE_METADATA_RESPONSE_V1, UPDATE_METADATA_RESPONSE_V2};
 
+    /* SASL handshake api */
+    public static final Schema SASL_HANDSHAKE_REQUEST_V0 = new Schema(
+            new Field("mechanism", STRING, "SASL Mechanism chosen by the client."));
+
+    public static final Schema SASL_HANDSHAKE_RESPONSE_V0 = new Schema(
+            new Field("error_code", INT16),
+            new Field("enabled_mechanisms", new ArrayOf(Type.STRING), "Array of mechanisms enabled in the server."));
+
+    public static final Schema[] SASL_HANDSHAKE_REQUEST = new Schema[] {SASL_HANDSHAKE_REQUEST_V0};
+    public static final Schema[] SASL_HANDSHAKE_RESPONSE = new Schema[] {SASL_HANDSHAKE_RESPONSE_V0};
+
+    /* ApiVersion api */
+    public static final Schema API_VERSIONS_REQUEST_V0 = new Schema();
+
+    public static final Schema API_VERSIONS_V0 = new Schema(new Field("api_key", INT16, "API key."),
+                                                           new Field("min_version", INT16, "Minimum supported version."),
+                                                           new Field("max_version", INT16, "Maximum supported version."));
+
+    public static final Schema API_VERSIONS_RESPONSE_V0 = new Schema(new Field("error_code", INT16, "Error code."),
+                                                                    new Field("api_versions", new ArrayOf(API_VERSIONS_V0), "API versions supported by the broker."));
+
+    public static final Schema[] API_VERSIONS_REQUEST = new Schema[]{API_VERSIONS_REQUEST_V0};
+    public static final Schema[] API_VERSIONS_RESPONSE = new Schema[]{API_VERSIONS_RESPONSE_V0};
+
     /* an array of all requests and responses with all schema versions; a null value in the inner array means that the
      * particular version is not supported */
     public static final Schema[][] REQUESTS = new Schema[ApiKeys.MAX_API_KEY + 1][];
     public static final Schema[][] RESPONSES = new Schema[ApiKeys.MAX_API_KEY + 1][];
+    public static final short[] MIN_VERSIONS = new short[ApiKeys.MAX_API_KEY + 1];
 
     /* the latest version of each api */
     public static final short[] CURR_VERSION = new short[ApiKeys.MAX_API_KEY + 1];
@@ -744,6 +799,8 @@
         REQUESTS[ApiKeys.SYNC_GROUP.id] = SYNC_GROUP_REQUEST;
         REQUESTS[ApiKeys.DESCRIBE_GROUPS.id] = DESCRIBE_GROUPS_REQUEST;
         REQUESTS[ApiKeys.LIST_GROUPS.id] = LIST_GROUPS_REQUEST;
+        REQUESTS[ApiKeys.SASL_HANDSHAKE.id] = SASL_HANDSHAKE_REQUEST;
+        REQUESTS[ApiKeys.API_VERSIONS.id] = API_VERSIONS_REQUEST;
 
         RESPONSES[ApiKeys.PRODUCE.id] = PRODUCE_RESPONSE;
         RESPONSES[ApiKeys.FETCH.id] = FETCH_RESPONSE;
@@ -762,16 +819,33 @@
         RESPONSES[ApiKeys.SYNC_GROUP.id] = SYNC_GROUP_RESPONSE;
         RESPONSES[ApiKeys.DESCRIBE_GROUPS.id] = DESCRIBE_GROUPS_RESPONSE;
         RESPONSES[ApiKeys.LIST_GROUPS.id] = LIST_GROUPS_RESPONSE;
+        RESPONSES[ApiKeys.SASL_HANDSHAKE.id] = SASL_HANDSHAKE_RESPONSE;
+        RESPONSES[ApiKeys.API_VERSIONS.id] = API_VERSIONS_RESPONSE;
 
-        /* set the maximum version of each api */
-        for (ApiKeys api : ApiKeys.values())
+        /* set the minimum and maximum version of each api */
+        for (ApiKeys api : ApiKeys.values()) {
             CURR_VERSION[api.id] = (short) (REQUESTS[api.id].length - 1);
+            for (int i = 0; i < REQUESTS[api.id].length; ++i)
+                if (REQUESTS[api.id][i] != null) {
+                    MIN_VERSIONS[api.id] = (short) i;
+                    break;
+                }
+        }
 
-        /* sanity check that we have the same number of request and response versions for each api */
-        for (ApiKeys api : ApiKeys.values())
+        /* sanity check that:
+         *   - we have the same number of request and response versions for each api
+         *   - we have a consistent set of request and response versions for each api */
+        for (ApiKeys api : ApiKeys.values()) {
             if (REQUESTS[api.id].length != RESPONSES[api.id].length)
                 throw new IllegalStateException(REQUESTS[api.id].length + " request versions for api " + api.name
                         + " but " + RESPONSES[api.id].length + " response versions.");
+
+            for (int i = 0; i < REQUESTS[api.id].length; ++i)
+                if ((REQUESTS[api.id][i] == null && RESPONSES[api.id][i] != null) ||
+                        (REQUESTS[api.id][i] != null && RESPONSES[api.id][i] == null))
+                    throw new IllegalStateException("Request and response for version " + i + " of API "
+                            + api.id + " are defined inconsistently. One is null while the other is not null.");
+        }
     }
 
     private static String indentString(int size) {
@@ -934,4 +1008,4 @@
         System.out.println(toHtml());
     }
 
-}
+}
\ No newline at end of file
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java
index 905c670..d5fbed7 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java
@@ -21,6 +21,7 @@
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
 
@@ -85,7 +86,7 @@
 
     /** Case insensitive lookup by protocol name */
     public static SecurityProtocol forName(String name) {
-        return SecurityProtocol.valueOf(name.toUpperCase());
+        return SecurityProtocol.valueOf(name.toUpperCase(Locale.ROOT));
     }
 
     /**
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java
index a08f876..207f108 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java
@@ -24,13 +24,33 @@
 public class ArrayOf extends Type {
 
     private final Type type;
+    private final boolean nullable;
 
     public ArrayOf(Type type) {
+        this(type, false);
+    }
+
+    public static ArrayOf nullable(Type type) {
+        return new ArrayOf(type, true);
+    }
+
+    private ArrayOf(Type type, boolean nullable) {
         this.type = type;
+        this.nullable = nullable;
+    }
+
+    @Override
+    public boolean isNullable() {
+        return nullable;
     }
 
     @Override
     public void write(ByteBuffer buffer, Object o) {
+        if (o == null) {
+            buffer.putInt(-1);
+            return;
+        }
+
         Object[] objs = (Object[]) o;
         int size = objs.length;
         buffer.putInt(size);
@@ -41,8 +61,11 @@
     @Override
     public Object read(ByteBuffer buffer) {
         int size = buffer.getInt();
-        if (size < 0)
+        if (size < 0 && isNullable())
+            return null;
+        else if (size < 0)
             throw new SchemaException("Array size " + size + " cannot be negative");
+
         if (size > buffer.remaining())
             throw new SchemaException("Error reading array of size " + size + ", only " + buffer.remaining() + " bytes available");
         Object[] objs = new Object[size];
@@ -53,8 +76,11 @@
 
     @Override
     public int sizeOf(Object o) {
-        Object[] objs = (Object[]) o;
         int size = 4;
+        if (o == null)
+            return size;
+
+        Object[] objs = (Object[]) o;
         for (int i = 0; i < objs.length; i++)
             size += type.sizeOf(objs[i]);
         return size;
@@ -72,6 +98,9 @@
     @Override
     public Object[] validate(Object item) {
         try {
+            if (isNullable() && item == null)
+                return null;
+
             Object[] array = (Object[]) item;
             for (int i = 0; i < array.length; i++)
                 type.validate(array[i]);
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
index 79f0638..212d701 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
@@ -3,9 +3,9 @@
  * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
  * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
  * License. You may obtain a copy of the License at
- * 
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
  * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
  * specific language governing permissions and limitations under the License.
@@ -41,7 +41,7 @@
 
     /**
      * Return the value of the given pre-validated field, or if the value is missing return the default value.
-     * 
+     *
      * @param field The field for which to get the default value
      * @throws SchemaException if the field has no value and has no default.
      */
@@ -59,7 +59,7 @@
 
     /**
      * Get the value for the field directly by the field index with no lookup needed (faster!)
-     * 
+     *
      * @param field The field to look up
      * @return The value for that field.
      * @throws SchemaException if the field has no value and has no default.
@@ -71,7 +71,7 @@
 
     /**
      * Get the record value for the field with the given name by doing a hash table lookup (slower!)
-     * 
+     *
      * @param name The name of the field
      * @return The value in the record
      * @throws SchemaException If no such field exists
@@ -148,6 +148,14 @@
         return (String) get(name);
     }
 
+    public Boolean getBoolean(Field field) {
+        return (Boolean) get(field);
+    }
+
+    public Boolean getBoolean(String name) {
+        return (Boolean) get(name);
+    }
+
     public ByteBuffer getBytes(Field field) {
         Object result = get(field);
         if (result instanceof byte[])
@@ -164,7 +172,7 @@
 
     /**
      * Set the given field to the specified value
-     * 
+     *
      * @param field The field
      * @param value The value
      * @throws SchemaException If the validation of the field failed
@@ -177,7 +185,7 @@
 
     /**
      * Set the field specified by the given name to the value
-     * 
+     *
      * @param name The name of the field
      * @param value The value to set
      * @throws SchemaException If the field is not known
@@ -194,7 +202,7 @@
      * Create a struct for the schema of a container type (struct or array). Note that for array type, this method
      * assumes that the type is an array of schema and creates a struct of that schema. Arrays of other types can't be
      * instantiated with this method.
-     * 
+     *
      * @param field The field to create an instance of
      * @return The struct
      * @throws SchemaException If the given field is not a container type
@@ -213,7 +221,7 @@
 
     /**
      * Create a struct instance for the given field which must be a container type (struct or array)
-     * 
+     *
      * @param field The name of the field to create (field must be a schema type)
      * @return The struct
      * @throws SchemaException If the given field is not a container type
@@ -282,7 +290,7 @@
             Field f = this.schema.get(i);
             b.append(f.name);
             b.append('=');
-            if (f.type() instanceof ArrayOf) {
+            if (f.type() instanceof ArrayOf && this.values[i] != null) {
                 Object[] arrayValue = (Object[]) this.values[i];
                 b.append('[');
                 for (int j = 0; j < arrayValue.length; j++) {
@@ -307,9 +315,11 @@
         for (int i = 0; i < this.values.length; i++) {
             Field f = this.schema.get(i);
             if (f.type() instanceof ArrayOf) {
-                Object[] arrayObject = (Object []) this.get(f);
-                for (Object arrayItem: arrayObject)
-                    result = prime * result + arrayItem.hashCode();
+                if (this.get(f) != null) {
+                    Object[] arrayObject = (Object []) this.get(f);
+                    for (Object arrayItem: arrayObject)
+                        result = prime * result + arrayItem.hashCode();
+                }
             } else {
                 Object field = this.get(f);
                 if (field != null) {
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java
index 92c1f7c..43b4a37 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java
@@ -59,6 +59,47 @@
         return false;
     }
 
+    /**
+     * The Boolean type represents a boolean value in a byte by using
+     * the value of 0 to represent false, and 1 to represent true.
+     *
+     * If for some reason a value that is not 0 or 1 is read,
+     * then any non-zero value will return true.
+     */
+    public static final Type BOOLEAN = new Type() {
+        @Override
+        public void write(ByteBuffer buffer, Object o) {
+            if ((Boolean) o)
+                buffer.put((byte) 1);
+            else
+                buffer.put((byte) 0);
+        }
+
+        @Override
+        public Object read(ByteBuffer buffer) {
+            byte value = buffer.get();
+            return value != 0;
+        }
+
+        @Override
+        public int sizeOf(Object o) {
+            return 1;
+        }
+
+        @Override
+        public String toString() {
+            return "BOOLEAN";
+        }
+
+        @Override
+        public Boolean validate(Object item) {
+            if (item instanceof Boolean)
+                return (Boolean) item;
+            else
+                throw new SchemaException(item + " is not a Boolean.");
+        }
+    };
+
     public static final Type INT8 = new Type() {
         @Override
         public void write(ByteBuffer buffer, Object o) {
@@ -196,7 +237,7 @@
                 throw new SchemaException("String length " + length + " cannot be negative");
             if (length > buffer.remaining())
                 throw new SchemaException("Error reading string of length " + length + ", only " + buffer.remaining() + " bytes available");
-            
+
             byte[] bytes = new byte[length];
             buffer.get(bytes);
             return Utils.utf8(bytes);
diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
index f37ef39..7175953 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
@@ -115,7 +115,10 @@
      * to accept this single record.
      */
     public boolean hasRoomFor(byte[] key, byte[] value) {
-        return this.writable && this.compressor.numRecordsWritten() == 0 ?
+        if (!this.writable)
+            return false;
+
+        return this.compressor.numRecordsWritten() == 0 ?
             this.initialCapacity >= Records.LOG_OVERHEAD + Record.recordSize(key, value) :
             this.writeLimit >= this.compressor.estimatedBytesWritten() + Records.LOG_OVERHEAD + Record.recordSize(key, value);
     }
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
index 5a40b7f..ab61c66 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
@@ -72,9 +72,13 @@
                 return DescribeGroupsRequest.parse(buffer, versionId);
             case LIST_GROUPS:
                 return ListGroupsRequest.parse(buffer, versionId);
+            case SASL_HANDSHAKE:
+                return SaslHandshakeRequest.parse(buffer, versionId);
+            case API_VERSIONS:
+                return ApiVersionsRequest.parse(buffer, versionId);
             default:
                 throw new AssertionError(String.format("ApiKey %s is not currently handled in `getRequest`, the " +
                         "code should be updated to do so.", apiKey));
         }
     }
-}
+}
\ No newline at end of file
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java
new file mode 100644
index 0000000..b78c759
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+
+public class ApiVersionsRequest extends AbstractRequest {
+
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.API_VERSIONS.id);
+
+    public ApiVersionsRequest() {
+        super(new Struct(CURRENT_SCHEMA));
+    }
+
+    public ApiVersionsRequest(Struct struct) {
+        super(struct);
+    }
+
+    @Override
+    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
+        switch (versionId) {
+            case 0:
+                short errorCode = Errors.forException(e).code();
+                return new ApiVersionsResponse(errorCode, Collections.<ApiVersionsResponse.ApiVersion>emptyList());
+            default:
+                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
+                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.API_VERSIONS.id)));
+        }
+    }
+
+    public static ApiVersionsRequest parse(ByteBuffer buffer, int versionId) {
+        return new ApiVersionsRequest(ProtoUtils.parseRequest(ApiKeys.API_VERSIONS.id, versionId, buffer));
+    }
+
+    public static ApiVersionsRequest parse(ByteBuffer buffer) {
+        return new ApiVersionsRequest(CURRENT_SCHEMA.read(buffer));
+    }
+}
\ No newline at end of file
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
new file mode 100644
index 0000000..fe995b2
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
@@ -0,0 +1,130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.Protocol;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class ApiVersionsResponse extends AbstractRequestResponse {
+
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.API_VERSIONS.id);
+    private static final ApiVersionsResponse API_VERSIONS_RESPONSE = createApiVersionsResponse();
+
+    public static final String ERROR_CODE_KEY_NAME = "error_code";
+    public static final String API_VERSIONS_KEY_NAME = "api_versions";
+    public static final String API_KEY_NAME = "api_key";
+    public static final String MIN_VERSION_KEY_NAME = "min_version";
+    public static final String MAX_VERSION_KEY_NAME = "max_version";
+
+    /**
+     * Possible error codes:
+     *
+     * UNSUPPORTED_VERSION (33)
+     */
+    private final short errorCode;
+    private final Map<Short, ApiVersion> apiKeyToApiVersion;
+
+    public static final class ApiVersion {
+        public final short apiKey;
+        public final short minVersion;
+        public final short maxVersion;
+
+        public ApiVersion(short apiKey, short minVersion, short maxVersion) {
+            this.apiKey = apiKey;
+            this.minVersion = minVersion;
+            this.maxVersion = maxVersion;
+        }
+    }
+
+    public ApiVersionsResponse(short errorCode, List<ApiVersion> apiVersions) {
+        super(new Struct(CURRENT_SCHEMA));
+        struct.set(ERROR_CODE_KEY_NAME, errorCode);
+        List<Struct> apiVersionList = new ArrayList<>();
+        for (ApiVersion apiVersion : apiVersions) {
+            Struct apiVersionStruct = struct.instance(API_VERSIONS_KEY_NAME);
+            apiVersionStruct.set(API_KEY_NAME, apiVersion.apiKey);
+            apiVersionStruct.set(MIN_VERSION_KEY_NAME, apiVersion.minVersion);
+            apiVersionStruct.set(MAX_VERSION_KEY_NAME, apiVersion.maxVersion);
+            apiVersionList.add(apiVersionStruct);
+        }
+        struct.set(API_VERSIONS_KEY_NAME, apiVersionList.toArray());
+        this.errorCode = errorCode;
+        this.apiKeyToApiVersion = buildApiKeyToApiVersion(apiVersions);
+    }
+
+    public ApiVersionsResponse(Struct struct) {
+        super(struct);
+        this.errorCode = struct.getShort(ERROR_CODE_KEY_NAME);
+        List<ApiVersion> tempApiVersions = new ArrayList<>();
+        for (Object apiVersionsObj : struct.getArray(API_VERSIONS_KEY_NAME)) {
+            Struct apiVersionStruct = (Struct) apiVersionsObj;
+            short apiKey = apiVersionStruct.getShort(API_KEY_NAME);
+            short minVersion = apiVersionStruct.getShort(MIN_VERSION_KEY_NAME);
+            short maxVersion = apiVersionStruct.getShort(MAX_VERSION_KEY_NAME);
+            tempApiVersions.add(new ApiVersion(apiKey, minVersion, maxVersion));
+        }
+        this.apiKeyToApiVersion = buildApiKeyToApiVersion(tempApiVersions);
+    }
+
+    public Collection<ApiVersion> apiVersions() {
+        return apiKeyToApiVersion.values();
+    }
+
+    public ApiVersion apiVersion(short apiKey) {
+        return apiKeyToApiVersion.get(apiKey);
+    }
+
+    public short errorCode() {
+        return errorCode;
+    }
+
+    public static ApiVersionsResponse parse(ByteBuffer buffer) {
+        return new ApiVersionsResponse(CURRENT_SCHEMA.read(buffer));
+    }
+
+    public static ApiVersionsResponse fromError(Errors error) {
+        return new ApiVersionsResponse(error.code(), Collections.<ApiVersion>emptyList());
+    }
+
+    public static ApiVersionsResponse apiVersionsResponse() {
+        return API_VERSIONS_RESPONSE;
+    }
+
+    private static ApiVersionsResponse createApiVersionsResponse() {
+        List<ApiVersion> versionList = new ArrayList<>();
+        for (ApiKeys apiKey : ApiKeys.values()) {
+            versionList.add(new ApiVersion(apiKey.id, Protocol.MIN_VERSIONS[apiKey.id], Protocol.CURR_VERSION[apiKey.id]));
+        }
+        return new ApiVersionsResponse(Errors.NONE.code(), versionList);
+    }
+
+    private Map<Short, ApiVersion> buildApiKeyToApiVersion(List<ApiVersion> apiVersions) {
+        Map<Short, ApiVersion> tempApiIdToApiVersion = new HashMap<>();
+        for (ApiVersion apiVersion: apiVersions) {
+            tempApiIdToApiVersion.put(apiVersion.apiKey, apiVersion);
+        }
+        return tempApiIdToApiVersion;
+    }
+}
\ No newline at end of file
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java
index 92d8c6d..f0cb8fc 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java
@@ -25,24 +25,41 @@
 import java.util.List;
 
 public class MetadataRequest extends AbstractRequest {
-    
+
     private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id);
     private static final String TOPICS_KEY_NAME = "topics";
 
+    private static final MetadataRequest ALL_TOPICS_REQUEST = new MetadataRequest((List<String>) null); // Unusual cast to work around constructor ambiguity
+
     private final List<String> topics;
 
+    public static MetadataRequest allTopics() {
+        return ALL_TOPICS_REQUEST;
+    }
+
+    /**
+     * In v0 null is not allowed and and empty list indicates requesting all topics.
+     * In v1 null indicates requesting all topics, and an empty list indicates requesting no topics.
+     */
     public MetadataRequest(List<String> topics) {
         super(new Struct(CURRENT_SCHEMA));
-        struct.set(TOPICS_KEY_NAME, topics.toArray());
+        if (topics == null)
+            struct.set(TOPICS_KEY_NAME, null);
+        else
+            struct.set(TOPICS_KEY_NAME, topics.toArray());
         this.topics = topics;
     }
 
     public MetadataRequest(Struct struct) {
         super(struct);
         Object[] topicArray = struct.getArray(TOPICS_KEY_NAME);
-        topics = new ArrayList<>();
-        for (Object topicObj: topicArray) {
-            topics.add((String) topicObj);
+        if (topicArray != null) {
+            topics = new ArrayList<>();
+            for (Object topicObj: topicArray) {
+                topics.add((String) topicObj);
+            }
+        } else {
+            topics = null;
         }
     }
 
@@ -52,18 +69,25 @@
         Errors error = Errors.forException(e);
         List<MetadataResponse.PartitionMetadata> partitions = Collections.emptyList();
 
-        for (String topic : topics)
-            topicMetadatas.add(new MetadataResponse.TopicMetadata(error, topic, partitions));
+        if (topics != null) {
+            for (String topic : topics)
+                topicMetadatas.add(new MetadataResponse.TopicMetadata(error, topic, false, partitions));
+        }
 
         switch (versionId) {
             case 0:
-                return new MetadataResponse(Collections.<Node>emptyList(), topicMetadatas);
+            case 1:
+                return new MetadataResponse(Collections.<Node>emptyList(), MetadataResponse.NO_CONTROLLER_ID, topicMetadatas, versionId);
             default:
                 throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
                         versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.METADATA.id)));
         }
     }
 
+    public boolean isAllTopics() {
+        return topics == null;
+    }
+
     public List<String> topics() {
         return topics;
     }
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java
index 13e0d8f..09a5bee 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java
@@ -18,7 +18,6 @@
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
 import org.apache.kafka.common.protocol.ProtoUtils;
-import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
@@ -32,7 +31,7 @@
 
 public class MetadataResponse extends AbstractRequestResponse {
 
-    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.METADATA.id);
+    private static final short CURRENT_VERSION = ProtoUtils.latestVersion(ApiKeys.METADATA.id);
     private static final String BROKERS_KEY_NAME = "brokers";
     private static final String TOPIC_METADATA_KEY_NAME = "topic_metadata";
 
@@ -40,6 +39,10 @@
     private static final String NODE_ID_KEY_NAME = "node_id";
     private static final String HOST_KEY_NAME = "host";
     private static final String PORT_KEY_NAME = "port";
+    private static final String RACK_KEY_NAME = "rack";
+
+    private static final String CONTROLLER_ID_KEY_NAME = "controller_id";
+    public static final int NO_CONTROLLER_ID = -1;
 
     // topic level field names
     private static final String TOPIC_ERROR_CODE_KEY_NAME = "topic_error_code";
@@ -54,6 +57,7 @@
      */
 
     private static final String TOPIC_KEY_NAME = "topic";
+    private static final String IS_INTERNAL_KEY_NAME = "is_internal";
     private static final String PARTITION_METADATA_KEY_NAME = "partition_metadata";
 
     // partition level field names
@@ -72,13 +76,24 @@
     private static final String ISR_KEY_NAME = "isr";
 
     private final Collection<Node> brokers;
+    private final Node controller;
     private final List<TopicMetadata> topicMetadata;
 
+    /**
+     * Constructor for the latest version
+     */
+    public MetadataResponse(List<Node> brokers, int controllerId, List<TopicMetadata> topicMetadata) {
+        this(brokers, controllerId, topicMetadata, CURRENT_VERSION);
+    }
 
-    public MetadataResponse(List<Node> brokers, List<TopicMetadata> topicMetadata) {
-        super(new Struct(CURRENT_SCHEMA));
+    /**
+     * Constructor for a specific version
+     */
+    public MetadataResponse(List<Node> brokers, int controllerId, List<TopicMetadata> topicMetadata, int version) {
+        super(new Struct(ProtoUtils.responseSchema(ApiKeys.METADATA.id, version)));
 
         this.brokers = brokers;
+        this.controller = getControllerNode(controllerId, brokers);
         this.topicMetadata = topicMetadata;
 
         List<Struct> brokerArray = new ArrayList<>();
@@ -87,15 +102,25 @@
             broker.set(NODE_ID_KEY_NAME, node.id());
             broker.set(HOST_KEY_NAME, node.host());
             broker.set(PORT_KEY_NAME, node.port());
+            // This field only exists in v1+
+            if (broker.hasField(RACK_KEY_NAME))
+                broker.set(RACK_KEY_NAME, node.rack());
             brokerArray.add(broker);
         }
         struct.set(BROKERS_KEY_NAME, brokerArray.toArray());
 
+        // This field only exists in v1+
+        if (struct.hasField(CONTROLLER_ID_KEY_NAME))
+            struct.set(CONTROLLER_ID_KEY_NAME, controllerId);
+
         List<Struct> topicMetadataArray = new ArrayList<>(topicMetadata.size());
         for (TopicMetadata metadata : topicMetadata) {
             Struct topicData = struct.instance(TOPIC_METADATA_KEY_NAME);
             topicData.set(TOPIC_KEY_NAME, metadata.topic);
             topicData.set(TOPIC_ERROR_CODE_KEY_NAME, metadata.error.code());
+            // This field only exists in v1+
+            if (topicData.hasField(IS_INTERNAL_KEY_NAME))
+                topicData.set(IS_INTERNAL_KEY_NAME, metadata.isInternal());
 
             List<Struct> partitionMetadataArray = new ArrayList<>(metadata.partitionMetadata.size());
             for (PartitionMetadata partitionMetadata : metadata.partitionMetadata()) {
@@ -130,15 +155,28 @@
             int nodeId = broker.getInt(NODE_ID_KEY_NAME);
             String host = broker.getString(HOST_KEY_NAME);
             int port = broker.getInt(PORT_KEY_NAME);
-            brokers.put(nodeId, new Node(nodeId, host, port));
+            // This field only exists in v1+
+            // When we can't know if a rack exists in a v0 response we default to null
+            String rack =  broker.hasField(RACK_KEY_NAME) ? broker.getString(RACK_KEY_NAME) : null;
+            brokers.put(nodeId, new Node(nodeId, host, port, rack));
         }
 
+        // This field only exists in v1+
+        // When we can't know the controller id in a v0 response we default to NO_CONTROLLER_ID
+        int controllerId = NO_CONTROLLER_ID;
+        if (struct.hasField(CONTROLLER_ID_KEY_NAME))
+            controllerId = struct.getInt(CONTROLLER_ID_KEY_NAME);
+
         List<TopicMetadata> topicMetadata = new ArrayList<>();
         Object[] topicInfos = (Object[]) struct.get(TOPIC_METADATA_KEY_NAME);
         for (int i = 0; i < topicInfos.length; i++) {
             Struct topicInfo = (Struct) topicInfos[i];
             Errors topicError = Errors.forCode(topicInfo.getShort(TOPIC_ERROR_CODE_KEY_NAME));
             String topic = topicInfo.getString(TOPIC_KEY_NAME);
+            // This field only exists in v1+
+            // When we can't know if a topic is internal or not in a v0 response we default to false
+            boolean isInternal = topicInfo.hasField(IS_INTERNAL_KEY_NAME) ? topicInfo.getBoolean(IS_INTERNAL_KEY_NAME) : false;
+
             List<PartitionMetadata> partitionMetadata = new ArrayList<>();
 
             Object[] partitionInfos = (Object[]) topicInfo.get(PARTITION_METADATA_KEY_NAME);
@@ -149,23 +187,41 @@
                 int leader = partitionInfo.getInt(LEADER_KEY_NAME);
                 Node leaderNode = leader == -1 ? null : brokers.get(leader);
                 Object[] replicas = (Object[]) partitionInfo.get(REPLICAS_KEY_NAME);
+
                 List<Node> replicaNodes = new ArrayList<>(replicas.length);
                 for (Object replicaNodeId : replicas)
-                    replicaNodes.add(brokers.get(replicaNodeId));
+                    if (brokers.containsKey(replicaNodeId))
+                        replicaNodes.add(brokers.get(replicaNodeId));
+                    else
+                        replicaNodes.add(new Node((int) replicaNodeId, "", -1));
+
                 Object[] isr = (Object[]) partitionInfo.get(ISR_KEY_NAME);
                 List<Node> isrNodes = new ArrayList<>(isr.length);
                 for (Object isrNode : isr)
-                    isrNodes.add(brokers.get(isrNode));
+                    if (brokers.containsKey(isrNode))
+                        isrNodes.add(brokers.get(isrNode));
+                    else
+                        isrNodes.add(new Node((int) isrNode, "", -1));
+
                 partitionMetadata.add(new PartitionMetadata(partitionError, partition, leaderNode, replicaNodes, isrNodes));
             }
 
-            topicMetadata.add(new TopicMetadata(topicError, topic, partitionMetadata));
+            topicMetadata.add(new TopicMetadata(topicError, topic, isInternal, partitionMetadata));
         }
 
         this.brokers = brokers.values();
+        this.controller = getControllerNode(controllerId, brokers.values());
         this.topicMetadata = topicMetadata;
     }
 
+    private Node getControllerNode(int controllerId, Collection<Node> brokers) {
+        for (Node broker : brokers) {
+            if (broker.id() == controllerId)
+                return broker;
+        }
+        return null;
+    }
+
     /**
      * Get a map of the topics which had metadata errors
      * @return the map
@@ -211,20 +267,43 @@
         return brokers;
     }
 
+    /**
+     * Get all topic metadata returned in the metadata response
+     * @return the topicMetadata
+     */
+    public Collection<TopicMetadata> topicMetadata() {
+        return topicMetadata;
+    }
+
+    /**
+     * The controller node returned in metadata response
+     * @return the controller node or null if it doesn't exist
+     */
+    public Node controller() {
+        return controller;
+    }
+
     public static MetadataResponse parse(ByteBuffer buffer) {
-        return new MetadataResponse(CURRENT_SCHEMA.read(buffer));
+        return parse(buffer, CURRENT_VERSION);
+    }
+
+    public static MetadataResponse parse(ByteBuffer buffer, int version) {
+        return new MetadataResponse(ProtoUtils.responseSchema(ApiKeys.METADATA.id, version).read(buffer));
     }
 
     public static class TopicMetadata {
         private final Errors error;
         private final String topic;
+        private final boolean isInternal;
         private final List<PartitionMetadata> partitionMetadata;
 
         public TopicMetadata(Errors error,
                              String topic,
+                             boolean isInternal,
                              List<PartitionMetadata> partitionMetadata) {
             this.error = error;
             this.topic = topic;
+            this.isInternal = isInternal;
             this.partitionMetadata = partitionMetadata;
         }
 
@@ -236,6 +315,10 @@
             return topic;
         }
 
+        public boolean isInternal() {
+            return isInternal;
+        }
+
         public List<PartitionMetadata> partitionMetadata() {
             return partitionMetadata;
         }
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ResponseSend.java b/clients/src/main/java/org/apache/kafka/common/requests/ResponseSend.java
index 12b06d1..9494de7 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ResponseSend.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ResponseSend.java
@@ -31,7 +31,7 @@
         this(destination, header, response.toStruct());
     }
 
-    private static ByteBuffer serialize(ResponseHeader header, Struct body) {
+    public static ByteBuffer serialize(ResponseHeader header, Struct body) {
         ByteBuffer buffer = ByteBuffer.allocate(header.sizeOf() + body.sizeOf());
         header.writeTo(buffer);
         body.writeTo(buffer);
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java
new file mode 100644
index 0000000..bddc9f0
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.requests;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+
+
+/**
+ * Request from SASL client containing client SASL mechanism.
+ * <p/>
+ * For interoperability with Kafka 0.9.0.x, the mechanism flow may be omitted when using GSSAPI. Hence
+ * this request should not conflict with the first GSSAPI client packet. For GSSAPI, the first context
+ * establishment packet starts with byte 0x60 (APPLICATION-0 tag) followed by a variable-length encoded size.
+ * This handshake request starts with a request header two-byte API key set to 17, followed by a mechanism name,
+ * making it easy to distinguish from a GSSAPI packet.
+ */
+public class SaslHandshakeRequest extends AbstractRequest {
+
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.SASL_HANDSHAKE.id);
+    public static final String MECHANISM_KEY_NAME = "mechanism";
+
+    private final String mechanism;
+
+    public SaslHandshakeRequest(String mechanism) {
+        super(new Struct(CURRENT_SCHEMA));
+        struct.set(MECHANISM_KEY_NAME, mechanism);
+        this.mechanism = mechanism;
+    }
+
+    public SaslHandshakeRequest(Struct struct) {
+        super(struct);
+        mechanism = struct.getString(MECHANISM_KEY_NAME);
+    }
+
+    public String mechanism() {
+        return mechanism;
+    }
+
+    @Override
+    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
+        switch (versionId) {
+            case 0:
+                List<String> enabledMechanisms = Collections.emptyList();
+                return new SaslHandshakeResponse(Errors.forException(e).code(), enabledMechanisms);
+            default:
+                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
+                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.SASL_HANDSHAKE.id)));
+        }
+    }
+
+    public static SaslHandshakeRequest parse(ByteBuffer buffer, int versionId) {
+        return new SaslHandshakeRequest(ProtoUtils.parseRequest(ApiKeys.SASL_HANDSHAKE.id, versionId, buffer));
+    }
+
+    public static SaslHandshakeRequest parse(ByteBuffer buffer) {
+        return new SaslHandshakeRequest(CURRENT_SCHEMA.read(buffer));
+    }
+}
+
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java
new file mode 100644
index 0000000..c0fc495
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java
@@ -0,0 +1,85 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.requests;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+
+
+/**
+ * Response from SASL server which indicates if the client-chosen mechanism is enabled in the server.
+ * For error responses, the list of enabled mechanisms is included in the response.
+ */
+public class SaslHandshakeResponse extends AbstractRequestResponse {
+
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.SASL_HANDSHAKE.id);
+
+    private static final String ERROR_CODE_KEY_NAME = "error_code";
+    private static final String ENABLED_MECHANISMS_KEY_NAME = "enabled_mechanisms";
+
+    /**
+     * Possible error codes:
+     *   UNSUPPORTED_SASL_MECHANISM(33): Client mechanism not enabled in server
+     *   ILLEGAL_SASL_STATE(34) : Invalid request during SASL handshake
+     */
+    private final short errorCode;
+    private final List<String> enabledMechanisms;
+
+    public SaslHandshakeResponse(short errorCode, Collection<String> enabledMechanisms) {
+        super(new Struct(CURRENT_SCHEMA));
+        struct.set(ERROR_CODE_KEY_NAME, errorCode);
+        struct.set(ENABLED_MECHANISMS_KEY_NAME, enabledMechanisms.toArray());
+        this.errorCode = errorCode;
+        this.enabledMechanisms = new ArrayList<>(enabledMechanisms);
+    }
+
+    public SaslHandshakeResponse(Struct struct) {
+        super(struct);
+        errorCode = struct.getShort(ERROR_CODE_KEY_NAME);
+        Object[] mechanisms = struct.getArray(ENABLED_MECHANISMS_KEY_NAME);
+        ArrayList<String> enabledMechanisms = new ArrayList<>();
+        for (Object mechanism : mechanisms)
+            enabledMechanisms.add((String) mechanism);
+        this.enabledMechanisms = enabledMechanisms;
+    }
+
+    public short errorCode() {
+        return errorCode;
+    }
+
+    public List<String> enabledMechanisms() {
+        return enabledMechanisms;
+    }
+
+    public static SaslHandshakeResponse parse(ByteBuffer buffer) {
+        return new SaslHandshakeResponse(CURRENT_SCHEMA.read(buffer));
+    }
+
+    public static SaslHandshakeResponse parse(ByteBuffer buffer, int version) {
+        return new SaslHandshakeResponse(ProtoUtils.parseResponse(ApiKeys.SASL_HANDSHAKE.id, version, buffer));
+    }
+}
+
diff --git a/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java b/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java
index ff5e008..63bbafc 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java
@@ -58,7 +58,7 @@
         return null;
     }
 
-    public static String defaultRealm()
+    public static String defaultKerberosRealm()
         throws ClassNotFoundException, NoSuchMethodException,
                IllegalArgumentException, IllegalAccessException,
                InvocationTargetException {
diff --git a/clients/src/main/java/org/apache/kafka/common/security/auth/AuthCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/auth/AuthCallbackHandler.java
new file mode 100644
index 0000000..ed2c087
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/auth/AuthCallbackHandler.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.security.auth;
+
+import java.util.Map;
+
+import org.apache.kafka.common.network.Mode;
+
+import javax.security.auth.Subject;
+import javax.security.auth.callback.CallbackHandler;
+
+/*
+ * Callback handler for SASL-based authentication
+ */
+public interface AuthCallbackHandler extends CallbackHandler {
+
+    /**
+     * Configures this callback handler.
+     *
+     * @param configs Configuration
+     * @param mode The mode that indicates if this is a client or server connection
+     * @param subject Subject from login context
+     * @param saslMechanism Negotiated SASL mechanism
+     */
+    void configure(Map<String, ?> configs, Mode mode, Subject subject, String saslMechanism);
+
+    /**
+     * Closes this instance.
+     */
+    void close();
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/auth/Login.java b/clients/src/main/java/org/apache/kafka/common/security/auth/Login.java
new file mode 100644
index 0000000..1ac779d
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/auth/Login.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.security.auth;
+
+import java.util.Map;
+
+import javax.security.auth.Subject;
+import javax.security.auth.login.LoginContext;
+import javax.security.auth.login.LoginException;
+
+/**
+ * Login interface for authentication.
+ */
+public interface Login {
+
+    /**
+     * Configures this login instance.
+     */
+    void configure(Map<String, ?> configs, String loginContextName);
+
+    /**
+     * Performs login for each login module specified for the login context of this instance.
+     */
+    LoginContext login() throws LoginException;
+
+    /**
+     * Returns the authenticated subject of this login context.
+     */
+    Subject subject();
+
+    /**
+     * Returns the service name to be used for SASL.
+     */
+    String serviceName();
+
+    /**
+     * Closes this instance.
+     */
+    void close();
+}
+
diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/AbstractLogin.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/AbstractLogin.java
new file mode 100644
index 0000000..2fe43ab
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/AbstractLogin.java
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.security.authenticator;
+
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginContext;
+import javax.security.auth.login.LoginException;
+import javax.security.sasl.RealmCallback;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.Subject;
+
+import org.apache.kafka.common.security.JaasUtils;
+import org.apache.kafka.common.security.auth.Login;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+/**
+ * Base login class that implements methods common to typical SASL mechanisms.
+ */
+public abstract class AbstractLogin implements Login {
+    private static final Logger log = LoggerFactory.getLogger(AbstractLogin.class);
+
+    private String loginContextName;
+    private LoginContext loginContext;
+
+
+    @Override
+    public void configure(Map<String, ?> configs, String loginContextName) {
+        this.loginContextName = loginContextName;
+    }
+
+    @Override
+    public LoginContext login() throws LoginException {
+        String jaasConfigFile = System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM);
+        if (jaasConfigFile == null) {
+            log.debug("System property '" + JaasUtils.JAVA_LOGIN_CONFIG_PARAM + "' is not set, using default JAAS configuration.");
+        }
+        AppConfigurationEntry[] configEntries = Configuration.getConfiguration().getAppConfigurationEntry(loginContextName);
+        if (configEntries == null) {
+            String errorMessage = "Could not find a '" + loginContextName + "' entry in the JAAS configuration. System property '" +
+                JaasUtils.JAVA_LOGIN_CONFIG_PARAM + "' is " + (jaasConfigFile == null ? "not set" : jaasConfigFile);
+            throw new IllegalArgumentException(errorMessage);
+        }
+
+        loginContext = new LoginContext(loginContextName, new LoginCallbackHandler());
+        loginContext.login();
+        log.info("Successfully logged in.");
+        return loginContext;
+    }
+
+    @Override
+    public Subject subject() {
+        return loginContext.getSubject();
+    }
+
+    /**
+     * Callback handler for creating login context. Login callback handlers
+     * should support the callbacks required for the login modules used by
+     * the KafkaServer and KafkaClient contexts. Kafka does not support
+     * callback handlers which require additional user input.
+     *
+     */
+    public static class LoginCallbackHandler implements CallbackHandler {
+
+        @Override
+        public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
+            for (Callback callback : callbacks) {
+                if (callback instanceof NameCallback) {
+                    NameCallback nc = (NameCallback) callback;
+                    nc.setName(nc.getDefaultName());
+                } else if (callback instanceof PasswordCallback) {
+                    String errorMessage = "Could not login: the client is being asked for a password, but the Kafka" +
+                                 " client code does not currently support obtaining a password from the user.";
+                    throw new UnsupportedCallbackException(callback, errorMessage);
+                } else if (callback instanceof RealmCallback) {
+                    RealmCallback rc = (RealmCallback) callback;
+                    rc.setText(rc.getDefaultText());
+                } else {
+                    throw new UnsupportedCallbackException(callback, "Unrecognized SASL Login callback");
+                }
+            }
+        }
+    }
+}
+
diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/DefaultLogin.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/DefaultLogin.java
new file mode 100644
index 0000000..0a405bc
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/DefaultLogin.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.security.authenticator;
+
+public class DefaultLogin extends AbstractLogin {
+
+    @Override
+    public String serviceName() {
+        return "kafka";
+    }
+
+    @Override
+    public void close() {
+    }
+}
+
diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/LoginManager.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java
similarity index 69%
rename from clients/src/main/java/org/apache/kafka/common/security/kerberos/LoginManager.java
rename to clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java
index e163ba8..9aec9a7 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/kerberos/LoginManager.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java
@@ -16,51 +16,34 @@
  * limitations under the License.
  */
 
-package org.apache.kafka.common.security.kerberos;
+package org.apache.kafka.common.security.authenticator;
 
 import javax.security.auth.Subject;
 import javax.security.auth.login.LoginException;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.EnumMap;
 import java.util.Map;
 
-import org.apache.kafka.common.config.SaslConfigs;
 import org.apache.kafka.common.network.LoginType;
-import org.apache.kafka.common.security.JaasUtils;
+import org.apache.kafka.common.security.auth.Login;
+import org.apache.kafka.common.security.kerberos.KerberosLogin;
 
 public class LoginManager {
 
     private static final EnumMap<LoginType, LoginManager> CACHED_INSTANCES = new EnumMap<>(LoginType.class);
 
     private final Login login;
-    private final String serviceName;
     private final LoginType loginType;
     private int refCount;
 
-    private LoginManager(LoginType loginType, Map<String, ?> configs) throws IOException, LoginException {
+    private LoginManager(LoginType loginType, boolean hasKerberos, Map<String, ?> configs) throws IOException, LoginException {
         this.loginType = loginType;
         String loginContext = loginType.contextName();
-        login = new Login(loginContext, configs);
-        this.serviceName = getServiceName(loginContext, configs);
-        login.startThreadIfNeeded();
-    }
-
-    private static String getServiceName(String loginContext, Map<String, ?> configs) throws IOException {
-        String jaasServiceName = JaasUtils.jaasConfig(loginContext, JaasUtils.SERVICE_NAME);
-        String configServiceName = (String) configs.get(SaslConfigs.SASL_KERBEROS_SERVICE_NAME);
-        if (jaasServiceName != null && configServiceName != null && !jaasServiceName.equals(configServiceName)) {
-            String message = "Conflicting serviceName values found in JAAS and Kafka configs " +
-                "value in JAAS file " + jaasServiceName + ", value in Kafka config " + configServiceName;
-            throw new IllegalArgumentException(message);
-        }
-
-        if (jaasServiceName != null)
-            return jaasServiceName;
-        if (configServiceName != null)
-            return configServiceName;
-
-        throw new IllegalArgumentException("No serviceName defined in either JAAS or Kafka config");
+        login = hasKerberos ? new KerberosLogin() : new DefaultLogin();
+        login.configure(configs, loginContext);
+        login.login();
     }
 
     /**
@@ -78,11 +61,11 @@
      *                  (i.e. consumer and producer)
      * @param configs configuration as key/value pairs
      */
-    public static final LoginManager acquireLoginManager(LoginType loginType, Map<String, ?> configs) throws IOException, LoginException {
+    public static final LoginManager acquireLoginManager(LoginType loginType, boolean hasKerberos, Map<String, ?> configs) throws IOException, LoginException {
         synchronized (LoginManager.class) {
             LoginManager loginManager = CACHED_INSTANCES.get(loginType);
             if (loginManager == null) {
-                loginManager = new LoginManager(loginType, configs);
+                loginManager = new LoginManager(loginType, hasKerberos, configs);
                 CACHED_INSTANCES.put(loginType, loginManager);
             }
             return loginManager.acquire();
@@ -94,7 +77,7 @@
     }
 
     public String serviceName() {
-        return serviceName;
+        return login.serviceName();
     }
 
     private LoginManager acquire() {
@@ -111,7 +94,7 @@
                 throw new IllegalStateException("release called on LoginManager with refCount == 0");
             else if (refCount == 1) {
                 CACHED_INSTANCES.remove(loginType);
-                login.shutdown();
+                login.close();
             }
             --refCount;
         }
@@ -122,9 +105,8 @@
         synchronized (LoginManager.class) {
             for (LoginType loginType : new ArrayList<>(CACHED_INSTANCES.keySet())) {
                 LoginManager loginManager = CACHED_INSTANCES.remove(loginType);
-                loginManager.login.shutdown();
+                loginManager.login.close();
             }
         }
     }
-
 }
diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
index 370e729..ba201dc 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
@@ -23,38 +23,44 @@
 import java.nio.channels.SelectionKey;
 import java.util.Arrays;
 import java.util.Map;
-
 import java.security.Principal;
 import java.security.PrivilegedActionException;
 import java.security.PrivilegedExceptionAction;
 
 import javax.security.auth.Subject;
-import javax.security.auth.callback.Callback;
-import javax.security.auth.callback.CallbackHandler;
-import javax.security.auth.callback.NameCallback;
-import javax.security.auth.callback.PasswordCallback;
-import javax.security.auth.callback.UnsupportedCallbackException;
-import javax.security.sasl.AuthorizeCallback;
-import javax.security.sasl.RealmCallback;
 import javax.security.sasl.Sasl;
 import javax.security.sasl.SaslClient;
 import javax.security.sasl.SaslException;
 
+import org.apache.kafka.common.errors.AuthenticationException;
+import org.apache.kafka.common.errors.IllegalSaslStateException;
+import org.apache.kafka.common.errors.UnsupportedSaslMechanismException;
 import org.apache.kafka.common.network.Authenticator;
+import org.apache.kafka.common.network.Mode;
 import org.apache.kafka.common.network.NetworkSend;
 import org.apache.kafka.common.network.NetworkReceive;
 import org.apache.kafka.common.network.TransportLayer;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.requests.RequestHeader;
+import org.apache.kafka.common.requests.RequestSend;
+import org.apache.kafka.common.requests.SaslHandshakeRequest;
+import org.apache.kafka.common.requests.SaslHandshakeResponse;
+import org.apache.kafka.common.security.auth.AuthCallbackHandler;
 import org.apache.kafka.common.security.auth.KafkaPrincipal;
 import org.apache.kafka.common.security.auth.PrincipalBuilder;
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.NetworkClient;
 import org.apache.kafka.common.KafkaException;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class SaslClientAuthenticator implements Authenticator {
 
     public enum SaslState {
-        INITIAL, INTERMEDIATE, COMPLETE, FAILED
+        SEND_HANDSHAKE_REQUEST, RECEIVE_HANDSHAKE_RESPONSE, INITIAL, INTERMEDIATE, COMPLETE, FAILED
     }
 
     private static final Logger LOG = LoggerFactory.getLogger(SaslClientAuthenticator.class);
@@ -63,33 +69,57 @@
     private final String servicePrincipal;
     private final String host;
     private final String node;
+    private final String mechanism;
+    private final boolean handshakeRequestEnable;
 
     // assigned in `configure`
     private SaslClient saslClient;
+    private Map<String, ?> configs;
     private String clientPrincipalName;
+    private AuthCallbackHandler callbackHandler;
     private TransportLayer transportLayer;
 
     // buffers used in `authenticate`
     private NetworkReceive netInBuffer;
     private NetworkSend netOutBuffer;
 
-    private SaslState saslState = SaslState.INITIAL;
+    // Current SASL state
+    private SaslState saslState;
+    // Next SASL state to be set when outgoing writes associated with the current SASL state complete
+    private SaslState pendingSaslState;
+    // Correlation ID for the next request
+    private int correlationId;
+    // Request header for which response from the server is pending
+    private RequestHeader currentRequestHeader;
 
-    public SaslClientAuthenticator(String node, Subject subject, String servicePrincipal, String host) throws IOException {
+    public SaslClientAuthenticator(String node, Subject subject, String servicePrincipal, String host, String mechanism, boolean handshakeRequestEnable) throws IOException {
         this.node = node;
         this.subject = subject;
         this.host = host;
         this.servicePrincipal = servicePrincipal;
+        this.mechanism = mechanism;
+        this.handshakeRequestEnable = handshakeRequestEnable;
+        this.correlationId = -1;
     }
 
     public void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder, Map<String, ?> configs) throws KafkaException {
         try {
             this.transportLayer = transportLayer;
+            this.configs = configs;
+
+            setSaslState(handshakeRequestEnable ? SaslState.SEND_HANDSHAKE_REQUEST : SaslState.INITIAL);
 
             // determine client principal from subject.
-            Principal clientPrincipal = subject.getPrincipals().iterator().next();
-            this.clientPrincipalName = clientPrincipal.getName();
-            this.saslClient = createSaslClient();
+            if (!subject.getPrincipals().isEmpty()) {
+                Principal clientPrincipal = subject.getPrincipals().iterator().next();
+                this.clientPrincipalName = clientPrincipal.getName();
+            } else {
+                clientPrincipalName = null;
+            }
+            callbackHandler = new SaslClientCallbackHandler();
+            callbackHandler.configure(configs, Mode.CLIENT, subject, mechanism);
+
+            saslClient = createSaslClient();
         } catch (Exception e) {
             throw new KafkaException("Failed to configure SaslClientAuthenticator", e);
         }
@@ -99,15 +129,14 @@
         try {
             return Subject.doAs(subject, new PrivilegedExceptionAction<SaslClient>() {
                 public SaslClient run() throws SaslException {
-                    String[] mechs = {"GSSAPI"};
+                    String[] mechs = {mechanism};
                     LOG.debug("Creating SaslClient: client={};service={};serviceHostname={};mechs={}",
                         clientPrincipalName, servicePrincipal, host, Arrays.toString(mechs));
-                    return Sasl.createSaslClient(mechs, clientPrincipalName, servicePrincipal, host, null,
-                            new ClientCallbackHandler());
+                    return Sasl.createSaslClient(mechs, clientPrincipalName, servicePrincipal, host, configs, callbackHandler);
                 }
             });
         } catch (PrivilegedActionException e) {
-            throw new KafkaException("Failed to create SaslClient", e.getCause());
+            throw new KafkaException("Failed to create SaslClient with mechanism " + mechanism, e.getCause());
         }
     }
 
@@ -123,22 +152,39 @@
             return;
 
         switch (saslState) {
+            case SEND_HANDSHAKE_REQUEST:
+                String clientId = (String) configs.get(CommonClientConfigs.CLIENT_ID_CONFIG);
+                currentRequestHeader = new RequestHeader(ApiKeys.SASL_HANDSHAKE.id, clientId, correlationId++);
+                SaslHandshakeRequest handshakeRequest = new SaslHandshakeRequest(mechanism);
+                send(RequestSend.serialize(currentRequestHeader, handshakeRequest.toStruct()));
+                setSaslState(SaslState.RECEIVE_HANDSHAKE_RESPONSE);
+                break;
+            case RECEIVE_HANDSHAKE_RESPONSE:
+                byte[] responseBytes = receiveResponseOrToken();
+                if (responseBytes == null)
+                    break;
+                else {
+                    try {
+                        handleKafkaResponse(currentRequestHeader, responseBytes);
+                        currentRequestHeader = null;
+                    } catch (Exception e) {
+                        setSaslState(SaslState.FAILED);
+                        throw e;
+                    }
+                    setSaslState(SaslState.INITIAL);
+                    // Fall through and start SASL authentication using the configured client mechanism
+                }
             case INITIAL:
-                sendSaslToken(new byte[0]);
-                saslState = SaslState.INTERMEDIATE;
+                sendSaslToken(new byte[0], true);
+                setSaslState(SaslState.INTERMEDIATE);
                 break;
             case INTERMEDIATE:
-                if (netInBuffer == null) netInBuffer = new NetworkReceive(node);
-                netInBuffer.readFrom(transportLayer);
-                if (netInBuffer.complete()) {
-                    netInBuffer.payload().rewind();
-                    byte[] serverToken = new byte[netInBuffer.payload().remaining()];
-                    netInBuffer.payload().get(serverToken, 0, serverToken.length);
-                    netInBuffer = null; // reset the networkReceive as we read all the data.
-                    sendSaslToken(serverToken);
+                byte[] serverToken = receiveResponseOrToken();
+                if (serverToken != null) {
+                    sendSaslToken(serverToken, false);
                 }
                 if (saslClient.isComplete()) {
-                    saslState = SaslState.COMPLETE;
+                    setSaslState(SaslState.COMPLETE);
                     transportLayer.removeInterestOps(SelectionKey.OP_WRITE);
                 }
                 break;
@@ -149,30 +195,58 @@
         }
     }
 
-    private void sendSaslToken(byte[] serverToken) throws IOException {
+    private void setSaslState(SaslState saslState) {
+        if (netOutBuffer != null && !netOutBuffer.completed())
+            pendingSaslState = saslState;
+        else {
+            this.pendingSaslState = null;
+            this.saslState = saslState;
+            LOG.debug("Set SASL client state to {}", saslState);
+        }
+    }
+
+    private void sendSaslToken(byte[] serverToken, boolean isInitial) throws IOException {
         if (!saslClient.isComplete()) {
-            try {
-                byte[] saslToken = createSaslToken(serverToken);
-                if (saslToken != null) {
-                    netOutBuffer = new NetworkSend(node, ByteBuffer.wrap(saslToken));
-                    flushNetOutBufferAndUpdateInterestOps();
-                }
-            } catch (IOException e) {
-                saslState = SaslState.FAILED;
-                throw e;
-            }
+            byte[] saslToken = createSaslToken(serverToken, isInitial);
+            if (saslToken != null)
+                send(ByteBuffer.wrap(saslToken));
+        }
+    }
+
+    private void send(ByteBuffer buffer) throws IOException {
+        try {
+            netOutBuffer = new NetworkSend(node, buffer);
+            flushNetOutBufferAndUpdateInterestOps();
+        } catch (IOException e) {
+            setSaslState(SaslState.FAILED);
+            throw e;
         }
     }
 
     private boolean flushNetOutBufferAndUpdateInterestOps() throws IOException {
         boolean flushedCompletely = flushNetOutBuffer();
-        if (flushedCompletely)
+        if (flushedCompletely) {
             transportLayer.removeInterestOps(SelectionKey.OP_WRITE);
-        else
+            if (pendingSaslState != null)
+                setSaslState(pendingSaslState);
+        } else
             transportLayer.addInterestOps(SelectionKey.OP_WRITE);
         return flushedCompletely;
     }
 
+    private byte[] receiveResponseOrToken() throws IOException {
+        if (netInBuffer == null) netInBuffer = new NetworkReceive(node);
+        netInBuffer.readFrom(transportLayer);
+        byte[] serverPacket = null;
+        if (netInBuffer.complete()) {
+            netInBuffer.payload().rewind();
+            serverPacket = new byte[netInBuffer.payload().remaining()];
+            netInBuffer.payload().get(serverPacket, 0, serverPacket.length);
+            netInBuffer = null; // reset the networkReceive as we read all the data.
+        }
+        return serverPacket;
+    }
+
     public Principal principal() {
         return new KafkaPrincipal(KafkaPrincipal.USER_TYPE, clientPrincipalName);
     }
@@ -182,19 +256,25 @@
     }
 
     public void close() throws IOException {
-        saslClient.dispose();
+        if (saslClient != null)
+            saslClient.dispose();
+        if (callbackHandler != null)
+            callbackHandler.close();
     }
 
-    private byte[] createSaslToken(final byte[] saslToken) throws SaslException {
+    private byte[] createSaslToken(final byte[] saslToken, boolean isInitial) throws SaslException {
         if (saslToken == null)
             throw new SaslException("Error authenticating with the Kafka Broker: received a `null` saslToken.");
 
         try {
-            return Subject.doAs(subject, new PrivilegedExceptionAction<byte[]>() {
-                public byte[] run() throws SaslException {
-                    return saslClient.evaluateChallenge(saslToken);
-                }
-            });
+            if (isInitial && !saslClient.hasInitialResponse())
+                return saslToken;
+            else
+                return Subject.doAs(subject, new PrivilegedExceptionAction<byte[]>() {
+                    public byte[] run() throws SaslException {
+                        return saslClient.evaluateChallenge(saslToken);
+                    }
+                });
         } catch (PrivilegedActionException e) {
             String error = "An error: (" + e + ") occurred when evaluating SASL token received from the Kafka Broker.";
             // Try to provide hints to use about what went wrong so they can fix their configuration.
@@ -221,35 +301,39 @@
         return netOutBuffer.completed();
     }
 
-    public static class ClientCallbackHandler implements CallbackHandler {
+    private void handleKafkaResponse(RequestHeader requestHeader, byte[] responseBytes) {
+        Struct struct;
+        ApiKeys apiKey;
+        try {
+            struct = NetworkClient.parseResponse(ByteBuffer.wrap(responseBytes), requestHeader);
+            apiKey = ApiKeys.forId(requestHeader.apiKey());
+        } catch (SchemaException | IllegalArgumentException e) {
+            LOG.debug("Invalid SASL mechanism response, server may be expecting only GSSAPI tokens");
+            throw new AuthenticationException("Invalid SASL mechanism response", e);
+        }
+        switch (apiKey) {
+            case SASL_HANDSHAKE:
+                handleSaslHandshakeResponse(new SaslHandshakeResponse(struct));
+                break;
+            default:
+                throw new IllegalStateException("Unexpected API key during handshake: " + apiKey);
+        }
+    }
 
-        public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
-            for (Callback callback : callbacks) {
-                if (callback instanceof NameCallback) {
-                    NameCallback nc = (NameCallback) callback;
-                    nc.setName(nc.getDefaultName());
-                } else if (callback instanceof PasswordCallback) {
-                    // Call `setPassword` once we support obtaining a password from the user and update message below
-                    throw new UnsupportedCallbackException(callback, "Could not login: the client is being asked for a password, but the Kafka" +
-                             " client code does not currently support obtaining a password from the user." +
-                             " Make sure -Djava.security.auth.login.config property passed to JVM and" +
-                             " the client is configured to use a ticket cache (using" +
-                             " the JAAS configuration setting 'useTicketCache=true)'. Make sure you are using" +
-                             " FQDN of the Kafka broker you are trying to connect to.");
-                } else if (callback instanceof RealmCallback) {
-                    RealmCallback rc = (RealmCallback) callback;
-                    rc.setText(rc.getDefaultText());
-                } else if (callback instanceof AuthorizeCallback) {
-                    AuthorizeCallback ac = (AuthorizeCallback) callback;
-                    String authId = ac.getAuthenticationID();
-                    String authzId = ac.getAuthorizationID();
-                    ac.setAuthorized(authId.equals(authzId));
-                    if (ac.isAuthorized())
-                        ac.setAuthorizedID(authzId);
-                } else {
-                    throw new UnsupportedCallbackException(callback, "Unrecognized SASL ClientCallback");
-                }
-            }
+    private void handleSaslHandshakeResponse(SaslHandshakeResponse response) {
+        Errors error = Errors.forCode(response.errorCode());
+        switch (error) {
+            case NONE:
+                break;
+            case UNSUPPORTED_SASL_MECHANISM:
+                throw new UnsupportedSaslMechanismException(String.format("Client SASL mechanism '%s' not enabled in the server, enabled mechanisms are %s",
+                    mechanism, response.enabledMechanisms()));
+            case ILLEGAL_SASL_STATE:
+                throw new IllegalSaslStateException(String.format("Unexpected handshake request with client mechanism %s, enabled mechanisms are %s",
+                    mechanism, response.enabledMechanisms()));
+            default:
+                throw new AuthenticationException(String.format("Unknown error code %d, client mechanism is %s, enabled mechanisms are %s",
+                    response.errorCode(), mechanism, response.enabledMechanisms()));
         }
     }
 }
diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java
new file mode 100644
index 0000000..8e0b8db
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java
@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.security.authenticator;
+
+import java.util.Map;
+
+import javax.security.auth.Subject;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.RealmCallback;
+
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.network.Mode;
+import org.apache.kafka.common.security.auth.AuthCallbackHandler;
+
+/**
+ * Callback handler for Sasl clients. The callbacks required for the SASL mechanism
+ * configured for the client should be supported by this callback handler. See
+ * <a href="https://docs.oracle.com/javase/8/docs/technotes/guides/security/sasl/sasl-refguide.html">Java SASL API</a>
+ * for the list of SASL callback handlers required for each SASL mechanism.
+ */
+public class SaslClientCallbackHandler implements AuthCallbackHandler {
+
+    private boolean isKerberos;
+    private Subject subject;
+
+    @Override
+    public void configure(Map<String, ?> configs, Mode mode, Subject subject, String mechanism) {
+        this.isKerberos = mechanism.equals(SaslConfigs.GSSAPI_MECHANISM);
+        this.subject = subject;
+    }
+
+    @Override
+    public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
+        for (Callback callback : callbacks) {
+            if (callback instanceof NameCallback) {
+                NameCallback nc = (NameCallback) callback;
+                if (!isKerberos && subject != null && !subject.getPublicCredentials(String.class).isEmpty()) {
+                    nc.setName(subject.getPublicCredentials(String.class).iterator().next());
+                } else
+                    nc.setName(nc.getDefaultName());
+            } else if (callback instanceof PasswordCallback) {
+                if (!isKerberos && subject != null && !subject.getPrivateCredentials(String.class).isEmpty()) {
+                    char [] password = subject.getPrivateCredentials(String.class).iterator().next().toCharArray();
+                    ((PasswordCallback) callback).setPassword(password);
+                } else {
+                    String errorMessage = "Could not login: the client is being asked for a password, but the Kafka" +
+                             " client code does not currently support obtaining a password from the user.";
+                    if (isKerberos) {
+                        errorMessage += " Make sure -Djava.security.auth.login.config property passed to JVM and" +
+                             " the client is configured to use a ticket cache (using" +
+                             " the JAAS configuration setting 'useTicketCache=true)'. Make sure you are using" +
+                             " FQDN of the Kafka broker you are trying to connect to.";
+                    }
+                    throw new UnsupportedCallbackException(callback, errorMessage);
+                }
+            } else if (callback instanceof RealmCallback) {
+                RealmCallback rc = (RealmCallback) callback;
+                rc.setText(rc.getDefaultText());
+            } else if (callback instanceof AuthorizeCallback) {
+                AuthorizeCallback ac = (AuthorizeCallback) callback;
+                String authId = ac.getAuthenticationID();
+                String authzId = ac.getAuthorizationID();
+                ac.setAuthorized(authId.equals(authzId));
+                if (ac.isAuthorized())
+                    ac.setAuthorizedID(authzId);
+            } else {
+                throw new UnsupportedCallbackException(callback, "Unrecognized SASL ClientCallback");
+            }
+        }
+    }
+
+    @Override
+    public void close() {
+    }
+}
\ No newline at end of file
diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java
index 1f925f9..a9c19a5 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java
@@ -18,11 +18,12 @@
 package org.apache.kafka.common.security.authenticator;
 
 import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
-
+import java.util.Set;
 import java.nio.ByteBuffer;
 import java.nio.channels.SelectionKey;
-
 import java.security.Principal;
 import java.security.PrivilegedActionException;
 import java.security.PrivilegedExceptionAction;
@@ -44,51 +45,104 @@
 import org.ietf.jgss.Oid;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
+import org.apache.kafka.common.security.auth.AuthCallbackHandler;
 import org.apache.kafka.common.security.auth.KafkaPrincipal;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.errors.AuthenticationException;
+import org.apache.kafka.common.errors.IllegalSaslStateException;
+import org.apache.kafka.common.errors.UnsupportedSaslMechanismException;
 import org.apache.kafka.common.network.Authenticator;
+import org.apache.kafka.common.network.Mode;
 import org.apache.kafka.common.network.NetworkSend;
 import org.apache.kafka.common.network.NetworkReceive;
 import org.apache.kafka.common.network.TransportLayer;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.requests.AbstractRequest;
+import org.apache.kafka.common.requests.AbstractRequestResponse;
+import org.apache.kafka.common.requests.ApiVersionsRequest;
+import org.apache.kafka.common.requests.ApiVersionsResponse;
+import org.apache.kafka.common.requests.RequestHeader;
+import org.apache.kafka.common.requests.ResponseHeader;
+import org.apache.kafka.common.requests.ResponseSend;
+import org.apache.kafka.common.requests.SaslHandshakeRequest;
+import org.apache.kafka.common.requests.SaslHandshakeResponse;
 import org.apache.kafka.common.security.auth.PrincipalBuilder;
 
 public class SaslServerAuthenticator implements Authenticator {
 
     private static final Logger LOG = LoggerFactory.getLogger(SaslServerAuthenticator.class);
 
-    private final SaslServer saslServer;
-    private final Subject subject;
+    public enum SaslState {
+        HANDSHAKE_REQUEST, AUTHENTICATE, COMPLETE, FAILED
+    }
+
     private final String node;
+    private final Subject subject;
     private final KerberosShortNamer kerberosNamer;
     private final int maxReceiveSize;
+    private final String host;
+
+    // Current SASL state
+    private SaslState saslState = SaslState.HANDSHAKE_REQUEST;
+    // Next SASL state to be set when outgoing writes associated with the current SASL state complete
+    private SaslState pendingSaslState = null;
+    private SaslServer saslServer;
+    private String saslMechanism;
+    private AuthCallbackHandler callbackHandler;
 
     // assigned in `configure`
     private TransportLayer transportLayer;
+    private Set<String> enabledMechanisms;
+    private Map<String, ?> configs;
 
     // buffers used in `authenticate`
     private NetworkReceive netInBuffer;
     private NetworkSend netOutBuffer;
 
-    public SaslServerAuthenticator(String node, final Subject subject, KerberosShortNamer kerberosNameParser, int maxReceiveSize) throws IOException {
+    public SaslServerAuthenticator(String node, final Subject subject, KerberosShortNamer kerberosNameParser, String host, int maxReceiveSize) throws IOException {
         if (subject == null)
             throw new IllegalArgumentException("subject cannot be null");
-        if (subject.getPrincipals().isEmpty())
-            throw new IllegalArgumentException("subject must have at least one principal");
         this.node = node;
         this.subject = subject;
         this.kerberosNamer = kerberosNameParser;
         this.maxReceiveSize = maxReceiveSize;
-        saslServer = createSaslServer();
+        this.host = host;
     }
 
     public void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder, Map<String, ?> configs) {
         this.transportLayer = transportLayer;
+        this.configs = configs;
+        List<String> enabledMechanisms = (List<String>) this.configs.get(SaslConfigs.SASL_ENABLED_MECHANISMS);
+        if (enabledMechanisms == null || enabledMechanisms.isEmpty())
+            throw new IllegalArgumentException("No SASL mechanisms are enabled");
+        this.enabledMechanisms = new HashSet<>(enabledMechanisms);
     }
 
-    private SaslServer createSaslServer() throws IOException {
+    private void createSaslServer(String mechanism) throws IOException {
+        this.saslMechanism = mechanism;
+        callbackHandler = new SaslServerCallbackHandler(Configuration.getConfiguration(), kerberosNamer);
+        callbackHandler.configure(configs, Mode.SERVER, subject, saslMechanism);
+        if (mechanism.equals(SaslConfigs.GSSAPI_MECHANISM)) {
+            if (subject.getPrincipals().isEmpty())
+                throw new IllegalArgumentException("subject must have at least one principal");
+            saslServer = createSaslKerberosServer(callbackHandler, configs);
+        } else {
+            try {
+                saslServer = Subject.doAs(subject, new PrivilegedExceptionAction<SaslServer>() {
+                    public SaslServer run() throws SaslException {
+                        return Sasl.createSaslServer(saslMechanism, "kafka", host, configs, callbackHandler);
+                    }
+                });
+            } catch (PrivilegedActionException e) {
+                throw new SaslException("Kafka Server failed to create a SaslServer to interact with a client during session authentication", e.getCause());
+            }
+        }
+    }
+
+    private SaslServer createSaslKerberosServer(final AuthCallbackHandler saslServerCallbackHandler, final Map<String, ?> configs) throws IOException {
         // server is using a JAAS-authenticated subject: determine service principal name and hostname from kafka server's subject.
-        final SaslServerCallbackHandler saslServerCallbackHandler = new SaslServerCallbackHandler(
-                Configuration.getConfiguration(), kerberosNamer);
         final Principal servicePrincipal = subject.getPrincipals().iterator().next();
         KerberosName kerberosName;
         try {
@@ -99,9 +153,7 @@
         final String servicePrincipalName = kerberosName.serviceName();
         final String serviceHostname = kerberosName.hostName();
 
-        final String mech = "GSSAPI";
-
-        LOG.debug("Creating SaslServer for {} with mechanism {}", kerberosName, mech);
+        LOG.debug("Creating SaslServer for {} with mechanism {}", kerberosName, saslMechanism);
 
         // As described in http://docs.oracle.com/javase/8/docs/technotes/guides/security/jgss/jgss-features.html:
         // "To enable Java GSS to delegate to the native GSS library and its list of native mechanisms,
@@ -127,7 +179,7 @@
         try {
             return Subject.doAs(subject, new PrivilegedExceptionAction<SaslServer>() {
                 public SaslServer run() throws SaslException {
-                    return Sasl.createSaslServer(mech, servicePrincipalName, serviceHostname, null, saslServerCallbackHandler);
+                    return Sasl.createSaslServer(saslMechanism, servicePrincipalName, serviceHostname, configs, saslServerCallbackHandler);
                 }
             });
         } catch (PrivilegedActionException e) {
@@ -146,8 +198,8 @@
         if (netOutBuffer != null && !flushNetOutBufferAndUpdateInterestOps())
             return;
 
-        if (saslServer.isComplete()) {
-            transportLayer.removeInterestOps(SelectionKey.OP_WRITE);
+        if (saslServer != null && saslServer.isComplete()) {
+            setSaslState(SaslState.COMPLETE);
             return;
         }
 
@@ -161,12 +213,28 @@
             netInBuffer.payload().get(clientToken, 0, clientToken.length);
             netInBuffer = null; // reset the networkReceive as we read all the data.
             try {
-                byte[] response = saslServer.evaluateResponse(clientToken);
-                if (response != null) {
-                    netOutBuffer = new NetworkSend(node, ByteBuffer.wrap(response));
-                    flushNetOutBufferAndUpdateInterestOps();
+                switch (saslState) {
+                    case HANDSHAKE_REQUEST:
+                        if (handleKafkaRequest(clientToken))
+                            break;
+                        // For default GSSAPI, fall through to authenticate using the client token as the first GSSAPI packet.
+                        // This is required for interoperability with 0.9.0.x clients which do not send handshake request
+                    case AUTHENTICATE:
+                        byte[] response = saslServer.evaluateResponse(clientToken);
+                        if (response != null) {
+                            netOutBuffer = new NetworkSend(node, ByteBuffer.wrap(response));
+                            flushNetOutBufferAndUpdateInterestOps();
+                        }
+                        // When the authentication exchange is complete and no more tokens are expected from the client,
+                        // update SASL state. Current SASL state will be updated when outgoing writes to the client complete.
+                        if (saslServer.isComplete())
+                            setSaslState(SaslState.COMPLETE);
+                        break;
+                    default:
+                        break;
                 }
             } catch (Exception e) {
+                setSaslState(SaslState.FAILED);
                 throw new IOException(e);
             }
         }
@@ -177,18 +245,33 @@
     }
 
     public boolean complete() {
-        return saslServer.isComplete();
+        return saslState == SaslState.COMPLETE;
     }
 
     public void close() throws IOException {
-        saslServer.dispose();
+        if (saslServer != null)
+            saslServer.dispose();
+        if (callbackHandler != null)
+            callbackHandler.close();
+    }
+
+    private void setSaslState(SaslState saslState) {
+        if (netOutBuffer != null && !netOutBuffer.completed())
+            pendingSaslState = saslState;
+        else {
+            this.pendingSaslState = null;
+            this.saslState = saslState;
+            LOG.debug("Set SASL server state to {}", saslState);
+        }
     }
 
     private boolean flushNetOutBufferAndUpdateInterestOps() throws IOException {
         boolean flushedCompletely = flushNetOutBuffer();
-        if (flushedCompletely)
+        if (flushedCompletely) {
             transportLayer.removeInterestOps(SelectionKey.OP_WRITE);
-        else
+            if (pendingSaslState != null)
+                setSaslState(pendingSaslState);
+        } else
             transportLayer.addInterestOps(SelectionKey.OP_WRITE);
         return flushedCompletely;
     }
@@ -198,4 +281,74 @@
             netOutBuffer.writeTo(transportLayer);
         return netOutBuffer.completed();
     }
+
+    private boolean handleKafkaRequest(byte[] requestBytes) throws IOException, AuthenticationException {
+        boolean isKafkaRequest = false;
+        String clientMechanism = null;
+        try {
+            ByteBuffer requestBuffer = ByteBuffer.wrap(requestBytes);
+            RequestHeader requestHeader = RequestHeader.parse(requestBuffer);
+            AbstractRequest request = AbstractRequest.getRequest(requestHeader.apiKey(), requestHeader.apiVersion(), requestBuffer);
+            isKafkaRequest = true;
+
+            ApiKeys apiKey = ApiKeys.forId(requestHeader.apiKey());
+            LOG.debug("Handle Kafka request {}", apiKey);
+            switch (apiKey) {
+                case API_VERSIONS:
+                    handleApiVersionsRequest(requestHeader, (ApiVersionsRequest) request);
+                    break;
+                case SASL_HANDSHAKE:
+                    clientMechanism = handleHandshakeRequest(requestHeader, (SaslHandshakeRequest) request);
+                    break;
+                default:
+                    throw new IllegalSaslStateException("Unexpected Kafka request of type " + apiKey + " during SASL handshake.");
+            }
+        } catch (SchemaException | IllegalArgumentException e) {
+            // SchemaException is thrown if the request is not in Kafka format. IIlegalArgumentException is thrown
+            // if the API key is invalid. For compatibility with 0.9.0.x where the first packet is a GSSAPI token
+            // starting with 0x60, revert to GSSAPI for both these exceptions.
+            if (LOG.isDebugEnabled()) {
+                StringBuilder tokenBuilder = new StringBuilder();
+                for (byte b : requestBytes) {
+                    tokenBuilder.append(String.format("%02x", b));
+                    if (tokenBuilder.length() >= 20)
+                         break;
+                }
+                LOG.debug("Received client packet of length {} starting with bytes 0x{}, process as GSSAPI packet", requestBytes.length, tokenBuilder);
+            }
+            if (enabledMechanisms.contains(SaslConfigs.GSSAPI_MECHANISM)) {
+                LOG.debug("First client packet is not a SASL mechanism request, using default mechanism GSSAPI");
+                clientMechanism = SaslConfigs.GSSAPI_MECHANISM;
+            } else
+                throw new UnsupportedSaslMechanismException("Exception handling first SASL packet from client, GSSAPI is not supported by server", e);
+        }
+        if (clientMechanism != null) {
+            createSaslServer(clientMechanism);
+            setSaslState(SaslState.AUTHENTICATE);
+        }
+        return isKafkaRequest;
+    }
+
+    private String handleHandshakeRequest(RequestHeader requestHeader, SaslHandshakeRequest handshakeRequest) throws IOException, UnsupportedSaslMechanismException {
+        String clientMechanism = handshakeRequest.mechanism();
+        if (enabledMechanisms.contains(clientMechanism)) {
+            LOG.debug("Using SASL mechanism '{}' provided by client", clientMechanism);
+            sendKafkaResponse(requestHeader, new SaslHandshakeResponse((short) 0, enabledMechanisms));
+            return clientMechanism;
+        } else {
+            LOG.debug("SASL mechanism '{}' requested by client is not supported", clientMechanism);
+            sendKafkaResponse(requestHeader, new SaslHandshakeResponse(Errors.UNSUPPORTED_SASL_MECHANISM.code(), enabledMechanisms));
+            throw new UnsupportedSaslMechanismException("Unsupported SASL mechanism " + clientMechanism);
+        }
+    }
+
+    private void handleApiVersionsRequest(RequestHeader requestHeader, ApiVersionsRequest versionRequest) throws IOException, UnsupportedSaslMechanismException {
+        sendKafkaResponse(requestHeader, ApiVersionsResponse.apiVersionsResponse());
+    }
+
+    private void sendKafkaResponse(RequestHeader requestHeader, AbstractRequestResponse response) throws IOException {
+        ResponseHeader responseHeader = new ResponseHeader(requestHeader.correlationId());
+        netOutBuffer = new NetworkSend(node, ResponseSend.serialize(responseHeader, response.toStruct()));
+        flushNetOutBufferAndUpdateInterestOps();
+    }
 }
diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerCallbackHandler.java
index 1de4a2e..c23e390 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerCallbackHandler.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerCallbackHandler.java
@@ -19,12 +19,15 @@
 package org.apache.kafka.common.security.authenticator;
 
 import java.io.IOException;
+import java.util.Map;
 
+import org.apache.kafka.common.security.auth.AuthCallbackHandler;
 import org.apache.kafka.common.security.kerberos.KerberosShortNamer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
+import javax.security.auth.Subject;
 import javax.security.auth.callback.Callback;
-import javax.security.auth.callback.CallbackHandler;
 import javax.security.auth.callback.UnsupportedCallbackException;
 import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.login.Configuration;
@@ -32,9 +35,16 @@
 import javax.security.sasl.RealmCallback;
 
 import org.apache.kafka.common.security.kerberos.KerberosName;
+import org.apache.kafka.common.network.Mode;
 import org.apache.kafka.common.security.JaasUtils;
 
-public class SaslServerCallbackHandler implements CallbackHandler {
+/**
+ * Callback handler for Sasl servers. The callbacks required for all the SASL
+ * mechanisms enabled in the server should be supported by this callback handler. See
+ * <a href="https://docs.oracle.com/javase/8/docs/technotes/guides/security/sasl/sasl-refguide.html">Java SASL API</a>
+ * for the list of SASL callback handlers required for each SASL mechanism.
+ */
+public class SaslServerCallbackHandler implements AuthCallbackHandler {
     private static final Logger LOG = LoggerFactory.getLogger(SaslServerCallbackHandler.class);
     private final KerberosShortNamer kerberosShortNamer;
 
@@ -45,6 +55,11 @@
         this.kerberosShortNamer = kerberosNameParser;
     }
 
+    @Override
+    public void configure(Map<String, ?> configs, Mode mode, Subject subject, String saslMechanism) {
+    }
+
+    @Override
     public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
         for (Callback callback : callbacks) {
             if (callback instanceof RealmCallback) {
@@ -78,4 +93,7 @@
         }
     }
 
+    @Override
+    public void close() {
+    }
 }
diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/Login.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java
similarity index 86%
rename from clients/src/main/java/org/apache/kafka/common/security/kerberos/Login.java
rename to clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java
index 2e1a056..58becdf 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/kerberos/Login.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java
@@ -23,21 +23,21 @@
 import javax.security.auth.login.Configuration;
 import javax.security.auth.login.LoginContext;
 import javax.security.auth.login.LoginException;
-import javax.security.auth.callback.CallbackHandler;
 import javax.security.auth.kerberos.KerberosTicket;
 import javax.security.auth.Subject;
 
-import org.apache.kafka.common.security.authenticator.SaslClientAuthenticator.ClientCallbackHandler;
+import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.security.JaasUtils;
+import org.apache.kafka.common.security.authenticator.AbstractLogin;
 import org.apache.kafka.common.config.SaslConfigs;
 import org.apache.kafka.common.utils.Shell;
 import org.apache.kafka.common.utils.Time;
 import org.apache.kafka.common.utils.SystemTime;
-
 import org.apache.kafka.common.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
 import java.util.Date;
 import java.util.Random;
 import java.util.Set;
@@ -47,40 +47,40 @@
  * This class is responsible for refreshing Kerberos credentials for
  * logins for both Kafka client and server.
  */
-public class Login {
-    private static final Logger log = LoggerFactory.getLogger(Login.class);
+public class KerberosLogin extends AbstractLogin {
+    private static final Logger log = LoggerFactory.getLogger(KerberosLogin.class);
 
     private static final Random RNG = new Random();
 
-    private final Thread t;
-    private final boolean isKrbTicket;
-    private final boolean isUsingTicketCache;
-
-    private final String loginContextName;
-    private final String principal;
     private final Time time = new SystemTime();
-    private final CallbackHandler callbackHandler = new ClientCallbackHandler();
+    private Thread t;
+    private boolean isKrbTicket;
+    private boolean isUsingTicketCache;
+
+    private String loginContextName;
+    private String principal;
 
     // LoginThread will sleep until 80% of time from last refresh to
     // ticket's expiry has been reached, at which time it will wake
     // and try to renew the ticket.
-    private final double ticketRenewWindowFactor;
+    private double ticketRenewWindowFactor;
 
     /**
      * Percentage of random jitter added to the renewal time
      */
-    private final double ticketRenewJitter;
+    private double ticketRenewJitter;
 
     // Regardless of ticketRenewWindowFactor setting above and the ticket expiry time,
     // thread will not sleep between refresh attempts any less than 1 minute (60*1000 milliseconds = 1 minute).
     // Change the '1' to e.g. 5, to change this to 5 minutes.
-    private final long minTimeBeforeRelogin;
+    private long minTimeBeforeRelogin;
 
-    private final String kinitCmd;
+    private String kinitCmd;
 
     private volatile Subject subject;
 
-    private LoginContext login;
+    private LoginContext loginContext;
+    private String serviceName;
     private long lastLogin;
 
     /**
@@ -93,16 +93,22 @@
      * @throws javax.security.auth.login.LoginException
      *               Thrown if authentication fails.
      */
-    public Login(final String loginContextName, Map<String, ?> configs) throws LoginException {
+    public void configure(Map<String, ?> configs, final String loginContextName) {
+        super.configure(configs, loginContextName);
         this.loginContextName = loginContextName;
         this.ticketRenewWindowFactor = (Double) configs.get(SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR);
         this.ticketRenewJitter = (Double) configs.get(SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER);
         this.minTimeBeforeRelogin = (Long) configs.get(SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN);
         this.kinitCmd = (String) configs.get(SaslConfigs.SASL_KERBEROS_KINIT_CMD);
+        this.serviceName = getServiceName(configs, loginContextName);
+    }
+
+    @Override
+    public LoginContext login() throws LoginException {
 
         this.lastLogin = currentElapsedTime();
-        login = login(loginContextName);
-        subject = login.getSubject();
+        loginContext = super.login();
+        subject = loginContext.getSubject();
         isKrbTicket = !subject.getPrivateCredentials(KerberosTicket.class).isEmpty();
 
         AppConfigurationEntry[] entries = Configuration.getConfiguration().getAppConfigurationEntry(loginContextName);
@@ -127,7 +133,7 @@
             log.debug("It is not a Kerberos ticket");
             t = null;
             // if no TGT, do not bother with ticket management.
-            return;
+            return loginContext;
         }
         log.debug("It is a Kerberos ticket");
 
@@ -259,16 +265,12 @@
                 }
             }
         }, true);
+        t.start();
+        return loginContext;
     }
 
-    public void startThreadIfNeeded() {
-        // thread object 't' will be null if a refresh thread is not needed.
-        if (t != null) {
-            t.start();
-        }
-    }
-
-    public void shutdown() {
+    @Override
+    public void close() {
         if ((t != null) && (t.isAlive())) {
             t.interrupt();
             try {
@@ -279,28 +281,39 @@
         }
     }
 
+    @Override
     public Subject subject() {
         return subject;
     }
 
-    private synchronized LoginContext login(final String loginContextName) throws LoginException {
-        String jaasConfigFile = System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM);
-        if (jaasConfigFile == null) {
-            log.debug("System property '" + JaasUtils.JAVA_LOGIN_CONFIG_PARAM + "' is not set, using default JAAS configuration.");
+    @Override
+    public String serviceName() {
+        return serviceName;
+    }
+
+    private String getServiceName(Map<String, ?> configs, String loginContext) {
+        String jaasServiceName;
+        try {
+            jaasServiceName = JaasUtils.jaasConfig(loginContext, JaasUtils.SERVICE_NAME);
+        } catch (IOException e) {
+            throw new KafkaException("Jaas configuration not found", e);
         }
-        AppConfigurationEntry[] configEntries = Configuration.getConfiguration().getAppConfigurationEntry(loginContextName);
-        if (configEntries == null) {
-            String errorMessage = "Could not find a '" + loginContextName + "' entry in the JAAS configuration. System property '" +
-                JaasUtils.JAVA_LOGIN_CONFIG_PARAM + "' is " + (jaasConfigFile == null ? "not set" : jaasConfigFile);
-            throw new IllegalArgumentException(errorMessage);
+        String configServiceName = (String) configs.get(SaslConfigs.SASL_KERBEROS_SERVICE_NAME);
+        if (jaasServiceName != null && configServiceName != null && !jaasServiceName.equals(configServiceName)) {
+            String message = "Conflicting serviceName values found in JAAS and Kafka configs " +
+                "value in JAAS file " + jaasServiceName + ", value in Kafka config " + configServiceName;
+            throw new IllegalArgumentException(message);
         }
 
-        LoginContext loginContext = new LoginContext(loginContextName, callbackHandler);
-        loginContext.login();
-        log.info("Successfully logged in.");
-        return loginContext;
+        if (jaasServiceName != null)
+            return jaasServiceName;
+        if (configServiceName != null)
+            return configServiceName;
+
+        throw new IllegalArgumentException("No serviceName defined in either JAAS or Kafka config");
     }
 
+
     private long getRefreshTime(KerberosTicket tgt) {
         long start = tgt.getStartTime().getTime();
         long expires = tgt.getEndTime().getTime();
@@ -346,25 +359,25 @@
         if (!isKrbTicket) {
             return;
         }
-        if (login == null) {
+        if (loginContext == null) {
             throw new LoginException("Login must be done first");
         }
         if (!hasSufficientTimeElapsed()) {
             return;
         }
         log.info("Initiating logout for {}", principal);
-        synchronized (Login.class) {
+        synchronized (KerberosLogin.class) {
             // register most recent relogin attempt
             lastLogin = currentElapsedTime();
             //clear up the kerberos state. But the tokens are not cleared! As per
             //the Java kerberos login module code, only the kerberos credentials
             //are cleared
-            login.logout();
+            loginContext.logout();
             //login and also update the subject field of this instance to
             //have the new credentials (pass it to the LoginContext constructor)
-            login = new LoginContext(loginContextName, subject);
+            loginContext = new LoginContext(loginContextName, subject);
             log.info("Initiating re-login for {}", principal);
-            login.login();
+            loginContext.login();
         }
     }
 
diff --git a/clients/src/main/java/org/apache/kafka/common/security/plain/PlainLoginModule.java b/clients/src/main/java/org/apache/kafka/common/security/plain/PlainLoginModule.java
new file mode 100644
index 0000000..f06fbf6
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/plain/PlainLoginModule.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.security.plain;
+
+import java.util.Map;
+
+import javax.security.auth.Subject;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.login.LoginException;
+import javax.security.auth.spi.LoginModule;
+
+public class PlainLoginModule implements LoginModule {
+
+    private static final String USERNAME_CONFIG = "username";
+    private static final String PASSWORD_CONFIG = "password";
+
+    static {
+        PlainSaslServerProvider.initialize();
+    }
+
+    @Override
+    public void initialize(Subject subject, CallbackHandler callbackHandler, Map<String, ?> sharedState, Map<String, ?> options) {
+        String username = (String) options.get(USERNAME_CONFIG);
+        if (username != null)
+            subject.getPublicCredentials().add(username);
+        String password = (String) options.get(PASSWORD_CONFIG);
+        if (password != null)
+            subject.getPrivateCredentials().add(password);
+    }
+
+    @Override
+    public boolean login() throws LoginException {
+        return true;
+    }
+
+    @Override
+    public boolean logout() throws LoginException {
+        return true;
+    }
+
+    @Override
+    public boolean commit() throws LoginException {
+        return true;
+    }
+
+    @Override
+    public boolean abort() throws LoginException {
+        return false;
+    }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServer.java b/clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServer.java
new file mode 100644
index 0000000..5c6fd78
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServer.java
@@ -0,0 +1,170 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.security.plain;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.util.Arrays;
+import java.util.Map;
+
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import javax.security.sasl.SaslServerFactory;
+
+import org.apache.kafka.common.network.LoginType;
+import org.apache.kafka.common.security.JaasUtils;
+
+/**
+ * Simple SaslServer implementation for SASL/PLAIN. In order to make this implementation
+ * fully pluggable, authentication of username/password is fully contained within the
+ * server implementation.
+ * <p>
+ * Valid users with passwords are specified in the Jaas configuration file. Each user
+ * is specified with user_<username> as key and <password> as value. This is consistent
+ * with Zookeeper Digest-MD5 implementation.
+ * <p>
+ * To avoid storing clear passwords on disk or to integrate with external authentication
+ * servers in production systems, this module can be replaced with a different implementation.
+ *
+ */
+public class PlainSaslServer implements SaslServer {
+
+    public static final String PLAIN_MECHANISM = "PLAIN";
+    private static final String JAAS_USER_PREFIX = "user_";
+
+    private boolean complete;
+    private String authorizationID;
+
+    public PlainSaslServer(CallbackHandler callbackHandler) {
+    }
+
+    @Override
+    public byte[] evaluateResponse(byte[] response) throws SaslException {
+        /*
+         * Message format (from https://tools.ietf.org/html/rfc4616):
+         *
+         * message   = [authzid] UTF8NUL authcid UTF8NUL passwd
+         * authcid   = 1*SAFE ; MUST accept up to 255 octets
+         * authzid   = 1*SAFE ; MUST accept up to 255 octets
+         * passwd    = 1*SAFE ; MUST accept up to 255 octets
+         * UTF8NUL   = %x00 ; UTF-8 encoded NUL character
+         *
+         * SAFE      = UTF1 / UTF2 / UTF3 / UTF4
+         *                ;; any UTF-8 encoded Unicode character except NUL
+         */
+
+        String[] tokens;
+        try {
+            tokens = new String(response, "UTF-8").split("\u0000");
+        } catch (UnsupportedEncodingException e) {
+            throw new SaslException("UTF-8 encoding not supported", e);
+        }
+        if (tokens.length != 3)
+            throw new SaslException("Invalid SASL/PLAIN response: expected 3 tokens, got " + tokens.length);
+        authorizationID = tokens[0];
+        String username = tokens[1];
+        String password = tokens[2];
+
+        if (username.isEmpty()) {
+            throw new SaslException("Authentication failed: username not specified");
+        }
+        if (password.isEmpty()) {
+            throw new SaslException("Authentication failed: password not specified");
+        }
+        if (authorizationID.isEmpty())
+            authorizationID = username;
+
+        try {
+            String expectedPassword = JaasUtils.jaasConfig(LoginType.SERVER.contextName(), JAAS_USER_PREFIX + username);
+            if (!password.equals(expectedPassword)) {
+                throw new SaslException("Authentication failed: Invalid username or password");
+            }
+        } catch (IOException e) {
+            throw new SaslException("Authentication failed: Invalid JAAS configuration", e);
+        }
+        complete = true;
+        return new byte[0];
+    }
+
+    @Override
+    public String getAuthorizationID() {
+        if (!complete)
+            throw new IllegalStateException("Authentication exchange has not completed");
+        return authorizationID;
+    }
+
+    @Override
+    public String getMechanismName() {
+        return PLAIN_MECHANISM;
+    }
+
+    @Override
+    public Object getNegotiatedProperty(String propName) {
+        if (!complete)
+            throw new IllegalStateException("Authentication exchange has not completed");
+        return null;
+    }
+
+    @Override
+    public boolean isComplete() {
+        return complete;
+    }
+
+    @Override
+    public byte[] unwrap(byte[] incoming, int offset, int len) throws SaslException {
+        if (!complete)
+            throw new IllegalStateException("Authentication exchange has not completed");
+        return Arrays.copyOfRange(incoming, offset, offset + len);
+    }
+
+    @Override
+    public byte[] wrap(byte[] outgoing, int offset, int len) throws SaslException {
+        if (!complete)
+            throw new IllegalStateException("Authentication exchange has not completed");
+        return Arrays.copyOfRange(outgoing, offset, offset + len);
+    }
+
+    @Override
+    public void dispose() throws SaslException {
+    }
+
+    public static class PlainSaslServerFactory implements SaslServerFactory {
+
+        @Override
+        public SaslServer createSaslServer(String mechanism, String protocol, String serverName, Map<String, ?> props, CallbackHandler cbh)
+            throws SaslException {
+
+            if (!PLAIN_MECHANISM.equals(mechanism)) {
+                throw new SaslException(String.format("Mechanism \'%s\' is not supported. Only PLAIN is supported.", mechanism));
+            }
+            return new PlainSaslServer(cbh);
+        }
+
+        @Override
+        public String[] getMechanismNames(Map<String, ?> props) {
+            String noPlainText = (String) props.get(Sasl.POLICY_NOPLAINTEXT);
+            if ("true".equals(noPlainText))
+                return new String[]{};
+            else
+                return new String[]{PLAIN_MECHANISM};
+        }
+    }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServerProvider.java b/clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServerProvider.java
new file mode 100644
index 0000000..c3db1f5
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServerProvider.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.security.plain;
+
+import java.security.Provider;
+import java.security.Security;
+
+import org.apache.kafka.common.security.plain.PlainSaslServer.PlainSaslServerFactory;
+
+public class PlainSaslServerProvider extends Provider {
+
+    private static final long serialVersionUID = 1L;
+
+    protected PlainSaslServerProvider() {
+        super("Simple SASL/PLAIN Server Provider", 1.0, "Simple SASL/PLAIN Server Provider for Kafka");
+        super.put("SaslServerFactory." + PlainSaslServer.PLAIN_MECHANISM, PlainSaslServerFactory.class.getName());
+    }
+
+    public static void initialize() {
+        Security.addProvider(new PlainSaslServerProvider());
+    }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/BytesDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/BytesDeserializer.java
new file mode 100644
index 0000000..ee6a57c
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/serialization/BytesDeserializer.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.serialization;
+
+import org.apache.kafka.common.utils.Bytes;
+
+import java.util.Map;
+
+public class BytesDeserializer implements Deserializer<Bytes> {
+
+    public void configure(Map<String, ?> configs, boolean isKey) {
+        // nothing to do
+    }
+
+    public Bytes deserialize(String topic, byte[] data) {
+        if (data == null)
+            return null;
+
+        return new Bytes(data);
+    }
+
+    public void close() {
+        // nothing to do
+    }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/BytesSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/BytesSerializer.java
new file mode 100644
index 0000000..3d04446
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/serialization/BytesSerializer.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.serialization;
+
+import org.apache.kafka.common.utils.Bytes;
+
+import java.util.Map;
+
+public class BytesSerializer implements Serializer<Bytes> {
+
+    public void configure(Map<String, ?> configs, boolean isKey) {
+        // nothing to do
+    }
+
+    public byte[] serialize(String topic, Bytes data) {
+        if (data == null)
+            return null;
+
+        return data.get();
+    }
+
+    public void close() {
+        // nothing to do
+    }
+}
+
diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java
index f27f74f..d744522 100644
--- a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java
+++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java
@@ -13,6 +13,8 @@
 
 package org.apache.kafka.common.serialization;
 
+import org.apache.kafka.common.utils.Bytes;
+
 import java.nio.ByteBuffer;
 
 /**
@@ -80,6 +82,18 @@
         }
     }
 
+    static public final class BytesSerde implements Serde<Bytes> {
+        @Override
+        public Serializer<Bytes> serializer() {
+            return new BytesSerializer();
+        }
+
+        @Override
+        public Deserializer<Bytes> deserializer() {
+            return new BytesDeserializer();
+        }
+    }
+
     static public final class ByteArraySerde implements Serde<byte[]> {
         @Override
         public Serializer<byte[]> serializer() {
@@ -114,10 +128,14 @@
             return (Serde<T>) ByteArray();
         }
 
-        if (ByteBufferSerde.class.isAssignableFrom(type)) {
+        if (ByteBuffer.class.isAssignableFrom(type)) {
             return (Serde<T>) ByteBuffer();
         }
 
+        if (Bytes.class.isAssignableFrom(type)) {
+            return (Serde<T>) Bytes();
+        }
+
         // TODO: we can also serializes objects of type T using generic Java serialization by default
         throw new IllegalArgumentException("Unknown class for built-in serializer");
     }
@@ -150,42 +168,49 @@
     }
 
     /*
-     * A serde for nullable long type.
+     * A serde for nullable {@code Long} type.
      */
     static public Serde<Long> Long() {
         return new LongSerde();
     }
 
     /*
-     * A serde for nullable int type.
+     * A serde for nullable {@code Integer} type.
      */
     static public Serde<Integer> Integer() {
         return new IntegerSerde();
     }
 
     /*
-     * A serde for nullable long type.
+     * A serde for nullable {@code Double} type.
      */
     static public Serde<Double> Double() {
         return new DoubleSerde();
     }
 
     /*
-     * A serde for nullable string type.
+     * A serde for nullable {@code String} type.
      */
     static public Serde<String> String() {
         return new StringSerde();
     }
 
     /*
-     * A serde for nullable byte array type.
+     * A serde for nullable {@code ByteBuffer} type.
      */
     static public Serde<ByteBuffer> ByteBuffer() {
         return new ByteBufferSerde();
     }
 
     /*
-     * A serde for nullable byte array type.
+     * A serde for nullable {@code Bytes} type.
+     */
+    static public Serde<Bytes> Bytes() {
+        return new BytesSerde();
+    }
+
+    /*
+     * A serde for nullable {@code byte[]} type.
      */
     static public Serde<byte[]> ByteArray() {
         return new ByteArraySerde();
diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Bytes.java b/clients/src/main/java/org/apache/kafka/common/utils/Bytes.java
new file mode 100644
index 0000000..78340e5
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/utils/Bytes.java
@@ -0,0 +1,178 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.utils;
+
+import java.util.Arrays;
+import java.util.Comparator;
+
+/**
+ * Utility class that handles immutable byte arrays.
+ */
+public class Bytes implements Comparable<Bytes> {
+
+    private static final char[] HEX_CHARS_UPPER = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'A', 'B', 'C', 'D', 'E', 'F'};
+
+    private final byte[] bytes;
+
+    // cache the hash code for the string, default to 0
+    private int hashCode;
+
+    public static Bytes wrap(byte[] bytes) {
+        return new Bytes(bytes);
+    }
+
+    /**
+     * Create a Bytes using the byte array.
+     *
+     * @param bytes This array becomes the backing storage for the object.
+     */
+    public Bytes(byte[] bytes) {
+        this.bytes = bytes;
+
+        // initialize hash code to 0
+        hashCode = 0;
+    }
+
+    /**
+     * Get the data from the Bytes.
+     * @return The data is only valid between offset and offset+length.
+     */
+    public byte[] get() {
+        return this.bytes;
+    }
+
+    /**
+     * The hashcode is cached except for the case where it is computed as 0, in which
+     * case we compute the hashcode on every call.
+     *
+     * @return the hashcode
+     */
+    @Override
+    public int hashCode() {
+        if (hashCode == 0) {
+            hashCode = Arrays.hashCode(bytes);
+        }
+
+        return hashCode;
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (this == other)
+            return true;
+
+        // we intentionally use the function to compute hashcode here
+        if (this.hashCode() != other.hashCode())
+            return false;
+
+        if (other instanceof Bytes)
+            return Arrays.equals(this.bytes, ((Bytes) other).get());
+
+        return false;
+    }
+
+    @Override
+    public int compareTo(Bytes that) {
+        return BYTES_LEXICO_COMPARATOR.compare(this.bytes, that.bytes);
+    }
+
+    @Override
+    public String toString() {
+        return Bytes.toString(bytes, 0, bytes.length);
+    }
+
+    /**
+     * Write a printable representation of a byte array. Non-printable
+     * characters are hex escaped in the format \\x%02X, eg:
+     * \x00 \x05 etc.
+     *
+     * This function is brought from org.apache.hadoop.hbase.util.Bytes
+     *
+     * @param b array to write out
+     * @param off offset to start at
+     * @param len length to write
+     * @return string output
+     */
+    private static String toString(final byte[] b, int off, int len) {
+        StringBuilder result = new StringBuilder();
+
+        if (b == null)
+            return result.toString();
+
+        // just in case we are passed a 'len' that is > buffer length...
+        if (off >= b.length)
+            return result.toString();
+
+        if (off + len > b.length)
+            len = b.length - off;
+
+        for (int i = off; i < off + len; ++i) {
+            int ch = b[i] & 0xFF;
+            if (ch >= ' ' && ch <= '~' && ch != '\\') {
+                result.append((char) ch);
+            } else {
+                result.append("\\x");
+                result.append(HEX_CHARS_UPPER[ch / 0x10]);
+                result.append(HEX_CHARS_UPPER[ch % 0x10]);
+            }
+        }
+        return result.toString();
+    }
+
+    /**
+     * A byte array comparator based on lexicograpic ordering.
+     */
+    public final static Comparator<byte[]> BYTES_LEXICO_COMPARATOR = new LexicographicByteArrayComparator();
+
+    private interface ByteArrayComparator extends Comparator<byte[]> {
+
+        int compare(final byte[] buffer1, int offset1, int length1,
+                    final byte[] buffer2, int offset2, int length2);
+    }
+
+    private static class LexicographicByteArrayComparator implements ByteArrayComparator {
+
+        @Override
+        public int compare(byte[] buffer1, byte[] buffer2) {
+            return compare(buffer1, 0, buffer1.length, buffer2, 0, buffer2.length);
+        }
+
+        public int compare(final byte[] buffer1, int offset1, int length1,
+                           final byte[] buffer2, int offset2, int length2) {
+
+            // short circuit equal case
+            if (buffer1 == buffer2 &&
+                    offset1 == offset2 &&
+                    length1 == length2) {
+                return 0;
+            }
+
+            // similar to Arrays.compare() but considers offset and length
+            int end1 = offset1 + length1;
+            int end2 = offset2 + length2;
+            for (int i = offset1, j = offset2; i < end1 && j < end2; i++, j++) {
+                int a = buffer1[i] & 0xff;
+                int b = buffer2[j] & 0xff;
+                if (a != b) {
+                    return a - b;
+                }
+            }
+            return length1 - length2;
+        }
+    }
+}
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
index ff07461..2272795 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
@@ -77,6 +77,8 @@
         consumer.unsubscribe();
         Assert.assertTrue(consumer.subscription().isEmpty());
         Assert.assertTrue(consumer.assignment().isEmpty());
+
+        consumer.close();
     }
 
     @Test(expected = IllegalArgumentException.class)
@@ -85,10 +87,13 @@
         props.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "testSeekNegative");
         props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
         props.setProperty(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName());
-
         KafkaConsumer<byte[], byte[]> consumer = newConsumer();
-        consumer.assign(Arrays.asList(new TopicPartition("nonExistTopic", 0)));
-        consumer.seek(new TopicPartition("nonExistTopic", 0), -1);
+        try {
+            consumer.assign(Arrays.asList(new TopicPartition("nonExistTopic", 0)));
+            consumer.seek(new TopicPartition("nonExistTopic", 0), -1);
+        } finally {
+            consumer.close();
+        }
     }
 
     @Test
@@ -129,6 +134,8 @@
 
         consumer.unsubscribe();
         Assert.assertTrue(consumer.paused().isEmpty());
+
+        consumer.close();
     }
 
     private KafkaConsumer<byte[], byte[]> newConsumer() {
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
index b864d69..5a174db 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
@@ -63,6 +63,7 @@
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -575,6 +576,61 @@
         assertTrue(subscriptions.partitionAssignmentNeeded());
     }
 
+
+    @Test
+    public void testUpdateMetadataDuringRebalance() {
+        final String topic1 = "topic1";
+        final String topic2 = "topic2";
+        TopicPartition tp1 = new TopicPartition(topic1, 0);
+        TopicPartition tp2 = new TopicPartition(topic2, 0);
+        final String consumerId = "leader";
+
+        List<String> topics = Arrays.asList(topic1, topic2);
+
+        subscriptions.subscribe(topics, rebalanceListener);
+        metadata.setTopics(topics);
+        subscriptions.needReassignment();
+
+        // we only have metadata for one topic initially
+        metadata.update(TestUtils.singletonCluster(topic1, 1), time.milliseconds());
+
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        // prepare initial rebalance
+        Map<String, List<String>> memberSubscriptions = Collections.singletonMap(consumerId, topics);
+        partitionAssignor.prepare(Collections.singletonMap(consumerId, Arrays.asList(tp1)));
+
+        client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE.code()));
+        client.prepareResponse(new MockClient.RequestMatcher() {
+            @Override
+            public boolean matches(ClientRequest request) {
+                SyncGroupRequest sync = new SyncGroupRequest(request.request().body());
+                if (sync.memberId().equals(consumerId) &&
+                        sync.generationId() == 1 &&
+                        sync.groupAssignment().containsKey(consumerId)) {
+                    // trigger the metadata update including both topics after the sync group request has been sent
+                    Map<String, Integer> topicPartitionCounts = new HashMap<>();
+                    topicPartitionCounts.put(topic1, 1);
+                    topicPartitionCounts.put(topic2, 1);
+                    metadata.update(TestUtils.singletonCluster(topicPartitionCounts), time.milliseconds());
+                    return true;
+                }
+                return false;
+            }
+        }, syncGroupResponse(Arrays.asList(tp1), Errors.NONE.code()));
+
+        // the metadata update should trigger a second rebalance
+        client.prepareResponse(joinGroupLeaderResponse(2, consumerId, memberSubscriptions, Errors.NONE.code()));
+        client.prepareResponse(syncGroupResponse(Arrays.asList(tp1, tp2), Errors.NONE.code()));
+
+        coordinator.ensurePartitionAssignment();
+
+        assertFalse(subscriptions.partitionAssignmentNeeded());
+        assertEquals(new HashSet<>(Arrays.asList(tp1, tp2)), subscriptions.assignedPartitions());
+    }
+
+
     @Test
     public void testExcludeInternalTopicsConfigOption() { 
         subscriptions.subscribe(Pattern.compile(".*"), rebalanceListener);
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
index 9002e81..49bff10 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
@@ -44,6 +44,7 @@
 import org.apache.kafka.common.requests.FetchResponse;
 import org.apache.kafka.common.requests.ListOffsetRequest;
 import org.apache.kafka.common.requests.ListOffsetResponse;
+import org.apache.kafka.common.requests.MetadataRequest;
 import org.apache.kafka.common.requests.MetadataResponse;
 import org.apache.kafka.common.serialization.ByteArrayDeserializer;
 import org.apache.kafka.common.utils.MockTime;
@@ -478,14 +479,14 @@
     @Test(expected = InvalidTopicException.class)
     public void testGetTopicMetadataInvalidTopic() {
         client.prepareResponse(newMetadataResponse(topicName, Errors.INVALID_TOPIC_EXCEPTION).toStruct());
-        fetcher.getTopicMetadata(Collections.singletonList(topicName), 5000L);
+        fetcher.getTopicMetadata(new MetadataRequest(Collections.singletonList(topicName)), 5000L);
     }
 
     @Test
     public void testGetTopicMetadataUnknownTopic() {
         client.prepareResponse(newMetadataResponse(topicName, Errors.UNKNOWN_TOPIC_OR_PARTITION).toStruct());
 
-        Map<String, List<PartitionInfo>> topicMetadata = fetcher.getTopicMetadata(Collections.singletonList(topicName), 5000L);
+        Map<String, List<PartitionInfo>> topicMetadata = fetcher.getTopicMetadata(new MetadataRequest(Collections.singletonList(topicName)), 5000L);
         assertNull(topicMetadata.get(topicName));
     }
 
@@ -494,7 +495,7 @@
         client.prepareResponse(newMetadataResponse(topicName, Errors.LEADER_NOT_AVAILABLE).toStruct());
         client.prepareResponse(newMetadataResponse(topicName, Errors.NONE).toStruct());
 
-        Map<String, List<PartitionInfo>> topicMetadata = fetcher.getTopicMetadata(Collections.singletonList(topicName), 5000L);
+        Map<String, List<PartitionInfo>> topicMetadata = fetcher.getTopicMetadata(new MetadataRequest(Collections.singletonList(topicName)), 5000L);
         assertTrue(topicMetadata.containsKey(topicName));
     }
 
@@ -570,8 +571,8 @@
             }
         }
 
-        MetadataResponse.TopicMetadata topicMetadata = new MetadataResponse.TopicMetadata(error, topic, partitionsMetadata);
-        return new MetadataResponse(cluster.nodes(), Arrays.asList(topicMetadata));
+        MetadataResponse.TopicMetadata topicMetadata = new MetadataResponse.TopicMetadata(error, topic, false, partitionsMetadata);
+        return new MetadataResponse(cluster.nodes(), MetadataResponse.NO_CONTROLLER_ID, Arrays.asList(topicMetadata));
     }
 
     private Fetcher<byte[], byte[]> createFetcher(int maxPollRecords,
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java
index 904aa73..a39d2e8 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java
@@ -316,11 +316,11 @@
         // Advance the clock to expire the batch.
         time.sleep(requestTimeout + 1);
         accum.mutePartition(tp1);
-        List<RecordBatch> expiredBatches = accum.abortExpiredBatches(requestTimeout, cluster, time.milliseconds());
+        List<RecordBatch> expiredBatches = accum.abortExpiredBatches(requestTimeout, time.milliseconds());
         assertEquals("The batch should not be expired when the partition is muted", 0, expiredBatches.size());
 
         accum.unmutePartition(tp1);
-        expiredBatches = accum.abortExpiredBatches(requestTimeout, cluster, time.milliseconds());
+        expiredBatches = accum.abortExpiredBatches(requestTimeout, time.milliseconds());
         assertEquals("The batch should be expired", 1, expiredBatches.size());
         assertEquals("No partitions should be ready.", 0, accum.ready(cluster, time.milliseconds()).readyNodes.size());
 
@@ -330,11 +330,11 @@
         time.sleep(requestTimeout + 1);
 
         accum.mutePartition(tp1);
-        expiredBatches = accum.abortExpiredBatches(requestTimeout, cluster, time.milliseconds());
+        expiredBatches = accum.abortExpiredBatches(requestTimeout, time.milliseconds());
         assertEquals("The batch should not be expired when metadata is still available and partition is muted", 0, expiredBatches.size());
 
         accum.unmutePartition(tp1);
-        expiredBatches = accum.abortExpiredBatches(requestTimeout, cluster, time.milliseconds());
+        expiredBatches = accum.abortExpiredBatches(requestTimeout, time.milliseconds());
         assertEquals("The batch should be expired when the partition is not muted", 1, expiredBatches.size());
         assertEquals("No partitions should be ready.", 0, accum.ready(cluster, time.milliseconds()).readyNodes.size());
 
@@ -351,16 +351,16 @@
 
         // test expiration.
         time.sleep(requestTimeout + retryBackoffMs);
-        expiredBatches = accum.abortExpiredBatches(requestTimeout, cluster, time.milliseconds());
+        expiredBatches = accum.abortExpiredBatches(requestTimeout, time.milliseconds());
         assertEquals("The batch should not be expired.", 0, expiredBatches.size());
         time.sleep(1L);
 
         accum.mutePartition(tp1);
-        expiredBatches = accum.abortExpiredBatches(requestTimeout, cluster, time.milliseconds());
+        expiredBatches = accum.abortExpiredBatches(requestTimeout, time.milliseconds());
         assertEquals("The batch should not be expired when the partition is muted", 0, expiredBatches.size());
 
         accum.unmutePartition(tp1);
-        expiredBatches = accum.abortExpiredBatches(requestTimeout, cluster, time.milliseconds());
+        expiredBatches = accum.abortExpiredBatches(requestTimeout, time.milliseconds());
         assertEquals("The batch should be expired when the partition is not muted.", 1, expiredBatches.size());
     }
 
diff --git a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java
index 022fb6b..e20e422 100644
--- a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java
@@ -236,12 +236,11 @@
         Map<String, ConfigValue> expected = new HashMap<>();
         String errorMessageB = "Missing required configuration \"b\" which has no default value.";
         String errorMessageC = "Missing required configuration \"c\" which has no default value.";
-        String errorMessageD = "Invalid value for configuration d";
 
         ConfigValue configA = new ConfigValue("a", 1, Arrays.<Object>asList(1, 2, 3), Collections.<String>emptyList());
         ConfigValue configB = new ConfigValue("b", null, Arrays.<Object>asList(4, 5), Arrays.asList(errorMessageB, errorMessageB));
         ConfigValue configC = new ConfigValue("c", null, Arrays.<Object>asList(4, 5), Arrays.asList(errorMessageC));
-        ConfigValue configD = new ConfigValue("d", 10, Arrays.<Object>asList(1, 2, 3), Arrays.asList(errorMessageD));
+        ConfigValue configD = new ConfigValue("d", 10, Arrays.<Object>asList(1, 2, 3), Collections.<String>emptyList());
 
         expected.put("a", configA);
         expected.put("b", configB);
diff --git a/clients/src/test/java/org/apache/kafka/common/network/CertStores.java b/clients/src/test/java/org/apache/kafka/common/network/CertStores.java
new file mode 100644
index 0000000..6f108b554
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/network/CertStores.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.network;
+
+import java.io.File;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.kafka.common.config.SslConfigs;
+import org.apache.kafka.test.TestSslUtils;
+
+public class CertStores {
+
+    private final Map<String, Object> sslConfig;
+
+    public CertStores(boolean server, String host) throws Exception {
+        String name = server ? "server" : "client";
+        Mode mode = server ? Mode.SERVER : Mode.CLIENT;
+        File truststoreFile = File.createTempFile(name + "TS", ".jks");
+        sslConfig = TestSslUtils.createSslConfig(!server, true, mode, truststoreFile, name, host);
+        if (server)
+            sslConfig.put(SslConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, Class.forName(SslConfigs.DEFAULT_PRINCIPAL_BUILDER_CLASS));
+    }
+
+    public Map<String, Object> getTrustingConfig(CertStores truststoreConfig) {
+        Map<String, Object> config = new HashMap<>(sslConfig);
+        config.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, truststoreConfig.sslConfig.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG));
+        config.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, truststoreConfig.sslConfig.get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG));
+        config.put(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, truststoreConfig.sslConfig.get(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG));
+        return config;
+    }
+
+    public Map<String, Object> getUntrustingConfig() {
+        return sslConfig;
+    }
+}
\ No newline at end of file
diff --git a/clients/src/test/java/org/apache/kafka/common/network/NetworkTestUtils.java b/clients/src/test/java/org/apache/kafka/common/network/NetworkTestUtils.java
new file mode 100644
index 0000000..969055d
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/network/NetworkTestUtils.java
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.network;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.protocol.SecurityProtocol;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.test.TestUtils;
+
+/**
+ * Common utility functions used by transport layer and authenticator tests.
+ */
+public class NetworkTestUtils {
+
+    public static NioEchoServer createEchoServer(SecurityProtocol securityProtocol, Map<String, Object> serverConfigs) throws Exception {
+        NioEchoServer server = new NioEchoServer(securityProtocol, serverConfigs, "localhost");
+        server.start();
+        return server;
+    }
+
+    public static Selector createSelector(ChannelBuilder channelBuilder) {
+        return new Selector(5000, new Metrics(), new MockTime(), "MetricGroup", channelBuilder);
+    }
+
+    public static void checkClientConnection(Selector selector, String node, int minMessageSize, int messageCount) throws Exception {
+
+        waitForChannelReady(selector, node);
+        String prefix = TestUtils.randomString(minMessageSize);
+        int requests = 0;
+        int responses = 0;
+        selector.send(new NetworkSend(node, ByteBuffer.wrap((prefix + "-0").getBytes())));
+        requests++;
+        while (responses < messageCount) {
+            selector.poll(0L);
+            assertEquals("No disconnects should have occurred.", 0, selector.disconnected().size());
+
+            for (NetworkReceive receive : selector.completedReceives()) {
+                assertEquals(prefix + "-" + responses, new String(Utils.toArray(receive.payload())));
+                responses++;
+            }
+
+            for (int i = 0; i < selector.completedSends().size() && requests < messageCount && selector.isChannelReady(node); i++, requests++) {
+                selector.send(new NetworkSend(node, ByteBuffer.wrap((prefix + "-" + requests).getBytes())));
+            }
+        }
+    }
+
+    public static void waitForChannelReady(Selector selector, String node) throws IOException {
+        // wait for handshake to finish
+        int secondsLeft = 30;
+        while (!selector.isChannelReady(node) && secondsLeft-- > 0) {
+            selector.poll(1000L);
+        }
+        assertTrue(selector.isChannelReady(node));
+    }
+
+    public static void waitForChannelClose(Selector selector, String node) throws IOException {
+        boolean closed = false;
+        for (int i = 0; i < 30; i++) {
+            selector.poll(1000L);
+            if (selector.channel(node) == null) {
+                closed = true;
+                break;
+            }
+        }
+        assertTrue(closed);
+    }
+}
diff --git a/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java b/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java
new file mode 100644
index 0000000..e99a399
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java
@@ -0,0 +1,145 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.network;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.ServerSocketChannel;
+import java.nio.channels.SocketChannel;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.protocol.SecurityProtocol;
+import org.apache.kafka.common.utils.MockTime;
+
+/**
+ * Non-blocking EchoServer implementation that uses ChannelBuilder to create channels
+ * with the configured security protocol.
+ *
+ */
+public class NioEchoServer extends Thread {
+    private final int port;
+    private final ServerSocketChannel serverSocketChannel;
+    private final List<SocketChannel> newChannels;
+    private final List<SocketChannel> socketChannels;
+    private final AcceptorThread acceptorThread;
+    private final Selector selector;
+    private final ConcurrentLinkedQueue<NetworkSend> inflightSends = new ConcurrentLinkedQueue<NetworkSend>();
+
+    public NioEchoServer(SecurityProtocol securityProtocol, Map<String, ?> configs, String serverHost) throws Exception {
+        serverSocketChannel = ServerSocketChannel.open();
+        serverSocketChannel.configureBlocking(false);
+        serverSocketChannel.socket().bind(new InetSocketAddress(serverHost, 0));
+        this.port = serverSocketChannel.socket().getLocalPort();
+        this.socketChannels = Collections.synchronizedList(new ArrayList<SocketChannel>());
+        this.newChannels = Collections.synchronizedList(new ArrayList<SocketChannel>());
+        ChannelBuilder channelBuilder = ChannelBuilders.create(securityProtocol, Mode.SERVER, LoginType.SERVER, configs, null, true);
+        this.selector = new Selector(5000, new Metrics(), new MockTime(), "MetricGroup", channelBuilder);
+        setName("echoserver");
+        setDaemon(true);
+        acceptorThread = new AcceptorThread();
+    }
+
+    public int port() {
+        return port;
+    }
+
+    @Override
+    public void run() {
+        try {
+            acceptorThread.start();
+            while (serverSocketChannel.isOpen()) {
+                selector.poll(1000);
+                for (SocketChannel socketChannel : newChannels) {
+                    String id = id(socketChannel);
+                    selector.register(id, socketChannel);
+                    socketChannels.add(socketChannel);
+                }
+                newChannels.clear();
+                while (true) {
+                    NetworkSend send = inflightSends.peek();
+                    if (send != null && !selector.channel(send.destination()).hasSend()) {
+                        send = inflightSends.poll();
+                        selector.send(send);
+                    } else
+                        break;
+                }
+                List<NetworkReceive> completedReceives = selector.completedReceives();
+                for (NetworkReceive rcv : completedReceives) {
+                    NetworkSend send = new NetworkSend(rcv.source(), rcv.payload());
+                    if (!selector.channel(send.destination()).hasSend())
+                        selector.send(send);
+                    else
+                        inflightSends.add(send);
+                }
+            }
+        } catch (IOException e) {
+            // ignore
+        }
+    }
+
+    private String id(SocketChannel channel) {
+        return channel.socket().getLocalAddress().getHostAddress() + ":" + channel.socket().getLocalPort() + "-" +
+                channel.socket().getInetAddress().getHostAddress() + ":" + channel.socket().getPort();
+    }
+
+    public void closeConnections() throws IOException {
+        for (SocketChannel channel : socketChannels)
+            channel.close();
+        socketChannels.clear();
+    }
+
+    public void close() throws IOException, InterruptedException {
+        this.serverSocketChannel.close();
+        closeConnections();
+        acceptorThread.interrupt();
+        acceptorThread.join();
+        interrupt();
+        join();
+    }
+
+    private class AcceptorThread extends Thread {
+        public AcceptorThread() throws IOException {
+            setName("acceptor");
+        }
+        public void run() {
+            try {
+                java.nio.channels.Selector acceptSelector = java.nio.channels.Selector.open();
+                serverSocketChannel.register(acceptSelector, SelectionKey.OP_ACCEPT);
+                while (serverSocketChannel.isOpen()) {
+                    if (acceptSelector.select(1000) > 0) {
+                        Iterator<SelectionKey> it = acceptSelector.selectedKeys().iterator();
+                        while (it.hasNext()) {
+                            SelectionKey key = it.next();
+                            if (key.isAcceptable()) {
+                                SocketChannel socketChannel = ((ServerSocketChannel) key.channel()).accept();
+                                socketChannel.configureBlocking(false);
+                                newChannels.add(socketChannel);
+                                selector.wakeup();
+                            }
+                            it.remove();
+                        }
+                    }
+                }
+            } catch (IOException e) {
+                // ignore
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
index d3302c8..4e96411 100644
--- a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
@@ -12,25 +12,15 @@
  */
 package org.apache.kafka.common.network;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
-import java.util.concurrent.ConcurrentLinkedQueue;
 import java.io.IOException;
-import java.io.File;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.nio.channels.SelectionKey;
-import java.nio.channels.ServerSocketChannel;
 import java.nio.channels.SocketChannel;
 
 import javax.net.ssl.SSLContext;
@@ -40,11 +30,9 @@
 import org.apache.kafka.common.config.SslConfigs;
 import org.apache.kafka.common.security.ssl.SslFactory;
 import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.protocol.SecurityProtocol;
 import org.apache.kafka.common.utils.MockTime;
 import org.apache.kafka.common.config.types.Password;
-import org.apache.kafka.common.utils.Utils;
-import org.apache.kafka.test.TestSslUtils;
-import org.apache.kafka.test.TestUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -56,7 +44,7 @@
 
     private static final int BUFFER_SIZE = 4 * 1024;
 
-    private SslEchoServer server;
+    private NioEchoServer server;
     private Selector selector;
     private ChannelBuilder channelBuilder;
     private CertStores serverCertStores;
@@ -91,13 +79,13 @@
     @Test
     public void testValidEndpointIdentification() throws Exception {
         String node = "0";
-        createEchoServer(sslServerConfigs);
+        server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs);
         sslClientConfigs.put(SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, "HTTPS");
         createSelector(sslClientConfigs);
-        InetSocketAddress addr = new InetSocketAddress("localhost", server.port);
+        InetSocketAddress addr = new InetSocketAddress("localhost", server.port());
         selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE);
 
-        testClientConnection(node, 100, 10);
+        NetworkTestUtils.checkClientConnection(selector, node, 100, 10);
     }
     
     /**
@@ -113,12 +101,12 @@
         sslServerConfigs = serverCertStores.getTrustingConfig(clientCertStores);
         sslClientConfigs = clientCertStores.getTrustingConfig(serverCertStores);
         sslClientConfigs.put(SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, "HTTPS");
-        createEchoServer(sslServerConfigs);
+        server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs);
         createSelector(sslClientConfigs);
-        InetSocketAddress addr = new InetSocketAddress("localhost", server.port);
+        InetSocketAddress addr = new InetSocketAddress("localhost", server.port());
         selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE);
 
-        waitForChannelClose(node);
+        NetworkTestUtils.waitForChannelClose(selector, node);
     }
     
     /**
@@ -129,14 +117,14 @@
     public void testEndpointIdentificationDisabled() throws Exception {
         String node = "0";
         String serverHost = InetAddress.getLocalHost().getHostAddress();
-        server = new SslEchoServer(sslServerConfigs, serverHost);
+        server = new NioEchoServer(SecurityProtocol.SSL, sslServerConfigs, serverHost);
         server.start();
         sslClientConfigs.remove(SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG);
         createSelector(sslClientConfigs);
-        InetSocketAddress addr = new InetSocketAddress(serverHost, server.port);
+        InetSocketAddress addr = new InetSocketAddress(serverHost, server.port());
         selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE);
 
-        testClientConnection(node, 100, 10);
+        NetworkTestUtils.checkClientConnection(selector, node, 100, 10);
     }
     
     /**
@@ -147,12 +135,12 @@
     public void testClientAuthenticationRequiredValidProvided() throws Exception {
         String node = "0";
         sslServerConfigs.put(SslConfigs.SSL_CLIENT_AUTH_CONFIG, "required");
-        createEchoServer(sslServerConfigs);
+        server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs);
         createSelector(sslClientConfigs);
-        InetSocketAddress addr = new InetSocketAddress("localhost", server.port);
+        InetSocketAddress addr = new InetSocketAddress("localhost", server.port());
         selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE);
 
-        testClientConnection(node, 100, 10);
+        NetworkTestUtils.checkClientConnection(selector, node, 100, 10);
     }
     
     /**
@@ -164,12 +152,12 @@
         String node = "0";
         sslServerConfigs = serverCertStores.getUntrustingConfig();
         sslServerConfigs.put(SslConfigs.SSL_CLIENT_AUTH_CONFIG, "required");
-        createEchoServer(sslServerConfigs);        
+        server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs);
         createSelector(sslClientConfigs);
-        InetSocketAddress addr = new InetSocketAddress("localhost", server.port);
+        InetSocketAddress addr = new InetSocketAddress("localhost", server.port());
         selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE);
 
-        waitForChannelClose(node);
+        NetworkTestUtils.waitForChannelClose(selector, node);
     }
     
     /**
@@ -180,16 +168,16 @@
     public void testClientAuthenticationRequiredNotProvided() throws Exception {
         String node = "0";
         sslServerConfigs.put(SslConfigs.SSL_CLIENT_AUTH_CONFIG, "required");
-        createEchoServer(sslServerConfigs);
+        server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs);
         
         sslClientConfigs.remove(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG);
         sslClientConfigs.remove(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG);
         sslClientConfigs.remove(SslConfigs.SSL_KEY_PASSWORD_CONFIG);
         createSelector(sslClientConfigs);
-        InetSocketAddress addr = new InetSocketAddress("localhost", server.port);
+        InetSocketAddress addr = new InetSocketAddress("localhost", server.port());
         selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE);
 
-        waitForChannelClose(node);
+        NetworkTestUtils.waitForChannelClose(selector, node);
     }
     
     /**
@@ -201,12 +189,12 @@
         String node = "0";
         sslServerConfigs = serverCertStores.getUntrustingConfig();
         sslServerConfigs.put(SslConfigs.SSL_CLIENT_AUTH_CONFIG, "none");
-        createEchoServer(sslServerConfigs);      
+        server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs);
         createSelector(sslClientConfigs);
-        InetSocketAddress addr = new InetSocketAddress("localhost", server.port);
+        InetSocketAddress addr = new InetSocketAddress("localhost", server.port());
         selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE);
 
-        testClientConnection(node, 100, 10);
+        NetworkTestUtils.checkClientConnection(selector, node, 100, 10);
     }
     
     /**
@@ -217,16 +205,16 @@
     public void testClientAuthenticationDisabledNotProvided() throws Exception {
         String node = "0";
         sslServerConfigs.put(SslConfigs.SSL_CLIENT_AUTH_CONFIG, "none");
-        createEchoServer(sslServerConfigs);
+        server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs);
         
         sslClientConfigs.remove(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG);
         sslClientConfigs.remove(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG);
         sslClientConfigs.remove(SslConfigs.SSL_KEY_PASSWORD_CONFIG);
         createSelector(sslClientConfigs);
-        InetSocketAddress addr = new InetSocketAddress("localhost", server.port);
+        InetSocketAddress addr = new InetSocketAddress("localhost", server.port());
         selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE);
 
-        testClientConnection(node, 100, 10);
+        NetworkTestUtils.checkClientConnection(selector, node, 100, 10);
     }
     
     /**
@@ -237,12 +225,12 @@
     public void testClientAuthenticationRequestedValidProvided() throws Exception {
         String node = "0";
         sslServerConfigs.put(SslConfigs.SSL_CLIENT_AUTH_CONFIG, "requested");
-        createEchoServer(sslServerConfigs);
+        server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs);
         createSelector(sslClientConfigs);
-        InetSocketAddress addr = new InetSocketAddress("localhost", server.port);
+        InetSocketAddress addr = new InetSocketAddress("localhost", server.port());
         selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE);
 
-        testClientConnection(node, 100, 10);
+        NetworkTestUtils.checkClientConnection(selector, node, 100, 10);
     }
     
     /**
@@ -253,16 +241,16 @@
     public void testClientAuthenticationRequestedNotProvided() throws Exception {
         String node = "0";
         sslServerConfigs.put(SslConfigs.SSL_CLIENT_AUTH_CONFIG, "requested");
-        createEchoServer(sslServerConfigs);
+        server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs);
         
         sslClientConfigs.remove(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG);
         sslClientConfigs.remove(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG);
         sslClientConfigs.remove(SslConfigs.SSL_KEY_PASSWORD_CONFIG);
         createSelector(sslClientConfigs);
-        InetSocketAddress addr = new InetSocketAddress("localhost", server.port);
+        InetSocketAddress addr = new InetSocketAddress("localhost", server.port());
         selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE);
 
-        testClientConnection(node, 100, 10);
+        NetworkTestUtils.checkClientConnection(selector, node, 100, 10);
     }
     
     /**
@@ -303,12 +291,12 @@
     public void testInvalidKeyPassword() throws Exception {
         String node = "0";
         sslServerConfigs.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, new Password("invalid"));
-        createEchoServer(sslServerConfigs);        
+        server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs);
         createSelector(sslClientConfigs);
-        InetSocketAddress addr = new InetSocketAddress("localhost", server.port);
+        InetSocketAddress addr = new InetSocketAddress("localhost", server.port());
         selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE);
 
-        waitForChannelClose(node);
+        NetworkTestUtils.waitForChannelClose(selector, node);
     }
     
     /**
@@ -318,14 +306,14 @@
     public void testUnsupportedTLSVersion() throws Exception {
         String node = "0";
         sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.2"));
-        createEchoServer(sslServerConfigs);
+        server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs);
         
         sslClientConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.1"));
         createSelector(sslClientConfigs);
-        InetSocketAddress addr = new InetSocketAddress("localhost", server.port);
+        InetSocketAddress addr = new InetSocketAddress("localhost", server.port());
         selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE);
 
-        waitForChannelClose(node);
+        NetworkTestUtils.waitForChannelClose(selector, node);
     }
     
     /**
@@ -336,14 +324,14 @@
         String node = "0";
         String[] cipherSuites = SSLContext.getDefault().getDefaultSSLParameters().getCipherSuites();
         sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuites[0]));
-        createEchoServer(sslServerConfigs);
+        server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs);
         
         sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuites[1]));
         createSelector(sslClientConfigs);
-        InetSocketAddress addr = new InetSocketAddress("localhost", server.port);
+        InetSocketAddress addr = new InetSocketAddress("localhost", server.port());
         selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE);
 
-        waitForChannelClose(node);
+        NetworkTestUtils.waitForChannelClose(selector, node);
     }
 
     /**
@@ -352,12 +340,12 @@
     @Test
     public void testNetReadBufferResize() throws Exception {
         String node = "0";
-        createEchoServer(sslServerConfigs);
+        server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs);
         createSelector(sslClientConfigs, 10, null, null);
-        InetSocketAddress addr = new InetSocketAddress("localhost", server.port);
+        InetSocketAddress addr = new InetSocketAddress("localhost", server.port());
         selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE);
 
-        testClientConnection(node, 64000, 10);
+        NetworkTestUtils.checkClientConnection(selector, node, 64000, 10);
     }
     
     /**
@@ -366,12 +354,12 @@
     @Test
     public void testNetWriteBufferResize() throws Exception {
         String node = "0";
-        createEchoServer(sslServerConfigs);
+        server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs);
         createSelector(sslClientConfigs, null, 10, null);
-        InetSocketAddress addr = new InetSocketAddress("localhost", server.port);
+        InetSocketAddress addr = new InetSocketAddress("localhost", server.port());
         selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE);
 
-        testClientConnection(node, 64000, 10);
+        NetworkTestUtils.checkClientConnection(selector, node, 64000, 10);
     }
 
     /**
@@ -380,55 +368,12 @@
     @Test
     public void testApplicationBufferResize() throws Exception {
         String node = "0";
-        createEchoServer(sslServerConfigs);
+        server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs);
         createSelector(sslClientConfigs, null, null, 10);
-        InetSocketAddress addr = new InetSocketAddress("localhost", server.port);
+        InetSocketAddress addr = new InetSocketAddress("localhost", server.port());
         selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE);
 
-        testClientConnection(node, 64000, 10);
-    }
-
-    private void testClientConnection(String node, int minMessageSize, int messageCount) throws Exception {
-
-        String prefix = TestUtils.randomString(minMessageSize);
-        int requests = 0;
-        int responses = 0;
-        // wait for handshake to finish
-        while (!selector.isChannelReady(node)) {
-            selector.poll(1000L);
-        }
-        selector.send(new NetworkSend(node, ByteBuffer.wrap((prefix + "-0").getBytes())));
-        requests++;
-        while (responses < messageCount) {
-            selector.poll(0L);
-            assertEquals("No disconnects should have occurred.", 0, selector.disconnected().size());
-
-            for (NetworkReceive receive : selector.completedReceives()) {
-                assertEquals(prefix + "-" + responses, new String(Utils.toArray(receive.payload())));
-                responses++;
-            }
-
-            for (int i = 0; i < selector.completedSends().size() && requests < messageCount && selector.isChannelReady(node); i++, requests++) {
-                selector.send(new NetworkSend(node, ByteBuffer.wrap((prefix + "-" + requests).getBytes())));
-            }
-        }
-    }
-    
-    private void waitForChannelClose(String node) throws IOException {
-        boolean closed = false;
-        for (int i = 0; i < 30; i++) {
-            selector.poll(1000L);
-            if (selector.channel(node) == null) {
-                closed = true;
-                break;
-            }
-        }
-        assertTrue(closed);
-    }
-    
-    private void createEchoServer(Map<String, Object> sslServerConfigs) throws Exception {
-        server = new SslEchoServer(sslServerConfigs, "localhost");
-        server.start();
+        NetworkTestUtils.checkClientConnection(selector, node, 64000, 10);
     }
     
     private void createSelector(Map<String, Object> sslClientConfigs) {
@@ -455,32 +400,6 @@
         this.selector = new Selector(5000, new Metrics(), new MockTime(), "MetricGroup", channelBuilder);
     }
     
-    private static class CertStores {
-        
-        Map<String, Object> sslConfig;
-        
-        CertStores(boolean server, String host) throws Exception {
-            String name = server ? "server" : "client";
-            Mode mode = server ? Mode.SERVER : Mode.CLIENT;
-            File truststoreFile = File.createTempFile(name + "TS", ".jks");
-            sslConfig = TestSslUtils.createSslConfig(!server, true, mode, truststoreFile, name, host);
-            if (server)
-                sslConfig.put(SslConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, Class.forName(SslConfigs.DEFAULT_PRINCIPAL_BUILDER_CLASS));
-        }
-       
-        private Map<String, Object> getTrustingConfig(CertStores truststoreConfig) {
-            Map<String, Object> config = new HashMap<>(sslConfig);
-            config.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, truststoreConfig.sslConfig.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG));
-            config.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, truststoreConfig.sslConfig.get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG));
-            config.put(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, truststoreConfig.sslConfig.get(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG));
-            return config;
-        }
-        
-        private Map<String, Object> getUntrustingConfig() {
-            return sslConfig;
-        }
-    }
-
     /**
      * SSLTransportLayer with overrides for packet and application buffer size to test buffer resize
      * code path. The overridden buffer size starts with a small value and increases in size when the buffer
@@ -537,117 +456,5 @@
             }
         }
     }
-    
-    // Non-blocking EchoServer implementation that uses SSLTransportLayer
-    private class SslEchoServer extends Thread {
-        private final int port;
-        private final ServerSocketChannel serverSocketChannel;
-        private final List<SocketChannel> newChannels;
-        private final List<SocketChannel> socketChannels;
-        private final AcceptorThread acceptorThread;
-        private SslFactory sslFactory;
-        private final Selector selector;
-        private final ConcurrentLinkedQueue<NetworkSend> inflightSends = new ConcurrentLinkedQueue<NetworkSend>();
-
-        public SslEchoServer(Map<String, ?> configs, String serverHost) throws Exception {
-            this.sslFactory = new SslFactory(Mode.SERVER);
-            this.sslFactory.configure(configs);
-            serverSocketChannel = ServerSocketChannel.open();
-            serverSocketChannel.configureBlocking(false);
-            serverSocketChannel.socket().bind(new InetSocketAddress(serverHost, 0));
-            this.port = serverSocketChannel.socket().getLocalPort();
-            this.socketChannels = Collections.synchronizedList(new ArrayList<SocketChannel>());
-            this.newChannels = Collections.synchronizedList(new ArrayList<SocketChannel>());
-            SslChannelBuilder channelBuilder = new SslChannelBuilder(Mode.SERVER);
-            channelBuilder.configure(sslServerConfigs);
-            this.selector = new Selector(5000, new Metrics(), new MockTime(), "MetricGroup", channelBuilder);
-            setName("echoserver");
-            setDaemon(true);
-            acceptorThread = new AcceptorThread();
-        }
-
-        @Override
-        public void run() {
-            try {
-                acceptorThread.start();
-                while (serverSocketChannel.isOpen()) {
-                    selector.poll(1000);
-                    for (SocketChannel socketChannel : newChannels) {
-                        String id = id(socketChannel);
-                        selector.register(id, socketChannel);
-                        socketChannels.add(socketChannel);
-                    }
-                    newChannels.clear();
-                    while (true) {
-                        NetworkSend send = inflightSends.peek();
-                        if (send != null && !selector.channel(send.destination()).hasSend()) {
-                            send = inflightSends.poll();
-                            selector.send(send);
-                        } else
-                            break;
-                    }
-                    List<NetworkReceive> completedReceives = selector.completedReceives();
-                    for (NetworkReceive rcv : completedReceives) {
-                        NetworkSend send = new NetworkSend(rcv.source(), rcv.payload());
-                        if (!selector.channel(send.destination()).hasSend())
-                            selector.send(send);
-                        else
-                            inflightSends.add(send);
-                    }
-                }
-            } catch (IOException e) {
-                // ignore
-            }
-        }
-        
-        private String id(SocketChannel channel) {
-            return channel.socket().getLocalAddress().getHostAddress() + ":" + channel.socket().getLocalPort() + "-" +
-                    channel.socket().getInetAddress().getHostAddress() + ":" + channel.socket().getPort();
-        }
-
-        public void closeConnections() throws IOException {
-            for (SocketChannel channel : socketChannels)
-                channel.close();
-            socketChannels.clear();
-        }
-
-        public void close() throws IOException, InterruptedException {
-            this.serverSocketChannel.close();
-            closeConnections();
-            acceptorThread.interrupt();
-            acceptorThread.join();
-            interrupt();
-            join();
-        }
-        
-        private class AcceptorThread extends Thread {
-            public AcceptorThread() throws IOException {
-                setName("acceptor");
-            }
-            public void run() {
-                try {
-
-                    java.nio.channels.Selector acceptSelector = java.nio.channels.Selector.open();
-                    serverSocketChannel.register(acceptSelector, SelectionKey.OP_ACCEPT);
-                    while (serverSocketChannel.isOpen()) {
-                        if (acceptSelector.select(1000) > 0) {
-                            Iterator<SelectionKey> it = acceptSelector.selectedKeys().iterator();
-                            while (it.hasNext()) {
-                                SelectionKey key = it.next();
-                                if (key.isAcceptable()) {
-                                    SocketChannel socketChannel = ((ServerSocketChannel) key.channel()).accept();
-                                    socketChannel.configureBlocking(false);
-                                    newChannels.add(socketChannel);
-                                    selector.wakeup();
-                                }
-                            }
-                        }
-                    }
-                } catch (IOException e) {
-                    // ignore
-                }
-            }
-        }
-    }
 
 }
diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java
index 5c34277..1633e89 100644
--- a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java
@@ -18,6 +18,7 @@
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
 
 import java.nio.ByteBuffer;
@@ -33,7 +34,8 @@
 
     @Before
     public void setup() {
-        this.schema = new Schema(new Field("int8", Type.INT8),
+        this.schema = new Schema(new Field("boolean", Type.BOOLEAN),
+                                 new Field("int8", Type.INT8),
                                  new Field("int16", Type.INT16),
                                  new Field("int32", Type.INT32),
                                  new Field("int64", Type.INT64),
@@ -42,8 +44,10 @@
                                  new Field("bytes", Type.BYTES),
                                  new Field("nullable_bytes", Type.NULLABLE_BYTES),
                                  new Field("array", new ArrayOf(Type.INT32)),
+                                 new Field("null_array", ArrayOf.nullable(Type.INT32)),
                                  new Field("struct", new Schema(new Field("field", new ArrayOf(Type.INT32)))));
-        this.struct = new Struct(this.schema).set("int8", (byte) 1)
+        this.struct = new Struct(this.schema).set("boolean", true)
+                                             .set("int8", (byte) 1)
                                              .set("int16", (short) 1)
                                              .set("int32", 1)
                                              .set("int64", 1L)
@@ -51,12 +55,15 @@
                                              .set("nullable_string", null)
                                              .set("bytes", ByteBuffer.wrap("1".getBytes()))
                                              .set("nullable_bytes", null)
-                                             .set("array", new Object[] {1});
+                                             .set("array", new Object[] {1})
+                                             .set("null_array", null);
         this.struct.set("struct", this.struct.instance("struct").set("field", new Object[] {1, 2, 3}));
     }
 
     @Test
     public void testSimple() {
+        check(Type.BOOLEAN, false);
+        check(Type.BOOLEAN, true);
         check(Type.INT8, (byte) -111);
         check(Type.INT16, (short) -11111);
         check(Type.INT32, -11111111);
@@ -75,6 +82,7 @@
         check(new ArrayOf(Type.INT32), new Object[] {1, 2, 3, 4});
         check(new ArrayOf(Type.STRING), new Object[] {});
         check(new ArrayOf(Type.STRING), new Object[] {"hello", "there", "beautiful"});
+        check(ArrayOf.nullable(Type.STRING), null);
     }
 
     @Test
@@ -224,6 +232,13 @@
         }
     }
 
+    @Test
+    public void testToString() {
+        String structStr = this.struct.toString();
+        assertNotNull("Struct string should not be null.", structStr);
+        assertFalse("Struct string should not be empty.", structStr.isEmpty());
+    }
+
     private Object roundtrip(Type type, Object obj) {
         ByteBuffer buffer = ByteBuffer.allocate(type.sizeOf(obj));
         type.write(buffer, obj);
diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java
index ed64f63..b1117f1 100644
--- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java
@@ -71,6 +71,17 @@
         }
     }
 
+    @Test
+    public void testHasRoomForMethod() {
+        MemoryRecords recs1 = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), compression);
+        recs1.append(0, new Record(0L, "a".getBytes(), "1".getBytes()));
+
+        assertTrue(recs1.hasRoomFor("b".getBytes(), "2".getBytes()));
+        recs1.close();
+        assertFalse(recs1.hasRoomFor("b".getBytes(), "2".getBytes()));
+
+    }
+
     @Parameterized.Parameters
     public static Collection<Object[]> data() {
         List<Object[]> values = new ArrayList<Object[]>();
diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
index 9def557..345de3f 100644
--- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
@@ -70,9 +70,10 @@
                 createListOffsetRequest(),
                 createListOffsetRequest().getErrorResponse(0, new UnknownServerException()),
                 createListOffsetResponse(),
-                createMetadataRequest(),
-                createMetadataRequest().getErrorResponse(0, new UnknownServerException()),
-                createMetadataResponse(),
+                MetadataRequest.allTopics(),
+                createMetadataRequest(Arrays.asList("topic1")),
+                createMetadataRequest(Arrays.asList("topic1")).getErrorResponse(1, new UnknownServerException()),
+                createMetadataResponse(1),
                 createOffsetCommitRequest(2),
                 createOffsetCommitRequest(2).getErrorResponse(2, new UnknownServerException()),
                 createOffsetCommitResponse(),
@@ -91,12 +92,20 @@
                 createUpdateMetadataResponse(),
                 createLeaderAndIsrRequest(),
                 createLeaderAndIsrRequest().getErrorResponse(0, new UnknownServerException()),
-                createLeaderAndIsrResponse()
+                createLeaderAndIsrResponse(),
+                createSaslHandshakeRequest(),
+                createSaslHandshakeRequest().getErrorResponse(0, new UnknownServerException()),
+                createSaslHandshakeResponse(),
+                createApiVersionRequest(),
+                createApiVersionRequest().getErrorResponse(0, new UnknownServerException()),
+                createApiVersionResponse()
         );
 
         for (AbstractRequestResponse req : requestResponseList)
             checkSerialization(req, null);
 
+        createMetadataResponse(0);
+        createMetadataRequest(Arrays.asList("topic1")).getErrorResponse(0, new UnknownServerException());
         checkSerialization(createFetchRequest().getErrorResponse(0, new UnknownServerException()), 0);
         checkSerialization(createOffsetCommitRequest(0), 0);
         checkSerialization(createOffsetCommitRequest(0).getErrorResponse(0, new UnknownServerException()), 0);
@@ -278,22 +287,22 @@
         return new ListOffsetResponse(responseData);
     }
 
-    private AbstractRequest createMetadataRequest() {
-        return new MetadataRequest(Arrays.asList("topic1"));
+    private AbstractRequest createMetadataRequest(List<String> topics) {
+        return new MetadataRequest(topics);
     }
 
-    private AbstractRequestResponse createMetadataResponse() {
+    private AbstractRequestResponse createMetadataResponse(int version) {
         Node node = new Node(1, "host1", 1001);
         List<Node> replicas = Arrays.asList(node);
         List<Node> isr = Arrays.asList(node);
 
         List<MetadataResponse.TopicMetadata> allTopicMetadata = new ArrayList<>();
-        allTopicMetadata.add(new MetadataResponse.TopicMetadata(Errors.NONE, "topic1",
+        allTopicMetadata.add(new MetadataResponse.TopicMetadata(Errors.NONE, "__consumer_offsets", true,
                 Arrays.asList(new MetadataResponse.PartitionMetadata(Errors.NONE, 1, node, replicas, isr))));
-        allTopicMetadata.add(new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, "topic2",
+        allTopicMetadata.add(new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, "topic2", false,
                 Collections.<MetadataResponse.PartitionMetadata>emptyList()));
 
-        return new MetadataResponse(Arrays.asList(node), allTopicMetadata);
+        return new MetadataResponse(Arrays.asList(node), MetadataResponse.NO_CONTROLLER_ID, allTopicMetadata, version);
     }
 
     private AbstractRequest createOffsetCommitRequest(int version) {
@@ -425,5 +434,20 @@
         return new UpdateMetadataResponse(Errors.NONE.code());
     }
 
+    private AbstractRequest createSaslHandshakeRequest() {
+        return new SaslHandshakeRequest("PLAIN");
+    }
 
-}
+    private AbstractRequestResponse createSaslHandshakeResponse() {
+        return new SaslHandshakeResponse(Errors.NONE.code(), Collections.singletonList("GSSAPI"));
+    }
+
+    private AbstractRequest createApiVersionRequest() {
+        return new ApiVersionsRequest();
+    }
+
+    private AbstractRequestResponse createApiVersionResponse() {
+        List<ApiVersionsResponse.ApiVersion> apiVersions = Arrays.asList(new ApiVersionsResponse.ApiVersion((short) 0, (short) 0, (short) 2));
+        return new ApiVersionsResponse(Errors.NONE.code(), apiVersions);
+    }
+}
\ No newline at end of file
diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java
new file mode 100644
index 0000000..368b5a7
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java
@@ -0,0 +1,553 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.security.authenticator;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import org.apache.kafka.clients.NetworkClient;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.network.CertStores;
+import org.apache.kafka.common.network.ChannelBuilder;
+import org.apache.kafka.common.network.ChannelBuilders;
+import org.apache.kafka.common.network.LoginType;
+import org.apache.kafka.common.network.Mode;
+import org.apache.kafka.common.network.NetworkSend;
+import org.apache.kafka.common.network.NetworkTestUtils;
+import org.apache.kafka.common.network.NioEchoServer;
+import org.apache.kafka.common.network.Selector;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.Protocol;
+import org.apache.kafka.common.protocol.SecurityProtocol;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.requests.AbstractRequestResponse;
+import org.apache.kafka.common.requests.ApiVersionsRequest;
+import org.apache.kafka.common.requests.ApiVersionsResponse;
+import org.apache.kafka.common.requests.MetadataRequest;
+import org.apache.kafka.common.requests.RequestHeader;
+import org.apache.kafka.common.requests.RequestSend;
+import org.apache.kafka.common.requests.SaslHandshakeRequest;
+import org.apache.kafka.common.requests.SaslHandshakeResponse;
+import org.apache.kafka.common.security.JaasUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Tests for the Sasl authenticator. These use a test harness that runs a simple socket server that echos back responses.
+ */
+public class SaslAuthenticatorTest {
+
+    private static final int BUFFER_SIZE = 4 * 1024;
+
+    private NioEchoServer server;
+    private Selector selector;
+    private ChannelBuilder channelBuilder;
+    private CertStores serverCertStores;
+    private CertStores clientCertStores;
+    private Map<String, Object> saslClientConfigs;
+    private Map<String, Object> saslServerConfigs;
+
+    @Before
+    public void setup() throws Exception {
+        serverCertStores = new CertStores(true, "localhost");
+        clientCertStores = new CertStores(false, "localhost");
+        saslServerConfigs = serverCertStores.getTrustingConfig(clientCertStores);
+        saslClientConfigs = clientCertStores.getTrustingConfig(serverCertStores);
+    }
+
+    @After
+    public void teardown() throws Exception {
+        if (server != null)
+            this.server.close();
+        if (selector != null)
+            this.selector.close();
+    }
+
+    /**
+     * Tests good path SASL/PLAIN client and server channels using SSL transport layer.
+     */
+    @Test
+    public void testValidSaslPlainOverSsl() throws Exception {
+        String node = "0";
+        SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL;
+        configureMechanisms("PLAIN", Arrays.asList("PLAIN"));
+
+        server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs);
+        createAndCheckClientConnection(securityProtocol, node);
+    }
+
+    /**
+     * Tests good path SASL/PLAIN client and server channels using PLAINTEXT transport layer.
+     */
+    @Test
+    public void testValidSaslPlainOverPlaintext() throws Exception {
+        String node = "0";
+        SecurityProtocol securityProtocol = SecurityProtocol.SASL_PLAINTEXT;
+        configureMechanisms("PLAIN", Arrays.asList("PLAIN"));
+
+        server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs);
+        createAndCheckClientConnection(securityProtocol, node);
+    }
+
+    /**
+     * Tests that SASL/PLAIN clients with invalid password fail authentication.
+     */
+    @Test
+    public void testInvalidPasswordSaslPlain() throws Exception {
+        String node = "0";
+        SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL;
+        TestJaasConfig jaasConfig = configureMechanisms("PLAIN", Arrays.asList("PLAIN"));
+        jaasConfig.setPlainClientOptions(TestJaasConfig.USERNAME, "invalidpassword");
+
+        server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs);
+        createClientConnection(securityProtocol, node);
+        NetworkTestUtils.waitForChannelClose(selector, node);
+    }
+
+    /**
+     * Tests that SASL/PLAIN clients with invalid username fail authentication.
+     */
+    @Test
+    public void testInvalidUsernameSaslPlain() throws Exception {
+        String node = "0";
+        SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL;
+        TestJaasConfig jaasConfig = configureMechanisms("PLAIN", Arrays.asList("PLAIN"));
+        jaasConfig.setPlainClientOptions("invaliduser", TestJaasConfig.PASSWORD);
+
+        server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs);
+        createClientConnection(securityProtocol, node);
+        NetworkTestUtils.waitForChannelClose(selector, node);
+    }
+
+    /**
+     * Tests that SASL/PLAIN clients without valid username fail authentication.
+     */
+    @Test
+    public void testMissingUsernameSaslPlain() throws Exception {
+        String node = "0";
+        TestJaasConfig jaasConfig = configureMechanisms("PLAIN", Arrays.asList("PLAIN"));
+        jaasConfig.setPlainClientOptions(null, "mypassword");
+
+        SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL;
+        server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs);
+        createSelector(securityProtocol, saslClientConfigs);
+        InetSocketAddress addr = new InetSocketAddress("127.0.0.1", server.port());
+        try {
+            selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE);
+            fail("SASL/PLAIN channel created without username");
+        } catch (KafkaException e) {
+            // Expected exception
+        }
+    }
+
+    /**
+     * Tests that SASL/PLAIN clients with missing password in JAAS configuration fail authentication.
+     */
+    @Test
+    public void testMissingPasswordSaslPlain() throws Exception {
+        String node = "0";
+        TestJaasConfig jaasConfig = configureMechanisms("PLAIN", Arrays.asList("PLAIN"));
+        jaasConfig.setPlainClientOptions("myuser", null);
+
+        SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL;
+        server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs);
+        createSelector(securityProtocol, saslClientConfigs);
+        InetSocketAddress addr = new InetSocketAddress("127.0.0.1", server.port());
+        try {
+            selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE);
+            fail("SASL/PLAIN channel created without password");
+        } catch (KafkaException e) {
+            // Expected exception
+        }
+    }
+
+    /**
+     * Tests that mechanisms that are not supported in Kafka can be plugged in without modifying
+     * Kafka code if Sasl client and server providers are available.
+     */
+    @Test
+    public void testMechanismPluggability() throws Exception {
+        String node = "0";
+        SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL;
+        configureMechanisms("DIGEST-MD5", Arrays.asList("DIGEST-MD5"));
+
+        server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs);
+        createAndCheckClientConnection(securityProtocol, node);
+    }
+
+    /**
+     * Tests that servers supporting multiple SASL mechanisms work with clients using
+     * any of the enabled mechanisms.
+     */
+    @Test
+    public void testMultipleServerMechanisms() throws Exception {
+        SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL;
+        configureMechanisms("DIGEST-MD5", Arrays.asList("DIGEST-MD5", "PLAIN"));
+        server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs);
+
+        String node1 = "1";
+        saslClientConfigs.put(SaslConfigs.SASL_MECHANISM, "PLAIN");
+        createAndCheckClientConnection(securityProtocol, node1);
+
+        String node2 = "2";
+        saslClientConfigs.put(SaslConfigs.SASL_MECHANISM, "DIGEST-MD5");
+        createSelector(securityProtocol, saslClientConfigs);
+        InetSocketAddress addr = new InetSocketAddress("127.0.0.1", server.port());
+        selector.connect(node2, addr, BUFFER_SIZE, BUFFER_SIZE);
+        NetworkTestUtils.checkClientConnection(selector, node2, 100, 10);
+    }
+
+    /**
+     * Tests that Kafka ApiVersionsRequests are handled by the SASL server authenticator
+     * prior to SASL handshake flow and that subsequent authentication succeeds
+     * when transport layer is PLAINTEXT. This test simulates SASL authentication using a
+     * (non-SASL) PLAINTEXT client and sends ApiVersionsRequest straight after
+     * connection to the server is established, before any SASL-related packets are sent.
+     */
+    @Test
+    public void testUnauthenticatedApiVersionsRequestOverPlaintext() throws Exception {
+        testUnauthenticatedApiVersionsRequest(SecurityProtocol.SASL_PLAINTEXT);
+    }
+
+    /**
+     * Tests that Kafka ApiVersionsRequests are handled by the SASL server authenticator
+     * prior to SASL handshake flow and that subsequent authentication succeeds
+     * when transport layer is SSL. This test simulates SASL authentication using a
+     * (non-SASL) SSL client and sends ApiVersionsRequest straight after
+     * SSL handshake, before any SASL-related packets are sent.
+     */
+    @Test
+    public void testUnauthenticatedApiVersionsRequestOverSsl() throws Exception {
+        testUnauthenticatedApiVersionsRequest(SecurityProtocol.SASL_SSL);
+    }
+
+    /**
+     * Tests that any invalid data during Kafka SASL handshake request flow
+     * or the actual SASL authentication flow result in authentication failure
+     * and do not cause any failures in the server.
+     */
+    @Test
+    public void testInvalidSaslPacket() throws Exception {
+        SecurityProtocol securityProtocol = SecurityProtocol.SASL_PLAINTEXT;
+        configureMechanisms("PLAIN", Arrays.asList("PLAIN"));
+        server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs);
+
+        // Send invalid SASL packet after valid handshake request
+        String node1 = "invalid1";
+        createClientConnection(SecurityProtocol.PLAINTEXT, node1);
+        sendHandshakeRequestReceiveResponse(node1);
+        Random random = new Random();
+        byte[] bytes = new byte[1024];
+        random.nextBytes(bytes);
+        selector.send(new NetworkSend(node1, ByteBuffer.wrap(bytes)));
+        NetworkTestUtils.waitForChannelClose(selector, node1);
+        selector.close();
+
+        // Test good connection still works
+        createAndCheckClientConnection(securityProtocol, "good1");
+
+        // Send invalid SASL packet before handshake request
+        String node2 = "invalid2";
+        createClientConnection(SecurityProtocol.PLAINTEXT, node2);
+        random.nextBytes(bytes);
+        selector.send(new NetworkSend(node2, ByteBuffer.wrap(bytes)));
+        NetworkTestUtils.waitForChannelClose(selector, node2);
+        selector.close();
+
+        // Test good connection still works
+        createAndCheckClientConnection(securityProtocol, "good2");
+    }
+
+    /**
+     * Tests that ApiVersionsRequest after Kafka SASL handshake request flow,
+     * but prior to actual SASL authentication, results in authentication failure.
+     * This is similar to {@link #testUnauthenticatedApiVersionsRequest(SecurityProtocol)}
+     * where a non-SASL client is used to send requests that are processed by
+     * {@link SaslServerAuthenticator} of the server prior to client authentication.
+     */
+    @Test
+    public void testInvalidApiVersionsRequestSequence() throws Exception {
+        SecurityProtocol securityProtocol = SecurityProtocol.SASL_PLAINTEXT;
+        configureMechanisms("PLAIN", Arrays.asList("PLAIN"));
+        server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs);
+
+        // Send handshake request followed by ApiVersionsRequest
+        String node1 = "invalid1";
+        createClientConnection(SecurityProtocol.PLAINTEXT, node1);
+        sendHandshakeRequestReceiveResponse(node1);
+
+        RequestHeader versionsHeader = new RequestHeader(ApiKeys.API_VERSIONS.id, "someclient", 2);
+        selector.send(new NetworkSend(node1, RequestSend.serialize(versionsHeader, new ApiVersionsRequest().toStruct())));
+        NetworkTestUtils.waitForChannelClose(selector, node1);
+        selector.close();
+
+        // Test good connection still works
+        createAndCheckClientConnection(securityProtocol, "good1");
+    }
+
+    /**
+     * Tests that packets that are too big during Kafka SASL handshake request flow
+     * or the actual SASL authentication flow result in authentication failure
+     * and do not cause any failures in the server.
+     */
+    @Test
+    public void testPacketSizeTooBig() throws Exception {
+        SecurityProtocol securityProtocol = SecurityProtocol.SASL_PLAINTEXT;
+        configureMechanisms("PLAIN", Arrays.asList("PLAIN"));
+        server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs);
+
+        // Send SASL packet with large size after valid handshake request
+        String node1 = "invalid1";
+        createClientConnection(SecurityProtocol.PLAINTEXT, node1);
+        sendHandshakeRequestReceiveResponse(node1);
+        ByteBuffer buffer = ByteBuffer.allocate(1024);
+        buffer.putInt(Integer.MAX_VALUE);
+        buffer.put(new byte[buffer.capacity() - 4]);
+        buffer.rewind();
+        selector.send(new NetworkSend(node1, buffer));
+        NetworkTestUtils.waitForChannelClose(selector, node1);
+        selector.close();
+
+        // Test good connection still works
+        createAndCheckClientConnection(securityProtocol, "good1");
+
+        // Send packet with large size before handshake request
+        String node2 = "invalid2";
+        createClientConnection(SecurityProtocol.PLAINTEXT, node2);
+        buffer.clear();
+        buffer.putInt(Integer.MAX_VALUE);
+        buffer.put(new byte[buffer.capacity() - 4]);
+        buffer.rewind();
+        selector.send(new NetworkSend(node2, buffer));
+        NetworkTestUtils.waitForChannelClose(selector, node2);
+        selector.close();
+
+        // Test good connection still works
+        createAndCheckClientConnection(securityProtocol, "good2");
+    }
+
+    /**
+     * Tests that Kafka requests that are forbidden until successful authentication result
+     * in authentication failure and do not cause any failures in the server.
+     */
+    @Test
+    public void testDisallowedKafkaRequestsBeforeAuthentication() throws Exception {
+        SecurityProtocol securityProtocol = SecurityProtocol.SASL_PLAINTEXT;
+        configureMechanisms("PLAIN", Arrays.asList("PLAIN"));
+        server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs);
+
+        // Send metadata request before Kafka SASL handshake request
+        String node1 = "invalid1";
+        createClientConnection(SecurityProtocol.PLAINTEXT, node1);
+        RequestHeader metadataRequestHeader1 = new RequestHeader(ApiKeys.METADATA.id, "someclient", 1);
+        MetadataRequest metadataRequest1 = new MetadataRequest(Collections.singletonList("sometopic"));
+        selector.send(new NetworkSend(node1, RequestSend.serialize(metadataRequestHeader1, metadataRequest1.toStruct())));
+        NetworkTestUtils.waitForChannelClose(selector, node1);
+        selector.close();
+
+        // Test good connection still works
+        createAndCheckClientConnection(securityProtocol, "good1");
+
+        // Send metadata request after Kafka SASL handshake request
+        String node2 = "invalid2";
+        createClientConnection(SecurityProtocol.PLAINTEXT, node2);
+        sendHandshakeRequestReceiveResponse(node2);
+        RequestHeader metadataRequestHeader2 = new RequestHeader(ApiKeys.METADATA.id, "someclient", 2);
+        MetadataRequest metadataRequest2 = new MetadataRequest(Collections.singletonList("sometopic"));
+        selector.send(new NetworkSend(node2, RequestSend.serialize(metadataRequestHeader2, metadataRequest2.toStruct())));
+        NetworkTestUtils.waitForChannelClose(selector, node2);
+        selector.close();
+
+        // Test good connection still works
+        createAndCheckClientConnection(securityProtocol, "good2");
+    }
+
+    /**
+     * Tests that connections cannot be created if the login module class is unavailable.
+     */
+    @Test
+    public void testInvalidLoginModule() throws Exception {
+        TestJaasConfig jaasConfig = configureMechanisms("PLAIN", Arrays.asList("PLAIN"));
+        jaasConfig.createOrUpdateEntry(JaasUtils.LOGIN_CONTEXT_CLIENT, "InvalidLoginModule", TestJaasConfig.defaultClientOptions());
+
+        SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL;
+        server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs);
+        try {
+            createSelector(securityProtocol, saslClientConfigs);
+            fail("SASL/PLAIN channel created without valid login module");
+        } catch (KafkaException e) {
+            // Expected exception
+        }
+    }
+
+    /**
+     * Tests that mechanisms with default implementation in Kafka may be disabled in
+     * the Kafka server by removing from the enabled mechanism list.
+     */
+    @Test
+    public void testDisabledMechanism() throws Exception {
+        String node = "0";
+        SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL;
+        configureMechanisms("PLAIN", Arrays.asList("DIGEST-MD5"));
+
+        server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs);
+        createClientConnection(securityProtocol, node);
+        NetworkTestUtils.waitForChannelClose(selector, node);
+    }
+
+    /**
+     * Tests that clients using invalid SASL mechanisms fail authentication.
+     */
+    @Test
+    public void testInvalidMechanism() throws Exception {
+        String node = "0";
+        SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL;
+        configureMechanisms("PLAIN", Arrays.asList("PLAIN"));
+        saslClientConfigs.put(SaslConfigs.SASL_MECHANISM, "INVALID");
+
+        server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs);
+        createClientConnection(securityProtocol, node);
+        NetworkTestUtils.waitForChannelClose(selector, node);
+    }
+
+    /**
+     * Tests that Kafka ApiVersionsRequests are handled by the SASL server authenticator
+     * prior to SASL handshake flow and that subsequent authentication succeeds
+     * when transport layer is PLAINTEXT/SSL. This test uses a non-SASL client that simulates
+     * SASL authentication after ApiVersionsRequest.
+     * <p>
+     * Test sequence (using <tt>securityProtocol=PLAINTEXT</tt> as an example):
+     * <ol>
+     *   <li>Starts a SASL_PLAINTEXT test server that simply echoes back client requests after authentication.</li>
+     *   <li>A (non-SASL) PLAINTEXT test client connects to the SASL server port. Client is now unauthenticated.<./li>
+     *   <li>The unauthenticated non-SASL client sends an ApiVersionsRequest and validates the response.
+     *       A valid response indicates that {@link SaslServerAuthenticator} of the test server responded to
+     *       the ApiVersionsRequest even though the client is not yet authenticated.</li>
+     *   <li>The unauthenticated non-SASL client sends a SaslHandshakeRequest and validates the response. A valid response
+     *       indicates that {@link SaslServerAuthenticator} of the test server responded to the SaslHandshakeRequest
+     *       after processing ApiVersionsRequest.</li>
+     *   <li>The unauthenticated non-SASL client sends the SASL/PLAIN packet containing username/password to authenticate
+     *       itself. The client is now authenticated by the server. At this point this test client is at the
+     *       same state as a regular SASL_PLAINTEXT client that is <tt>ready</tt>.</li>
+     *   <li>The authenticated client sends random data to the server and checks that the data is echoed
+     *       back by the test server (ie, not Kafka request-response) to ensure that the client now
+     *       behaves exactly as a regular SASL_PLAINTEXT client that has completed authentication.</li>
+     * </ol>
+     */
+    private void testUnauthenticatedApiVersionsRequest(SecurityProtocol securityProtocol) throws Exception {
+        configureMechanisms("PLAIN", Arrays.asList("PLAIN"));
+        server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs);
+
+        // Create non-SASL connection to manually authenticate after ApiVersionsRequest
+        String node = "1";
+        SecurityProtocol clientProtocol;
+        switch (securityProtocol) {
+            case SASL_PLAINTEXT:
+                clientProtocol = SecurityProtocol.PLAINTEXT;
+                break;
+            case SASL_SSL:
+                clientProtocol = SecurityProtocol.SSL;
+                break;
+            default:
+                throw new IllegalArgumentException("Server protocol " + securityProtocol + " is not SASL");
+        }
+        createClientConnection(clientProtocol, node);
+        NetworkTestUtils.waitForChannelReady(selector, node);
+
+        // Send ApiVersionsRequest and check response
+        ApiVersionsResponse versionsResponse = sendVersionRequestReceiveResponse(node);
+        assertEquals(Protocol.MIN_VERSIONS[ApiKeys.SASL_HANDSHAKE.id], versionsResponse.apiVersion(ApiKeys.SASL_HANDSHAKE.id).minVersion);
+        assertEquals(Protocol.CURR_VERSION[ApiKeys.SASL_HANDSHAKE.id], versionsResponse.apiVersion(ApiKeys.SASL_HANDSHAKE.id).maxVersion);
+
+        // Send SaslHandshakeRequest and check response
+        SaslHandshakeResponse handshakeResponse = sendHandshakeRequestReceiveResponse(node);
+        assertEquals(Collections.singletonList("PLAIN"), handshakeResponse.enabledMechanisms());
+
+        // Authenticate using PLAIN username/password
+        String authString = "\u0000" + TestJaasConfig.USERNAME + "\u0000" + TestJaasConfig.PASSWORD;
+        selector.send(new NetworkSend(node, ByteBuffer.wrap(authString.getBytes("UTF-8"))));
+        waitForResponse();
+
+        // Check send/receive on the manually authenticated connection
+        NetworkTestUtils.checkClientConnection(selector, node, 100, 10);
+    }
+
+    private TestJaasConfig configureMechanisms(String clientMechanism, List<String> serverMechanisms) {
+        saslClientConfigs.put(SaslConfigs.SASL_MECHANISM, clientMechanism);
+        saslServerConfigs.put(SaslConfigs.SASL_ENABLED_MECHANISMS, serverMechanisms);
+        return TestJaasConfig.createConfiguration(clientMechanism, serverMechanisms);
+    }
+
+    private void createSelector(SecurityProtocol securityProtocol, Map<String, Object> clientConfigs) {
+        String saslMechanism = (String) saslClientConfigs.get(SaslConfigs.SASL_MECHANISM);
+        this.channelBuilder = ChannelBuilders.create(securityProtocol, Mode.CLIENT, LoginType.CLIENT, clientConfigs, saslMechanism, true);
+        this.selector = NetworkTestUtils.createSelector(channelBuilder);
+    }
+
+    private void createClientConnection(SecurityProtocol securityProtocol, String node) throws Exception {
+        createSelector(securityProtocol, saslClientConfigs);
+        InetSocketAddress addr = new InetSocketAddress("127.0.0.1", server.port());
+        selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE);
+    }
+
+    private void createAndCheckClientConnection(SecurityProtocol securityProtocol, String node) throws Exception {
+        createClientConnection(securityProtocol, node);
+        NetworkTestUtils.checkClientConnection(selector, node, 100, 10);
+        selector.close();
+        selector = null;
+    }
+
+    private Struct sendKafkaRequestReceiveResponse(String node, ApiKeys apiKey, AbstractRequestResponse request) throws IOException {
+        RequestHeader header = new RequestHeader(apiKey.id, "someclient", 1);
+        selector.send(new NetworkSend(node, RequestSend.serialize(header, request.toStruct())));
+        ByteBuffer responseBuffer = waitForResponse();
+        return NetworkClient.parseResponse(responseBuffer, header);
+    }
+
+    private SaslHandshakeResponse sendHandshakeRequestReceiveResponse(String node) throws Exception {
+        SaslHandshakeRequest handshakeRequest = new SaslHandshakeRequest("PLAIN");
+        Struct responseStruct = sendKafkaRequestReceiveResponse(node, ApiKeys.SASL_HANDSHAKE, handshakeRequest);
+        SaslHandshakeResponse response = new SaslHandshakeResponse(responseStruct);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        return response;
+    }
+
+    private ApiVersionsResponse sendVersionRequestReceiveResponse(String node) throws Exception {
+        ApiVersionsRequest handshakeRequest = new ApiVersionsRequest();
+        Struct responseStruct = sendKafkaRequestReceiveResponse(node, ApiKeys.API_VERSIONS, handshakeRequest);
+        ApiVersionsResponse response = new ApiVersionsResponse(responseStruct);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        return response;
+    }
+
+    private ByteBuffer waitForResponse() throws IOException {
+        int waitSeconds = 10;
+        do {
+            selector.poll(1000);
+        } while (selector.completedReceives().isEmpty() && waitSeconds-- > 0);
+        assertEquals(1, selector.completedReceives().size());
+        return selector.completedReceives().get(0).payload();
+    }
+}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/TestDigestLoginModule.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/TestDigestLoginModule.java
new file mode 100644
index 0000000..2923a5a
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/TestDigestLoginModule.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.security.authenticator;
+
+import java.io.IOException;
+import java.security.Provider;
+import java.security.Security;
+import java.util.Arrays;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.RealmCallback;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import javax.security.sasl.SaslServerFactory;
+
+import org.apache.kafka.common.security.plain.PlainLoginModule;
+
+/**
+ * Digest-MD5 login module for multi-mechanism tests. Since callback handlers are not configurable in Kafka
+ * yet, this replaces the standard Digest-MD5 SASL server provider with one that invokes the test callback handler.
+ * This login module uses the same format as PlainLoginModule and hence simply reuses the same methods.
+ *
+ */
+public class TestDigestLoginModule extends PlainLoginModule {
+
+    private static final SaslServerFactory STANDARD_DIGEST_SASL_SERVER_FACTORY;
+    static {
+        SaslServerFactory digestSaslServerFactory = null;
+        Enumeration<SaslServerFactory> factories = Sasl.getSaslServerFactories();
+        Map<String, Object> emptyProps = new HashMap<>();
+        while (factories.hasMoreElements()) {
+            SaslServerFactory factory = factories.nextElement();
+            if (Arrays.asList(factory.getMechanismNames(emptyProps)).contains("DIGEST-MD5")) {
+                digestSaslServerFactory = factory;
+                break;
+            }
+        }
+        STANDARD_DIGEST_SASL_SERVER_FACTORY = digestSaslServerFactory;
+        Security.insertProviderAt(new DigestSaslServerProvider(), 1);
+    }
+
+    public static class DigestServerCallbackHandler implements CallbackHandler {
+
+        @Override
+        public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+            for (Callback callback : callbacks) {
+                if (callback instanceof NameCallback) {
+                    NameCallback nameCallback = (NameCallback) callback;
+                    nameCallback.setName(nameCallback.getDefaultName());
+                } else if (callback instanceof PasswordCallback) {
+                    PasswordCallback passwordCallback = (PasswordCallback) callback;
+                    passwordCallback.setPassword(TestJaasConfig.PASSWORD.toCharArray());
+                } else if (callback instanceof RealmCallback) {
+                    RealmCallback realmCallback = (RealmCallback) callback;
+                    realmCallback.setText(realmCallback.getDefaultText());
+                } else if (callback instanceof AuthorizeCallback) {
+                    AuthorizeCallback authCallback = (AuthorizeCallback) callback;
+                    if (TestJaasConfig.USERNAME.equals(authCallback.getAuthenticationID())) {
+                        authCallback.setAuthorized(true);
+                        authCallback.setAuthorizedID(authCallback.getAuthenticationID());
+                    }
+                }
+            }
+        }
+    }
+
+    public static class DigestSaslServerFactory implements SaslServerFactory {
+
+        @Override
+        public SaslServer createSaslServer(String mechanism, String protocol, String serverName, Map<String, ?> props, CallbackHandler cbh)
+                throws SaslException {
+            return STANDARD_DIGEST_SASL_SERVER_FACTORY.createSaslServer(mechanism, protocol, serverName, props, new DigestServerCallbackHandler());
+        }
+
+        @Override
+        public String[] getMechanismNames(Map<String, ?> props) {
+            return new String[] {"DIGEST-MD5"};
+        }
+    }
+
+    public static class DigestSaslServerProvider extends Provider {
+
+        private static final long serialVersionUID = 1L;
+
+        protected DigestSaslServerProvider() {
+            super("Test SASL/Digest-MD5 Server Provider", 1.0, "Test SASL/Digest-MD5 Server Provider for Kafka");
+            super.put("SaslServerFactory.DIGEST-MD5", TestDigestLoginModule.DigestSaslServerFactory.class.getName());
+        }
+    }
+}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/TestJaasConfig.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/TestJaasConfig.java
new file mode 100644
index 0000000..2291cc1
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/TestJaasConfig.java
@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.security.authenticator;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag;
+
+import org.apache.kafka.common.security.JaasUtils;
+import org.apache.kafka.common.security.plain.PlainLoginModule;
+
+public class TestJaasConfig extends Configuration {
+
+    static final String USERNAME = "myuser";
+    static final String PASSWORD = "mypassword";
+
+    private Map<String, AppConfigurationEntry[]> entryMap = new HashMap<>();
+
+    public static TestJaasConfig createConfiguration(String clientMechanism, List<String> serverMechanisms) {
+        TestJaasConfig config = new TestJaasConfig();
+        config.createOrUpdateEntry(JaasUtils.LOGIN_CONTEXT_CLIENT, loginModule(clientMechanism), defaultClientOptions());
+        for (String mechanism : serverMechanisms) {
+            config.createOrUpdateEntry(JaasUtils.LOGIN_CONTEXT_SERVER, loginModule(mechanism), defaultServerOptions());
+        }
+        Configuration.setConfiguration(config);
+        return config;
+    }
+
+    public void setPlainClientOptions(String clientUsername, String clientPassword) {
+        Map<String, Object> options = new HashMap<>();
+        if (clientUsername != null)
+            options.put("username", clientUsername);
+        if (clientPassword != null)
+            options.put("password", clientPassword);
+        createOrUpdateEntry(JaasUtils.LOGIN_CONTEXT_CLIENT, PlainLoginModule.class.getName(), options);
+    }
+
+    public void createOrUpdateEntry(String name, String loginModule, Map<String, Object> options) {
+        AppConfigurationEntry entry = new AppConfigurationEntry(loginModule, LoginModuleControlFlag.REQUIRED, options);
+        entryMap.put(name, new AppConfigurationEntry[] {entry});
+    }
+
+    @Override
+    public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
+        return entryMap.get(name);
+    }
+
+    private static String loginModule(String mechanism) {
+        String loginModule;
+        switch (mechanism) {
+            case "PLAIN":
+                loginModule = PlainLoginModule.class.getName();
+                break;
+            case "DIGEST-MD5":
+                loginModule = TestDigestLoginModule.class.getName();
+                break;
+            default:
+                throw new IllegalArgumentException("Unsupported mechanism " + mechanism);
+        }
+        return loginModule;
+    }
+
+    public static Map<String, Object> defaultClientOptions() {
+        Map<String, Object> options = new HashMap<>();
+        options.put("username", USERNAME);
+        options.put("password", PASSWORD);
+        return options;
+    }
+
+    public static Map<String, Object> defaultServerOptions() {
+        Map<String, Object> options = new HashMap<>();
+        options.put("user_" + USERNAME, PASSWORD);
+        return options;
+    }
+}
\ No newline at end of file
diff --git a/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java b/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java
index 0c187cb..cff12a3 100644
--- a/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java
+++ b/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java
@@ -29,6 +29,7 @@
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -59,7 +60,7 @@
                                              record.timestamp(), record.timestampType(),
                                              record.checksum(), record.serializedKeySize(),
                                              record.serializedValueSize(),
-                                             record.key(), record.value().toUpperCase()));
+                                             record.key(), record.value().toUpperCase(Locale.ROOT)));
             }
             recordMap.put(tp, lst);
         }
diff --git a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java
index fbe2630..91e921f 100644
--- a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java
+++ b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java
@@ -210,6 +210,7 @@
             X509Certificate cCert = generateCertificate("CN=" + host + ", O=A client", cKP, 30, "SHA1withRSA");
             createKeyStore(keyStoreFile.getPath(), password, "client", cKP.getPrivate(), cCert);
             certs.put(certAlias, cCert);
+            keyStoreFile.deleteOnExit();
         } else if (mode == Mode.SERVER) {
             keyStoreFile = File.createTempFile("serverKS", ".jks");
             KeyPair sKP = generateKeyPair("RSA");
@@ -217,10 +218,12 @@
                                                         "SHA1withRSA");
             createKeyStore(keyStoreFile.getPath(), password, password, "server", sKP.getPrivate(), sCert);
             certs.put(certAlias, sCert);
+            keyStoreFile.deleteOnExit();
         }
 
         if (trustStore) {
             createTrustStore(trustStoreFile.getPath(), trustStorePassword, certs);
+            trustStoreFile.deleteOnExit();
         }
 
         return createSslConfig(mode, keyStoreFile, password, password, trustStoreFile, trustStorePassword);
diff --git a/clients/src/test/java/org/apache/kafka/test/TestUtils.java b/clients/src/test/java/org/apache/kafka/test/TestUtils.java
index 7ffc54a..1bfe578 100644
--- a/clients/src/test/java/org/apache/kafka/test/TestUtils.java
+++ b/clients/src/test/java/org/apache/kafka/test/TestUtils.java
@@ -20,14 +20,18 @@
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.Random;
 
 import org.apache.kafka.common.Cluster;
 import org.apache.kafka.common.Node;
 import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.utils.Utils;
 
 
 /**
@@ -45,20 +49,32 @@
     public static final Random SEEDED_RANDOM = new Random(192348092834L);
     public static final Random RANDOM = new Random();
 
+    public static Cluster singletonCluster(Map<String, Integer> topicPartitionCounts) {
+        return clusterWith(1, topicPartitionCounts);
+    }
+
     public static Cluster singletonCluster(String topic, int partitions) {
         return clusterWith(1, topic, partitions);
     }
 
-    public static Cluster clusterWith(int nodes, String topic, int partitions) {
+    public static Cluster clusterWith(int nodes, Map<String, Integer> topicPartitionCounts) {
         Node[] ns = new Node[nodes];
         for (int i = 0; i < nodes; i++)
             ns[i] = new Node(i, "localhost", 1969);
-        List<PartitionInfo> parts = new ArrayList<PartitionInfo>();
-        for (int i = 0; i < partitions; i++)
-            parts.add(new PartitionInfo(topic, i, ns[i % ns.length], ns, ns));
+        List<PartitionInfo> parts = new ArrayList<>();
+        for (Map.Entry<String, Integer> topicPartition : topicPartitionCounts.entrySet()) {
+            String topic = topicPartition.getKey();
+            int partitions = topicPartition.getValue();
+            for (int i = 0; i < partitions; i++)
+                parts.add(new PartitionInfo(topic, i, ns[i % ns.length], ns, ns));
+        }
         return new Cluster(asList(ns), parts, Collections.<String>emptySet());
     }
 
+    public static Cluster clusterWith(int nodes, String topic, int partitions) {
+        return clusterWith(nodes, Collections.singletonMap(topic, partitions));
+    }
+
     /**
      * Generate an array of random bytes
      * 
@@ -84,12 +100,44 @@
     }
 
     /**
-     * Creates an empty file in the default temporary-file directory, using `kafka` as the prefix and `tmp` as the
+     * Create an empty file in the default temporary-file directory, using `kafka` as the prefix and `tmp` as the
      * suffix to generate its name.
      */
     public static File tempFile() throws IOException {
         File file = File.createTempFile("kafka", ".tmp");
         file.deleteOnExit();
+
+        return file;
+    }
+
+    /**
+     * Create a temporary relative directory in the default temporary-file directory with the given prefix.
+     *
+     * @param prefix The prefix of the temporary directory, if null using "kafka-" as default prefix
+     */
+    public static File tempDirectory(String prefix) throws IOException {
+        return tempDirectory(null, prefix);
+    }
+
+    /**
+     * Create a temporary relative directory in the specified parent directory with the given prefix.
+     *
+     * @param parent The parent folder path name, if null using the default temporary-file directory
+     * @param prefix The prefix of the temporary directory, if null using "kafka-" as default prefix
+     */
+    public static File tempDirectory(Path parent, String prefix) throws IOException {
+        final File file = parent == null ?
+                Files.createTempDirectory(prefix == null ? "kafka-" : prefix).toFile() :
+                Files.createTempDirectory(parent, prefix == null ? "kafka-" : prefix).toFile();
+        file.deleteOnExit();
+
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            @Override
+            public void run() {
+                Utils.delete(file);
+            }
+        });
+
         return file;
     }
 
diff --git a/config/connect-distributed.properties b/config/connect-distributed.properties
index 46bd3bc..b25339f 100644
--- a/config/connect-distributed.properties
+++ b/config/connect-distributed.properties
@@ -30,7 +30,7 @@
 value.converter.schemas.enable=true
 
 # The internal converter used for offsets and config data is configurable and must be specified, but most users will
-# always want to use the built-in default. Offset and config data is never visible outside of Copcyat in this format.
+# always want to use the built-in default. Offset and config data is never visible outside of Kafka Connect in this format.
 internal.key.converter=org.apache.kafka.connect.json.JsonConverter
 internal.value.converter=org.apache.kafka.connect.json.JsonConverter
 internal.key.converter.schemas.enable=false
diff --git a/config/connect-standalone.properties b/config/connect-standalone.properties
index 8c4f98e..8760590 100644
--- a/config/connect-standalone.properties
+++ b/config/connect-standalone.properties
@@ -26,7 +26,7 @@
 value.converter.schemas.enable=true
 
 # The internal converter used for offsets and config data is configurable and must be specified, but most users will
-# always want to use the built-in default. Offset and config data is never visible outside of Copcyat in this format.
+# always want to use the built-in default. Offset and config data is never visible outside of Kafka Connect in this format.
 internal.key.converter=org.apache.kafka.connect.json.JsonConverter
 internal.value.converter=org.apache.kafka.connect.json.JsonConverter
 internal.key.converter.schemas.enable=false
diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java b/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java
index 37c056a..591644c 100644
--- a/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java
+++ b/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java
@@ -299,7 +299,7 @@
 
 
     /**
-     * Get the {@link Schema.Type} associated with the the given class.
+     * Get the {@link Schema.Type} associated with the given class.
      *
      * @param klass the Class to
      * @return the corresponding type, nor null if there is no matching type
diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/Schema.java b/connect/api/src/main/java/org/apache/kafka/connect/data/Schema.java
index 3c0e40c..ae2eeb5 100644
--- a/connect/api/src/main/java/org/apache/kafka/connect/data/Schema.java
+++ b/connect/api/src/main/java/org/apache/kafka/connect/data/Schema.java
@@ -18,6 +18,7 @@
 package org.apache.kafka.connect.data;
 
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 
 /**
@@ -48,7 +49,7 @@
         private String name;
 
         Type() {
-            this.name = this.name().toLowerCase();
+            this.name = this.name().toLowerCase(Locale.ROOT);
         }
 
         public String getName() {
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java
index 849fa2f..e7a0c36 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java
@@ -26,6 +26,8 @@
 import org.apache.kafka.connect.runtime.distributed.DistributedConfig;
 import org.apache.kafka.connect.runtime.distributed.DistributedHerder;
 import org.apache.kafka.connect.runtime.rest.RestServer;
+import org.apache.kafka.connect.storage.ConfigBackingStore;
+import org.apache.kafka.connect.storage.KafkaConfigBackingStore;
 import org.apache.kafka.connect.storage.KafkaOffsetBackingStore;
 import org.apache.kafka.connect.storage.KafkaStatusBackingStore;
 import org.apache.kafka.connect.storage.StatusBackingStore;
@@ -74,8 +76,12 @@
         StatusBackingStore statusBackingStore = new KafkaStatusBackingStore(time, worker.getInternalValueConverter());
         statusBackingStore.configure(config);
 
-        DistributedHerder herder = new DistributedHerder(config, time, worker, statusBackingStore, advertisedUrl.toString());
-        final Connect connect = new Connect(worker, herder, rest);
+        ConfigBackingStore configBackingStore = new KafkaConfigBackingStore(worker.getInternalValueConverter());
+        configBackingStore.configure(config);
+
+        DistributedHerder herder = new DistributedHerder(config, time, worker, statusBackingStore, configBackingStore,
+                advertisedUrl.toString());
+        final Connect connect = new Connect(herder, rest);
         try {
             connect.start();
         } catch (Exception e) {
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java
index 6c4335e..4ade18c 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java
@@ -76,7 +76,7 @@
         Worker worker = new Worker(workerId, time, config, new FileOffsetBackingStore());
 
         Herder herder = new StandaloneHerder(worker);
-        final Connect connect = new Connect(worker, herder, rest);
+        final Connect connect = new Connect(herder, rest);
         connect.start();
 
         try {
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
index a97c4db..83f56e2 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
@@ -19,6 +19,7 @@
 import org.apache.kafka.common.config.Config;
 import org.apache.kafka.common.config.ConfigDef;
 import org.apache.kafka.common.config.ConfigDef.ConfigKey;
+import org.apache.kafka.common.config.ConfigDef.Type;
 import org.apache.kafka.common.config.ConfigValue;
 import org.apache.kafka.connect.connector.Connector;
 import org.apache.kafka.connect.errors.NotFoundException;
@@ -28,10 +29,12 @@
 import org.apache.kafka.connect.runtime.rest.entities.ConfigValueInfo;
 import org.apache.kafka.connect.runtime.rest.entities.ConnectorPluginInfo;
 import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo;
+import org.apache.kafka.connect.storage.ConfigBackingStore;
 import org.apache.kafka.connect.storage.StatusBackingStore;
 import org.apache.kafka.connect.tools.VerifiableSinkConnector;
 import org.apache.kafka.connect.tools.VerifiableSourceConnector;
 import org.apache.kafka.connect.util.ConnectorTaskId;
+import org.apache.kafka.connect.util.ReflectionsUtil;
 import org.reflections.Reflections;
 import org.reflections.util.ClasspathHelper;
 import org.reflections.util.ConfigurationBuilder;
@@ -74,28 +77,47 @@
  */
 public abstract class AbstractHerder implements Herder, TaskStatus.Listener, ConnectorStatus.Listener {
 
+    private final String workerId;
     protected final Worker worker;
     protected final StatusBackingStore statusBackingStore;
-    private final String workerId;
+    protected final ConfigBackingStore configBackingStore;
 
     private Map<String, Connector> tempConnectors = new ConcurrentHashMap<>();
-    private static final List<Class<? extends Connector>> SKIPPED_CONNECTORS = Arrays.<Class<? extends Connector>>asList(VerifiableSourceConnector.class, VerifiableSinkConnector.class);
     private static List<ConnectorPluginInfo> validConnectorPlugins;
+    private static final Object LOCK = new Object();
+    private Thread classPathTraverser;
+    private static final List<Class<? extends Connector>> EXCLUDES = Arrays.<Class<? extends Connector>>asList(VerifiableSourceConnector.class, VerifiableSinkConnector.class);
 
-    public AbstractHerder(Worker worker, StatusBackingStore statusBackingStore, String workerId) {
+    public AbstractHerder(Worker worker,
+                          String workerId,
+                          StatusBackingStore statusBackingStore,
+                          ConfigBackingStore configBackingStore) {
         this.worker = worker;
-        this.statusBackingStore = statusBackingStore;
         this.workerId = workerId;
+        this.statusBackingStore = statusBackingStore;
+        this.configBackingStore = configBackingStore;
     }
 
     protected abstract int generation();
 
     protected void startServices() {
+        this.worker.start();
         this.statusBackingStore.start();
+        this.configBackingStore.start();
+        traverseClassPath();
     }
 
     protected void stopServices() {
         this.statusBackingStore.stop();
+        this.configBackingStore.stop();
+        this.worker.stop();
+        if (this.classPathTraverser != null) {
+            try {
+                this.classPathTraverser.join();
+            } catch (InterruptedException e) {
+                // ignore as it can only happen during shutdown
+            }
+        }
     }
 
     @Override
@@ -105,6 +127,18 @@
     }
 
     @Override
+    public void onPause(String connector) {
+        statusBackingStore.put(new ConnectorStatus(connector, ConnectorStatus.State.PAUSED,
+                workerId, generation()));
+    }
+
+    @Override
+    public void onResume(String connector) {
+        statusBackingStore.put(new ConnectorStatus(connector, TaskStatus.State.RUNNING,
+                workerId, generation()));
+    }
+
+    @Override
     public void onShutdown(String connector) {
         statusBackingStore.putSafe(new ConnectorStatus(connector, ConnectorStatus.State.UNASSIGNED,
                 workerId, generation()));
@@ -132,6 +166,16 @@
     }
 
     @Override
+    public void onResume(ConnectorTaskId id) {
+        statusBackingStore.put(new TaskStatus(id, TaskStatus.State.RUNNING, workerId, generation()));
+    }
+
+    @Override
+    public void onPause(ConnectorTaskId id) {
+        statusBackingStore.put(new TaskStatus(id, TaskStatus.State.PAUSED, workerId, generation()));
+    }
+
+    @Override
     public void onDeletion(String connector) {
         for (TaskStatus status : statusBackingStore.getAll(connector))
             statusBackingStore.put(new TaskStatus(status.id(), TaskStatus.State.DESTROYED, workerId, generation()));
@@ -139,6 +183,20 @@
     }
 
     @Override
+    public void pauseConnector(String connector) {
+        if (!configBackingStore.contains(connector))
+            throw new NotFoundException("Unknown connector " + connector);
+        configBackingStore.putTargetState(connector, TargetState.PAUSED);
+    }
+
+    @Override
+    public void resumeConnector(String connector) {
+        if (!configBackingStore.contains(connector))
+            throw new NotFoundException("Unknown connector " + connector);
+        configBackingStore.putTargetState(connector, TargetState.STARTED);
+    }
+
+    @Override
     public ConnectorStateInfo connectorStatus(String connName) {
         ConnectorStatus connector = statusBackingStore.get(connName);
         if (connector == null)
@@ -201,22 +259,26 @@
     }
 
     public static List<ConnectorPluginInfo> connectorPlugins() {
-        if (validConnectorPlugins != null) {
-            return validConnectorPlugins;
-        }
-
-        Reflections reflections = new Reflections(new ConfigurationBuilder().setUrls(ClasspathHelper.forJavaClassPath()));
-        Set<Class<? extends Connector>> connectorClasses = reflections.getSubTypesOf(Connector.class);
-        connectorClasses.removeAll(SKIPPED_CONNECTORS);
-        List<ConnectorPluginInfo> connectorPlugins = new LinkedList<>();
-        for (Class<? extends Connector> connectorClass: connectorClasses) {
-            int mod = connectorClass.getModifiers();
-            if (!Modifier.isAbstract(mod) && !Modifier.isInterface(mod)) {
-                connectorPlugins.add(new ConnectorPluginInfo(connectorClass.getCanonicalName()));
+        synchronized (LOCK) {
+            if (validConnectorPlugins != null) {
+                return validConnectorPlugins;
             }
+            ReflectionsUtil.registerUrlTypes();
+            ConfigurationBuilder builder = new ConfigurationBuilder().setUrls(ClasspathHelper.forJavaClassPath());
+            Reflections reflections = new Reflections(builder);
+
+            Set<Class<? extends Connector>> connectorClasses = reflections.getSubTypesOf(Connector.class);
+            connectorClasses.removeAll(EXCLUDES);
+            List<ConnectorPluginInfo> connectorPlugins = new LinkedList<>();
+            for (Class<? extends Connector> connectorClass : connectorClasses) {
+                int mod = connectorClass.getModifiers();
+                if (!Modifier.isAbstract(mod) && !Modifier.isInterface(mod)) {
+                    connectorPlugins.add(new ConnectorPluginInfo(connectorClass.getCanonicalName()));
+                }
+            }
+            validConnectorPlugins = connectorPlugins;
+            return connectorPlugins;
         }
-        validConnectorPlugins = connectorPlugins;
-        return connectorPlugins;
     }
 
     // public for testing
@@ -230,16 +292,18 @@
             configValueMap.put(configName, configValue);
             if (!configKeys.containsKey(configName)) {
                 configValue.addErrorMessage("Configuration is not defined: " + configName);
-                configInfoList.add(new ConfigInfo(null, convertConfigValue(configValue)));
+                configInfoList.add(new ConfigInfo(null, convertConfigValue(configValue, null)));
             }
         }
 
-        for (String configName: configKeys.keySet()) {
-            ConfigKeyInfo configKeyInfo = convertConfigKey(configKeys.get(configName));
+        for (Map.Entry<String, ConfigKey> entry : configKeys.entrySet()) {
+            String configName = entry.getKey();
+            ConfigKeyInfo configKeyInfo = convertConfigKey(entry.getValue());
+            Type type = entry.getValue().type;
             ConfigValueInfo configValueInfo = null;
             if (configValueMap.containsKey(configName)) {
                 ConfigValue configValue = configValueMap.get(configName);
-                configValueInfo = convertConfigValue(configValue);
+                configValueInfo = convertConfigValue(configValue, type);
                 errorCount += configValue.errorMessages().size();
             }
             configInfoList.add(new ConfigInfo(configKeyInfo, configValueInfo));
@@ -249,11 +313,16 @@
 
     private static ConfigKeyInfo convertConfigKey(ConfigKey configKey) {
         String name = configKey.name;
-        String type = configKey.type.name();
-        Object defaultValue = configKey.defaultValue;
+        Type type = configKey.type;
+        String typeName = configKey.type.name();
+
         boolean required = false;
-        if (defaultValue == ConfigDef.NO_DEFAULT_VALUE) {
+        String defaultValue;
+        if (configKey.defaultValue == ConfigDef.NO_DEFAULT_VALUE) {
+            defaultValue = (String) configKey.defaultValue;
             required = true;
+        } else {
+            defaultValue = ConfigDef.convertToString(configKey.defaultValue, type);
         }
         String importance = configKey.importance.name();
         String documentation = configKey.documentation;
@@ -262,11 +331,23 @@
         String width = configKey.width.name();
         String displayName = configKey.displayName;
         List<String> dependents = configKey.dependents;
-        return new ConfigKeyInfo(name, type, required, defaultValue, importance, documentation, group, orderInGroup, width, displayName, dependents);
+        return new ConfigKeyInfo(name, typeName, required, defaultValue, importance, documentation, group, orderInGroup, width, displayName, dependents);
     }
 
-    private static ConfigValueInfo convertConfigValue(ConfigValue configValue) {
-        return new ConfigValueInfo(configValue.name(), configValue.value(), configValue.recommendedValues(), configValue.errorMessages(), configValue.visible());
+    private static ConfigValueInfo convertConfigValue(ConfigValue configValue, Type type) {
+        String value = ConfigDef.convertToString(configValue.value(), type);
+        List<String> recommendedValues = new LinkedList<>();
+
+        if (type == Type.LIST) {
+            for (Object object: configValue.recommendedValues()) {
+                recommendedValues.add(ConfigDef.convertToString(object, Type.STRING));
+            }
+        } else {
+            for (Object object : configValue.recommendedValues()) {
+                recommendedValues.add(ConfigDef.convertToString(object, type));
+            }
+        }
+        return new ConfigValueInfo(configValue.name(), value, recommendedValues, configValue.errorMessages(), configValue.visible());
     }
 
     private Connector getConnector(String connType) {
@@ -288,4 +369,14 @@
             return null;
         }
     }
+
+    private void traverseClassPath() {
+        classPathTraverser = new Thread(new Runnable() {
+            @Override
+            public void run() {
+                connectorPlugins();
+            }
+        }, "CLASSPATH traversal thread.");
+        classPathTraverser.start();
+    }
 }
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractStatus.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractStatus.java
index 4f31be1..d00b81f 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractStatus.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractStatus.java
@@ -21,6 +21,7 @@
     public enum State {
         UNASSIGNED,
         RUNNING,
+        PAUSED,
         FAILED,
         DESTROYED,
     }
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java
index 49cf4bc..86f7f23 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java
@@ -33,7 +33,6 @@
 public class Connect {
     private static final Logger log = LoggerFactory.getLogger(Connect.class);
 
-    private final Worker worker;
     private final Herder herder;
     private final RestServer rest;
     private final CountDownLatch startLatch = new CountDownLatch(1);
@@ -41,9 +40,8 @@
     private final AtomicBoolean shutdown = new AtomicBoolean(false);
     private final ShutdownHook shutdownHook;
 
-    public Connect(Worker worker, Herder herder, RestServer rest) {
+    public Connect(Herder herder, RestServer rest) {
         log.debug("Kafka Connect instance created");
-        this.worker = worker;
         this.herder = herder;
         this.rest = rest;
         shutdownHook = new ShutdownHook();
@@ -54,7 +52,6 @@
             log.info("Kafka Connect starting");
             Runtime.getRuntime().addShutdownHook(shutdownHook);
 
-            worker.start();
             herder.start();
             rest.start(herder);
 
@@ -72,7 +69,6 @@
 
                 rest.stop();
                 herder.stop();
-                worker.stop();
 
                 log.info("Kafka Connect stopped");
             }
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorStatus.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorStatus.java
index d9a2eba..de5d393 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorStatus.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorStatus.java
@@ -16,6 +16,7 @@
  **/
 package org.apache.kafka.connect.runtime;
 
+
 public class ConnectorStatus extends AbstractStatus<String> {
 
     public ConnectorStatus(String connector, State state, String msg, String workerUrl, int generation) {
@@ -35,7 +36,9 @@
         void onShutdown(String connector);
 
         /**
-         * Invoked from the Connector using {@link org.apache.kafka.connect.connector.ConnectorContext#raiseError(Exception)}.
+         * Invoked from the Connector using {@link org.apache.kafka.connect.connector.ConnectorContext#raiseError(Exception)}
+         * or if either {@link org.apache.kafka.connect.connector.Connector#start(java.util.Map)} or
+         * {@link org.apache.kafka.connect.connector.Connector#stop()} throw an exception.
          * Note that no shutdown event will follow after the task has been failed.
          * @param connector The connector name
          * @param cause Error raised from the connector.
@@ -43,6 +46,18 @@
         void onFailure(String connector, Throwable cause);
 
         /**
+         * Invoked when the connector is paused through the REST API
+         * @param connector The connector name
+         */
+        void onPause(String connector);
+
+        /**
+         * Invoked after the connector has been resumed.
+         * @param connector The connector name
+         */
+        void onResume(String connector);
+
+        /**
          * Invoked after successful startup of the connector.
          * @param connector The connector name
          */
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java
index 3ea4a81..ce8bcf9 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java
@@ -61,9 +61,9 @@
      * from the current configuration. However, note
      *
      * @returns A list of connector names
-     * @throws org.apache.kafka.connect.runtime.distributed.NotLeaderException if this node can not resolve the request
+     * @throws org.apache.kafka.connect.runtime.distributed.RequestTargetException if this node can not resolve the request
      *         (e.g., because it has not joined the cluster or does not have configs in sync with the group) and it is
-     *         also not the leader
+     *         not the leader or the task owner (e.g., task restart must be handled by the worker which owns the task)
      * @throws org.apache.kafka.connect.errors.ConnectException if this node is the leader, but still cannot resolve the
      *         request (e.g., it is not in sync with other worker's config state)
      */
@@ -135,6 +135,35 @@
      */
     ConfigInfos validateConfigs(String connType, Map<String, String> connectorConfig);
 
+    /**
+     * Restart the task with the given id.
+     * @param id id of the task
+     * @param cb callback to invoke upon completion
+     */
+    void restartTask(ConnectorTaskId id, Callback<Void> cb);
+
+    /**
+     * Restart the connector.
+     * @param connName name of the connector
+     * @param cb callback to invoke upon completion
+     */
+    void restartConnector(String connName, Callback<Void> cb);
+
+    /**
+     * Pause the connector. This call will asynchronously suspend processing by the connector and all
+     * of its tasks.
+     * @param connector name of the connector
+     */
+    void pauseConnector(String connector);
+
+    /**
+     * Resume the connector. This call will asynchronously start the connector and its tasks (if
+     * not started already).
+     * @param connector name of the connector
+     */
+    void resumeConnector(String connector);
+
+
     class Created<T> {
         private final boolean created;
         private final T result;
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TargetState.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TargetState.java
new file mode 100644
index 0000000..b59b3bb
--- /dev/null
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TargetState.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ **/
+package org.apache.kafka.connect.runtime;
+
+/**
+ * The target state of a connector is its desired state as indicated by the user
+ * through interaction with the REST API. When a connector is first created, its
+ * target state is "STARTED." This does not mean it has actually started, just that
+ * the Connect framework will attempt to start it after its tasks have been assigned.
+ * After the connector has been paused, the target state will change to PAUSED,
+ * and all the tasks will stop doing work.
+ *
+ * Target states are persisted in the config topic, which is read by all of the
+ * workers in the group. When a worker sees a new target state for a connector which
+ * is running, it will transition any tasks which it owns (i.e. which have been
+ * assigned to it by the leader) into the desired target state. Upon completion of
+ * a task rebalance, the worker will start the task in the last known target state.
+ */
+public enum TargetState {
+    STARTED,
+    PAUSED,
+}
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskStatus.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskStatus.java
index 3542eb8..173a694 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskStatus.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskStatus.java
@@ -37,6 +37,18 @@
         void onStartup(ConnectorTaskId id);
 
         /**
+         * Invoked after the task has been paused.
+         * @param id The id of the task
+         */
+        void onPause(ConnectorTaskId id);
+
+        /**
+         * Invoked after the task has been resumed.
+         * @param id The id of the task
+         */
+        void onResume(ConnectorTaskId id);
+
+        /**
          * Invoked if the task raises an error. No shutdown event will follow.
          * @param id The id of the task
          * @param cause The error raised by the task.
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
index e1a806a..a88d0f9 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
@@ -26,7 +26,6 @@
 import org.apache.kafka.connect.connector.ConnectorContext;
 import org.apache.kafka.connect.connector.Task;
 import org.apache.kafka.connect.errors.ConnectException;
-import org.apache.kafka.connect.sink.SinkConnector;
 import org.apache.kafka.connect.sink.SinkTask;
 import org.apache.kafka.connect.source.SourceTask;
 import org.apache.kafka.connect.storage.Converter;
@@ -132,10 +131,10 @@
         long limit = started + config.getLong(WorkerConfig.TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_CONFIG);
 
         for (Map.Entry<String, WorkerConnector> entry : connectors.entrySet()) {
-            WorkerConnector conn = entry.getValue();
+            WorkerConnector workerConnector = entry.getValue();
             log.warn("Shutting down connector {} uncleanly; herder should have shut down connectors before the" +
-                    "Worker is stopped.", conn);
-            conn.stop();
+                    "Worker is stopped.", entry.getKey());
+            workerConnector.shutdown();
         }
 
         Collection<ConnectorTaskId> taskIds = tasks.keySet();
@@ -157,8 +156,12 @@
      * @param connConfig connector configuration
      * @param ctx context for the connector
      * @param statusListener listener for notifications of connector status changes
+     * @param initialState the initial target state that the connector should be initialized to
      */
-    public void startConnector(ConnectorConfig connConfig, ConnectorContext ctx, ConnectorStatus.Listener statusListener) {
+    public void startConnector(ConnectorConfig connConfig,
+                               ConnectorContext ctx,
+                               ConnectorStatus.Listener statusListener,
+                               TargetState initialState) {
         String connName = connConfig.getString(ConnectorConfig.NAME_CONFIG);
         Class<? extends Connector> connClass = getConnectorClass(connConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG));
 
@@ -171,22 +174,17 @@
         WorkerConnector workerConnector = new WorkerConnector(connName, connector, ctx, statusListener);
 
         log.info("Instantiated connector {} with version {} of type {}", connName, connector.version(), connClass.getName());
-        workerConnector.initialize();
-        try {
-            workerConnector.start(connConfig.originalsStrings());
-        } catch (ConnectException e) {
-            throw new ConnectException("Connector threw an exception while starting", e);
-        }
+        workerConnector.initialize(connConfig);
+        workerConnector.transitionTo(initialState);
 
         connectors.put(connName, workerConnector);
-
         log.info("Finished creating connector {}", connName);
     }
 
     /* Now that the configuration doesn't contain the actual class name, we need to be able to tell the herder whether a connector is a Sink */
     public boolean isSinkConnector(String connName) {
         WorkerConnector workerConnector = connectors.get(connName);
-        return SinkConnector.class.isAssignableFrom(workerConnector.delegate.getClass());
+        return workerConnector.isSinkConnector();
     }
 
     public Connector getConnector(String connType) {
@@ -242,6 +240,9 @@
         return names.substring(0, names.toString().length() - 2);
     }
 
+    public boolean ownsTask(ConnectorTaskId taskId) {
+        return tasks.containsKey(taskId);
+    }
 
     private static Connector instantiateConnector(Class<? extends Connector> connClass) {
         try {
@@ -260,7 +261,7 @@
         if (workerConnector == null)
             throw new ConnectException("Connector " + connName + " not found in this worker.");
 
-        Connector connector = workerConnector.delegate;
+        Connector connector = workerConnector.connector();
         List<Map<String, String>> result = new ArrayList<>();
         String taskClassName = connector.taskClass().getName();
         for (Map<String, String> taskProps : connector.taskConfigs(maxTasks)) {
@@ -280,7 +281,7 @@
         if (connector == null)
             throw new ConnectException("Connector " + connName + " not found in this worker.");
 
-        connector.stop();
+        connector.shutdown();
         connectors.remove(connName);
 
         log.info("Stopped connector {}", connName);
@@ -293,13 +294,24 @@
         return connectors.keySet();
     }
 
+    public boolean isRunning(String connName) {
+        WorkerConnector connector = connectors.get(connName);
+        if (connector == null)
+            throw new ConnectException("Connector " + connName + " not found in this worker.");
+        return connector.isRunning();
+    }
+
     /**
      * Add a new task.
      * @param id Globally unique ID for this task.
      * @param taskConfig the parsed task configuration
      * @param statusListener listener for notifications of task status changes
+     * @param initialState the initial target state that the task should be initialized to
      */
-    public void startTask(ConnectorTaskId id, TaskConfig taskConfig, TaskStatus.Listener statusListener) {
+    public void startTask(ConnectorTaskId id,
+                          TaskConfig taskConfig,
+                          TaskStatus.Listener statusListener,
+                          TargetState initialState) {
         log.info("Creating task {}", id);
 
         if (tasks.containsKey(id)) {
@@ -313,11 +325,11 @@
         final Task task = instantiateTask(taskClass);
         log.info("Instantiated task {} with version {} of type {}", id, task.version(), taskClass.getName());
 
-        final WorkerTask workerTask = buildWorkerTask(id, task, statusListener);
+        final WorkerTask workerTask = buildWorkerTask(id, task, statusListener, initialState);
 
         // Start the task before adding modifying any state, any exceptions are caught higher up the
         // call chain and there's no cleanup to do here
-        workerTask.initialize(taskConfig.originalsStrings());
+        workerTask.initialize(taskConfig);
         executor.submit(workerTask);
 
         if (task instanceof SourceTask) {
@@ -327,17 +339,21 @@
         tasks.put(id, workerTask);
     }
 
-    private WorkerTask buildWorkerTask(ConnectorTaskId id, Task task, TaskStatus.Listener lifecycleListener) {
+    private WorkerTask buildWorkerTask(ConnectorTaskId id,
+                                       Task task,
+                                       TaskStatus.Listener statusListener,
+                                       TargetState initialState) {
         // Decide which type of worker task we need based on the type of task.
         if (task instanceof SourceTask) {
             OffsetStorageReader offsetReader = new OffsetStorageReaderImpl(offsetBackingStore, id.connector(),
                     internalKeyConverter, internalValueConverter);
             OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetBackingStore, id.connector(),
                     internalKeyConverter, internalValueConverter);
-            return new WorkerSourceTask(id, (SourceTask) task, lifecycleListener, keyConverter, valueConverter, producer,
-                    offsetReader, offsetWriter, config, time);
+            return new WorkerSourceTask(id, (SourceTask) task, statusListener, initialState, keyConverter,
+                     valueConverter, producer, offsetReader, offsetWriter, config, time);
         } else if (task instanceof SinkTask) {
-            return new WorkerSinkTask(id, (SinkTask) task, lifecycleListener, config, keyConverter, valueConverter, time);
+            return new WorkerSinkTask(id, (SinkTask) task, statusListener, initialState, config, keyConverter,
+                    valueConverter, time);
         } else {
             log.error("Tasks must be a subclass of either SourceTask or SinkTask", task);
             throw new ConnectException("Tasks must be a subclass of either SourceTask or SinkTask");
@@ -415,51 +431,21 @@
         return workerId;
     }
 
-    private static class WorkerConnector  {
-        private final String connName;
-        private final ConnectorStatus.Listener lifecycleListener;
-        private final ConnectorContext ctx;
-        private final Connector delegate;
+    public boolean ownsConnector(String connName) {
+        return this.connectors.containsKey(connName);
+    }
 
-        public WorkerConnector(String connName,
-                               Connector delegate,
-                               ConnectorContext ctx,
-                               ConnectorStatus.Listener lifecycleListener) {
-            this.connName = connName;
-            this.ctx = ctx;
-            this.delegate = delegate;
-            this.lifecycleListener = lifecycleListener;
+    public void setTargetState(String connName, TargetState state) {
+        log.info("Setting connector {} state to {}", connName, state);
+
+        WorkerConnector connector = connectors.get(connName);
+        if (connector != null)
+            connector.transitionTo(state);
+
+        for (Map.Entry<ConnectorTaskId, WorkerTask> taskEntry : tasks.entrySet()) {
+            if (taskEntry.getKey().connector().equals(connName))
+                taskEntry.getValue().transitionTo(state);
         }
-
-        public void initialize() {
-            delegate.initialize(ctx);
-        }
-
-        public void start(Map<String, String> props) {
-            try {
-                delegate.start(props);
-                lifecycleListener.onStartup(connName);
-            } catch (Throwable t) {
-                log.error("Error while starting connector {}", connName, t);
-                lifecycleListener.onFailure(connName, t);
-            }
-        }
-
-        public void stop() {
-            try {
-                delegate.stop();
-                lifecycleListener.onShutdown(connName);
-            } catch (Throwable t) {
-                log.error("Error while shutting down connector {}", connName, t);
-                lifecycleListener.onFailure(connName, t);
-            }
-        }
-
-        @Override
-        public String toString() {
-            return delegate.toString();
-        }
-
     }
 
 }
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java
index 471e4a5..7ede130 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java
@@ -109,6 +109,11 @@
                     " from the domain of the REST API.";
     protected static final String ACCESS_CONTROL_ALLOW_ORIGIN_DEFAULT = "";
 
+    public static final String ACCESS_CONTROL_ALLOW_METHODS_CONFIG = "access.control.allow.methods";
+    protected static final String ACCESS_CONTROL_ALLOW_METHODS_DOC =
+        "Sets the methods supported for cross origin requests by setting the Access-Control-Allow-Methods header. "
+        + "The default value of the Access-Control-Allow-Methods header allows cross origin requests for GET, POST and HEAD.";
+    protected static final String ACCESS_CONTROL_ALLOW_METHODS_DEFAULT = "";
 
     /**
      * Get a basic ConfigDef for a WorkerConfig. This includes all the common settings. Subclasses can use this to
@@ -141,7 +146,10 @@
                 .define(REST_ADVERTISED_PORT_CONFIG, Type.INT,  null, Importance.LOW, REST_ADVERTISED_PORT_DOC)
                 .define(ACCESS_CONTROL_ALLOW_ORIGIN_CONFIG, Type.STRING,
                         ACCESS_CONTROL_ALLOW_ORIGIN_DEFAULT, Importance.LOW,
-                        ACCESS_CONTROL_ALLOW_ORIGIN_DOC);
+                        ACCESS_CONTROL_ALLOW_ORIGIN_DOC)
+                .define(ACCESS_CONTROL_ALLOW_METHODS_CONFIG, Type.STRING,
+                        ACCESS_CONTROL_ALLOW_METHODS_DEFAULT, Importance.LOW,
+                        ACCESS_CONTROL_ALLOW_METHODS_DOC);
     }
 
     public WorkerConfig(ConfigDef definition, Map<String, String> props) {
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java
new file mode 100644
index 0000000..7880095
--- /dev/null
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java
@@ -0,0 +1,206 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ **/
+package org.apache.kafka.connect.runtime;
+
+import org.apache.kafka.connect.connector.Connector;
+import org.apache.kafka.connect.connector.ConnectorContext;
+import org.apache.kafka.connect.sink.SinkConnector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+/**
+ * Container for connectors which is responsible for managing their lifecycle (e.g. handling startup,
+ * shutdown, pausing, etc.). Internally, we manage the runtime state of the connector and transition according
+ * to target state changes. Note that unlike connector tasks, the connector does not really have a "pause"
+ * state which is distinct from being stopped. We therefore treat pause operations as requests to momentarily
+ * stop the connector, and resume operations as requests to restart it (without reinitialization). Connector
+ * failures, whether in initialization or after startup, are treated as fatal, which means that we will not attempt
+ * to restart this connector instance after failure. What this means from a user perspective is that you must
+ * use the /restart REST API to restart a failed task. This behavior is consistent with task failures.
+ *
+ * Note that this class is NOT thread-safe.
+ */
+public class WorkerConnector {
+    private static final Logger log = LoggerFactory.getLogger(WorkerConnector.class);
+
+    private enum State {
+        INIT,    // initial state before startup
+        STOPPED, // the connector has been stopped/paused.
+        STARTED, // the connector has been started/resumed.
+        FAILED,  // the connector has failed (no further transitions are possible after this state)
+    }
+
+    private final String connName;
+    private final ConnectorStatus.Listener statusListener;
+    private final ConnectorContext ctx;
+    private final Connector connector;
+
+    private Map<String, String> config;
+    private State state;
+
+    public WorkerConnector(String connName,
+                           Connector connector,
+                           ConnectorContext ctx,
+                           ConnectorStatus.Listener statusListener) {
+        this.connName = connName;
+        this.ctx = ctx;
+        this.connector = connector;
+        this.statusListener = statusListener;
+        this.state = State.INIT;
+    }
+
+    public void initialize(ConnectorConfig connectorConfig) {
+        log.debug("Initializing connector {} with config {}", connName, config);
+
+        try {
+            this.config = connectorConfig.originalsStrings();
+
+            connector.initialize(new ConnectorContext() {
+                @Override
+                public void requestTaskReconfiguration() {
+                    ctx.requestTaskReconfiguration();
+                }
+
+                @Override
+                public void raiseError(Exception e) {
+                    log.error("Connector raised an error {}", connName, e);
+                    onFailure(e);
+                    ctx.raiseError(e);
+                }
+            });
+        } catch (Throwable t) {
+            log.error("Error initializing connector {}", connName, t);
+            onFailure(t);
+        }
+    }
+
+    private boolean doStart() {
+        try {
+            switch (state) {
+                case STARTED:
+                    return false;
+
+                case INIT:
+                case STOPPED:
+                    connector.start(config);
+                    this.state = State.STARTED;
+                    return true;
+
+                default:
+                    throw new IllegalArgumentException("Cannot start connector in state " + state);
+            }
+        } catch (Throwable t) {
+            log.error("Error while starting connector {}", connName, t);
+            onFailure(t);
+            return false;
+        }
+    }
+
+    private void onFailure(Throwable t) {
+        statusListener.onFailure(connName, t);
+        this.state = State.FAILED;
+    }
+
+    private void resume() {
+        if (doStart())
+            statusListener.onResume(connName);
+    }
+
+    private void start() {
+        if (doStart())
+            statusListener.onStartup(connName);
+    }
+
+    public boolean isRunning() {
+        return state == State.STARTED;
+    }
+
+    private void pause() {
+        try {
+            switch (state) {
+                case STOPPED:
+                    return;
+
+                case STARTED:
+                    connector.stop();
+                    // fall through
+
+                case INIT:
+                    statusListener.onPause(connName);
+                    this.state = State.STOPPED;
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Cannot pause connector in state " + state);
+            }
+        } catch (Throwable t) {
+            log.error("Error while shutting down connector {}", connName, t);
+            statusListener.onFailure(connName, t);
+            this.state = State.FAILED;
+        }
+    }
+
+    public void shutdown() {
+        try {
+            if (state == State.STARTED)
+                connector.stop();
+            this.state = State.STOPPED;
+        } catch (Throwable t) {
+            log.error("Error while shutting down connector {}", connName, t);
+            this.state = State.FAILED;
+        } finally {
+            statusListener.onShutdown(connName);
+        }
+    }
+
+    public void transitionTo(TargetState targetState) {
+        if (state == State.FAILED) {
+            log.warn("Cannot transition connector {} to {} since it has failed", connName, targetState);
+            return;
+        }
+
+        log.debug("Transition connector {} to {}", connName, targetState);
+        if (targetState == TargetState.PAUSED) {
+            pause();
+        } else if (targetState == TargetState.STARTED) {
+            if (state == State.INIT)
+                start();
+            else
+                resume();
+        } else {
+            throw new IllegalArgumentException("Unhandled target state " + targetState);
+        }
+    }
+
+    public boolean isSinkConnector() {
+        return SinkConnector.class.isAssignableFrom(connector.getClass());
+    }
+
+    public Connector connector() {
+        return connector;
+    }
+
+    @Override
+    public String toString() {
+        return "WorkerConnector{" +
+                "connName='" + connName + '\'' +
+                ", connector=" + connector +
+                '}';
+    }
+}
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java
index 6293455..f5eaac4 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java
@@ -43,11 +43,8 @@
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import static java.util.Collections.singleton;
 
@@ -78,12 +75,13 @@
 
     public WorkerSinkTask(ConnectorTaskId id,
                           SinkTask task,
-                          TaskStatus.Listener lifecycleListener,
+                          TaskStatus.Listener statusListener,
+                          TargetState initialState,
                           WorkerConfig workerConfig,
                           Converter keyConverter,
                           Converter valueConverter,
                           Time time) {
-        super(id, lifecycleListener);
+        super(id, statusListener, initialState);
 
         this.workerConfig = workerConfig;
         this.task = task;
@@ -103,10 +101,15 @@
     }
 
     @Override
-    public void initialize(Map<String, String> taskConfig) {
-        this.taskConfig = taskConfig;
-        this.consumer = createConsumer();
-        this.context = new WorkerSinkTaskContext(consumer);
+    public void initialize(TaskConfig taskConfig) {
+        try {
+            this.taskConfig = taskConfig.originalsStrings();
+            this.consumer = createConsumer();
+            this.context = new WorkerSinkTaskContext(consumer);
+        } catch (Throwable t) {
+            log.error("Task {} failed initialization and will not be started.", t);
+            onFailure(t);
+        }
     }
 
     @Override
@@ -126,6 +129,12 @@
     }
 
     @Override
+    public void transitionTo(TargetState state) {
+        super.transitionTo(state);
+        consumer.wakeup();
+    }
+
+    @Override
     public void execute() {
         initializeAndStart();
         try {
@@ -218,6 +227,12 @@
             deliverMessages();
         } catch (WakeupException we) {
             log.trace("{} consumer woken up", id);
+
+            if (shouldPause()) {
+                pauseAll();
+            } else if (!pausedForRedelivery) {
+                resumeAll();
+            }
         }
     }
 
@@ -338,6 +353,16 @@
         }
     }
 
+    private void resumeAll() {
+        for (TopicPartition tp : consumer.assignment())
+            if (!context.pausedPartitions().contains(tp))
+                consumer.resume(singleton(tp));
+    }
+
+    private void pauseAll() {
+        consumer.pause(consumer.assignment());
+    }
+
     private void deliverMessages() {
         // Finally, deliver this batch to the sink
         try {
@@ -350,9 +375,8 @@
             // If we had paused all consumer topic partitions to try to redeliver data, then we should resume any that
             // the task had not explicitly paused
             if (pausedForRedelivery) {
-                for (TopicPartition tp : consumer.assignment())
-                    if (!context.pausedPartitions().contains(tp))
-                        consumer.resume(singleton(tp));
+                if (!shouldPause())
+                    resumeAll();
                 pausedForRedelivery = false;
             }
         } catch (RetriableException e) {
@@ -360,7 +384,7 @@
             // If we're retrying a previous batch, make sure we've paused all topic partitions so we don't get new data,
             // but will still be able to poll in order to handle user-requested timeouts, keep group membership, etc.
             pausedForRedelivery = true;
-            consumer.pause(consumer.assignment());
+            pauseAll();
             // Let this exit normally, the batch will be reprocessed on the next loop.
         } catch (Throwable t) {
             log.error("Task {} threw an uncaught and unrecoverable exception", id, t);
@@ -412,24 +436,14 @@
             // If we paused everything for redelivery (which is no longer relevant since we discarded the data), make
             // sure anything we paused that the task didn't request to be paused *and* which we still own is resumed.
             // Also make sure our tracking of paused partitions is updated to remove any partitions we no longer own.
-            if (pausedForRedelivery) {
-                pausedForRedelivery = false;
+            pausedForRedelivery = false;
 
-                Set<TopicPartition> assigned = new HashSet<>(partitions);
-                Set<TopicPartition> taskPaused = context.pausedPartitions();
-
-                for (TopicPartition tp : partitions) {
-                    if (!taskPaused.contains(tp))
-                        consumer.resume(singleton(tp));
-                }
-
-                Iterator<TopicPartition> tpIter = taskPaused.iterator();
-                while (tpIter.hasNext()) {
-                    TopicPartition tp = tpIter.next();
-                    if (assigned.contains(tp))
-                        tpIter.remove();
-                }
-            }
+            // Ensure that the paused partitions contains only assigned partitions and repause as necessary
+            context.pausedPartitions().retainAll(partitions);
+            if (shouldPause())
+                pauseAll();
+            else if (!context.pausedPartitions().isEmpty())
+                consumer.pause(context.pausedPartitions());
 
             // Instead of invoking the assignment callback on initialization, we guarantee the consumer is ready upon
             // task start. Since this callback gets invoked during that initial setup before we've started the task, we
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java
index 3a43f96..602af4a 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java
@@ -76,7 +76,8 @@
 
     public WorkerSourceTask(ConnectorTaskId id,
                             SourceTask task,
-                            TaskStatus.Listener lifecycleListener,
+                            TaskStatus.Listener statusListener,
+                            TargetState initialState,
                             Converter keyConverter,
                             Converter valueConverter,
                             KafkaProducer<byte[], byte[]> producer,
@@ -84,7 +85,7 @@
                             OffsetStorageWriter offsetWriter,
                             WorkerConfig workerConfig,
                             Time time) {
-        super(id, lifecycleListener);
+        super(id, statusListener, initialState);
 
         this.workerConfig = workerConfig;
         this.task = task;
@@ -104,8 +105,13 @@
     }
 
     @Override
-    public void initialize(Map<String, String> config) {
-        this.taskConfig = config;
+    public void initialize(TaskConfig taskConfig) {
+        try {
+            this.taskConfig = taskConfig.originalsStrings();
+        } catch (Throwable t) {
+            log.error("Task {} failed initialization and will not be started.", t);
+            onFailure(t);
+        }
     }
 
     protected void close() {
@@ -139,6 +145,11 @@
             }
 
             while (!isStopping()) {
+                if (shouldPause()) {
+                    awaitUnpause();
+                    continue;
+                }
+
                 if (toSend == null) {
                     log.debug("Nothing to send to Kafka. Polling source for additional records");
                     toSend = task.poll();
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java
index 7979fb0..846ca95 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java
@@ -21,33 +21,41 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Map;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
  * Handles processing for an individual task. This interface only provides the basic methods
  * used by {@link Worker} to manage the tasks. Implementations combine a user-specified Task with
  * Kafka to create a data flow.
+ *
+ * Note on locking: since the task runs in its own thread, special care must be taken to ensure
+ * that state transitions are reported correctly, in particular since some state transitions are
+ * asynchronous (e.g. pause/resume). For example, changing the state to paused could cause a race
+ * if the task fails at the same time. To protect from these cases, we synchronize status updates
+ * using the WorkerTask's monitor.
  */
 abstract class WorkerTask implements Runnable {
     private static final Logger log = LoggerFactory.getLogger(WorkerTask.class);
 
     protected final ConnectorTaskId id;
-    private final AtomicBoolean stopping;
-    private final AtomicBoolean running;
-    private final AtomicBoolean cancelled;
+    private final AtomicBoolean stopping;   // indicates whether the Worker has asked the task to stop
+    private final AtomicBoolean cancelled;  // indicates whether the Worker has cancelled the task (e.g. because of slow shutdown)
     private final CountDownLatch shutdownLatch;
-    private final TaskStatus.Listener lifecycleListener;
+    private final TaskStatus.Listener statusListener;
+    private final AtomicReference<TargetState> targetState;
 
-    public WorkerTask(ConnectorTaskId id, TaskStatus.Listener lifecycleListener) {
+    public WorkerTask(ConnectorTaskId id,
+                      TaskStatus.Listener statusListener,
+                      TargetState initialState) {
         this.id = id;
         this.stopping = new AtomicBoolean(false);
-        this.running = new AtomicBoolean(false);
         this.cancelled = new AtomicBoolean(false);
         this.shutdownLatch = new CountDownLatch(1);
-        this.lifecycleListener = lifecycleListener;
+        this.statusListener = statusListener;
+        this.targetState = new AtomicReference<>(initialState);
     }
 
     public ConnectorTaskId id() {
@@ -58,14 +66,24 @@
      * Initialize the task for execution.
      * @param props initial configuration
      */
-    public abstract void initialize(Map<String, String> props);
+    public abstract void initialize(TaskConfig taskConfig);
+
+
+    private void triggerStop() {
+        synchronized (this) {
+            this.stopping.set(true);
+
+            // wakeup any threads that are waiting for unpause
+            this.notifyAll();
+        }
+    }
 
     /**
      * Stop this task from processing messages. This method does not block, it only triggers
      * shutdown. Use #{@link #awaitStop} to block until completion.
      */
     public void stop() {
-        this.stopping.set(true);
+        triggerStop();
     }
 
     /**
@@ -83,9 +101,6 @@
      * @return true if successful, false if the timeout was reached
      */
     public boolean awaitStop(long timeoutMs) {
-        if (!running.get())
-            return true;
-
         try {
             return shutdownLatch.await(timeoutMs, TimeUnit.MILLISECONDS);
         } catch (InterruptedException e) {
@@ -101,10 +116,6 @@
         return stopping.get();
     }
 
-    protected boolean isStopped() {
-        return !running.get();
-    }
-
     private void doClose() {
         try {
             close();
@@ -115,14 +126,17 @@
     }
 
     private void doRun() {
-        if (!this.running.compareAndSet(false, true))
-            throw new IllegalStateException("The task cannot be started while still running");
-
         try {
-            if (stopping.get())
-                return;
+            synchronized (this) {
+                if (stopping.get())
+                    return;
 
-            lifecycleListener.onStartup(id);
+                if (targetState.get() == TargetState.PAUSED)
+                    statusListener.onPause(id);
+                else
+                    statusListener.onStartup(id);
+            }
+
             execute();
         } catch (Throwable t) {
             log.error("Task {} threw an uncaught and unrecoverable exception", id, t);
@@ -133,22 +147,80 @@
         }
     }
 
+    private void onShutdown() {
+        synchronized (this) {
+            triggerStop();
+
+            // if we were cancelled, skip the status update since the task may have already been
+            // started somewhere else
+            if (!cancelled.get())
+                statusListener.onShutdown(id);
+        }
+    }
+
+    protected void onFailure(Throwable t) {
+        synchronized (this) {
+            triggerStop();
+
+            // if we were cancelled, skip the status update since the task may have already been
+            // started somewhere else
+            if (!cancelled.get())
+                statusListener.onFailure(id, t);
+        }
+    }
+
     @Override
     public void run() {
         try {
             doRun();
-            if (!cancelled.get())
-                lifecycleListener.onShutdown(id);
+            onShutdown();
         } catch (Throwable t) {
-            if (!cancelled.get())
-                lifecycleListener.onFailure(id, t);
+            onFailure(t);
 
             if (t instanceof Error)
                 throw t;
         } finally {
-            running.set(false);
             shutdownLatch.countDown();
         }
     }
 
+    public boolean shouldPause() {
+        return this.targetState.get() == TargetState.PAUSED;
+    }
+
+    /**
+     * Await task resumption.
+     * @return true if the task's target state is not paused, false if the task is shutdown before resumption
+     * @throws InterruptedException
+     */
+    protected boolean awaitUnpause() throws InterruptedException {
+        synchronized (this) {
+            while (targetState.get() == TargetState.PAUSED) {
+                if (stopping.get())
+                    return false;
+                this.wait();
+            }
+            return true;
+        }
+    }
+
+    public void transitionTo(TargetState state) {
+        synchronized (this) {
+            // ignore the state change if we are stopping
+            if (stopping.get())
+                return;
+
+            TargetState oldState = this.targetState.getAndSet(state);
+            if (state != oldState) {
+                if (state == TargetState.PAUSED) {
+                    statusListener.onPause(id);
+                } else if (state == TargetState.STARTED) {
+                    statusListener.onResume(id);
+                    this.notifyAll();
+                } else
+                    throw new IllegalArgumentException("Unhandled target state " + state);
+            }
+        }
+    }
+
 }
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ClusterConfigState.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ClusterConfigState.java
index cc4a3c1..ea5ba82 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ClusterConfigState.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ClusterConfigState.java
@@ -17,36 +17,47 @@
 
 package org.apache.kafka.connect.runtime.distributed;
 
+import org.apache.kafka.connect.runtime.TargetState;
 import org.apache.kafka.connect.util.ConnectorTaskId;
 
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.TreeMap;
 
 /**
  * An immutable snapshot of the configuration state of connectors and tasks in a Kafka Connect cluster.
  */
 public class ClusterConfigState {
-    public static final ClusterConfigState EMPTY = new ClusterConfigState(-1, Collections.<String, Integer>emptyMap(),
-            Collections.<String, Map<String, String>>emptyMap(), Collections.<ConnectorTaskId, Map<String, String>>emptyMap(),
+    public static final long NO_OFFSET = -1;
+    public static final ClusterConfigState EMPTY = new ClusterConfigState(
+            NO_OFFSET,
+            Collections.<String, Integer>emptyMap(),
+            Collections.<String, Map<String, String>>emptyMap(),
+            Collections.<String, TargetState>emptyMap(),
+            Collections.<ConnectorTaskId, Map<String, String>>emptyMap(),
             Collections.<String>emptySet());
 
     private final long offset;
     private final Map<String, Integer> connectorTaskCounts;
     private final Map<String, Map<String, String>> connectorConfigs;
+    private final Map<String, TargetState> connectorTargetStates;
     private final Map<ConnectorTaskId, Map<String, String>> taskConfigs;
     private final Set<String> inconsistentConnectors;
 
     public ClusterConfigState(long offset,
                               Map<String, Integer> connectorTaskCounts,
                               Map<String, Map<String, String>> connectorConfigs,
+                              Map<String, TargetState> connectorTargetStates,
                               Map<ConnectorTaskId, Map<String, String>> taskConfigs,
                               Set<String> inconsistentConnectors) {
         this.offset = offset;
         this.connectorTaskCounts = connectorTaskCounts;
         this.connectorConfigs = connectorConfigs;
+        this.connectorTargetStates = connectorTargetStates;
         this.taskConfigs = taskConfigs;
         this.inconsistentConnectors = inconsistentConnectors;
     }
@@ -61,6 +72,15 @@
     }
 
     /**
+     * Check whether this snapshot contains configuration for a connector.
+     * @param connector name of the connector
+     * @return true if this state contains configuration for the connector, false otherwise
+     */
+    public boolean contains(String connector) {
+        return connectorConfigs.containsKey(connector);
+    }
+
+    /**
      * Get a list of the connectors in this configuration
      */
     public Set<String> connectors() {
@@ -77,6 +97,15 @@
     }
 
     /**
+     * Get the target state of the connector
+     * @param connector name of the connector
+     * @return the target state
+     */
+    public TargetState targetState(String connector) {
+        return connectorTargetStates.get(connector);
+    }
+
+    /**
      * Get the configuration for a task.
      * @param task id of the task
      * @return a map containing configuration parameters
@@ -86,6 +115,20 @@
     }
 
     /**
+     * Get all task configs for a connector.
+     * @param connector name of the connector
+     * @return a list of task configurations
+     */
+    public List<Map<String, String>> allTaskConfigs(String connector) {
+        Map<Integer, Map<String, String>> taskConfigs = new TreeMap<>();
+        for (Map.Entry<ConnectorTaskId, Map<String, String>> taskConfigEntry : this.taskConfigs.entrySet()) {
+            if (taskConfigEntry.getKey().connector().equals(connector))
+                taskConfigs.put(taskConfigEntry.getKey().task(), taskConfigEntry.getValue());
+        }
+        return new LinkedList<>(taskConfigs.values());
+    }
+
+    /**
      * Get the number of tasks assigned for the given connector.
      * @param connectorName name of the connector to look up tasks for
      * @return the number of tasks
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java
index 2fc8297..037eba7 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java
@@ -28,12 +28,13 @@
 import org.apache.kafka.connect.runtime.AbstractHerder;
 import org.apache.kafka.connect.runtime.ConnectorConfig;
 import org.apache.kafka.connect.runtime.HerderConnectorContext;
+import org.apache.kafka.connect.runtime.TargetState;
 import org.apache.kafka.connect.runtime.TaskConfig;
 import org.apache.kafka.connect.runtime.Worker;
 import org.apache.kafka.connect.runtime.rest.RestServer;
 import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo;
 import org.apache.kafka.connect.runtime.rest.entities.TaskInfo;
-import org.apache.kafka.connect.storage.KafkaConfigStorage;
+import org.apache.kafka.connect.storage.ConfigBackingStore;
 import org.apache.kafka.connect.storage.StatusBackingStore;
 import org.apache.kafka.connect.util.Callback;
 import org.apache.kafka.connect.util.ConnectorTaskId;
@@ -43,7 +44,6 @@
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -52,7 +52,6 @@
 import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
@@ -78,19 +77,27 @@
  *     by a single node at a time. Most importantly, this includes writing updated configurations for connectors and tasks,
  *     (and therefore, also for creating, destroy, and scaling up/down connectors).
  * </p>
+ * <p>
+ *     The DistributedHerder uses a single thread for most of its processing. This includes processing
+ *     config changes, handling task rebalances and serving requests from the HTTP layer. The latter are pushed
+ *     into a queue until the thread has time to handle them. A consequence of this is that requests can get blocked
+ *     behind a worker rebalance. When the herder knows that a rebalance is expected, it typically returns an error
+ *     immediately to the request, but this is not always possible (in particular when another worker has requested
+ *     the rebalance). Similar to handling HTTP requests, config changes which are observed asynchronously by polling
+ *     the config log are batched for handling in the work thread.
+ * </p>
  */
 public class DistributedHerder extends AbstractHerder implements Runnable {
     private static final Logger log = LoggerFactory.getLogger(DistributedHerder.class);
 
     private static final long RECONFIGURE_CONNECTOR_TASKS_BACKOFF_MS = 250;
 
-    private final KafkaConfigStorage configStorage;
-    private ClusterConfigState configState;
     private final Time time;
 
     private final int workerSyncTimeoutMs;
     private final int workerUnsyncBackoffMs;
 
+    private final ExecutorService forwardRequestExecutor;
     private final WorkerGroupMember member;
     private final AtomicBoolean stopping;
     private final CountDownLatch stopLatch = new CountDownLatch(1);
@@ -100,6 +107,7 @@
     private boolean rebalanceResolved;
     private ConnectProtocol.Assignment assignment;
     private boolean canReadConfigs;
+    private ClusterConfigState configState;
 
     // To handle most external requests, like creating or destroying a connector, we can use a generic request where
     // the caller specifies all the code that should be executed.
@@ -107,51 +115,44 @@
     // Config updates can be collected and applied together when possible. Also, we need to take care to rebalance when
     // needed (e.g. task reconfiguration, which requires everyone to coordinate offset commits).
     private Set<String> connectorConfigUpdates = new HashSet<>();
+    // Similarly collect target state changes (when observed by the config storage listener) for handling in the
+    // herder's main thread.
+    private Set<String> connectorTargetStateChanges = new HashSet<>();
     private boolean needsReconfigRebalance;
     private volatile int generation;
 
-    private final ExecutorService forwardRequestExecutor;
-
     public DistributedHerder(DistributedConfig config,
                              Time time,
                              Worker worker,
                              StatusBackingStore statusBackingStore,
+                             ConfigBackingStore configBackingStore,
                              String restUrl) {
-        this(config, worker.workerId(), worker, statusBackingStore, null, null, restUrl, time);
+        this(config, worker, worker.workerId(), statusBackingStore, configBackingStore, null, restUrl, time);
+        configBackingStore.setUpdateListener(new ConfigUpdateListener());
     }
 
     // visible for testing
     DistributedHerder(DistributedConfig config,
-                      String workerId,
                       Worker worker,
+                      String workerId,
                       StatusBackingStore statusBackingStore,
-                      KafkaConfigStorage configStorage,
+                      ConfigBackingStore configStorage,
                       WorkerGroupMember member,
                       String restUrl,
                       Time time) {
-        super(worker, statusBackingStore, workerId);
+        super(worker, workerId, statusBackingStore, configStorage);
 
-        if (configStorage != null) {
-            // For testing. Assume configuration has already been performed
-            this.configStorage = configStorage;
-        } else {
-            this.configStorage = new KafkaConfigStorage(worker.getInternalValueConverter(), connectorConfigCallback(), taskConfigCallback());
-            this.configStorage.configure(config);
-        }
-        configState = ClusterConfigState.EMPTY;
         this.time = time;
-
         this.workerSyncTimeoutMs = config.getInt(DistributedConfig.WORKER_SYNC_TIMEOUT_MS_CONFIG);
         this.workerUnsyncBackoffMs = config.getInt(DistributedConfig.WORKER_UNSYNC_BACKOFF_MS_CONFIG);
+        this.member = member != null ? member : new WorkerGroupMember(config, restUrl, this.configBackingStore, new RebalanceListener(), time);
+        this.forwardRequestExecutor = Executors.newSingleThreadExecutor();
 
-        this.member = member != null ? member : new WorkerGroupMember(config, restUrl, this.configStorage, rebalanceListener(), time);
         stopping = new AtomicBoolean(false);
-
+        configState = ClusterConfigState.EMPTY;
         rebalanceResolved = true; // If we still need to follow up after a rebalance occurred, starting up tasks
         needsReconfigRebalance = false;
         canReadConfigs = true; // We didn't try yet, but Configs are readable until proven otherwise
-
-        forwardRequestExecutor = Executors.newSingleThreadExecutor();
     }
 
     @Override
@@ -161,19 +162,6 @@
     }
 
     @Override
-    protected void startServices() {
-        super.startServices();
-        configStorage.start();
-    }
-
-    @Override
-    protected void stopServices() {
-        super.stopServices();
-        if (configStorage != null)
-            configStorage.stop();
-    }
-
-    @Override
     public void run() {
         try {
             log.info("Herder starting");
@@ -248,50 +236,48 @@
 
         // Process any configuration updates
         Set<String> connectorConfigUpdatesCopy = null;
+        Set<String> connectorTargetStateChangesCopy = null;
         synchronized (this) {
-            if (needsReconfigRebalance || !connectorConfigUpdates.isEmpty()) {
+            if (needsReconfigRebalance || !connectorConfigUpdates.isEmpty() || !connectorTargetStateChanges.isEmpty()) {
                 // Connector reconfigs only need local updates since there is no coordination between workers required.
                 // However, if connectors were added or removed, work needs to be rebalanced since we have more work
                 // items to distribute among workers.
-                ClusterConfigState newConfigState = configStorage.snapshot();
-                if (!newConfigState.connectors().equals(configState.connectors()))
-                    needsReconfigRebalance = true;
-                configState = newConfigState;
+                configState = configBackingStore.snapshot();
+
                 if (needsReconfigRebalance) {
                     // Task reconfigs require a rebalance. Request the rebalance, clean out state, and then restart
                     // this loop, which will then ensure the rebalance occurs without any other requests being
                     // processed until it completes.
                     member.requestRejoin();
-                    // Any connector config updates will be addressed during the rebalance too
+                    // Any connector config updates or target state changes will be addressed during the rebalance too
                     connectorConfigUpdates.clear();
+                    connectorTargetStateChanges.clear();
                     needsReconfigRebalance = false;
                     return;
-                } else if (!connectorConfigUpdates.isEmpty()) {
-                    // We can't start/stop while locked since starting connectors can cause task updates that will
-                    // require writing configs, which in turn make callbacks into this class from another thread that
-                    // require acquiring a lock. This leads to deadlock. Instead, just copy the info we need and process
-                    // the updates after unlocking.
-                    connectorConfigUpdatesCopy = connectorConfigUpdates;
-                    connectorConfigUpdates = new HashSet<>();
+                } else {
+                    if (!connectorConfigUpdates.isEmpty()) {
+                        // We can't start/stop while locked since starting connectors can cause task updates that will
+                        // require writing configs, which in turn make callbacks into this class from another thread that
+                        // require acquiring a lock. This leads to deadlock. Instead, just copy the info we need and process
+                        // the updates after unlocking.
+                        connectorConfigUpdatesCopy = connectorConfigUpdates;
+                        connectorConfigUpdates = new HashSet<>();
+                    }
+
+                    if (!connectorTargetStateChanges.isEmpty()) {
+                        // Similarly for target state changes which can cause connectors to be restarted
+                        connectorTargetStateChangesCopy = connectorTargetStateChanges;
+                        connectorTargetStateChanges = new HashSet<>();
+                    }
                 }
             }
         }
-        if (connectorConfigUpdatesCopy != null) {
-            // If we only have connector config updates, we can just bounce the updated connectors that are
-            // currently assigned to this worker.
-            Set<String> localConnectors = assignment == null ? Collections.<String>emptySet() : new HashSet<>(assignment.connectors());
-            for (String connectorName : connectorConfigUpdatesCopy) {
-                if (!localConnectors.contains(connectorName))
-                    continue;
-                boolean remains = configState.connectors().contains(connectorName);
-                log.info("Handling connector-only config update by {} connector {}",
-                        remains ? "restarting" : "stopping", connectorName);
-                worker.stopConnector(connectorName);
-                // The update may be a deletion, so verify we actually need to restart the connector
-                if (remains)
-                    startConnector(connectorName);
-            }
-        }
+
+        if (connectorConfigUpdatesCopy != null)
+            processConnectorConfigUpdates(connectorConfigUpdatesCopy);
+
+        if (connectorTargetStateChangesCopy != null)
+            processTargetStateChanges(connectorTargetStateChangesCopy);
 
         // Let the group take any actions it needs to
         try {
@@ -303,6 +289,36 @@
         }
     }
 
+    private void processConnectorConfigUpdates(Set<String> connectorConfigUpdates) {
+        // If we only have connector config updates, we can just bounce the updated connectors that are
+        // currently assigned to this worker.
+        Set<String> localConnectors = assignment == null ? Collections.<String>emptySet() : new HashSet<>(assignment.connectors());
+        for (String connectorName : connectorConfigUpdates) {
+            if (!localConnectors.contains(connectorName))
+                continue;
+            boolean remains = configState.contains(connectorName);
+            log.info("Handling connector-only config update by {} connector {}",
+                    remains ? "restarting" : "stopping", connectorName);
+            worker.stopConnector(connectorName);
+            // The update may be a deletion, so verify we actually need to restart the connector
+            if (remains)
+                startConnector(connectorName);
+        }
+    }
+
+    private void processTargetStateChanges(Set<String> connectorTargetStateChanges) {
+        if (!connectorTargetStateChanges.isEmpty()) {
+            for (String connector : connectorTargetStateChanges) {
+                if (worker.connectorNames().contains(connector)) {
+                    TargetState targetState = configState.targetState(connector);
+                    worker.setTargetState(connector, targetState);
+                    if (targetState == TargetState.STARTED)
+                        reconfigureConnectorTasksWithRetry(connector);
+                }
+            }
+        }
+    }
+
     // public for testing
     public void halt() {
         synchronized (this) {
@@ -347,7 +363,6 @@
             }
         }
 
-
         forwardRequestExecutor.shutdown();
         try {
             if (!forwardRequestExecutor.awaitTermination(10000, TimeUnit.MILLISECONDS))
@@ -367,7 +382,7 @@
                 new Callable<Void>() {
                     @Override
                     public Void call() throws Exception {
-                        if (!checkConfigSynced(callback))
+                        if (checkRebalanceNeeded(callback))
                             return null;
 
                         callback.onCompletion(null, configState.connectors());
@@ -386,10 +401,10 @@
                 new Callable<Void>() {
                     @Override
                     public Void call() throws Exception {
-                        if (!checkConfigSynced(callback))
+                        if (checkRebalanceNeeded(callback))
                             return null;
 
-                        if (!configState.connectors().contains(connName)) {
+                        if (!configState.contains(connName)) {
                             callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null);
                         } else {
                             callback.onCompletion(null, new ConnectorInfo(connName, configState.connectorConfig(connName), configState.tasks(connName)));
@@ -419,16 +434,6 @@
     @Override
     public void putConnectorConfig(final String connName, final Map<String, String> config, final boolean allowReplace,
                                    final Callback<Created<ConnectorInfo>> callback) {
-        final Map<String, String> connConfig;
-        if (config == null) {
-            connConfig = null;
-        } else if (!config.containsKey(ConnectorConfig.NAME_CONFIG)) {
-            connConfig = new HashMap<>(config);
-            connConfig.put(ConnectorConfig.NAME_CONFIG, connName);
-        } else {
-            connConfig = config;
-        }
-
         log.trace("Submitting connector config write request {}", connName);
 
         addRequest(
@@ -441,27 +446,30 @@
                             return null;
                         }
 
-                        boolean exists = configState.connectors().contains(connName);
+                        boolean exists = configState.contains(connName);
                         if (!allowReplace && exists) {
                             callback.onCompletion(new AlreadyExistsException("Connector " + connName + " already exists"), null);
                             return null;
                         }
 
-                        if (connConfig == null && !exists) {
-                            callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null);
+                        if (config == null) {
+                            if (!exists) {
+                                callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null);
+                            } else {
+                                log.trace("Removing connector config {} {} {}", connName, allowReplace, configState.connectors());
+                                configBackingStore.removeConnectorConfig(connName);
+                                callback.onCompletion(null, new Created<ConnectorInfo>(false, null));
+                            }
                             return null;
                         }
 
                         log.trace("Submitting connector config {} {} {}", connName, allowReplace, configState.connectors());
-                        configStorage.putConnectorConfig(connName, connConfig);
+                        configBackingStore.putConnectorConfig(connName, config);
 
-                        boolean created = !exists && connConfig != null;
                         // Note that we use the updated connector config despite the fact that we don't have an updated
                         // snapshot yet. The existing task info should still be accurate.
-                        ConnectorInfo info = connConfig == null ? null :
-                                new ConnectorInfo(connName, connConfig, configState.tasks(connName));
-                        callback.onCompletion(null, new Created<>(created, info));
-
+                        ConnectorInfo info = new ConnectorInfo(connName, config, configState.tasks(connName));
+                        callback.onCompletion(null, new Created<>(!exists, info));
                         return null;
                     }
                 },
@@ -501,10 +509,10 @@
                 new Callable<Void>() {
                     @Override
                     public Void call() throws Exception {
-                        if (!checkConfigSynced(callback))
+                        if (checkRebalanceNeeded(callback))
                             return null;
 
-                        if (!configState.connectors().contains(connName)) {
+                        if (!configState.contains(connName)) {
                             callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null);
                         } else {
                             List<TaskInfo> result = new ArrayList<>();
@@ -531,10 +539,10 @@
                     public Void call() throws Exception {
                         if (!isLeader())
                             callback.onCompletion(new NotLeaderException("Only the leader may write task configurations.", leaderUrl()), null);
-                        else if (!configState.connectors().contains(connName))
+                        else if (!configState.contains(connName))
                             callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null);
                         else {
-                            configStorage.putTaskConfigs(taskConfigListAsMap(connName, configs));
+                            configBackingStore.putTaskConfigs(connName, configs);
                             callback.onCompletion(null, null);
                         }
                         return null;
@@ -545,6 +553,73 @@
     }
 
     @Override
+    public synchronized void restartConnector(final String connName, final Callback<Void> callback) {
+        addRequest(new Callable<Void>() {
+            @Override
+            public Void call() throws Exception {
+                if (checkRebalanceNeeded(callback))
+                    return null;
+
+                if (!configState.connectors().contains(connName)) {
+                    callback.onCompletion(new NotFoundException("Unknown connector: " + connName), null);
+                    return null;
+                }
+
+                if (worker.ownsConnector(connName)) {
+                    try {
+                        worker.stopConnector(connName);
+                        startConnector(connName);
+                        callback.onCompletion(null, null);
+                    } catch (Throwable t) {
+                        callback.onCompletion(t, null);
+                    }
+                } else if (isLeader()) {
+                    callback.onCompletion(new NotAssignedException("Cannot restart connector since it is not assigned to this member", member.ownerUrl(connName)), null);
+                } else {
+                    callback.onCompletion(new NotLeaderException("Cannot restart connector since it is not assigned to this member", leaderUrl()), null);
+                }
+                return null;
+            }
+        }, forwardErrorCallback(callback));
+    }
+
+    @Override
+    public synchronized void restartTask(final ConnectorTaskId id, final Callback<Void> callback) {
+        addRequest(new Callable<Void>() {
+            @Override
+            public Void call() throws Exception {
+                if (checkRebalanceNeeded(callback))
+                    return null;
+
+                if (!configState.connectors().contains(id.connector())) {
+                    callback.onCompletion(new NotFoundException("Unknown connector: " + id.connector()), null);
+                    return null;
+                }
+
+                if (configState.taskConfig(id) == null) {
+                    callback.onCompletion(new NotFoundException("Unknown task: " + id), null);
+                    return null;
+                }
+
+                if (worker.ownsTask(id)) {
+                    try {
+                        worker.stopAndAwaitTask(id);
+                        startTask(id);
+                        callback.onCompletion(null, null);
+                    } catch (Throwable t) {
+                        callback.onCompletion(t, null);
+                    }
+                } else if (isLeader()) {
+                    callback.onCompletion(new NotAssignedException("Cannot restart task since it is not assigned to this member", member.ownerUrl(id)), null);
+                } else {
+                    callback.onCompletion(new NotLeaderException("Cannot restart task since it is not assigned to this member", leaderUrl()), null);
+                }
+                return null;
+            }
+        }, forwardErrorCallback(callback));
+    }
+
+    @Override
     public int generation() {
         return generation;
     }
@@ -646,8 +721,8 @@
     private boolean readConfigToEnd(long timeoutMs) {
         log.info("Current config state offset {} is behind group assignment {}, reading to end of config log", configState.offset(), assignment.offset());
         try {
-            configStorage.readToEnd().get(timeoutMs, TimeUnit.MILLISECONDS);
-            configState = configStorage.snapshot();
+            configBackingStore.refresh(timeoutMs, TimeUnit.MILLISECONDS);
+            configState = configBackingStore.snapshot();
             log.info("Finished reading to end of log and updated config snapshot, new config log offset: {}", configState.offset());
             return true;
         } catch (TimeoutException e) {
@@ -657,8 +732,6 @@
             member.maybeLeaveGroup();
             backoff(workerUnsyncBackoffMs);
             return false;
-        } catch (InterruptedException | ExecutionException e) {
-            throw new ConnectException("Error trying to catch up after assignment", e);
         }
     }
 
@@ -679,10 +752,7 @@
         }
         for (ConnectorTaskId taskId : assignment.tasks()) {
             try {
-                log.info("Starting task {}", taskId);
-                Map<String, String> configs = configState.taskConfig(taskId);
-                TaskConfig taskConfig = new TaskConfig(configs);
-                worker.startTask(taskId, taskConfig, this);
+                startTask(taskId);
             } catch (ConfigException e) {
                 log.error("Couldn't instantiate task " + taskId + " because it has an invalid task " +
                         "configuration. This task will not execute until reconfigured.", e);
@@ -691,6 +761,14 @@
         log.info("Finished starting connectors and tasks");
     }
 
+    private void startTask(ConnectorTaskId taskId) {
+        log.info("Starting task {}", taskId);
+        TargetState initialState = configState.targetState(taskId.connector());
+        Map<String, String> configs = configState.taskConfig(taskId);
+        TaskConfig taskConfig = new TaskConfig(configs);
+        worker.startTask(taskId, taskConfig, this, initialState);
+    }
+
     // Helper for starting a connector with the given name, which will extract & parse the config, generate connector
     // context and add to the worker. This needs to be called from within the main worker thread for this herder.
     private void startConnector(String connectorName) {
@@ -699,11 +777,14 @@
         ConnectorConfig connConfig = new ConnectorConfig(configs);
         String connName = connConfig.getString(ConnectorConfig.NAME_CONFIG);
         ConnectorContext ctx = new HerderConnectorContext(DistributedHerder.this, connName);
-        worker.startConnector(connConfig, ctx, this);
+        TargetState initialState = configState.targetState(connectorName);
+        worker.startConnector(connConfig, ctx, this, initialState);
+
         // Immediately request configuration since this could be a brand new connector. However, also only update those
         // task configs if they are actually different from the existing ones to avoid unnecessary updates when this is
         // just restoring an existing connector.
-        reconfigureConnectorTasksWithRetry(connName);
+        if (initialState == TargetState.STARTED)
+            reconfigureConnectorTasksWithRetry(connName);
     }
 
     private void reconfigureConnectorTasksWithRetry(final String connName) {
@@ -740,6 +821,11 @@
     // by the system, then checks whether any configs have actually changed before submitting the new configs to storage
     private void reconfigureConnector(final String connName, final Callback<Void> cb) {
         try {
+            if (!worker.isRunning(connName)) {
+                log.info("Skipping reconfiguration of connector {} since it is not running", connName);
+                return;
+            }
+
             Map<String, String> configs = configState.connectorConfig(connName);
             ConnectorConfig connConfig = new ConnectorConfig(configs);
             List<String> sinkTopics = null;
@@ -766,11 +852,11 @@
             }
             if (changed) {
                 if (isLeader()) {
-                    configStorage.putTaskConfigs(taskConfigListAsMap(connName, taskProps));
+                    configBackingStore.putTaskConfigs(connName, taskProps);
                     cb.onCompletion(null, null);
                 } else {
-                    // We cannot forward the request on the same thread because this reconfiguration can happen in as a
-                    // result of . If we blocked
+                    // We cannot forward the request on the same thread because this reconfiguration can happen as a result of connector
+                    // addition or removal. If we blocked waiting for the response from leader, we may be kicked out of the worker group.
                     forwardRequestExecutor.submit(new Runnable() {
                         @Override
                         public void run() {
@@ -791,17 +877,14 @@
         }
     }
 
-    // Common handling for requests that get config data. Checks if we are in sync with the current config, which allows
-    // us to answer requests directly. If we are not, handles invoking the callback with the appropriate error.
-    private boolean checkConfigSynced(Callback<?> callback) {
-        if (assignment == null || configState.offset() != assignment.offset()) {
-            if (!isLeader())
-                callback.onCompletion(new NotLeaderException("Cannot get config data because config is not in sync and this is not the leader", leaderUrl()), null);
-            else
-                callback.onCompletion(new ConnectException("Cannot get config data because this is the leader node, but it does not have the most up to date configs"), null);
-            return false;
+    private boolean checkRebalanceNeeded(Callback<?> callback) {
+        // Raise an error if we are expecting a rebalance to begin. This prevents us from forwarding requests
+        // based on stale leadership or assignment information
+        if (needsReconfigRebalance) {
+            callback.onCompletion(new RebalanceNeededException("Request cannot be completed because a rebalance is expected"), null);
+            return true;
         }
-        return true;
+        return false;
     }
 
     private void addRequest(Callable<Void> action, Callback<Void> callback) {
@@ -815,6 +898,60 @@
             member.wakeup();
     }
 
+    public class ConfigUpdateListener implements ConfigBackingStore.UpdateListener {
+        @Override
+        public void onConnectorConfigRemove(String connector) {
+            log.info("Connector {} config removed", connector);
+
+            synchronized (DistributedHerder.this) {
+                // rebalance after connector removal to ensure that existing tasks are balanced among workers
+                if (configState.contains(connector))
+                    needsReconfigRebalance = true;
+                connectorConfigUpdates.add(connector);
+            }
+            member.wakeup();
+        }
+
+        @Override
+        public void onConnectorConfigUpdate(String connector) {
+            log.info("Connector {} config updated", connector);
+
+            // Stage the update and wake up the work thread. Connector config *changes* only need the one connector
+            // to be bounced. However, this callback may also indicate a connector *addition*, which does require
+            // a rebalance, so we need to be careful about what operation we request.
+            synchronized (DistributedHerder.this) {
+                if (!configState.contains(connector))
+                    needsReconfigRebalance = true;
+                connectorConfigUpdates.add(connector);
+            }
+            member.wakeup();
+        }
+
+        @Override
+        public void onTaskConfigUpdate(Collection<ConnectorTaskId> tasks) {
+            log.info("Tasks {} configs updated", tasks);
+
+            // Stage the update and wake up the work thread. No need to record the set of tasks here because task reconfigs
+            // always need a rebalance to ensure offsets get committed.
+            // TODO: As an optimization, some task config updates could avoid a rebalance. In particular, single-task
+            // connectors clearly don't need any coordination.
+            synchronized (DistributedHerder.this) {
+                needsReconfigRebalance = true;
+            }
+            member.wakeup();
+        }
+
+        @Override
+        public void onConnectorTargetStateChange(String connector) {
+            log.info("Connector {} target state change", connector);
+
+            synchronized (DistributedHerder.this) {
+                connectorTargetStateChanges.add(connector);
+            }
+            member.wakeup();
+        }
+    }
+
     private class HerderRequest implements Comparable<HerderRequest> {
         private final long at;
         private final Callable<Void> action;
@@ -850,43 +987,8 @@
         };
     }
 
-
-    // Config callbacks are triggered from the KafkaConfigStorage thread
-    private Callback<String> connectorConfigCallback() {
-        return new Callback<String>() {
-            @Override
-            public void onCompletion(Throwable error, String connector) {
-                log.info("Connector {} config updated", connector);
-                // Stage the update and wake up the work thread. Connector config *changes* only need the one connector
-                // to be bounced. However, this callback may also indicate a connector *addition*, which does require
-                // a rebalance, so we need to be careful about what operation we request.
-                synchronized (DistributedHerder.this) {
-                    connectorConfigUpdates.add(connector);
-                }
-                member.wakeup();
-            }
-        };
-    }
-
-    private Callback<List<ConnectorTaskId>> taskConfigCallback() {
-        return new Callback<List<ConnectorTaskId>>() {
-            @Override
-            public void onCompletion(Throwable error, List<ConnectorTaskId> tasks) {
-                log.info("Tasks {} configs updated", tasks);
-                // Stage the update and wake up the work thread. No need to record the set of tasks here because task reconfigs
-                // always need a rebalance to ensure offsets get committed.
-                // TODO: As an optimization, some task config updates could avoid a rebalance. In particular, single-task
-                // connectors clearly don't need any coordination.
-                synchronized (DistributedHerder.this) {
-                    needsReconfigRebalance = true;
-                }
-                member.wakeup();
-            }
-        };
-    }
-
     private void updateDeletedConnectorStatus() {
-        ClusterConfigState snapshot = configStorage.snapshot();
+        ClusterConfigState snapshot = configBackingStore.snapshot();
         Set<String> connectors = snapshot.connectors();
         for (String connector : statusBackingStore.connectors()) {
             if (!connectors.contains(connector)) {
@@ -897,81 +999,68 @@
     }
 
     // Rebalances are triggered internally from the group member, so these are always executed in the work thread.
-    private WorkerRebalanceListener rebalanceListener() {
-        return new WorkerRebalanceListener() {
-            @Override
-            public void onAssigned(ConnectProtocol.Assignment assignment, int generation) {
-                // This callback just logs the info and saves it. The actual response is handled in the main loop, which
-                // ensures the group member's logic for rebalancing can complete, potentially long-running steps to
-                // catch up (or backoff if we fail) not executed in a callback, and so we'll be able to invoke other
-                // group membership actions (e.g., we may need to explicitly leave the group if we cannot handle the
-                // assigned tasks).
-                log.info("Joined group and got assignment: {}", assignment);
-                synchronized (DistributedHerder.this) {
-                    DistributedHerder.this.assignment = assignment;
-                    DistributedHerder.this.generation = generation;
-                    rebalanceResolved = false;
-                }
-
-                // Delete the statuses of all connectors removed prior to the start of this reblaance. This has to
-                // be done after the rebalance completes to avoid race conditions as the previous generation attempts
-                // to change the state to UNASSIGNED after tasks have been stopped.
-                if (isLeader())
-                    updateDeletedConnectorStatus();
-
-                // We *must* interrupt any poll() call since this could occur when the poll starts, and we might then
-                // sleep in the poll() for a long time. Forcing a wakeup ensures we'll get to process this event in the
-                // main thread.
-                member.wakeup();
+    public class RebalanceListener implements WorkerRebalanceListener {
+        @Override
+        public void onAssigned(ConnectProtocol.Assignment assignment, int generation) {
+            // This callback just logs the info and saves it. The actual response is handled in the main loop, which
+            // ensures the group member's logic for rebalancing can complete, potentially long-running steps to
+            // catch up (or backoff if we fail) not executed in a callback, and so we'll be able to invoke other
+            // group membership actions (e.g., we may need to explicitly leave the group if we cannot handle the
+            // assigned tasks).
+            log.info("Joined group and got assignment: {}", assignment);
+            synchronized (DistributedHerder.this) {
+                DistributedHerder.this.assignment = assignment;
+                DistributedHerder.this.generation = generation;
+                rebalanceResolved = false;
             }
 
-            @Override
-            public void onRevoked(String leader, Collection<String> connectors, Collection<ConnectorTaskId> tasks) {
-                log.info("Rebalance started");
+            // Delete the statuses of all connectors removed prior to the start of this reblaance. This has to
+            // be done after the rebalance completes to avoid race conditions as the previous generation attempts
+            // to change the state to UNASSIGNED after tasks have been stopped.
+            if (isLeader())
+                updateDeletedConnectorStatus();
 
-                // Note that since we don't reset the assignment, we we don't revoke leadership here. During a rebalance,
-                // it is still important to have a leader that can write configs, offsets, etc.
-
-                if (rebalanceResolved) {
-                    // TODO: Parallelize this. We should be able to request all connectors and tasks to stop, then wait on all of
-                    // them to finish
-                    // TODO: Technically we don't have to stop connectors at all until we know they've really been removed from
-                    // this worker. Instead, we can let them continue to run but buffer any update requests (which should be
-                    // rare anyway). This would avoid a steady stream of start/stop, which probably also includes lots of
-                    // unnecessary repeated connections to the source/sink system.
-                    for (String connectorName : connectors)
-                        worker.stopConnector(connectorName);
-
-                    // TODO: We need to at least commit task offsets, but if we could commit offsets & pause them instead of
-                    // stopping them then state could continue to be reused when the task remains on this worker. For example,
-                    // this would avoid having to close a connection and then reopen it when the task is assigned back to this
-                    // worker again.
-                    if (!tasks.isEmpty()) {
-                        worker.stopTasks(tasks); // trigger stop() for all tasks
-                        worker.awaitStopTasks(tasks); // await stopping tasks
-                    }
-
-                    // Ensure that all status updates have been pushed to the storage system before rebalancing.
-                    // Otherwise, we may inadvertently overwrite the state with a stale value after the rebalance
-                    // completes.
-                    statusBackingStore.flush();
-                    log.info("Finished stopping tasks in preparation for rebalance");
-                } else {
-                    log.info("Wasn't unable to resume work after last rebalance, can skip stopping connectors and tasks");
-                }
-            }
-        };
-    }
-
-
-    private static Map<ConnectorTaskId, Map<String, String>> taskConfigListAsMap(String connName, List<Map<String, String>> configs) {
-        int index = 0;
-        Map<ConnectorTaskId, Map<String, String>> result = new HashMap<>();
-        for (Map<String, String> taskConfigMap : configs) {
-            ConnectorTaskId taskId = new ConnectorTaskId(connName, index);
-            result.put(taskId, taskConfigMap);
-            index++;
+            // We *must* interrupt any poll() call since this could occur when the poll starts, and we might then
+            // sleep in the poll() for a long time. Forcing a wakeup ensures we'll get to process this event in the
+            // main thread.
+            member.wakeup();
         }
-        return result;
+
+        @Override
+        public void onRevoked(String leader, Collection<String> connectors, Collection<ConnectorTaskId> tasks) {
+            log.info("Rebalance started");
+
+            // Note that since we don't reset the assignment, we we don't revoke leadership here. During a rebalance,
+            // it is still important to have a leader that can write configs, offsets, etc.
+
+            if (rebalanceResolved) {
+                // TODO: Parallelize this. We should be able to request all connectors and tasks to stop, then wait on all of
+                // them to finish
+                // TODO: Technically we don't have to stop connectors at all until we know they've really been removed from
+                // this worker. Instead, we can let them continue to run but buffer any update requests (which should be
+                // rare anyway). This would avoid a steady stream of start/stop, which probably also includes lots of
+                // unnecessary repeated connections to the source/sink system.
+                for (String connectorName : connectors)
+                    worker.stopConnector(connectorName);
+
+                // TODO: We need to at least commit task offsets, but if we could commit offsets & pause them instead of
+                // stopping them then state could continue to be reused when the task remains on this worker. For example,
+                // this would avoid having to close a connection and then reopen it when the task is assigned back to this
+                // worker again.
+                if (!tasks.isEmpty()) {
+                    worker.stopTasks(tasks); // trigger stop() for all tasks
+                    worker.awaitStopTasks(tasks); // await stopping tasks
+                }
+
+                // Ensure that all status updates have been pushed to the storage system before rebalancing.
+                // Otherwise, we may inadvertently overwrite the state with a stale value after the rebalance
+                // completes.
+                statusBackingStore.flush();
+                log.info("Finished stopping tasks in preparation for rebalance");
+            } else {
+                log.info("Wasn't unable to resume work after last rebalance, can skip stopping connectors and tasks");
+            }
+        }
     }
+
 }
diff --git a/streams/src/test/java/org/apache/kafka/test/NoOpKeyValueMapper.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotAssignedException.java
similarity index 62%
copy from streams/src/test/java/org/apache/kafka/test/NoOpKeyValueMapper.java
copy to connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotAssignedException.java
index 828b5ae..a4211cc 100644
--- a/streams/src/test/java/org/apache/kafka/test/NoOpKeyValueMapper.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotAssignedException.java
@@ -5,25 +5,25 @@
  * The ASF licenses this file to You under the Apache License, Version 2.0
  * (the "License"); you may not use this file except in compliance with
  * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  * See the License for the specific language governing permissions and
  * limitations under the License.
+ **/
+package org.apache.kafka.connect.runtime.distributed;
+
+/**
+ * Thrown when a request intended for the owner of a task or connector is received by a worker which doesn't
+ * own it (typically the leader).
  */
+public class NotAssignedException extends RequestTargetException {
 
-package org.apache.kafka.test;
-
-import org.apache.kafka.streams.KeyValue;
-import org.apache.kafka.streams.kstream.KeyValueMapper;
-
-public class NoOpKeyValueMapper<K, V> implements KeyValueMapper<K, V, KeyValue<K, V>> {
-
-    @Override
-    public KeyValue<K, V> apply(K key, V value) {
-        return new KeyValue<>(key, value);
+    public NotAssignedException(String message, String ownerUrl) {
+        super(message, ownerUrl);
     }
+
 }
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotLeaderException.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotLeaderException.java
index 5f94b53..9340eda 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotLeaderException.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotLeaderException.java
@@ -17,31 +17,14 @@
 
 package org.apache.kafka.connect.runtime.distributed;
 
-import org.apache.kafka.connect.errors.ConnectException;
-
 /**
  * Indicates an operation was not permitted because it can only be performed on the leader and this worker is not currently
  * the leader.
  */
-public class NotLeaderException extends ConnectException {
-    private final String leaderUrl;
+public class NotLeaderException extends RequestTargetException {
 
     public NotLeaderException(String msg, String leaderUrl) {
-        super(msg);
-        this.leaderUrl = leaderUrl;
+        super(msg, leaderUrl);
     }
 
-    public NotLeaderException(String msg, String leaderUrl, Throwable throwable) {
-        super(msg, throwable);
-        this.leaderUrl = leaderUrl;
-    }
-
-    public NotLeaderException(String leaderUrl, Throwable throwable) {
-        super(throwable);
-        this.leaderUrl = leaderUrl;
-    }
-
-    public String leaderUrl() {
-        return leaderUrl;
-    }
 }
diff --git a/streams/src/test/java/org/apache/kafka/test/NoOpKeyValueMapper.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/RebalanceNeededException.java
similarity index 65%
copy from streams/src/test/java/org/apache/kafka/test/NoOpKeyValueMapper.java
copy to connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/RebalanceNeededException.java
index 828b5ae..80c08a3 100644
--- a/streams/src/test/java/org/apache/kafka/test/NoOpKeyValueMapper.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/RebalanceNeededException.java
@@ -5,25 +5,23 @@
  * The ASF licenses this file to You under the Apache License, Version 2.0
  * (the "License"); you may not use this file except in compliance with
  * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  * See the License for the specific language governing permissions and
  * limitations under the License.
- */
+ **/
+package org.apache.kafka.connect.runtime.distributed;
 
-package org.apache.kafka.test;
+import org.apache.kafka.connect.errors.ConnectException;
 
-import org.apache.kafka.streams.KeyValue;
-import org.apache.kafka.streams.kstream.KeyValueMapper;
+public class RebalanceNeededException extends ConnectException {
 
-public class NoOpKeyValueMapper<K, V> implements KeyValueMapper<K, V, KeyValue<K, V>> {
-
-    @Override
-    public KeyValue<K, V> apply(K key, V value) {
-        return new KeyValue<>(key, value);
+    public RebalanceNeededException(String s) {
+        super(s);
     }
+
 }
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/RequestTargetException.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/RequestTargetException.java
new file mode 100644
index 0000000..42a5f5d
--- /dev/null
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/RequestTargetException.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ **/
+package org.apache.kafka.connect.runtime.distributed;
+
+import org.apache.kafka.connect.errors.ConnectException;
+
+/**
+ * Raised when a request has been received by a worker which cannot handle it,
+ * but can forward it to the right target
+ */
+public class RequestTargetException extends ConnectException {
+    private final String forwardUrl;
+
+    public RequestTargetException(String s, String forwardUrl) {
+        super(s);
+        this.forwardUrl = forwardUrl;
+    }
+
+    public RequestTargetException(String s, Throwable throwable, String forwardUrl) {
+        super(s, throwable);
+        this.forwardUrl = forwardUrl;
+    }
+
+    public RequestTargetException(Throwable throwable, String forwardUrl) {
+        super(throwable);
+        this.forwardUrl = forwardUrl;
+    }
+
+    public String forwardUrl() {
+        return forwardUrl;
+    }
+
+}
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java
index fa50fbf..9c74960 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java
@@ -24,7 +24,7 @@
 import org.apache.kafka.common.requests.JoinGroupRequest.ProtocolMetadata;
 import org.apache.kafka.common.utils.CircularIterator;
 import org.apache.kafka.common.utils.Time;
-import org.apache.kafka.connect.storage.KafkaConfigStorage;
+import org.apache.kafka.connect.storage.ConfigBackingStore;
 import org.apache.kafka.connect.util.ConnectorTaskId;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -49,11 +49,12 @@
     public static final String DEFAULT_SUBPROTOCOL = "default";
 
     private final String restUrl;
-    private final KafkaConfigStorage configStorage;
+    private final ConfigBackingStore configStorage;
     private ConnectProtocol.Assignment assignmentSnapshot;
     private final WorkerCoordinatorMetrics sensors;
     private ClusterConfigState configSnapshot;
     private final WorkerRebalanceListener listener;
+    private LeaderState leaderState;
 
     private boolean rejoinRequested;
 
@@ -69,7 +70,7 @@
                              Time time,
                              long retryBackoffMs,
                              String restUrl,
-                             KafkaConfigStorage configStorage,
+                             ConfigBackingStore configStorage,
                              WorkerRebalanceListener listener) {
         super(client,
                 groupId,
@@ -198,6 +199,8 @@
             }
         }
 
+        this.leaderState = new LeaderState(allConfigs, connectorAssignments, taskAssignments);
+
         return fillAssignmentsAndSerialize(allConfigs.keySet(), ConnectProtocol.Assignment.NO_ERROR,
                 leaderId, allConfigs.get(leaderId).url(), maxOffset, connectorAssignments, taskAssignments);
     }
@@ -228,6 +231,7 @@
 
     @Override
     protected void onJoinPrepare(int generation, String memberId) {
+        this.leaderState = null;
         log.debug("Revoking previous assignment {}", assignmentSnapshot);
         if (assignmentSnapshot != null && !assignmentSnapshot.failed())
             listener.onRevoked(assignmentSnapshot.leader(), assignmentSnapshot.connectors(), assignmentSnapshot.tasks());
@@ -247,6 +251,22 @@
         super.close();
     }
 
+    private boolean isLeader() {
+        return assignmentSnapshot != null && memberId.equals(assignmentSnapshot.leader());
+    }
+
+    public String ownerUrl(String connector) {
+        if (needRejoin() || !isLeader())
+            return null;
+        return leaderState.ownerUrl(connector);
+    }
+
+    public String ownerUrl(ConnectorTaskId task) {
+        if (needRejoin() || !isLeader())
+            return null;
+        return leaderState.ownerUrl(task);
+    }
+
     private class WorkerCoordinatorMetrics {
         public final Metrics metrics;
         public final String metricGrpName;
@@ -282,4 +302,43 @@
         return res;
     }
 
+    private static <K, V> Map<V, K> invertAssignment(Map<K, List<V>> assignment) {
+        Map<V, K> inverted = new HashMap<>();
+        for (Map.Entry<K, List<V>> assignmentEntry : assignment.entrySet()) {
+            K key = assignmentEntry.getKey();
+            for (V value : assignmentEntry.getValue())
+                inverted.put(value, key);
+        }
+        return inverted;
+    }
+
+    private static class LeaderState {
+        private final Map<String, ConnectProtocol.WorkerState> allMembers;
+        private final Map<String, String> connectorOwners;
+        private final Map<ConnectorTaskId, String> taskOwners;
+
+        public LeaderState(Map<String, ConnectProtocol.WorkerState> allMembers,
+                           Map<String, List<String>> connectorAssignment,
+                           Map<String, List<ConnectorTaskId>> taskAssignment) {
+            this.allMembers = allMembers;
+            this.connectorOwners = invertAssignment(connectorAssignment);
+            this.taskOwners = invertAssignment(taskAssignment);
+        }
+
+        private String ownerUrl(ConnectorTaskId id) {
+            String ownerId = taskOwners.get(id);
+            if (ownerId == null)
+                return null;
+            return allMembers.get(ownerId).url();
+        }
+
+        private String ownerUrl(String connector) {
+            String ownerId = connectorOwners.get(connector);
+            if (ownerId == null)
+                return null;
+            return allMembers.get(ownerId).url();
+        }
+
+    }
+
 }
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java
index 57028ef..85af549 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java
@@ -32,7 +32,8 @@
 import org.apache.kafka.common.network.Selector;
 import org.apache.kafka.common.utils.AppInfoParser;
 import org.apache.kafka.common.utils.Time;
-import org.apache.kafka.connect.storage.KafkaConfigStorage;
+import org.apache.kafka.connect.util.ConnectorTaskId;
+import org.apache.kafka.connect.storage.ConfigBackingStore;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -68,7 +69,7 @@
 
     public WorkerGroupMember(DistributedConfig config,
                              String restUrl,
-                             KafkaConfigStorage configStorage,
+                             ConfigBackingStore configStorage,
                              WorkerRebalanceListener listener,
                              Time time) {
         try {
@@ -175,6 +176,14 @@
         coordinator.maybeLeaveGroup();
     }
 
+    public String ownerUrl(String connector) {
+        return coordinator.ownerUrl(connector);
+    }
+
+    public String ownerUrl(ConnectorTaskId task) {
+        return coordinator.ownerUrl(task);
+    }
+
     private void stop(boolean swallowException) {
         log.trace("Stopping the Connect group member.");
         AtomicReference<Throwable> firstException = new AtomicReference<Throwable>();
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
index 1505a01..a878fb0 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
@@ -117,7 +117,11 @@
         if (allowedOrigins != null && !allowedOrigins.trim().isEmpty()) {
             FilterHolder filterHolder = new FilterHolder(new CrossOriginFilter());
             filterHolder.setName("cross-origin");
-            filterHolder.setInitParameter("allowedOrigins", allowedOrigins);
+            filterHolder.setInitParameter(CrossOriginFilter.ALLOWED_ORIGINS_PARAM, allowedOrigins);
+            String allowedMethods = config.getString(WorkerConfig.ACCESS_CONTROL_ALLOW_METHODS_CONFIG);
+            if (allowedMethods != null && !allowedOrigins.trim().isEmpty()) {
+                filterHolder.setInitParameter(CrossOriginFilter.ALLOWED_METHODS_PARAM, allowedMethods);
+            }
             context.addFilter(filterHolder, "/*", EnumSet.of(DispatcherType.REQUEST));
         }
 
@@ -272,4 +276,5 @@
         else
             return base + path;
     }
+
 }
\ No newline at end of file
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigKeyInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigKeyInfo.java
index f813709..ead24c5 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigKeyInfo.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigKeyInfo.java
@@ -28,7 +28,7 @@
     private final String name;
     private final String type;
     private final boolean required;
-    private final Object defaultValue;
+    private final String defaultValue;
     private final String importance;
     private final String documentation;
     private final String group;
@@ -41,7 +41,7 @@
     public ConfigKeyInfo(@JsonProperty("name") String name,
                          @JsonProperty("type") String type,
                          @JsonProperty("required") boolean required,
-                         @JsonProperty("default_value") Object defaultValue,
+                         @JsonProperty("default_value") String defaultValue,
                          @JsonProperty("importance") String importance,
                          @JsonProperty("documentation") String documentation,
                          @JsonProperty("group") String group,
@@ -78,7 +78,7 @@
     }
 
     @JsonProperty("default_value")
-    public Object defaultValue() {
+    public String defaultValue() {
         return defaultValue;
     }
 
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigValueInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigValueInfo.java
index 51e7ee5..a6ae006 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigValueInfo.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigValueInfo.java
@@ -25,16 +25,16 @@
 
 public class ConfigValueInfo {
     private String name;
-    private Object value;
-    private List<Object> recommendedValues;
+    private String value;
+    private List<String> recommendedValues;
     private List<String> errors;
     private boolean visible;
 
     @JsonCreator
     public ConfigValueInfo(
         @JsonProperty("name") String name,
-        @JsonProperty("value") Object value,
-        @JsonProperty("recommended_values") List<Object> recommendedValues,
+        @JsonProperty("value") String value,
+        @JsonProperty("recommended_values") List<String> recommendedValues,
         @JsonProperty("errors") List<String> errors,
         @JsonProperty("visible") boolean visible) {
         this.name = name;
@@ -50,12 +50,12 @@
     }
 
     @JsonProperty
-    public Object value() {
+    public String value() {
         return value;
     }
 
     @JsonProperty("recommended_values")
-    public List<Object> recommendedValues() {
+    public List<String> recommendedValues() {
         return recommendedValues;
     }
 
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java
index b6e9f61..2ec35f4 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java
@@ -18,10 +18,10 @@
 package org.apache.kafka.connect.runtime.rest.resources;
 
 import com.fasterxml.jackson.core.type.TypeReference;
-
 import org.apache.kafka.connect.runtime.ConnectorConfig;
 import org.apache.kafka.connect.runtime.Herder;
-import org.apache.kafka.connect.runtime.distributed.NotLeaderException;
+import org.apache.kafka.connect.runtime.distributed.RebalanceNeededException;
+import org.apache.kafka.connect.runtime.distributed.RequestTargetException;
 import org.apache.kafka.connect.runtime.rest.RestServer;
 import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo;
 import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo;
@@ -33,15 +33,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.net.URI;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
 import javax.servlet.ServletContext;
+import javax.ws.rs.BadRequestException;
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DELETE;
 import javax.ws.rs.GET;
@@ -50,8 +43,17 @@
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
+import javax.ws.rs.core.UriBuilder;
+import java.net.URI;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
 @Path("/connectors")
 @Produces(MediaType.APPLICATION_JSON)
@@ -75,16 +77,17 @@
 
     @GET
     @Path("/")
-    public Collection<String> listConnectors() throws Throwable {
+    public Collection<String> listConnectors(final @QueryParam("forward") Boolean forward) throws Throwable {
         FutureCallback<Collection<String>> cb = new FutureCallback<>();
         herder.connectors(cb);
         return completeOrForwardRequest(cb, "/connectors", "GET", null, new TypeReference<Collection<String>>() {
-        });
+        }, forward);
     }
 
     @POST
     @Path("/")
-    public Response createConnector(final CreateConnectorRequest createRequest) throws Throwable {
+    public Response createConnector(final @QueryParam("forward") Boolean forward,
+                                    final CreateConnectorRequest createRequest) throws Throwable {
         String name = createRequest.name();
         Map<String, String> configs = createRequest.config();
         if (!configs.containsKey(ConnectorConfig.NAME_CONFIG))
@@ -93,24 +96,26 @@
         FutureCallback<Herder.Created<ConnectorInfo>> cb = new FutureCallback<>();
         herder.putConnectorConfig(name, configs, false, cb);
         Herder.Created<ConnectorInfo> info = completeOrForwardRequest(cb, "/connectors", "POST", createRequest,
-                new TypeReference<ConnectorInfo>() { }, new CreatedConnectorInfoTranslator());
+                new TypeReference<ConnectorInfo>() { }, new CreatedConnectorInfoTranslator(), forward);
         return Response.created(URI.create("/connectors/" + name)).entity(info.result()).build();
     }
 
     @GET
     @Path("/{connector}")
-    public ConnectorInfo getConnector(final @PathParam("connector") String connector) throws Throwable {
+    public ConnectorInfo getConnector(final @PathParam("connector") String connector,
+                                      final @QueryParam("forward") Boolean forward) throws Throwable {
         FutureCallback<ConnectorInfo> cb = new FutureCallback<>();
         herder.connectorInfo(connector, cb);
-        return completeOrForwardRequest(cb, "/connectors/" + connector, "GET", null);
+        return completeOrForwardRequest(cb, "/connectors/" + connector, "GET", null, forward);
     }
 
     @GET
     @Path("/{connector}/config")
-    public Map<String, String> getConnectorConfig(final @PathParam("connector") String connector) throws Throwable {
+    public Map<String, String> getConnectorConfig(final @PathParam("connector") String connector,
+                                                  final @QueryParam("forward") Boolean forward) throws Throwable {
         FutureCallback<Map<String, String>> cb = new FutureCallback<>();
         herder.connectorConfig(connector, cb);
-        return completeOrForwardRequest(cb, "/connectors/" + connector + "/config", "GET", null);
+        return completeOrForwardRequest(cb, "/connectors/" + connector + "/config", "GET", null, forward);
     }
 
     @GET
@@ -122,11 +127,20 @@
     @PUT
     @Path("/{connector}/config")
     public Response putConnectorConfig(final @PathParam("connector") String connector,
-                                   final Map<String, String> connectorConfig) throws Throwable {
+                                       final @QueryParam("forward") Boolean forward,
+                                       final Map<String, String> connectorConfig) throws Throwable {
         FutureCallback<Herder.Created<ConnectorInfo>> cb = new FutureCallback<>();
+        String includedName = connectorConfig.get(ConnectorConfig.NAME_CONFIG);
+        if (includedName != null) {
+            if (!includedName.equals(connector))
+                throw new BadRequestException("Connector name configuration (" + includedName + ") doesn't match connector name in the URL (" + connector + ")");
+        } else {
+            connectorConfig.put(ConnectorConfig.NAME_CONFIG, connector);
+        }
+
         herder.putConnectorConfig(connector, connectorConfig, true, cb);
         Herder.Created<ConnectorInfo> createdInfo = completeOrForwardRequest(cb, "/connectors/" + connector + "/config",
-                "PUT", connectorConfig, new TypeReference<ConnectorInfo>() { }, new CreatedConnectorInfoTranslator());
+                "PUT", connectorConfig, new TypeReference<ConnectorInfo>() { }, new CreatedConnectorInfoTranslator(), forward);
         Response.ResponseBuilder response;
         if (createdInfo.created())
             response = Response.created(URI.create("/connectors/" + connector));
@@ -135,55 +149,116 @@
         return response.entity(createdInfo.result()).build();
     }
 
+    @POST
+    @Path("/{connector}/restart")
+    public void restartConnector(final @PathParam("connector") String connector,
+                                 final @QueryParam("forward") Boolean forward) throws Throwable {
+        FutureCallback<Void> cb = new FutureCallback<>();
+        herder.restartConnector(connector, cb);
+        completeOrForwardRequest(cb, "/connectors/" + connector + "/restart", "POST", null, forward);
+    }
+
+    @PUT
+    @Path("/{connector}/pause")
+    public Response pauseConnector(@PathParam("connector") String connector) {
+        herder.pauseConnector(connector);
+        return Response.accepted().build();
+    }
+
+    @PUT
+    @Path("/{connector}/resume")
+    public Response resumeConnector(@PathParam("connector") String connector) {
+        herder.resumeConnector(connector);
+        return Response.accepted().build();
+    }
+
     @GET
     @Path("/{connector}/tasks")
-    public List<TaskInfo> getTaskConfigs(final @PathParam("connector") String connector) throws Throwable {
+    public List<TaskInfo> getTaskConfigs(final @PathParam("connector") String connector,
+                                         final @QueryParam("forward") Boolean forward) throws Throwable {
         FutureCallback<List<TaskInfo>> cb = new FutureCallback<>();
         herder.taskConfigs(connector, cb);
         return completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks", "GET", null, new TypeReference<List<TaskInfo>>() {
-        });
+        }, forward);
     }
 
     @POST
     @Path("/{connector}/tasks")
     public void putTaskConfigs(final @PathParam("connector") String connector,
+                               final @QueryParam("forward") Boolean forward,
                                final List<Map<String, String>> taskConfigs) throws Throwable {
         FutureCallback<Void> cb = new FutureCallback<>();
         herder.putTaskConfigs(connector, taskConfigs, cb);
-        completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks", "POST", taskConfigs);
+        completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks", "POST", taskConfigs, forward);
     }
 
     @GET
     @Path("/{connector}/tasks/{task}/status")
-    public ConnectorStateInfo.TaskState getTaskStatus(@PathParam("connector") String connector,
-                                                      @PathParam("task") Integer task) throws Throwable {
+    public ConnectorStateInfo.TaskState getTaskStatus(final @PathParam("connector") String connector,
+                                                      final @PathParam("task") Integer task) throws Throwable {
         return herder.taskStatus(new ConnectorTaskId(connector, task));
     }
 
+    @POST
+    @Path("/{connector}/tasks/{task}/restart")
+    public void restartTask(final @PathParam("connector") String connector,
+                            final @PathParam("task") Integer task,
+                            final @QueryParam("forward") Boolean forward) throws Throwable {
+        FutureCallback<Void> cb = new FutureCallback<>();
+        ConnectorTaskId taskId = new ConnectorTaskId(connector, task);
+        herder.restartTask(taskId, cb);
+        completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks/" + task + "/restart", "POST", null, forward);
+    }
+
     @DELETE
     @Path("/{connector}")
-    public void destroyConnector(final @PathParam("connector") String connector) throws Throwable {
+    public void destroyConnector(final @PathParam("connector") String connector,
+                                 final @QueryParam("forward") Boolean forward) throws Throwable {
         FutureCallback<Herder.Created<ConnectorInfo>> cb = new FutureCallback<>();
         herder.putConnectorConfig(connector, null, true, cb);
-        completeOrForwardRequest(cb, "/connectors/" + connector, "DELETE", null);
+        completeOrForwardRequest(cb, "/connectors/" + connector, "DELETE", null, forward);
     }
 
     // Wait for a FutureCallback to complete. If it succeeds, return the parsed response. If it fails, try to forward the
     // request to the leader.
-    private <T, U> T completeOrForwardRequest(
-            FutureCallback<T> cb, String path, String method, Object body, TypeReference<U> resultType,
-            Translator<T, U> translator) throws Throwable {
+    private <T, U> T completeOrForwardRequest(FutureCallback<T> cb,
+                                              String path,
+                                              String method,
+                                              Object body,
+                                              TypeReference<U> resultType,
+                                              Translator<T, U> translator,
+                                              Boolean forward) throws Throwable {
         try {
             return cb.get(REQUEST_TIMEOUT_MS, TimeUnit.MILLISECONDS);
         } catch (ExecutionException e) {
-            if (e.getCause() instanceof NotLeaderException) {
-                NotLeaderException notLeaderError = (NotLeaderException) e.getCause();
-                String forwardUrl = RestServer.urlJoin(notLeaderError.leaderUrl(), path);
-                log.debug("Forwarding request to leader: {} {} {}", forwardUrl, method, body);
-                return translator.translate(RestServer.httpRequest(forwardUrl, method, body, resultType));
+            Throwable cause = e.getCause();
+
+            if (cause instanceof RequestTargetException) {
+                if (forward == null || forward) {
+                    // the only time we allow recursive forwarding is when no forward flag has
+                    // been set, which should only be seen by the first worker to handle a user request.
+                    // this gives two total hops to resolve the request before giving up.
+                    boolean recursiveForward = forward == null;
+                    RequestTargetException targetException = (RequestTargetException) cause;
+                    String forwardUrl = UriBuilder.fromUri(targetException.forwardUrl())
+                            .path(path)
+                            .queryParam("forward", recursiveForward)
+                            .build()
+                            .toString();
+                    log.debug("Forwarding request {} {} {}", forwardUrl, method, body);
+                    return translator.translate(RestServer.httpRequest(forwardUrl, method, body, resultType));
+                } else {
+                    // we should find the right target for the query within two hops, so if
+                    // we don't, it probably means that a rebalance has taken place.
+                    throw new ConnectRestException(Response.Status.CONFLICT.getStatusCode(),
+                            "Cannot complete request because of a conflicting operation (e.g. worker rebalance)");
+                }
+            } else if (cause instanceof RebalanceNeededException) {
+                throw new ConnectRestException(Response.Status.CONFLICT.getStatusCode(),
+                        "Cannot complete request momentarily due to stale configuration (typically caused by a concurrent config change)");
             }
 
-            throw e.getCause();
+            throw cause;
         } catch (TimeoutException e) {
             // This timeout is for the operation itself. None of the timeout error codes are relevant, so internal server
             // error is the best option
@@ -193,12 +268,14 @@
         }
     }
 
-    private <T> T completeOrForwardRequest(FutureCallback<T> cb, String path, String method, Object body, TypeReference<T> resultType) throws Throwable {
-        return completeOrForwardRequest(cb, path, method, body, resultType, new IdentityTranslator<T>());
+    private <T> T completeOrForwardRequest(FutureCallback<T> cb, String path, String method, Object body,
+                                           TypeReference<T> resultType, Boolean forward) throws Throwable {
+        return completeOrForwardRequest(cb, path, method, body, resultType, new IdentityTranslator<T>(), forward);
     }
 
-    private <T> T completeOrForwardRequest(FutureCallback<T> cb, String path, String method, Object body) throws Throwable {
-        return completeOrForwardRequest(cb, path, method, body, null, new IdentityTranslator<T>());
+    private <T> T completeOrForwardRequest(FutureCallback<T> cb, String path, String method,
+                                           Object body, Boolean forward) throws Throwable {
+        return completeOrForwardRequest(cb, path, method, body, null, new IdentityTranslator<T>(), forward);
     }
 
     private interface Translator<T, U> {
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java
index 9c48ed7..2316bae 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java
@@ -23,10 +23,14 @@
 import org.apache.kafka.connect.runtime.AbstractHerder;
 import org.apache.kafka.connect.runtime.ConnectorConfig;
 import org.apache.kafka.connect.runtime.HerderConnectorContext;
+import org.apache.kafka.connect.runtime.TargetState;
 import org.apache.kafka.connect.runtime.TaskConfig;
 import org.apache.kafka.connect.runtime.Worker;
+import org.apache.kafka.connect.runtime.distributed.ClusterConfigState;
 import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo;
 import org.apache.kafka.connect.runtime.rest.entities.TaskInfo;
+import org.apache.kafka.connect.storage.ConfigBackingStore;
+import org.apache.kafka.connect.storage.MemoryConfigBackingStore;
 import org.apache.kafka.connect.storage.MemoryStatusBackingStore;
 import org.apache.kafka.connect.storage.StatusBackingStore;
 import org.apache.kafka.connect.util.Callback;
@@ -36,12 +40,8 @@
 
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
-
 
 /**
  * Single process, in-memory "herder". Useful for a standalone Kafka Connect process.
@@ -49,17 +49,20 @@
 public class StandaloneHerder extends AbstractHerder {
     private static final Logger log = LoggerFactory.getLogger(StandaloneHerder.class);
 
-    private HashMap<String, ConnectorState> connectors = new HashMap<>();
+    private ClusterConfigState configState;
 
     public StandaloneHerder(Worker worker) {
-        this(worker.workerId(), worker, new MemoryStatusBackingStore());
+        this(worker, worker.workerId(), new MemoryStatusBackingStore(), new MemoryConfigBackingStore());
     }
 
     // visible for testing
-    StandaloneHerder(String workerId,
-                     Worker worker,
-                     StatusBackingStore statusBackingStore) {
-        super(worker, statusBackingStore, workerId);
+    StandaloneHerder(Worker worker,
+                     String workerId,
+                     StatusBackingStore statusBackingStore,
+                     MemoryConfigBackingStore configBackingStore) {
+        super(worker, workerId, statusBackingStore, configBackingStore);
+        this.configState = ClusterConfigState.EMPTY;
+        configBackingStore.setUpdateListener(new ConfigUpdateListener());
     }
 
     public synchronized void start() {
@@ -74,7 +77,7 @@
         // There's no coordination/hand-off to do here since this is all standalone. Instead, we
         // should just clean up the stuff we normally would, i.e. cleanly checkpoint and shutdown all
         // the tasks.
-        for (String connName : new HashSet<>(connectors.keySet())) {
+        for (String connName : configState.connectors()) {
             removeConnectorTasks(connName);
             try {
                 worker.stopConnector(connName);
@@ -82,8 +85,7 @@
                 log.error("Error shutting down connector {}: ", connName, e);
             }
         }
-        connectors.clear();
-
+        stopServices();
         log.info("Herder stopped");
     }
 
@@ -94,27 +96,24 @@
 
     @Override
     public synchronized void connectors(Callback<Collection<String>> callback) {
-        callback.onCompletion(null, new ArrayList<>(connectors.keySet()));
+        callback.onCompletion(null, configState.connectors());
     }
 
     @Override
     public synchronized void connectorInfo(String connName, Callback<ConnectorInfo> callback) {
-        ConnectorState state = connectors.get(connName);
-        if (state == null) {
+        ConnectorInfo connectorInfo = createConnectorInfo(connName);
+        if (connectorInfo == null) {
             callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null);
             return;
         }
-        callback.onCompletion(null, createConnectorInfo(state));
+        callback.onCompletion(null, connectorInfo);
     }
 
-    private ConnectorInfo createConnectorInfo(ConnectorState state) {
-        if (state == null)
+    private ConnectorInfo createConnectorInfo(String connector) {
+        if (!configState.contains(connector))
             return null;
-
-        List<ConnectorTaskId> taskIds = new ArrayList<>();
-        for (int i = 0; i < state.taskConfigs.size(); i++)
-            taskIds.add(new ConnectorTaskId(state.name, i));
-        return new ConnectorInfo(state.name, state.configOriginals, taskIds);
+        Map<String, String> config = configState.connectorConfig(connector);
+        return new ConnectorInfo(connector, config, configState.tasks(connector));
     }
 
     @Override
@@ -133,12 +132,13 @@
     }
 
     @Override
-    public synchronized void putConnectorConfig(String connName, final Map<String, String> config,
+    public synchronized void putConnectorConfig(String connName,
+                                                final Map<String, String> config,
                                                 boolean allowReplace,
                                                 final Callback<Created<ConnectorInfo>> callback) {
         try {
             boolean created = false;
-            if (connectors.containsKey(connName)) {
+            if (configState.contains(connName)) {
                 if (!allowReplace) {
                     callback.onCompletion(new AlreadyExistsException("Connector " + connName + " already exists"), null);
                     return;
@@ -147,7 +147,7 @@
                     removeConnectorTasks(connName);
                 worker.stopConnector(connName);
                 if (config == null) {
-                    connectors.remove(connName);
+                    configBackingStore.removeConnectorConfig(connName);
                     onDeletion(connName);
                 }
             } else {
@@ -163,7 +163,7 @@
                 updateConnectorTasks(connName);
             }
             if (config != null)
-                callback.onCompletion(null, new Created<>(created, createConnectorInfo(connectors.get(connName))));
+                callback.onCompletion(null, new Created<>(created, createConnectorInfo(connName)));
             else
                 callback.onCompletion(null, new Created<ConnectorInfo>(false, null));
         } catch (ConnectException e) {
@@ -183,17 +183,14 @@
 
     @Override
     public synchronized void taskConfigs(String connName, Callback<List<TaskInfo>> callback) {
-        ConnectorState state = connectors.get(connName);
-        if (state == null) {
+        if (!configState.contains(connName)) {
             callback.onCompletion(new NotFoundException("Connector " + connName + " not found", null), null);
             return;
         }
 
         List<TaskInfo> result = new ArrayList<>();
-        for (int i = 0; i < state.taskConfigs.size(); i++) {
-            TaskInfo info = new TaskInfo(new ConnectorTaskId(connName, i), state.taskConfigs.get(i));
-            result.add(info);
-        }
+        for (ConnectorTaskId taskId : configState.tasks(connName))
+            result.add(new TaskInfo(taskId, configState.taskConfig(taskId)));
         callback.onCompletion(null, result);
     }
 
@@ -202,6 +199,42 @@
         throw new UnsupportedOperationException("Kafka Connect in standalone mode does not support externally setting task configurations.");
     }
 
+    @Override
+    public synchronized void restartTask(ConnectorTaskId taskId, Callback<Void> cb) {
+        if (!configState.contains(taskId.connector()))
+            cb.onCompletion(new NotFoundException("Connector " + taskId.connector() + " not found", null), null);
+
+        Map<String, String> taskConfig = configState.taskConfig(taskId);
+        if (taskConfig == null)
+            cb.onCompletion(new NotFoundException("Task " + taskId + " not found", null), null);
+
+        TargetState targetState = configState.targetState(taskId.connector());
+        try {
+            worker.stopAndAwaitTask(taskId);
+            worker.startTask(taskId, new TaskConfig(taskConfig), this, targetState);
+            cb.onCompletion(null, null);
+        } catch (Exception e) {
+            log.error("Failed to restart task {}", taskId, e);
+            cb.onCompletion(e, null);
+        }
+    }
+
+    @Override
+    public synchronized void restartConnector(String connName, Callback<Void> cb) {
+        if (!configState.contains(connName))
+            cb.onCompletion(new NotFoundException("Connector " + connName + " not found", null), null);
+
+        Map<String, String> config = configState.connectorConfig(connName);
+        try {
+            worker.stopConnector(connName);
+            startConnector(config);
+            cb.onCompletion(null, null);
+        } catch (Exception e) {
+            log.error("Failed to restart connector {}", connName, e);
+            cb.onCompletion(e, null);
+        }
+    }
+
     /**
      * Start a connector in the worker and record its state.
      * @param connectorProps new connector configuration
@@ -210,82 +243,103 @@
     private String startConnector(Map<String, String> connectorProps) {
         ConnectorConfig connConfig = new ConnectorConfig(connectorProps);
         String connName = connConfig.getString(ConnectorConfig.NAME_CONFIG);
-        ConnectorState state = connectors.get(connName);
-        worker.startConnector(connConfig, new HerderConnectorContext(this, connName), this);
-        if (state == null) {
-            connectors.put(connName, new ConnectorState(connectorProps, connConfig));
-        } else {
-            state.configOriginals = connectorProps;
-            state.config = connConfig;
-        }
+        configBackingStore.putConnectorConfig(connName, connectorProps);
+        TargetState targetState = configState.targetState(connName);
+        worker.startConnector(connConfig, new HerderConnectorContext(this, connName), this, targetState);
         return connName;
     }
 
-
     private List<Map<String, String>> recomputeTaskConfigs(String connName) {
-        ConnectorState state = connectors.get(connName);
+        Map<String, String> config = configState.connectorConfig(connName);
+        ConnectorConfig connConfig = new ConnectorConfig(config);
+
         return worker.connectorTaskConfigs(connName,
-                state.config.getInt(ConnectorConfig.TASKS_MAX_CONFIG),
-                state.config.getList(ConnectorConfig.TOPICS_CONFIG));
+                connConfig.getInt(ConnectorConfig.TASKS_MAX_CONFIG),
+                connConfig.getList(ConnectorConfig.TOPICS_CONFIG));
     }
 
-    private void createConnectorTasks(String connName) {
-        ConnectorState state = connectors.get(connName);
-        int index = 0;
-        for (Map<String, String> taskConfigMap : state.taskConfigs) {
-            ConnectorTaskId taskId = new ConnectorTaskId(connName, index);
+    private void createConnectorTasks(String connName, TargetState initialState) {
+        for (ConnectorTaskId taskId : configState.tasks(connName)) {
+            Map<String, String> taskConfigMap = configState.taskConfig(taskId);
             TaskConfig config = new TaskConfig(taskConfigMap);
             try {
-                worker.startTask(taskId, config, this);
+                worker.startTask(taskId, config, this, initialState);
             } catch (Throwable e) {
                 log.error("Failed to add task {}: ", taskId, e);
                 // Swallow this so we can continue updating the rest of the tasks
                 // FIXME what's the proper response? Kill all the tasks? Consider this the same as a task
                 // that died after starting successfully.
             }
-            index++;
         }
     }
 
-    private Set<ConnectorTaskId> tasksFor(ConnectorState state) {
-        Set<ConnectorTaskId> tasks = new HashSet<>();
-        for (int i = 0; i < state.taskConfigs.size(); i++)
-            tasks.add(new ConnectorTaskId(state.name, i));
-        return tasks;
-    }
-
     private void removeConnectorTasks(String connName) {
-        ConnectorState state = connectors.get(connName);
-        Set<ConnectorTaskId> tasks = tasksFor(state);
+        Collection<ConnectorTaskId> tasks = configState.tasks(connName);
         if (!tasks.isEmpty()) {
             worker.stopTasks(tasks);
             worker.awaitStopTasks(tasks);
-            state.taskConfigs = new ArrayList<>();
+            configBackingStore.removeTaskConfigs(connName);
         }
     }
 
     private void updateConnectorTasks(String connName) {
+        if (!worker.isRunning(connName)) {
+            log.info("Skipping reconfiguration of connector {} since it is not running", connName);
+            return;
+        }
+
         List<Map<String, String>> newTaskConfigs = recomputeTaskConfigs(connName);
-        ConnectorState state = connectors.get(connName);
-        if (!newTaskConfigs.equals(state.taskConfigs)) {
+        List<Map<String, String>> oldTaskConfigs = configState.allTaskConfigs(connName);
+
+        if (!newTaskConfigs.equals(oldTaskConfigs)) {
             removeConnectorTasks(connName);
-            state.taskConfigs = newTaskConfigs;
-            createConnectorTasks(connName);
+            configBackingStore.putTaskConfigs(connName, newTaskConfigs);
+            createConnectorTasks(connName, configState.targetState(connName));
         }
     }
 
+    // This update listener assumes synchronous updates the ConfigBackingStore, which only works
+    // with the MemoryConfigBackingStore. This allows us to write a change (e.g. through
+    // ConfigBackingStore.putConnectorConfig()) and then immediately read it back from an updated
+    // snapshot.
+    // TODO: To get any real benefit from the backing store abstraction, we should move some of
+    // the handling into the callbacks in this listener.
+    private class ConfigUpdateListener implements ConfigBackingStore.UpdateListener {
 
-    private static class ConnectorState {
-        public String name;
-        public Map<String, String> configOriginals;
-        public ConnectorConfig config;
-        List<Map<String, String>> taskConfigs;
+        @Override
+        public void onConnectorConfigRemove(String connector) {
+            synchronized (StandaloneHerder.this) {
+                configState = configBackingStore.snapshot();
+            }
+        }
 
-        public ConnectorState(Map<String, String> configOriginals, ConnectorConfig config) {
-            this.name = config.getString(ConnectorConfig.NAME_CONFIG);
-            this.configOriginals = configOriginals;
-            this.config = config;
-            this.taskConfigs = new ArrayList<>();
+        @Override
+        public void onConnectorConfigUpdate(String connector) {
+            // TODO: move connector configuration update handling here to be consistent with
+            //       the semantics of the config backing store
+
+            synchronized (StandaloneHerder.this) {
+                configState = configBackingStore.snapshot();
+            }
+        }
+
+        @Override
+        public void onTaskConfigUpdate(Collection<ConnectorTaskId> tasks) {
+            synchronized (StandaloneHerder.this) {
+                configState = configBackingStore.snapshot();
+            }
+        }
+
+        @Override
+        public void onConnectorTargetStateChange(String connector) {
+            synchronized (StandaloneHerder.this) {
+                configState = configBackingStore.snapshot();
+                TargetState targetState = configState.targetState(connector);
+                worker.setTargetState(connector, targetState);
+                if (targetState == TargetState.STARTED)
+                    updateConnectorTasks(connector);
+            }
         }
     }
+
 }
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConfigBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConfigBackingStore.java
new file mode 100644
index 0000000..77fc43b
--- /dev/null
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConfigBackingStore.java
@@ -0,0 +1,127 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ **/
+package org.apache.kafka.connect.storage;
+
+import org.apache.kafka.connect.runtime.TargetState;
+import org.apache.kafka.connect.runtime.WorkerConfig;
+import org.apache.kafka.connect.runtime.distributed.ClusterConfigState;
+import org.apache.kafka.connect.util.ConnectorTaskId;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+public interface ConfigBackingStore {
+
+    void configure(WorkerConfig config);
+
+    void start();
+
+    void stop();
+
+    /**
+     * Get a snapshot of the current configuration state including all connector and task
+     * configurations.
+     * @return the cluster config state
+     */
+    ClusterConfigState snapshot();
+
+    /**
+     * Check if the store has configuration for a connector.
+     * @param connector name of the connector
+     * @return true if the backing store contains configuration for the connector
+     */
+    boolean contains(String connector);
+
+    /**
+     * Update the configuration for a connector.
+     * @param connector name of the connector
+     * @param properties the connector configuration
+     */
+    void putConnectorConfig(String connector, Map<String, String> properties);
+
+    /**
+     * Remove configuration for a connector
+     * @param connector name of the connector
+     */
+    void removeConnectorConfig(String connector);
+
+    /**
+     * Update the task configurations for a connector.
+     * @param connector name of the connector
+     * @param configs the new task configs for the connector
+     */
+    void putTaskConfigs(String connector, List<Map<String, String>> configs);
+
+    /**
+     * Remove the task configs associated with a connector.
+     * @param connector name of the connector
+     */
+    void removeTaskConfigs(String connector);
+
+    /**
+     * Refresh the backing store. This forces the store to ensure that it has the latest
+     * configs that have been written.
+     * @param timeout max time to wait for the refresh to complete
+     * @param unit unit of timeout
+     * @throws TimeoutException if the timeout expires before the refresh has completed
+     */
+    void refresh(long timeout, TimeUnit unit) throws TimeoutException;
+
+    /**
+     * Transition a connector to a new target state (e.g. paused).
+     * @param connector name of the connector
+     * @param state the state to transition to
+     */
+    void putTargetState(String connector, TargetState state);
+
+    /**
+     * Set an update listener to get notifications when there are config/target state
+     * changes.
+     * @param listener non-null listener
+     */
+    void setUpdateListener(UpdateListener listener);
+
+    interface UpdateListener {
+        /**
+         * Invoked when a connector configuration has been removed
+         * @param connector name of the connector
+         */
+        void onConnectorConfigRemove(String connector);
+
+        /**
+         * Invoked when a connector configuration has been updated.
+         * @param connector name of the connector
+         */
+        void onConnectorConfigUpdate(String connector);
+
+        /**
+         * Invoked when task configs are updated.
+         * @param tasks all the tasks whose configs have been updated
+         */
+        void onTaskConfigUpdate(Collection<ConnectorTaskId> tasks);
+
+        /**
+         * Invoked when the user has set a new target state (e.g. paused)
+         * @param connector name of the connector
+         */
+        void onConnectorTargetStateChange(String connector);
+    }
+
+}
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigStorage.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java
similarity index 78%
rename from connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigStorage.java
rename to connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java
index 08c528c..9412e42 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigStorage.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java
@@ -33,6 +33,8 @@
 import org.apache.kafka.connect.data.Struct;
 import org.apache.kafka.connect.errors.ConnectException;
 import org.apache.kafka.connect.errors.DataException;
+import org.apache.kafka.connect.runtime.TargetState;
+import org.apache.kafka.connect.runtime.WorkerConfig;
 import org.apache.kafka.connect.runtime.distributed.ClusterConfigState;
 import org.apache.kafka.connect.runtime.distributed.DistributedConfig;
 import org.apache.kafka.connect.util.Callback;
@@ -50,7 +52,6 @@
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
@@ -146,8 +147,14 @@
  * rebalance must be deferred.
  * </p>
  */
-public class KafkaConfigStorage {
-    private static final Logger log = LoggerFactory.getLogger(KafkaConfigStorage.class);
+public class KafkaConfigBackingStore implements ConfigBackingStore {
+    private static final Logger log = LoggerFactory.getLogger(KafkaConfigBackingStore.class);
+
+    public static final String TARGET_STATE_PREFIX = "target-state-";
+
+    public static String TARGET_STATE_KEY(String connectorName) {
+        return TARGET_STATE_PREFIX + connectorName;
+    }
 
     public static final String CONNECTOR_PREFIX = "connector-";
 
@@ -178,14 +185,17 @@
     public static final Schema CONNECTOR_TASKS_COMMIT_V0 = SchemaBuilder.struct()
             .field("tasks", Schema.INT32_SCHEMA)
             .build();
+    public static final Schema TARGET_STATE_V0 = SchemaBuilder.struct()
+            .field("state", Schema.STRING_SCHEMA)
+            .build();
 
     private static final long READ_TO_END_TIMEOUT_MS = 30000;
 
     private final Object lock;
     private boolean starting;
     private final Converter converter;
-    private final Callback<String> connectorConfigCallback;
-    private final Callback<List<ConnectorTaskId>> tasksConfigCallback;
+    private UpdateListener updateListener;
+
     private String topic;
     // Data is passed to the log already serialized. We use a converter to handle translating to/from generic Connect
     // format to serialized form
@@ -195,28 +205,33 @@
     // Connector and task configs: name or id -> config map
     private Map<String, Map<String, String>> connectorConfigs = new HashMap<>();
     private Map<ConnectorTaskId, Map<String, String>> taskConfigs = new HashMap<>();
+
     // Set of connectors where we saw a task commit with an incomplete set of task config updates, indicating the data
     // is in an inconsistent state and we cannot safely use them until they have been refreshed.
     private Set<String> inconsistent = new HashSet<>();
     // The most recently read offset. This does not take into account deferred task updates/commits, so we may have
     // outstanding data to be applied.
-    private long offset;
+    private volatile long offset;
 
     // Connector -> Map[ConnectorTaskId -> Configs]
-    private Map<String, Map<ConnectorTaskId, Map<String, String>>> deferredTaskUpdates = new HashMap<>();
+    private final Map<String, Map<ConnectorTaskId, Map<String, String>>> deferredTaskUpdates = new HashMap<>();
 
+    private final Map<String, TargetState> connectorTargetStates = new HashMap<>();
 
-    public KafkaConfigStorage(Converter converter, Callback<String> connectorConfigCallback, Callback<List<ConnectorTaskId>> tasksConfigCallback) {
+    public KafkaConfigBackingStore(Converter converter) {
         this.lock = new Object();
         this.starting = false;
         this.converter = converter;
-        this.connectorConfigCallback = connectorConfigCallback;
-        this.tasksConfigCallback = tasksConfigCallback;
-
-        offset = -1;
+        this.offset = -1;
     }
 
-    public void configure(DistributedConfig config) {
+    @Override
+    public void setUpdateListener(UpdateListener listener) {
+        this.updateListener = listener;
+    }
+
+    @Override
+    public void configure(WorkerConfig config) {
         topic = config.getString(DistributedConfig.CONFIG_TOPIC_CONFIG);
         if (topic.equals(""))
             throw new ConfigException("Must specify topic for connector configuration.");
@@ -232,28 +247,31 @@
         consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
         consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
 
-        configLog = createKafkaBasedLog(topic, producerProps, consumerProps, consumedCallback);
+        configLog = createKafkaBasedLog(topic, producerProps, consumerProps, new ConsumeCallback());
     }
 
+    @Override
     public void start() {
-        log.info("Starting KafkaConfigStorage");
+        log.info("Starting KafkaConfigBackingStore");
         // During startup, callbacks are *not* invoked. You can grab a snapshot after starting -- just take care that
         // updates can continue to occur in the background
         starting = true;
         configLog.start();
         starting = false;
-        log.info("Started KafkaConfigStorage");
+        log.info("Started KafkaConfigBackingStore");
     }
 
+    @Override
     public void stop() {
-        log.info("Closing KafkaConfigStorage");
+        log.info("Closing KafkaConfigBackingStore");
         configLog.stop();
-        log.info("Closed KafkaConfigStorage");
+        log.info("Closed KafkaConfigBackingStore");
     }
 
     /**
      * Get a snapshot of the current state of the cluster.
      */
+    @Override
     public ClusterConfigState snapshot() {
         synchronized (lock) {
             // Doing a shallow copy of the data is safe here because the complex nested data that is copied should all be
@@ -262,12 +280,20 @@
                     offset,
                     new HashMap<>(connectorTaskCounts),
                     new HashMap<>(connectorConfigs),
+                    new HashMap<>(connectorTargetStates),
                     new HashMap<>(taskConfigs),
                     new HashSet<>(inconsistent)
             );
         }
     }
 
+    @Override
+    public boolean contains(String connector) {
+        synchronized (lock) {
+            return connectorConfigs.containsKey(connector);
+        }
+    }
+
     /**
      * Write this connector configuration to persistent storage and wait until it has been acknowledge and read back by
      * tailing the Kafka log with a consumer.
@@ -275,18 +301,33 @@
      * @param connector  name of the connector to write data for
      * @param properties the configuration to write
      */
+    @Override
     public void putConnectorConfig(String connector, Map<String, String> properties) {
-        byte[] serializedConfig;
-        if (properties == null) {
-            serializedConfig = null;
-        } else {
-            Struct connectConfig = new Struct(CONNECTOR_CONFIGURATION_V0);
-            connectConfig.put("properties", properties);
-            serializedConfig = converter.fromConnectData(topic, CONNECTOR_CONFIGURATION_V0, connectConfig);
-        }
+        log.debug("Writing connector configuration {} for connector {} configuration", properties, connector);
+        Struct connectConfig = new Struct(CONNECTOR_CONFIGURATION_V0);
+        connectConfig.put("properties", properties);
+        byte[] serializedConfig = converter.fromConnectData(topic, CONNECTOR_CONFIGURATION_V0, connectConfig);
+        updateConnectorConfig(connector, serializedConfig);
+    }
 
+    /**
+     * Remove configuration for a given connector.
+     * @param connector name of the connector to remove
+     */
+    @Override
+    public void removeConnectorConfig(String connector) {
+        log.debug("Removing connector configuration for connector {}", connector);
+        updateConnectorConfig(connector, null);
+        configLog.send(TARGET_STATE_KEY(connector), null);
+    }
+
+    @Override
+    public void removeTaskConfigs(String connector) {
+        throw new UnsupportedOperationException("Removal of tasks is not currently supported");
+    }
+
+    private void updateConnectorConfig(String connector, byte[] serializedConfig) {
         try {
-            log.debug("Writing connector configuration for connector " + connector + " configuration: " + properties);
             configLog.send(CONNECTOR_KEY(connector), serializedConfig);
             configLog.readToEnd().get(READ_TO_END_TIMEOUT_MS, TimeUnit.MILLISECONDS);
         } catch (InterruptedException | ExecutionException | TimeoutException e) {
@@ -299,11 +340,13 @@
      * Write these task configurations and associated commit messages, unless an inconsistency is found that indicates
      * that we would be leaving one of the referenced connectors with an inconsistent state.
      *
-     * @param configs map containing task configurations
+     * @param connector the connector to write task configuration
+     * @param configs list of task configurations for the connector
      * @throws ConnectException if the task configurations do not resolve inconsistencies found in the existing root
      *                          and task configurations.
      */
-    public void putTaskConfigs(Map<ConnectorTaskId, Map<String, String>> configs) {
+    @Override
+    public void putTaskConfigs(String connector, List<Map<String, String>> configs) {
         // Make sure we're at the end of the log. We should be the only writer, but we want to make sure we don't have
         // any outstanding lagging data to consume.
         try {
@@ -313,46 +356,33 @@
             throw new ConnectException("Error writing root configuration to Kafka", e);
         }
 
-        // In theory, there is only a single writer and we shouldn't need this lock since the background thread should
-        // not invoke any callbacks that would conflict, but in practice this guards against inconsistencies due to
-        // the root config being updated.
-        Map<String, Integer> newTaskCounts = new HashMap<>();
-        synchronized (lock) {
-            // Validate tasks in this assignment. Any task configuration updates should include updates for *all* tasks
-            // in the connector -- we should have all task IDs 0 - N-1 within a connector if any task is included here
-            Map<String, Set<Integer>> updatedConfigIdsByConnector = taskIdsByConnector(configs);
-            for (Map.Entry<String, Set<Integer>> taskConfigSetEntry : updatedConfigIdsByConnector.entrySet()) {
-                if (!completeTaskIdSet(taskConfigSetEntry.getValue(), taskConfigSetEntry.getValue().size())) {
-                    log.error("Submitted task configuration contain invalid range of task IDs, ignoring this submission");
-                    throw new ConnectException("Error writing task configurations: found some connectors with invalid connectors");
-                }
-                newTaskCounts.put(taskConfigSetEntry.getKey(), taskConfigSetEntry.getValue().size());
-            }
-        }
+        int taskCount = configs.size();
 
         // Start sending all the individual updates
-        for (Map.Entry<ConnectorTaskId, Map<String, String>> taskConfigEntry : configs.entrySet()) {
+        int index = 0;
+        for (Map<String, String> taskConfig: configs) {
             Struct connectConfig = new Struct(TASK_CONFIGURATION_V0);
-            connectConfig.put("properties", taskConfigEntry.getValue());
+            connectConfig.put("properties", taskConfig);
             byte[] serializedConfig = converter.fromConnectData(topic, TASK_CONFIGURATION_V0, connectConfig);
-            log.debug("Writing configuration for task " + taskConfigEntry.getKey() + " configuration: " + taskConfigEntry.getValue());
-            configLog.send(TASK_KEY(taskConfigEntry.getKey()), serializedConfig);
+            log.debug("Writing configuration for task " + index + " configuration: " + taskConfig);
+            ConnectorTaskId connectorTaskId = new ConnectorTaskId(connector, index);
+            configLog.send(TASK_KEY(connectorTaskId), serializedConfig);
+            index++;
         }
 
         // Finally, send the commit to update the number of tasks and apply the new configs, then wait until we read to
         // the end of the log
         try {
             // Read to end to ensure all the task configs have been written
-            configLog.readToEnd().get(READ_TO_END_TIMEOUT_MS, TimeUnit.MILLISECONDS);
-
-            // Write all the commit messages
-            for (Map.Entry<String, Integer> taskCountEntry : newTaskCounts.entrySet()) {
-                Struct connectConfig = new Struct(CONNECTOR_TASKS_COMMIT_V0);
-                connectConfig.put("tasks", taskCountEntry.getValue());
-                byte[] serializedConfig = converter.fromConnectData(topic, CONNECTOR_TASKS_COMMIT_V0, connectConfig);
-                log.debug("Writing commit for connector " + taskCountEntry.getKey() + " with " + taskCountEntry.getValue() + " tasks.");
-                configLog.send(COMMIT_TASKS_KEY(taskCountEntry.getKey()), serializedConfig);
+            if (taskCount > 0) {
+                configLog.readToEnd().get(READ_TO_END_TIMEOUT_MS, TimeUnit.MILLISECONDS);
             }
+            // Write the commit message
+            Struct connectConfig = new Struct(CONNECTOR_TASKS_COMMIT_V0);
+            connectConfig.put("tasks", taskCount);
+            byte[] serializedConfig = converter.fromConnectData(topic, CONNECTOR_TASKS_COMMIT_V0, connectConfig);
+            log.debug("Writing commit for connector " + connector + " with " + taskCount + " tasks.");
+            configLog.send(COMMIT_TASKS_KEY(connector), serializedConfig);
 
             // Read to end to ensure all the commit messages have been written
             configLog.readToEnd().get(READ_TO_END_TIMEOUT_MS, TimeUnit.MILLISECONDS);
@@ -362,12 +392,22 @@
         }
     }
 
-    public Future<Void> readToEnd() {
-        return configLog.readToEnd();
+    @Override
+    public void refresh(long timeout, TimeUnit unit) throws TimeoutException {
+        try {
+            configLog.readToEnd().get(timeout, unit);
+        } catch (InterruptedException | ExecutionException e) {
+            throw new ConnectException("Error trying to read to end of config log", e);
+        }
     }
 
-    public void readToEnd(Callback<Void> cb) {
-        configLog.readToEnd(cb);
+    @Override
+    public void putTargetState(String connector, TargetState state) {
+        Struct connectTargetState = new Struct(TARGET_STATE_V0);
+        connectTargetState.put("state", state.name());
+        byte[] serializedTargetState = converter.fromConnectData(topic, TARGET_STATE_V0, connectTargetState);
+        log.debug("Writing target state {} for connector {}", state, connector);
+        configLog.send(TARGET_STATE_KEY(connector), serializedTargetState);
     }
 
     private KafkaBasedLog<String, byte[]> createKafkaBasedLog(String topic, Map<String, Object> producerProps,
@@ -376,11 +416,11 @@
     }
 
     @SuppressWarnings("unchecked")
-    private final Callback<ConsumerRecord<String, byte[]>> consumedCallback = new Callback<ConsumerRecord<String, byte[]>>() {
+    private class ConsumeCallback implements Callback<ConsumerRecord<String, byte[]>> {
         @Override
         public void onCompletion(Throwable error, ConsumerRecord<String, byte[]> record) {
             if (error != null) {
-                log.error("Unexpected in consumer callback for KafkaConfigStorage: ", error);
+                log.error("Unexpected in consumer callback for KafkaConfigBackingStore: ", error);
                 return;
             }
 
@@ -395,13 +435,43 @@
             // *next record*, not the last one consumed.
             offset = record.offset() + 1;
 
-            if (record.key().startsWith(CONNECTOR_PREFIX)) {
+            if (record.key().startsWith(TARGET_STATE_PREFIX)) {
+                String connectorName = record.key().substring(TARGET_STATE_PREFIX.length());
+                synchronized (lock) {
+                    if (value.value() != null) {
+                        if (!(value.value() instanceof Map)) {
+                            log.error("Found target state ({}) in wrong format: {}",  record.key(), value.value().getClass());
+                            return;
+                        }
+                        Object targetState = ((Map<String, Object>) value.value()).get("state");
+                        if (!(targetState instanceof String)) {
+                            log.error("Invalid data for target state for connector ({}): 'state' field should be a Map but is {}",
+                                    connectorName, targetState == null ? null : targetState.getClass());
+                            return;
+                        }
+
+                        try {
+                            TargetState state = TargetState.valueOf((String) targetState);
+                            log.trace("Setting target state for connector {} to {}", connectorName, targetState);
+                            connectorTargetStates.put(connectorName, state);
+                        } catch (IllegalArgumentException e) {
+                            log.error("Invalid target state for connector ({}): {}", connectorName, targetState);
+                            return;
+                        }
+                    }
+                }
+
+                if (!starting)
+                    updateListener.onConnectorTargetStateChange(connectorName);
+            } else if (record.key().startsWith(CONNECTOR_PREFIX)) {
                 String connectorName = record.key().substring(CONNECTOR_PREFIX.length());
+                boolean removed = false;
                 synchronized (lock) {
                     if (value.value() == null) {
                         // Connector deletion will be written as a null value
                         log.info("Removed connector " + connectorName + " due to null configuration. This is usually intentional and does not indicate an issue.");
                         connectorConfigs.remove(connectorName);
+                        removed = true;
                     } else {
                         // Connector configs can be applied and callbacks invoked immediately
                         if (!(value.value() instanceof Map)) {
@@ -410,15 +480,23 @@
                         }
                         Object newConnectorConfig = ((Map<String, Object>) value.value()).get("properties");
                         if (!(newConnectorConfig instanceof Map)) {
-                            log.error("Invalid data for connector config (" + connectorName + "): properties filed should be a Map but is " + newConnectorConfig.getClass());
+                            log.error("Invalid data for connector config ({}): properties field should be a Map but is {}", connectorName,
+                                    newConnectorConfig == null ? null : newConnectorConfig.getClass());
                             return;
                         }
-                        log.debug("Updating configuration for connector " + connectorName + " configuation: " + newConnectorConfig);
+                        log.debug("Updating configuration for connector " + connectorName + " configuration: " + newConnectorConfig);
                         connectorConfigs.put(connectorName, (Map<String, String>) newConnectorConfig);
+
+                        if (!connectorTargetStates.containsKey(connectorName))
+                            connectorTargetStates.put(connectorName, TargetState.STARTED);
                     }
                 }
-                if (!starting)
-                    connectorConfigCallback.onCompletion(null, connectorName);
+                if (!starting) {
+                    if (removed)
+                        updateListener.onConnectorConfigRemove(connectorName);
+                    else
+                        updateListener.onConnectorConfigUpdate(connectorName);
+                }
             } else if (record.key().startsWith(TASK_PREFIX)) {
                 synchronized (lock) {
                     ConnectorTaskId taskId = parseTaskId(record.key());
@@ -474,20 +552,13 @@
                         log.error("Ignoring connector tasks configuration commit for connector " + connectorName + " because it is in the wrong format: " + value.value());
                         return;
                     }
-
                     Map<ConnectorTaskId, Map<String, String>> deferred = deferredTaskUpdates.get(connectorName);
 
                     int newTaskCount = intValue(((Map<String, Object>) value.value()).get("tasks"));
 
                     // Validate the configs we're supposed to update to ensure we're getting a complete configuration
                     // update of all tasks that are expected based on the number of tasks in the commit message.
-                    Map<String, Set<Integer>> updatedConfigIdsByConnector = taskIdsByConnector(deferred);
-                    Set<Integer> taskIdSet = updatedConfigIdsByConnector.get(connectorName);
-                    if (taskIdSet == null) {
-                        //TODO: Figure out why this happens (KAFKA-3321)
-                        log.error("Received a commit message for connector " + connectorName + " but there is no matching configuration for tasks in this connector. This should never happen.");
-                        return;
-                    }
+                    Set<Integer> taskIdSet = taskIds(connectorName, deferred);
                     if (!completeTaskIdSet(taskIdSet, newTaskCount)) {
                         // Given the logic for writing commit messages, we should only hit this condition due to compacted
                         // historical data, in which case we would not have applied any updates yet and there will be no
@@ -512,12 +583,13 @@
                 }
 
                 if (!starting)
-                    tasksConfigCallback.onCompletion(null, updatedTasks);
+                    updateListener.onTaskConfigUpdate(updatedTasks);
             } else {
                 log.error("Discarding config update record with invalid key: " + record.key());
             }
         }
-    };
+
+    }
 
     private ConnectorTaskId parseTaskId(String key) {
         String[] parts = key.split("-");
@@ -533,19 +605,18 @@
     }
 
     /**
-     * Given task configurations, get a set of integer task IDs organized by connector name.
+     * Given task configurations, get a set of integer task IDs for the connector.
      */
-    private Map<String, Set<Integer>> taskIdsByConnector(Map<ConnectorTaskId, Map<String, String>> configs) {
-        Map<String, Set<Integer>> connectorTaskIds = new HashMap<>();
-        if (configs == null)
-            return connectorTaskIds;
-        for (Map.Entry<ConnectorTaskId, Map<String, String>> taskConfigEntry : configs.entrySet()) {
-            ConnectorTaskId taskId = taskConfigEntry.getKey();
-            if (!connectorTaskIds.containsKey(taskId.connector()))
-                connectorTaskIds.put(taskId.connector(), new TreeSet<Integer>());
-            connectorTaskIds.get(taskId.connector()).add(taskId.task());
+    private Set<Integer> taskIds(String connector, Map<ConnectorTaskId, Map<String, String>> configs) {
+        Set<Integer> tasks = new TreeSet<>();
+        if (configs == null) {
+            return tasks;
         }
-        return connectorTaskIds;
+        for (ConnectorTaskId taskId : configs.keySet()) {
+            assert taskId.connector().equals(connector);
+            tasks.add(taskId.task());
+        }
+        return tasks;
     }
 
     private boolean completeTaskIdSet(Set<Integer> idSet, int expectedSize) {
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java
new file mode 100644
index 0000000..212022d
--- /dev/null
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java
@@ -0,0 +1,166 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ **/
+package org.apache.kafka.connect.storage;
+
+import org.apache.kafka.connect.runtime.TargetState;
+import org.apache.kafka.connect.runtime.WorkerConfig;
+import org.apache.kafka.connect.runtime.distributed.ClusterConfigState;
+import org.apache.kafka.connect.util.ConnectorTaskId;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+
+public class MemoryConfigBackingStore implements ConfigBackingStore {
+
+    private Map<String, ConnectorState> connectors = new HashMap<>();
+    private UpdateListener updateListener;
+
+    @Override
+    public void configure(WorkerConfig config) {
+    }
+
+    @Override
+    public synchronized void start() {
+    }
+
+    @Override
+    public synchronized void stop() {
+    }
+
+    @Override
+    public synchronized ClusterConfigState snapshot() {
+        Map<String, Integer> connectorTaskCounts = new HashMap<>();
+        Map<String, Map<String, String>> connectorConfigs = new HashMap<>();
+        Map<String, TargetState> connectorTargetStates = new HashMap<>();
+        Map<ConnectorTaskId, Map<String, String>> taskConfigs = new HashMap<>();
+
+        for (Map.Entry<String, ConnectorState> connectorStateEntry : connectors.entrySet()) {
+            String connector = connectorStateEntry.getKey();
+            ConnectorState connectorState = connectorStateEntry.getValue();
+            connectorTaskCounts.put(connector, connectorState.taskConfigs.size());
+            connectorConfigs.put(connector, connectorState.connConfig);
+            connectorTargetStates.put(connector, connectorState.targetState);
+            taskConfigs.putAll(connectorState.taskConfigs);
+        }
+
+        return new ClusterConfigState(
+                ClusterConfigState.NO_OFFSET,
+                connectorTaskCounts,
+                connectorConfigs,
+                connectorTargetStates,
+                taskConfigs,
+                Collections.<String>emptySet());
+    }
+
+    @Override
+    public synchronized boolean contains(String connector) {
+        return connectors.containsKey(connector);
+    }
+
+    @Override
+    public synchronized void putConnectorConfig(String connector, Map<String, String> properties) {
+        ConnectorState state = connectors.get(connector);
+        if (state == null)
+            connectors.put(connector, new ConnectorState(properties));
+        else
+            state.connConfig = properties;
+
+        if (updateListener != null)
+            updateListener.onConnectorConfigUpdate(connector);
+    }
+
+    @Override
+    public synchronized void removeConnectorConfig(String connector) {
+        ConnectorState state = connectors.remove(connector);
+
+        if (updateListener != null && state != null)
+            updateListener.onConnectorConfigRemove(connector);
+    }
+
+    @Override
+    public synchronized void removeTaskConfigs(String connector) {
+        ConnectorState state = connectors.get(connector);
+        if (state == null)
+            throw new IllegalArgumentException("Cannot remove tasks for non-existing connector");
+
+        HashSet<ConnectorTaskId> taskIds = new HashSet<>(state.taskConfigs.keySet());
+        state.taskConfigs.clear();
+
+        if (updateListener != null)
+            updateListener.onTaskConfigUpdate(taskIds);
+    }
+
+    @Override
+    public synchronized void putTaskConfigs(String connector, List<Map<String, String>> configs) {
+        ConnectorState state = connectors.get(connector);
+        if (state == null)
+            throw new IllegalArgumentException("Cannot put tasks for non-existing connector");
+
+        Map<ConnectorTaskId, Map<String, String>> taskConfigsMap = taskConfigListAsMap(connector, configs);
+        state.taskConfigs = taskConfigsMap;
+
+        if (updateListener != null)
+            updateListener.onTaskConfigUpdate(taskConfigsMap.keySet());
+    }
+
+    @Override
+    public void refresh(long timeout, TimeUnit unit) {
+    }
+
+    @Override
+    public synchronized void putTargetState(String connector, TargetState state) {
+        ConnectorState connectorState = connectors.get(connector);
+        if (connectorState == null)
+            throw new IllegalArgumentException("No connector `" + connector + "` configured");
+
+        connectorState.targetState = state;
+
+        if (updateListener != null)
+            updateListener.onConnectorTargetStateChange(connector);
+    }
+
+    @Override
+    public synchronized void setUpdateListener(UpdateListener listener) {
+        this.updateListener = listener;
+    }
+
+    private static class ConnectorState {
+        private TargetState targetState;
+        private Map<String, String> connConfig;
+        private Map<ConnectorTaskId, Map<String, String>> taskConfigs;
+
+        public ConnectorState(Map<String, String> connConfig) {
+            this.targetState = TargetState.STARTED;
+            this.connConfig = connConfig;
+            this.taskConfigs = new HashMap<>();
+        }
+    }
+
+    private static Map<ConnectorTaskId, Map<String, String>> taskConfigListAsMap(String connector, List<Map<String, String>> configs) {
+        int index = 0;
+        Map<ConnectorTaskId, Map<String, String>> result = new TreeMap<>();
+        for (Map<String, String> taskConfigMap: configs) {
+            result.put(new ConnectorTaskId(connector, index++), taskConfigMap);
+        }
+        return result;
+    }
+}
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/StatusBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/StatusBackingStore.java
index 268c5df..69f1fdc 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/StatusBackingStore.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/StatusBackingStore.java
@@ -102,5 +102,5 @@
      * Configure class with the given key-value pairs
      * @param config config for StatusBackingStore
      */
-    public void configure(WorkerConfig config);
+    void configure(WorkerConfig config);
 }
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/ReflectionsUtil.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/ReflectionsUtil.java
new file mode 100644
index 0000000..fc3a0dd
--- /dev/null
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/ReflectionsUtil.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements.  See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.  You may obtain a
+ * copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ **/
+package org.apache.kafka.connect.util;
+
+import org.reflections.vfs.Vfs;
+import org.reflections.vfs.Vfs.Dir;
+import org.reflections.vfs.Vfs.File;
+import org.reflections.vfs.Vfs.UrlType;
+
+import java.net.URL;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * CLASSPATH on OSX contains .mar and .jnilib file extensions. Vfs used by Reflections does not recognize
+ * urls with those extensions and log WARNs when scan them. Those WARNs can be eliminated by registering
+ * URL types before using reflection.
+ */
+public class ReflectionsUtil {
+
+    private static final String FILE_PROTOCOL = "file";
+    private static final List<String> ENDINGS = Arrays.asList(".mar", ".jnilib", "*");
+
+    public static void registerUrlTypes() {
+        final List<UrlType> urlTypes = new LinkedList<>();
+        urlTypes.add(new EmptyUrlType(ENDINGS));
+        urlTypes.addAll(Arrays.asList(Vfs.DefaultUrlTypes.values()));
+        Vfs.setDefaultURLTypes(urlTypes);
+    }
+
+    private static class EmptyUrlType implements UrlType {
+
+        private final List<String> endings;
+
+        private EmptyUrlType(final List<String> endings) {
+            this.endings = endings;
+        }
+
+        public boolean matches(URL url) {
+            final String protocol = url.getProtocol();
+            final String externalForm = url.toExternalForm();
+            if (!protocol.equals(FILE_PROTOCOL)) {
+                return false;
+            }
+            for (String ending : endings) {
+                if (externalForm.endsWith(ending)) {
+                    return true;
+                }
+            }
+            return false;
+        }
+
+        public Dir createDir(final URL url) throws Exception {
+            return emptyVfsDir(url);
+        }
+
+        private static Dir emptyVfsDir(final URL url) {
+            return new Dir() {
+                @Override
+                public String getPath() {
+                    return url.toExternalForm();
+                }
+
+                @Override
+                public Iterable<File> getFiles() {
+                    return Collections.emptyList();
+                }
+
+                @Override
+                public void close() {
+
+                }
+            };
+        }
+    }
+}
\ No newline at end of file
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java
index e4084a8..d7a2b2a 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java
@@ -17,6 +17,7 @@
 package org.apache.kafka.connect.runtime;
 
 import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo;
+import org.apache.kafka.connect.storage.ConfigBackingStore;
 import org.apache.kafka.connect.storage.StatusBackingStore;
 import org.apache.kafka.connect.util.ConnectorTaskId;
 import org.easymock.Capture;
@@ -40,20 +41,21 @@
         int generation = 5;
         ConnectorTaskId taskId = new ConnectorTaskId(connector, 0);
 
-        StatusBackingStore store = strictMock(StatusBackingStore.class);
+        ConfigBackingStore configStore = strictMock(ConfigBackingStore.class);
+        StatusBackingStore statusStore = strictMock(StatusBackingStore.class);
 
         AbstractHerder herder = partialMockBuilder(AbstractHerder.class)
-                .withConstructor(Worker.class, StatusBackingStore.class, String.class)
-                .withArgs(worker, store, workerId)
+                .withConstructor(Worker.class, String.class, StatusBackingStore.class, ConfigBackingStore.class)
+                .withArgs(worker, workerId, statusStore, configStore)
                 .addMockedMethod("generation")
                 .createMock();
 
         EasyMock.expect(herder.generation()).andStubReturn(generation);
 
-        EasyMock.expect(store.get(connector))
+        EasyMock.expect(statusStore.get(connector))
                 .andReturn(new ConnectorStatus(connector, AbstractStatus.State.RUNNING, workerId, generation));
 
-        EasyMock.expect(store.getAll(connector))
+        EasyMock.expect(statusStore.getAll(connector))
                 .andReturn(Collections.singletonList(
                         new TaskStatus(taskId, AbstractStatus.State.UNASSIGNED, workerId, generation)));
 
@@ -81,21 +83,22 @@
         ConnectorTaskId taskId = new ConnectorTaskId("connector", 0);
         String workerId = "workerId";
 
-        StatusBackingStore store = strictMock(StatusBackingStore.class);
+        ConfigBackingStore configStore = strictMock(ConfigBackingStore.class);
+        StatusBackingStore statusStore = strictMock(StatusBackingStore.class);
 
         AbstractHerder herder = partialMockBuilder(AbstractHerder.class)
-                .withConstructor(Worker.class, StatusBackingStore.class, String.class)
-                .withArgs(worker, store, workerId)
+                .withConstructor(Worker.class, String.class, StatusBackingStore.class, ConfigBackingStore.class)
+                .withArgs(worker, workerId, statusStore, configStore)
                 .addMockedMethod("generation")
                 .createMock();
 
         EasyMock.expect(herder.generation()).andStubReturn(5);
 
         final Capture<TaskStatus> statusCapture = EasyMock.newCapture();
-        store.putSafe(EasyMock.capture(statusCapture));
+        statusStore.putSafe(EasyMock.capture(statusCapture));
         EasyMock.expectLastCall();
 
-        EasyMock.expect(store.get(taskId)).andAnswer(new IAnswer<TaskStatus>() {
+        EasyMock.expect(statusStore.get(taskId)).andAnswer(new IAnswer<TaskStatus>() {
             @Override
             public TaskStatus answer() throws Throwable {
                 return statusCapture.getValue();
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConnectorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConnectorTest.java
new file mode 100644
index 0000000..a5f7409
--- /dev/null
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConnectorTest.java
@@ -0,0 +1,336 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ **/
+package org.apache.kafka.connect.runtime;
+
+import org.apache.kafka.connect.connector.Connector;
+import org.apache.kafka.connect.connector.ConnectorContext;
+import org.easymock.EasyMock;
+import org.easymock.EasyMockRunner;
+import org.easymock.EasyMockSupport;
+import org.easymock.Mock;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.easymock.EasyMock.expectLastCall;
+
+@RunWith(EasyMockRunner.class)
+public class WorkerConnectorTest extends EasyMockSupport {
+
+    public static final String CONNECTOR = "connector";
+    public static final Map<String, String> CONFIG = new HashMap<>();
+    static {
+        CONFIG.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, TestConnector.class.getName());
+        CONFIG.put(ConnectorConfig.NAME_CONFIG, CONNECTOR);
+    }
+    public static final ConnectorConfig CONNECTOR_CONFIG = new ConnectorConfig(CONFIG);
+
+    @Mock Connector connector;
+    @Mock ConnectorContext ctx;
+    @Mock ConnectorStatus.Listener listener;
+
+    @Test
+    public void testInitializeFailure() {
+        RuntimeException exception = new RuntimeException();
+
+        connector.initialize(EasyMock.notNull(ConnectorContext.class));
+        expectLastCall().andThrow(exception);
+
+        listener.onFailure(CONNECTOR, exception);
+        expectLastCall();
+
+        listener.onShutdown(CONNECTOR);
+        expectLastCall();
+
+        replayAll();
+
+        WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, ctx, listener);
+
+        workerConnector.initialize(CONNECTOR_CONFIG);
+        workerConnector.shutdown();
+
+        verifyAll();
+    }
+
+    @Test
+    public void testFailureIsFinalState() {
+        RuntimeException exception = new RuntimeException();
+
+        connector.initialize(EasyMock.notNull(ConnectorContext.class));
+        expectLastCall().andThrow(exception);
+
+        listener.onFailure(CONNECTOR, exception);
+        expectLastCall();
+
+        // expect no call to onStartup() after failure
+
+        listener.onShutdown(CONNECTOR);
+        expectLastCall();
+
+        replayAll();
+
+        WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, ctx, listener);
+
+        workerConnector.initialize(CONNECTOR_CONFIG);
+        workerConnector.transitionTo(TargetState.STARTED);
+        workerConnector.shutdown();
+
+        verifyAll();
+    }
+
+    @Test
+    public void testStartupAndShutdown() {
+        connector.initialize(EasyMock.notNull(ConnectorContext.class));
+        expectLastCall();
+
+        connector.start(CONFIG);
+        expectLastCall();
+
+        listener.onStartup(CONNECTOR);
+        expectLastCall();
+
+        connector.stop();
+        expectLastCall();
+
+        listener.onShutdown(CONNECTOR);
+        expectLastCall();
+
+        replayAll();
+
+        WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, ctx, listener);
+
+        workerConnector.initialize(CONNECTOR_CONFIG);
+        workerConnector.transitionTo(TargetState.STARTED);
+        workerConnector.shutdown();
+
+        verifyAll();
+    }
+
+    @Test
+    public void testStartupAndPause() {
+        connector.initialize(EasyMock.notNull(ConnectorContext.class));
+        expectLastCall();
+
+        connector.start(CONFIG);
+        expectLastCall();
+
+        listener.onStartup(CONNECTOR);
+        expectLastCall();
+
+        connector.stop();
+        expectLastCall();
+
+        listener.onPause(CONNECTOR);
+        expectLastCall();
+
+        listener.onShutdown(CONNECTOR);
+        expectLastCall();
+
+        replayAll();
+
+        WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, ctx, listener);
+
+        workerConnector.initialize(CONNECTOR_CONFIG);
+        workerConnector.transitionTo(TargetState.STARTED);
+        workerConnector.transitionTo(TargetState.PAUSED);
+        workerConnector.shutdown();
+
+        verifyAll();
+    }
+
+    @Test
+    public void testOnResume() {
+        connector.initialize(EasyMock.notNull(ConnectorContext.class));
+        expectLastCall();
+
+        listener.onPause(CONNECTOR);
+        expectLastCall();
+
+        connector.start(CONFIG);
+        expectLastCall();
+
+        listener.onResume(CONNECTOR);
+        expectLastCall();
+
+        connector.stop();
+        expectLastCall();
+
+        listener.onShutdown(CONNECTOR);
+        expectLastCall();
+
+        replayAll();
+
+        WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, ctx, listener);
+
+        workerConnector.initialize(CONNECTOR_CONFIG);
+        workerConnector.transitionTo(TargetState.PAUSED);
+        workerConnector.transitionTo(TargetState.STARTED);
+        workerConnector.shutdown();
+
+        verifyAll();
+    }
+
+    @Test
+    public void testStartupPaused() {
+        connector.initialize(EasyMock.notNull(ConnectorContext.class));
+        expectLastCall();
+
+        // connector never gets started
+
+        listener.onPause(CONNECTOR);
+        expectLastCall();
+
+        listener.onShutdown(CONNECTOR);
+        expectLastCall();
+
+        replayAll();
+
+        WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, ctx, listener);
+
+        workerConnector.initialize(CONNECTOR_CONFIG);
+        workerConnector.transitionTo(TargetState.PAUSED);
+        workerConnector.shutdown();
+
+        verifyAll();
+    }
+
+    @Test
+    public void testStartupFailure() {
+        RuntimeException exception = new RuntimeException();
+
+        connector.initialize(EasyMock.notNull(ConnectorContext.class));
+        expectLastCall();
+
+        connector.start(CONFIG);
+        expectLastCall().andThrow(exception);
+
+        listener.onFailure(CONNECTOR, exception);
+        expectLastCall();
+
+        listener.onShutdown(CONNECTOR);
+        expectLastCall();
+
+        replayAll();
+
+        WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, ctx, listener);
+
+        workerConnector.initialize(CONNECTOR_CONFIG);
+        workerConnector.transitionTo(TargetState.STARTED);
+        workerConnector.shutdown();
+
+        verifyAll();
+    }
+
+    @Test
+    public void testShutdownFailure() {
+        RuntimeException exception = new RuntimeException();
+
+        connector.initialize(EasyMock.notNull(ConnectorContext.class));
+        expectLastCall();
+
+        connector.start(CONFIG);
+        expectLastCall();
+
+        listener.onStartup(CONNECTOR);
+        expectLastCall();
+
+        connector.stop();
+        expectLastCall().andThrow(exception);
+
+        listener.onShutdown(CONNECTOR);
+        expectLastCall();
+
+        replayAll();
+
+        WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, ctx, listener);
+
+        workerConnector.initialize(CONNECTOR_CONFIG);
+        workerConnector.transitionTo(TargetState.STARTED);
+        workerConnector.shutdown();
+
+        verifyAll();
+    }
+
+    @Test
+    public void testTransitionStartedToStarted() {
+        connector.initialize(EasyMock.notNull(ConnectorContext.class));
+        expectLastCall();
+
+        connector.start(CONFIG);
+        expectLastCall();
+
+        // expect only one call to onStartup()
+        listener.onStartup(CONNECTOR);
+        expectLastCall();
+
+        connector.stop();
+        expectLastCall();
+
+        listener.onShutdown(CONNECTOR);
+        expectLastCall();
+
+        replayAll();
+
+        WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, ctx, listener);
+
+        workerConnector.initialize(CONNECTOR_CONFIG);
+        workerConnector.transitionTo(TargetState.STARTED);
+        workerConnector.transitionTo(TargetState.STARTED);
+        workerConnector.shutdown();
+
+        verifyAll();
+    }
+
+    @Test
+    public void testTransitionPausedToPaused() {
+        connector.initialize(EasyMock.notNull(ConnectorContext.class));
+        expectLastCall();
+
+        connector.start(CONFIG);
+        expectLastCall();
+
+        listener.onStartup(CONNECTOR);
+        expectLastCall();
+
+        connector.stop();
+        expectLastCall();
+
+        listener.onPause(CONNECTOR);
+        expectLastCall();
+
+        listener.onShutdown(CONNECTOR);
+        expectLastCall();
+
+        replayAll();
+
+        WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, ctx, listener);
+
+        workerConnector.initialize(CONNECTOR_CONFIG);
+        workerConnector.transitionTo(TargetState.STARTED);
+        workerConnector.transitionTo(TargetState.PAUSED);
+        workerConnector.transitionTo(TargetState.PAUSED);
+        workerConnector.shutdown();
+
+        verifyAll();
+    }
+
+    private static abstract class TestConnector extends Connector {
+    }
+
+}
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java
index 7bc83de..835e30f 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java
@@ -23,6 +23,7 @@
 import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.clients.consumer.OffsetAndMetadata;
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.WakeupException;
 import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.utils.Time;
 import org.apache.kafka.connect.data.Schema;
@@ -49,14 +50,15 @@
 import org.powermock.modules.junit4.PowerMockRunner;
 
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
+import static java.util.Arrays.asList;
 import static java.util.Collections.singleton;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
@@ -84,10 +86,12 @@
     private static final Map<String, String> TASK_PROPS = new HashMap<>();
     static {
         TASK_PROPS.put(SinkConnector.TOPICS_CONFIG, TOPIC);
+        TASK_PROPS.put(TaskConfig.TASK_CLASS_CONFIG, TestSinkTask.class.getName());
     }
-
+    private static final TaskConfig TASK_CONFIG = new TaskConfig(TASK_PROPS);
 
     private ConnectorTaskId taskId = new ConnectorTaskId("job", 0);
+    private TargetState initialState = TargetState.STARTED;
     private Time time;
     private WorkerSinkTask workerTask;
     @Mock
@@ -120,12 +124,91 @@
         workerConfig = new StandaloneConfig(workerProps);
         workerTask = PowerMock.createPartialMock(
                 WorkerSinkTask.class, new String[]{"createConsumer"},
-                taskId, sinkTask, statusListener, workerConfig, keyConverter, valueConverter, time);
+                taskId, sinkTask, statusListener, initialState, workerConfig, keyConverter, valueConverter, time);
 
         recordsReturned = 0;
     }
 
     @Test
+    public void testStartPaused() throws Exception {
+        workerTask = PowerMock.createPartialMock(
+                WorkerSinkTask.class, new String[]{"createConsumer"},
+                taskId, sinkTask, statusListener, TargetState.PAUSED, workerConfig, keyConverter, valueConverter, time);
+
+        expectInitializeTask();
+        expectPollInitialAssignment();
+
+        Set<TopicPartition> partitions = new HashSet<>(asList(TOPIC_PARTITION, TOPIC_PARTITION2));
+        EasyMock.expect(consumer.assignment()).andReturn(partitions);
+        consumer.pause(partitions);
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        workerTask.initialize(TASK_CONFIG);
+        workerTask.initializeAndStart();
+        workerTask.poll(Long.MAX_VALUE);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testPause() throws Exception {
+        expectInitializeTask();
+        expectPollInitialAssignment();
+
+        expectConsumerPoll(1);
+        expectConvertMessages(1);
+        sinkTask.put(EasyMock.<Collection<SinkRecord>>anyObject());
+        EasyMock.expectLastCall();
+
+        Set<TopicPartition> partitions = new HashSet<>(asList(TOPIC_PARTITION, TOPIC_PARTITION2));
+
+        // Pause
+        statusListener.onPause(taskId);
+        EasyMock.expectLastCall();
+        expectConsumerWakeup();
+        EasyMock.expect(consumer.assignment()).andReturn(partitions);
+        consumer.pause(partitions);
+        PowerMock.expectLastCall();
+
+        // No records returned
+        expectConsumerPoll(0);
+        sinkTask.put(Collections.<SinkRecord>emptyList());
+        EasyMock.expectLastCall();
+
+        // And unpause
+        statusListener.onResume(taskId);
+        EasyMock.expectLastCall();
+        expectConsumerWakeup();
+        EasyMock.expect(consumer.assignment()).andReturn(new HashSet<>(asList(TOPIC_PARTITION, TOPIC_PARTITION2)));
+        consumer.resume(singleton(TOPIC_PARTITION));
+        PowerMock.expectLastCall();
+        consumer.resume(singleton(TOPIC_PARTITION2));
+        PowerMock.expectLastCall();
+
+        expectConsumerPoll(1);
+        expectConvertMessages(1);
+        sinkTask.put(EasyMock.<Collection<SinkRecord>>anyObject());
+        EasyMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        workerTask.initialize(TASK_CONFIG);
+        workerTask.initializeAndStart();
+        workerTask.poll(Long.MAX_VALUE); // initial assignment
+        workerTask.poll(Long.MAX_VALUE); // fetch some data
+        workerTask.transitionTo(TargetState.PAUSED);
+        workerTask.poll(Long.MAX_VALUE); // wakeup
+        workerTask.poll(Long.MAX_VALUE); // now paused
+        workerTask.transitionTo(TargetState.STARTED);
+        workerTask.poll(Long.MAX_VALUE); // wakeup
+        workerTask.poll(Long.MAX_VALUE); // now unpaused
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
     public void testPollRedelivery() throws Exception {
         expectInitializeTask();
         expectPollInitialAssignment();
@@ -137,7 +220,7 @@
         sinkTask.put(EasyMock.capture(records));
         EasyMock.expectLastCall().andThrow(new RetriableException("retry"));
         // Pause
-        HashSet<TopicPartition> partitions = new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2));
+        HashSet<TopicPartition> partitions = new HashSet<>(asList(TOPIC_PARTITION, TOPIC_PARTITION2));
         EasyMock.expect(consumer.assignment()).andReturn(partitions);
         consumer.pause(partitions);
         PowerMock.expectLastCall();
@@ -155,7 +238,7 @@
 
         PowerMock.replayAll();
 
-        workerTask.initialize(TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         workerTask.initializeAndStart();
         workerTask.poll(Long.MAX_VALUE);
         workerTask.poll(Long.MAX_VALUE);
@@ -174,7 +257,7 @@
 
         PowerMock.replayAll();
 
-        workerTask.initialize(TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         workerTask.initializeAndStart();
         workerTask.poll(Long.MAX_VALUE);
         try {
@@ -197,7 +280,7 @@
 
         PowerMock.replayAll();
 
-        workerTask.initialize(TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         workerTask.initializeAndStart();
         workerTask.poll(Long.MAX_VALUE);
         try {
@@ -213,7 +296,7 @@
 
     private void expectInitializeTask() throws Exception {
         PowerMock.expectPrivate(workerTask, "createConsumer").andReturn(consumer);
-        consumer.subscribe(EasyMock.eq(Arrays.asList(TOPIC)), EasyMock.capture(rebalanceListener));
+        consumer.subscribe(EasyMock.eq(asList(TOPIC)), EasyMock.capture(rebalanceListener));
         PowerMock.expectLastCall();
 
         sinkTask.initialize(EasyMock.capture(sinkTaskContext));
@@ -223,7 +306,7 @@
     }
 
     private void expectRebalanceRevocationError(RuntimeException e) {
-        final List<TopicPartition> partitions = Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2);
+        final List<TopicPartition> partitions = asList(TOPIC_PARTITION, TOPIC_PARTITION2);
 
         sinkTask.close(new HashSet<>(partitions));
         EasyMock.expectLastCall().andThrow(e);
@@ -239,7 +322,7 @@
     }
 
     private void expectRebalanceAssignmentError(RuntimeException e) {
-        final List<TopicPartition> partitions = Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2);
+        final List<TopicPartition> partitions = asList(TOPIC_PARTITION, TOPIC_PARTITION2);
 
         sinkTask.close(new HashSet<>(partitions));
         EasyMock.expectLastCall();
@@ -268,7 +351,7 @@
     }
 
     private void expectPollInitialAssignment() {
-        final List<TopicPartition> partitions = Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2);
+        final List<TopicPartition> partitions = asList(TOPIC_PARTITION, TOPIC_PARTITION2);
 
         sinkTask.open(partitions);
         EasyMock.expectLastCall();
@@ -287,6 +370,12 @@
         EasyMock.expectLastCall();
     }
 
+    private void expectConsumerWakeup() {
+        consumer.wakeup();
+        EasyMock.expectLastCall();
+        EasyMock.expect(consumer.poll(EasyMock.anyLong())).andThrow(new WakeupException());
+    }
+
     private void expectConsumerPoll(final int numMessages) {
         EasyMock.expect(consumer.poll(EasyMock.anyLong())).andAnswer(
                 new IAnswer<ConsumerRecords<byte[], byte[]>>() {
@@ -309,4 +398,8 @@
         EasyMock.expect(keyConverter.toConnectData(TOPIC, RAW_KEY)).andReturn(new SchemaAndValue(KEY_SCHEMA, KEY)).times(numMessages);
         EasyMock.expect(valueConverter.toConnectData(TOPIC, RAW_VALUE)).andReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE)).times(numMessages);
     }
+
+    private abstract static class TestSinkTask extends SinkTask  {
+    }
+
 }
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java
index 25f0bf4..25dbff5 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java
@@ -89,9 +89,12 @@
     private static final Map<String, String> TASK_PROPS = new HashMap<>();
     static {
         TASK_PROPS.put(SinkConnector.TOPICS_CONFIG, TOPIC);
+        TASK_PROPS.put(TaskConfig.TASK_CLASS_CONFIG, TestSinkTask.class.getName());
     }
+    private static final TaskConfig TASK_CONFIG = new TaskConfig(TASK_PROPS);
 
     private ConnectorTaskId taskId = new ConnectorTaskId("job", 0);
+    private TargetState initialState = TargetState.STARTED;
     private Time time;
     @Mock private SinkTask sinkTask;
     private Capture<WorkerSinkTaskContext> sinkTaskContext = EasyMock.newCapture();
@@ -105,6 +108,7 @@
 
     private long recordsReturned;
 
+
     @SuppressWarnings("unchecked")
     @Override
     public void setup() {
@@ -121,7 +125,7 @@
         workerConfig = new StandaloneConfig(workerProps);
         workerTask = PowerMock.createPartialMock(
                 WorkerSinkTask.class, new String[]{"createConsumer"},
-                taskId, sinkTask, statusListener, workerConfig, keyConverter, valueConverter, time);
+                taskId, sinkTask, statusListener, initialState, workerConfig, keyConverter, valueConverter, time);
 
         recordsReturned = 0;
     }
@@ -136,7 +140,7 @@
 
         PowerMock.replayAll();
 
-        workerTask.initialize(TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         workerTask.initializeAndStart();
 
         // First iteration initializes partition assignment
@@ -147,7 +151,6 @@
             workerTask.iteration();
         }
         workerTask.stop();
-        workerTask.awaitStop(Long.MAX_VALUE);
         workerTask.close();
 
         // Verify contents match expected values, i.e. that they were translated properly. With max
@@ -180,7 +183,7 @@
 
         PowerMock.replayAll();
 
-        workerTask.initialize(TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         workerTask.initializeAndStart();
 
         // Initialize partition assignment
@@ -193,7 +196,6 @@
         // Commit finishes synchronously for testing so we can check this immediately
         assertEquals(0, workerTask.commitFailures());
         workerTask.stop();
-        workerTask.awaitStop(Long.MAX_VALUE);
         workerTask.close();
 
         assertEquals(2, capturedRecords.getValues().size());
@@ -220,7 +222,7 @@
 
         PowerMock.replayAll();
 
-        workerTask.initialize(TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         workerTask.initializeAndStart();
 
         // Initialize partition assignment
@@ -233,7 +235,6 @@
         assertEquals(1, workerTask.commitFailures());
         assertEquals(false, Whitebox.getInternalState(workerTask, "committing"));
         workerTask.stop();
-        workerTask.awaitStop(Long.MAX_VALUE);
         workerTask.close();
 
         PowerMock.verifyAll();
@@ -259,7 +260,7 @@
 
         PowerMock.replayAll();
 
-        workerTask.initialize(TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         workerTask.initializeAndStart();
 
         // Initialize partition assignment
@@ -274,7 +275,6 @@
         assertEquals(1, workerTask.commitFailures());
         assertEquals(false, Whitebox.getInternalState(workerTask, "committing"));
         workerTask.stop();
-        workerTask.awaitStop(Long.MAX_VALUE);
         workerTask.close();
 
         PowerMock.verifyAll();
@@ -292,7 +292,7 @@
 
         PowerMock.replayAll();
 
-        workerTask.initialize(TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         workerTask.initializeAndStart();
 
         // Initialize partition assignment
@@ -306,7 +306,6 @@
         assertEquals(1, workerTask.commitFailures());
         assertEquals(false, Whitebox.getInternalState(workerTask, "committing"));
         workerTask.stop();
-        workerTask.awaitStop(Long.MAX_VALUE);
         workerTask.close();
 
         PowerMock.verifyAll();
@@ -325,7 +324,7 @@
 
         PowerMock.replayAll();
 
-        workerTask.initialize(TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         workerTask.initializeAndStart();
 
         // Initialize partition assignment
@@ -342,7 +341,6 @@
         assertEquals(1, workerTask.commitFailures());
         assertEquals(false, Whitebox.getInternalState(workerTask, "committing"));
         workerTask.stop();
-        workerTask.awaitStop(Long.MAX_VALUE);
         workerTask.close();
 
         PowerMock.verifyAll();
@@ -405,13 +403,12 @@
 
         PowerMock.replayAll();
 
-        workerTask.initialize(TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         workerTask.initializeAndStart();
         workerTask.iteration();
         workerTask.iteration();
         workerTask.iteration();
         workerTask.stop();
-        workerTask.awaitStop(Long.MAX_VALUE);
         workerTask.close();
 
         PowerMock.verifyAll();
@@ -449,13 +446,12 @@
         expectStopTask(3);
         PowerMock.replayAll();
 
-        workerTask.initialize(TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         workerTask.initializeAndStart();
         workerTask.iteration();
         workerTask.iteration();
         workerTask.iteration();
         workerTask.stop();
-        workerTask.awaitStop(Long.MAX_VALUE);
         workerTask.close();
 
         PowerMock.verifyAll();
@@ -597,4 +593,7 @@
         return capturedCallback;
     }
 
+    private static abstract class TestSinkTask extends SinkTask {
+    }
+
 }
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java
index 3dd07a6..0d805da 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java
@@ -58,6 +58,7 @@
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
@@ -94,7 +95,12 @@
 
     private Capture<org.apache.kafka.clients.producer.Callback> producerCallbacks;
 
-    private static final Map<String, String> EMPTY_TASK_PROPS = Collections.emptyMap();
+    private static final Map<String, String> TASK_PROPS = new HashMap<>();
+    static {
+        TASK_PROPS.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName());
+    }
+    private static final TaskConfig TASK_CONFIG = new TaskConfig(TASK_PROPS);
+
     private static final List<SourceRecord> RECORDS = Arrays.asList(
             new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD)
     );
@@ -115,8 +121,94 @@
     }
 
     private void createWorkerTask() {
-        workerTask = new WorkerSourceTask(taskId, sourceTask, statusListener, keyConverter, valueConverter, producer,
-                offsetReader, offsetWriter, config, new SystemTime());
+        createWorkerTask(TargetState.STARTED);
+    }
+
+    private void createWorkerTask(TargetState initialState) {
+        workerTask = new WorkerSourceTask(taskId, sourceTask, statusListener, initialState, keyConverter,
+                valueConverter, producer, offsetReader, offsetWriter, config, new SystemTime());
+    }
+
+    @Test
+    public void testStartPaused() throws Exception {
+        final CountDownLatch startupLatch = new CountDownLatch(1);
+
+        createWorkerTask(TargetState.PAUSED);
+
+        sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class));
+        EasyMock.expectLastCall();
+        sourceTask.start(TASK_PROPS);
+        EasyMock.expectLastCall().andAnswer(new IAnswer<Void>() {
+            @Override
+            public Void answer() throws Throwable {
+                startupLatch.countDown();
+                return null;
+            }
+        });
+        statusListener.onPause(taskId);
+        EasyMock.expectLastCall();
+
+        // we shouldn't get any calls to poll()
+
+        sourceTask.stop();
+        EasyMock.expectLastCall();
+        expectOffsetFlush(true);
+
+        statusListener.onShutdown(taskId);
+        EasyMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        workerTask.initialize(TASK_CONFIG);
+        executor.submit(workerTask);
+        assertTrue(startupLatch.await(5, TimeUnit.SECONDS));
+        workerTask.stop();
+        assertTrue(workerTask.awaitStop(1000));
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testPause() throws Exception {
+        createWorkerTask();
+
+        sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class));
+        EasyMock.expectLastCall();
+        sourceTask.start(TASK_PROPS);
+        EasyMock.expectLastCall();
+        statusListener.onStartup(taskId);
+        EasyMock.expectLastCall();
+
+        AtomicInteger count = new AtomicInteger(0);
+        CountDownLatch pollLatch = expectPolls(10, count);
+        // In this test, we don't flush, so nothing goes any further than the offset writer
+
+        statusListener.onPause(taskId);
+        EasyMock.expectLastCall();
+
+        sourceTask.stop();
+        EasyMock.expectLastCall();
+        expectOffsetFlush(true);
+
+        statusListener.onShutdown(taskId);
+        EasyMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        workerTask.initialize(TASK_CONFIG);
+        executor.submit(workerTask);
+        awaitPolls(pollLatch);
+
+        workerTask.transitionTo(TargetState.PAUSED);
+
+        int priorCount = count.get();
+        Thread.sleep(100);
+        assertEquals(priorCount, count.get());
+
+        workerTask.stop();
+        assertTrue(workerTask.awaitStop(1000));
+
+        PowerMock.verifyAll();
     }
 
     @Test
@@ -125,7 +217,7 @@
 
         sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class));
         EasyMock.expectLastCall();
-        sourceTask.start(EMPTY_TASK_PROPS);
+        sourceTask.start(TASK_PROPS);
         EasyMock.expectLastCall();
         statusListener.onStartup(taskId);
         EasyMock.expectLastCall();
@@ -142,7 +234,7 @@
 
         PowerMock.replayAll();
 
-        workerTask.initialize(EMPTY_TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         executor.submit(workerTask);
         awaitPolls(pollLatch);
         workerTask.stop();
@@ -157,7 +249,7 @@
 
         sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class));
         EasyMock.expectLastCall();
-        sourceTask.start(EMPTY_TASK_PROPS);
+        sourceTask.start(TASK_PROPS);
         EasyMock.expectLastCall();
         statusListener.onStartup(taskId);
         EasyMock.expectLastCall();
@@ -175,7 +267,7 @@
 
         PowerMock.replayAll();
 
-        workerTask.initialize(EMPTY_TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         executor.submit(workerTask);
         awaitPolls(pollLatch);
         workerTask.stop();
@@ -191,7 +283,7 @@
 
         sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class));
         EasyMock.expectLastCall();
-        sourceTask.start(EMPTY_TASK_PROPS);
+        sourceTask.start(TASK_PROPS);
         EasyMock.expectLastCall();
         statusListener.onStartup(taskId);
         EasyMock.expectLastCall();
@@ -210,7 +302,7 @@
 
         PowerMock.replayAll();
 
-        workerTask.initialize(EMPTY_TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         executor.submit(workerTask);
         awaitPolls(pollLatch);
         assertTrue(workerTask.commitOffsets());
@@ -227,7 +319,7 @@
 
         sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class));
         EasyMock.expectLastCall();
-        sourceTask.start(EMPTY_TASK_PROPS);
+        sourceTask.start(TASK_PROPS);
         EasyMock.expectLastCall();
         statusListener.onStartup(taskId);
         EasyMock.expectLastCall();
@@ -245,7 +337,7 @@
 
         PowerMock.replayAll();
 
-        workerTask.initialize(EMPTY_TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         executor.submit(workerTask);
         awaitPolls(pollLatch);
         assertTrue(workerTask.commitOffsets());
@@ -317,7 +409,7 @@
 
         sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class));
         EasyMock.expectLastCall();
-        sourceTask.start(EMPTY_TASK_PROPS);
+        sourceTask.start(TASK_PROPS);
         EasyMock.expectLastCall();
 
         statusListener.onStartup(taskId);
@@ -336,7 +428,7 @@
 
         PowerMock.replayAll();
 
-        workerTask.initialize(EMPTY_TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         executor.submit(workerTask);
         // Stopping immediately while the other thread has work to do should result in no polling, no offset commits,
         // exiting the work thread immediately, and the stop() method will be invoked in the background thread since it
@@ -348,8 +440,8 @@
         PowerMock.verifyAll();
     }
 
-    private CountDownLatch expectPolls(int count) throws InterruptedException {
-        final CountDownLatch latch = new CountDownLatch(count);
+    private CountDownLatch expectPolls(int minimum, final AtomicInteger count) throws InterruptedException {
+        final CountDownLatch latch = new CountDownLatch(minimum);
         // Note that we stub these to allow any number of calls because the thread will continue to
         // run. The count passed in + latch returned just makes sure we get *at least* that number of
         // calls
@@ -357,6 +449,7 @@
                 .andStubAnswer(new IAnswer<List<SourceRecord>>() {
                     @Override
                     public List<SourceRecord> answer() throws Throwable {
+                        count.incrementAndGet();
                         latch.countDown();
                         return RECORDS;
                     }
@@ -366,6 +459,10 @@
         return latch;
     }
 
+    private CountDownLatch expectPolls(int count) throws InterruptedException {
+        return expectPolls(count, new AtomicInteger());
+    }
+
     @SuppressWarnings("unchecked")
     private void expectSendRecordSyncFailure(Throwable error) throws InterruptedException {
         expectConvertKeyValue(false);
@@ -446,8 +543,8 @@
             convertValueExpect.andReturn(SERIALIZED_RECORD);
     }
 
-    private void awaitPolls(CountDownLatch latch) throws InterruptedException {
-        latch.await(1000, TimeUnit.MILLISECONDS);
+    private boolean awaitPolls(CountDownLatch latch) throws InterruptedException {
+        return latch.await(1000, TimeUnit.MILLISECONDS);
     }
 
     @SuppressWarnings("unchecked")
@@ -469,4 +566,7 @@
         }
     }
 
+    private abstract static class TestSourceTask extends SourceTask {
+    }
+
 }
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java
index 20e3fe2..36803db 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java
@@ -16,12 +16,13 @@
  **/
 package org.apache.kafka.connect.runtime;
 
+import org.apache.kafka.connect.sink.SinkTask;
 import org.apache.kafka.connect.util.ConnectorTaskId;
 import org.easymock.EasyMock;
 import org.easymock.IAnswer;
 import org.junit.Test;
 
-import java.util.Collections;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.CountDownLatch;
 
@@ -32,7 +33,11 @@
 
 public class WorkerTaskTest {
 
-    private static final Map<String, String> EMPTY_TASK_PROPS = Collections.emptyMap();
+    private static final Map<String, String> TASK_PROPS = new HashMap<>();
+    static {
+        TASK_PROPS.put(TaskConfig.TASK_CLASS_CONFIG, TestSinkTask.class.getName());
+    }
+    private static final TaskConfig TASK_CONFIG = new TaskConfig(TASK_PROPS);
 
     @Test
     public void standardStartup() {
@@ -41,14 +46,14 @@
         TaskStatus.Listener statusListener = EasyMock.createMock(TaskStatus.Listener.class);
 
         WorkerTask workerTask = partialMockBuilder(WorkerTask.class)
-                .withConstructor(ConnectorTaskId.class, TaskStatus.Listener.class)
-                .withArgs(taskId, statusListener)
+                .withConstructor(ConnectorTaskId.class, TaskStatus.Listener.class, TargetState.class)
+                .withArgs(taskId, statusListener, TargetState.STARTED)
                 .addMockedMethod("initialize")
                 .addMockedMethod("execute")
                 .addMockedMethod("close")
                 .createStrictMock();
 
-        workerTask.initialize(EMPTY_TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         expectLastCall();
 
         workerTask.execute();
@@ -65,7 +70,7 @@
 
         replay(workerTask);
 
-        workerTask.initialize(EMPTY_TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         workerTask.run();
         workerTask.stop();
         workerTask.awaitStop(1000L);
@@ -80,14 +85,14 @@
         TaskStatus.Listener statusListener = EasyMock.createMock(TaskStatus.Listener.class);
 
         WorkerTask workerTask = partialMockBuilder(WorkerTask.class)
-                .withConstructor(ConnectorTaskId.class, TaskStatus.Listener.class)
-                .withArgs(taskId, statusListener)
+                .withConstructor(ConnectorTaskId.class, TaskStatus.Listener.class, TargetState.class)
+                .withArgs(taskId, statusListener, TargetState.STARTED)
                 .addMockedMethod("initialize")
                 .addMockedMethod("execute")
                 .addMockedMethod("close")
                 .createStrictMock();
 
-        workerTask.initialize(EMPTY_TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         EasyMock.expectLastCall();
 
         workerTask.close();
@@ -95,7 +100,7 @@
 
         replay(workerTask);
 
-        workerTask.initialize(EMPTY_TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         workerTask.stop();
         workerTask.awaitStop(1000L);
 
@@ -112,8 +117,8 @@
         TaskStatus.Listener statusListener = EasyMock.createMock(TaskStatus.Listener.class);
 
         WorkerTask workerTask = partialMockBuilder(WorkerTask.class)
-                .withConstructor(ConnectorTaskId.class, TaskStatus.Listener.class)
-                .withArgs(taskId, statusListener)
+                .withConstructor(ConnectorTaskId.class, TaskStatus.Listener.class, TargetState.class)
+                .withArgs(taskId, statusListener, TargetState.STARTED)
                 .addMockedMethod("initialize")
                 .addMockedMethod("execute")
                 .addMockedMethod("close")
@@ -130,7 +135,7 @@
             }
         };
 
-        workerTask.initialize(EMPTY_TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         EasyMock.expectLastCall();
 
         workerTask.execute();
@@ -152,7 +157,7 @@
 
         replay(workerTask);
 
-        workerTask.initialize(EMPTY_TASK_PROPS);
+        workerTask.initialize(TASK_CONFIG);
         workerTask.run();
 
         workerTask.stop();
@@ -163,4 +168,7 @@
         verify(workerTask);
     }
 
+    private static abstract class TestSinkTask extends SinkTask {
+    }
+
 }
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java
index 557d789..2004c99 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java
@@ -102,7 +102,7 @@
         props.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_ID);
         props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, WorkerTestConnector.class.getName());
 
-        connector.initialize(ctx);
+        connector.initialize(EasyMock.anyObject(ConnectorContext.class));
         EasyMock.expectLastCall();
         connector.start(props);
         EasyMock.expectLastCall();
@@ -126,10 +126,10 @@
 
         ConnectorConfig config = new ConnectorConfig(props);
         assertEquals(Collections.emptySet(), worker.connectorNames());
-        worker.startConnector(config, ctx, connectorStatusListener);
+        worker.startConnector(config, ctx, connectorStatusListener, TargetState.STARTED);
         assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames());
         try {
-            worker.startConnector(config, ctx, connectorStatusListener);
+            worker.startConnector(config, ctx, connectorStatusListener, TargetState.STARTED);
             fail("Should have thrown exception when trying to add connector with same name.");
         } catch (ConnectException e) {
             // expected
@@ -160,7 +160,7 @@
         props.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_ID);
         props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, "WorkerTestConnector");
 
-        connector.initialize(ctx);
+        connector.initialize(EasyMock.anyObject(ConnectorContext.class));
         EasyMock.expectLastCall();
         connector.start(props);
         EasyMock.expectLastCall();
@@ -184,7 +184,7 @@
 
         ConnectorConfig config = new ConnectorConfig(props);
         assertEquals(Collections.emptySet(), worker.connectorNames());
-        worker.startConnector(config, ctx, connectorStatusListener);
+        worker.startConnector(config, ctx, connectorStatusListener, TargetState.STARTED);
         assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames());
 
         worker.stopConnector(CONNECTOR_ID);
@@ -213,7 +213,7 @@
         props.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_ID);
         props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, "WorkerTest");
 
-        connector.initialize(ctx);
+        connector.initialize(EasyMock.anyObject(ConnectorContext.class));
         EasyMock.expectLastCall();
         connector.start(props);
         EasyMock.expectLastCall();
@@ -237,7 +237,7 @@
 
         ConnectorConfig config = new ConnectorConfig(props);
         assertEquals(Collections.emptySet(), worker.connectorNames());
-        worker.startConnector(config, ctx, connectorStatusListener);
+        worker.startConnector(config, ctx, connectorStatusListener, TargetState.STARTED);
         assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames());
 
         worker.stopConnector(CONNECTOR_ID);
@@ -279,7 +279,7 @@
         props.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_ID);
         props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, WorkerTestConnector.class.getName());
 
-        connector.initialize(ctx);
+        connector.initialize(EasyMock.anyObject(ConnectorContext.class));
         EasyMock.expectLastCall();
         connector.start(props);
         EasyMock.expectLastCall();
@@ -309,10 +309,10 @@
 
         ConnectorConfig config = new ConnectorConfig(props);
         assertEquals(Collections.emptySet(), worker.connectorNames());
-        worker.startConnector(config, ctx, connectorStatusListener);
+        worker.startConnector(config, ctx, connectorStatusListener, TargetState.STARTED);
         assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames());
         try {
-            worker.startConnector(config, ctx, connectorStatusListener);
+            worker.startConnector(config, ctx, connectorStatusListener, TargetState.STARTED);
             fail("Should have thrown exception when trying to add connector with same name.");
         } catch (ConnectException e) {
             // expected
@@ -351,6 +351,7 @@
                 WorkerSourceTask.class, EasyMock.eq(TASK_ID),
                 EasyMock.eq(task),
                 EasyMock.anyObject(TaskStatus.Listener.class),
+                EasyMock.eq(TargetState.STARTED),
                 EasyMock.anyObject(Converter.class),
                 EasyMock.anyObject(Converter.class),
                 EasyMock.anyObject(KafkaProducer.class),
@@ -361,7 +362,7 @@
                 .andReturn(workerTask);
         Map<String, String> origProps = new HashMap<>();
         origProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName());
-        workerTask.initialize(origProps);
+        workerTask.initialize(new TaskConfig(origProps));
         EasyMock.expectLastCall();
         workerTask.run();
         EasyMock.expectLastCall();
@@ -379,7 +380,7 @@
         worker = new Worker(WORKER_ID, new MockTime(), config, offsetBackingStore);
         worker.start();
         assertEquals(Collections.emptySet(), worker.taskIds());
-        worker.startTask(TASK_ID, new TaskConfig(origProps), taskStatusListener);
+        worker.startTask(TASK_ID, new TaskConfig(origProps), taskStatusListener, TargetState.STARTED);
         assertEquals(new HashSet<>(Arrays.asList(TASK_ID)), worker.taskIds());
         worker.stopAndAwaitTask(TASK_ID);
         assertEquals(Collections.emptySet(), worker.taskIds());
@@ -418,6 +419,7 @@
                 WorkerSourceTask.class, EasyMock.eq(TASK_ID),
                 EasyMock.eq(task),
                 EasyMock.anyObject(TaskStatus.Listener.class),
+                EasyMock.eq(TargetState.STARTED),
                 EasyMock.anyObject(Converter.class),
                 EasyMock.anyObject(Converter.class),
                 EasyMock.anyObject(KafkaProducer.class),
@@ -428,7 +430,7 @@
                 .andReturn(workerTask);
         Map<String, String> origProps = new HashMap<>();
         origProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName());
-        workerTask.initialize(origProps);
+        workerTask.initialize(new TaskConfig(origProps));
         EasyMock.expectLastCall();
         workerTask.run();
         EasyMock.expectLastCall();
@@ -447,7 +449,7 @@
 
         worker = new Worker(WORKER_ID, new MockTime(), config, offsetBackingStore);
         worker.start();
-        worker.startTask(TASK_ID, new TaskConfig(origProps), taskStatusListener);
+        worker.startTask(TASK_ID, new TaskConfig(origProps), taskStatusListener, TargetState.STARTED);
         worker.stop();
 
         PowerMock.verifyAll();
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java
index aa747f6..e62b663 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java
@@ -21,8 +21,10 @@
 import org.apache.kafka.common.utils.MockTime;
 import org.apache.kafka.connect.connector.ConnectorContext;
 import org.apache.kafka.connect.errors.AlreadyExistsException;
+import org.apache.kafka.connect.errors.NotFoundException;
 import org.apache.kafka.connect.runtime.ConnectorConfig;
 import org.apache.kafka.connect.runtime.Herder;
+import org.apache.kafka.connect.runtime.TargetState;
 import org.apache.kafka.connect.runtime.TaskConfig;
 import org.apache.kafka.connect.runtime.Worker;
 import org.apache.kafka.connect.runtime.WorkerConfig;
@@ -30,12 +32,12 @@
 import org.apache.kafka.connect.runtime.rest.entities.TaskInfo;
 import org.apache.kafka.connect.source.SourceConnector;
 import org.apache.kafka.connect.source.SourceTask;
-import org.apache.kafka.connect.storage.KafkaConfigStorage;
+import org.apache.kafka.connect.storage.ConfigBackingStore;
+import org.apache.kafka.connect.storage.KafkaConfigBackingStore;
 import org.apache.kafka.connect.storage.StatusBackingStore;
 import org.apache.kafka.connect.util.Callback;
 import org.apache.kafka.connect.util.ConnectorTaskId;
 import org.apache.kafka.connect.util.FutureCallback;
-import org.apache.kafka.connect.util.TestFuture;
 import org.easymock.Capture;
 import org.easymock.EasyMock;
 import org.easymock.IAnswer;
@@ -47,7 +49,6 @@
 import org.powermock.core.classloader.annotations.PowerMockIgnore;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
-import org.powermock.reflect.Whitebox;
 
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -56,10 +57,13 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(DistributedHerder.class)
@@ -123,13 +127,15 @@
         TASK_CONFIGS_MAP.put(TASK2, TASK_CONFIG);
     }
     private static final ClusterConfigState SNAPSHOT = new ClusterConfigState(1, Collections.singletonMap(CONN1, 3),
-            Collections.singletonMap(CONN1, CONN1_CONFIG), TASK_CONFIGS_MAP, Collections.<String>emptySet());
+            Collections.singletonMap(CONN1, CONN1_CONFIG), Collections.singletonMap(CONN1, TargetState.STARTED),
+            TASK_CONFIGS_MAP, Collections.<String>emptySet());
     private static final ClusterConfigState SNAPSHOT_UPDATED_CONN1_CONFIG = new ClusterConfigState(1, Collections.singletonMap(CONN1, 3),
-            Collections.singletonMap(CONN1, CONN1_CONFIG_UPDATED), TASK_CONFIGS_MAP, Collections.<String>emptySet());
+            Collections.singletonMap(CONN1, CONN1_CONFIG_UPDATED), Collections.singletonMap(CONN1, TargetState.STARTED),
+            TASK_CONFIGS_MAP, Collections.<String>emptySet());
 
     private static final String WORKER_ID = "localhost:8083";
 
-    @Mock private KafkaConfigStorage configStorage;
+    @Mock private KafkaConfigBackingStore configStorage;
     @Mock private StatusBackingStore statusBackingStore;
     @Mock private WorkerGroupMember member;
     private MockTime time;
@@ -137,8 +143,7 @@
     @Mock private Worker worker;
     @Mock private Callback<Herder.Created<ConnectorInfo>> putConnectorCallback;
 
-    private Callback<String> connectorConfigCallback;
-    private Callback<List<ConnectorTaskId>> taskConfigCallback;
+    private ConfigBackingStore.UpdateListener configUpdateListener;
     private WorkerRebalanceListener rebalanceListener;
 
     @Before
@@ -148,24 +153,27 @@
         time = new MockTime();
 
         herder = PowerMock.createPartialMock(DistributedHerder.class, new String[]{"backoff", "updateDeletedConnectorStatus"},
-                new DistributedConfig(HERDER_CONFIG), WORKER_ID, worker, statusBackingStore, configStorage, member, MEMBER_URL, time);
-        connectorConfigCallback = Whitebox.invokeMethod(herder, "connectorConfigCallback");
-        taskConfigCallback = Whitebox.invokeMethod(herder, "taskConfigCallback");
-        rebalanceListener = Whitebox.invokeMethod(herder, "rebalanceListener");
+                new DistributedConfig(HERDER_CONFIG), worker, WORKER_ID, statusBackingStore, configStorage, member, MEMBER_URL, time);
+
+        configUpdateListener = herder.new ConfigUpdateListener();
+        rebalanceListener = herder.new RebalanceListener();
+
         PowerMock.expectPrivate(herder, "updateDeletedConnectorStatus").andVoid().anyTimes();
     }
 
     @Test
-    public void testJoinAssignment() {
+    public void testJoinAssignment() throws Exception {
         // Join group and get assignment
         EasyMock.expect(member.memberId()).andStubReturn("member");
         expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1));
         expectPostRebalanceCatchup(SNAPSHOT);
         worker.startConnector(EasyMock.<ConnectorConfig>anyObject(), EasyMock.<ConnectorContext>anyObject(),
-                EasyMock.eq(herder));
+                EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
         PowerMock.expectLastCall();
+        EasyMock.expect(worker.isRunning(CONN1)).andReturn(true);
+
         EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS);
-        worker.startTask(EasyMock.eq(TASK1), EasyMock.<TaskConfig>anyObject(), EasyMock.eq(herder));
+        worker.startTask(EasyMock.eq(TASK1), EasyMock.<TaskConfig>anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
         PowerMock.expectLastCall();
         member.poll(EasyMock.anyInt());
         PowerMock.expectLastCall();
@@ -178,16 +186,17 @@
     }
 
     @Test
-    public void testRebalance() {
+    public void testRebalance() throws Exception {
         // Join group and get assignment
         EasyMock.expect(member.memberId()).andStubReturn("member");
         expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1));
         expectPostRebalanceCatchup(SNAPSHOT);
         worker.startConnector(EasyMock.<ConnectorConfig>anyObject(), EasyMock.<ConnectorContext>anyObject(),
-                EasyMock.eq(herder));
+                EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
         PowerMock.expectLastCall();
+        EasyMock.expect(worker.isRunning(CONN1)).andReturn(true);
         EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS);
-        worker.startTask(EasyMock.eq(TASK1), EasyMock.<TaskConfig>anyObject(), EasyMock.eq(herder));
+        worker.startTask(EasyMock.eq(TASK1), EasyMock.<TaskConfig>anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
         PowerMock.expectLastCall();
         member.poll(EasyMock.anyInt());
         PowerMock.expectLastCall();
@@ -197,8 +206,9 @@
 
         // and the new assignment started
         worker.startConnector(EasyMock.<ConnectorConfig>anyObject(), EasyMock.<ConnectorContext>anyObject(),
-                EasyMock.eq(herder));
+                EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
         PowerMock.expectLastCall();
+        EasyMock.expect(worker.isRunning(CONN1)).andReturn(true);
         EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS);
         member.poll(EasyMock.anyInt());
         PowerMock.expectLastCall();
@@ -212,6 +222,44 @@
     }
 
     @Test
+    public void testRebalanceFailedConnector() throws Exception {
+        // Join group and get assignment
+        EasyMock.expect(member.memberId()).andStubReturn("member");
+        expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1));
+        expectPostRebalanceCatchup(SNAPSHOT);
+        worker.startConnector(EasyMock.<ConnectorConfig>anyObject(), EasyMock.<ConnectorContext>anyObject(),
+                EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
+        PowerMock.expectLastCall();
+        EasyMock.expect(worker.isRunning(CONN1)).andReturn(true);
+        EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS);
+        worker.startTask(EasyMock.eq(TASK1), EasyMock.<TaskConfig>anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
+        PowerMock.expectLastCall();
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        expectRebalance(Arrays.asList(CONN1), Arrays.asList(TASK1), ConnectProtocol.Assignment.NO_ERROR,
+                1, Arrays.asList(CONN1), Arrays.<ConnectorTaskId>asList());
+
+        // and the new assignment started
+        worker.startConnector(EasyMock.<ConnectorConfig>anyObject(), EasyMock.<ConnectorContext>anyObject(),
+                EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
+        PowerMock.expectLastCall();
+        EasyMock.expect(worker.isRunning(CONN1)).andReturn(false);
+
+        // worker is not running, so we should see no call to connectorTaskConfigs()
+
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        herder.tick();
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
     public void testHaltCleansUpWorker() {
         EasyMock.expect(worker.connectorNames()).andReturn(Collections.singleton(CONN1));
         worker.stopConnector(CONN1);
@@ -227,6 +275,8 @@
         PowerMock.expectLastCall();
         statusBackingStore.stop();
         PowerMock.expectLastCall();
+        worker.stop();
+        PowerMock.expectLastCall();
 
         PowerMock.replayAll();
 
@@ -291,14 +341,15 @@
         expectRebalance(1, Arrays.asList(CONN1), Collections.<ConnectorTaskId>emptyList());
         expectPostRebalanceCatchup(SNAPSHOT);
         worker.startConnector(EasyMock.<ConnectorConfig>anyObject(), EasyMock.<ConnectorContext>anyObject(),
-                EasyMock.eq(herder));
+                EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
         PowerMock.expectLastCall();
+        EasyMock.expect(worker.isRunning(CONN1)).andReturn(true);
         EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS);
 
         // And delete the connector
         member.wakeup();
         PowerMock.expectLastCall();
-        configStorage.putConnectorConfig(CONN1, null);
+        configStorage.removeConnectorConfig(CONN1);
         PowerMock.expectLastCall();
         putConnectorCallback.onCompletion(null, new Herder.Created<ConnectorInfo>(false, null));
         PowerMock.expectLastCall();
@@ -315,6 +366,305 @@
     }
 
     @Test
+    public void testRestartConnector() throws Exception {
+        EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andStubReturn(TASK_CONFIGS);
+
+        // get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        expectRebalance(1, Collections.singletonList(CONN1), Collections.<ConnectorTaskId>emptyList());
+        expectPostRebalanceCatchup(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+        worker.startConnector(EasyMock.<ConnectorConfig>anyObject(), EasyMock.<ConnectorContext>anyObject(),
+                EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
+        PowerMock.expectLastCall();
+        EasyMock.expect(worker.isRunning(CONN1)).andReturn(true);
+
+        // now handle the connector restart
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        EasyMock.expect(worker.ownsConnector(CONN1)).andReturn(true);
+
+        worker.stopConnector(CONN1);
+        PowerMock.expectLastCall();
+        worker.startConnector(EasyMock.<ConnectorConfig>anyObject(), EasyMock.<ConnectorContext>anyObject(),
+                EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
+        PowerMock.expectLastCall();
+        EasyMock.expect(worker.isRunning(CONN1)).andReturn(true);
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Void> callback = new FutureCallback<>();
+        herder.restartConnector(CONN1, callback);
+        herder.tick();
+        callback.get(1000L, TimeUnit.MILLISECONDS);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testRestartUnknownConnector() throws Exception {
+        // get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        expectRebalance(1, Collections.<String>emptyList(), Collections.<ConnectorTaskId>emptyList());
+        expectPostRebalanceCatchup(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+
+        // now handle the connector restart
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Void> callback = new FutureCallback<>();
+        herder.restartConnector(CONN2, callback);
+        herder.tick();
+        try {
+            callback.get(1000L, TimeUnit.MILLISECONDS);
+            fail("Expected NotLeaderException to be raised");
+        } catch (ExecutionException e) {
+            assertTrue(e.getCause() instanceof NotFoundException);
+        }
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testRestartConnectorRedirectToLeader() throws Exception {
+        // get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("member");
+        expectRebalance(1, Collections.<String>emptyList(), Collections.<ConnectorTaskId>emptyList());
+        expectPostRebalanceCatchup(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // now handle the connector restart
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        EasyMock.expect(worker.ownsConnector(CONN1)).andReturn(false);
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Void> callback = new FutureCallback<>();
+        herder.restartConnector(CONN1, callback);
+        herder.tick();
+
+        try {
+            callback.get(1000L, TimeUnit.MILLISECONDS);
+            fail("Expected NotLeaderException to be raised");
+        } catch (ExecutionException e) {
+            assertTrue(e.getCause() instanceof NotLeaderException);
+        }
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testRestartConnectorRedirectToOwner() throws Exception {
+        // get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        expectRebalance(1, Collections.<String>emptyList(), Collections.<ConnectorTaskId>emptyList());
+        expectPostRebalanceCatchup(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // now handle the connector restart
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        String ownerUrl = "ownerUrl";
+        EasyMock.expect(worker.ownsConnector(CONN1)).andReturn(false);
+        EasyMock.expect(member.ownerUrl(CONN1)).andReturn(ownerUrl);
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Void> callback = new FutureCallback<>();
+        herder.restartConnector(CONN1, callback);
+        herder.tick();
+
+        try {
+            callback.get(1000L, TimeUnit.MILLISECONDS);
+            fail("Expected NotLeaderException to be raised");
+        } catch (ExecutionException e) {
+            assertTrue(e.getCause() instanceof NotAssignedException);
+            NotAssignedException notAssignedException = (NotAssignedException) e.getCause();
+            assertEquals(ownerUrl, notAssignedException.forwardUrl());
+        }
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testRestartTask() throws Exception {
+        EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andStubReturn(TASK_CONFIGS);
+
+        // get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        expectRebalance(1, Collections.<String>emptyList(), Collections.singletonList(TASK0));
+        expectPostRebalanceCatchup(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+        worker.startTask(EasyMock.eq(TASK0), EasyMock.<TaskConfig>anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
+        PowerMock.expectLastCall();
+
+        // now handle the task restart
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        EasyMock.expect(worker.ownsTask(TASK0)).andReturn(true);
+
+        worker.stopAndAwaitTask(TASK0);
+        PowerMock.expectLastCall();
+        worker.startTask(EasyMock.eq(TASK0), EasyMock.<TaskConfig>anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Void> callback = new FutureCallback<>();
+        herder.restartTask(TASK0, callback);
+        herder.tick();
+        callback.get(1000L, TimeUnit.MILLISECONDS);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testRestartUnknownTask() throws Exception {
+        // get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("member");
+        expectRebalance(1, Collections.<String>emptyList(), Collections.<ConnectorTaskId>emptyList());
+        expectPostRebalanceCatchup(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        FutureCallback<Void> callback = new FutureCallback<>();
+        herder.tick();
+        herder.restartTask(new ConnectorTaskId("blah", 0), callback);
+        herder.tick();
+
+        try {
+            callback.get(1000L, TimeUnit.MILLISECONDS);
+            fail("Expected NotLeaderException to be raised");
+        } catch (ExecutionException e) {
+            assertTrue(e.getCause() instanceof NotFoundException);
+        }
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testRestartTaskRedirectToLeader() throws Exception {
+        // get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("member");
+        expectRebalance(1, Collections.<String>emptyList(), Collections.<ConnectorTaskId>emptyList());
+        expectPostRebalanceCatchup(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // now handle the task restart
+        EasyMock.expect(worker.ownsTask(TASK0)).andReturn(false);
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Void> callback = new FutureCallback<>();
+        herder.restartTask(TASK0, callback);
+        herder.tick();
+
+        try {
+            callback.get(1000L, TimeUnit.MILLISECONDS);
+            fail("Expected NotLeaderException to be raised");
+        } catch (ExecutionException e) {
+            assertTrue(e.getCause() instanceof NotLeaderException);
+        }
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testRestartTaskRedirectToOwner() throws Exception {
+        // get the initial assignment
+        EasyMock.expect(member.memberId()).andStubReturn("leader");
+        expectRebalance(1, Collections.<String>emptyList(), Collections.<ConnectorTaskId>emptyList());
+        expectPostRebalanceCatchup(SNAPSHOT);
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        // now handle the task restart
+        String ownerUrl = "ownerUrl";
+        EasyMock.expect(worker.ownsTask(TASK0)).andReturn(false);
+        EasyMock.expect(member.ownerUrl(TASK0)).andReturn(ownerUrl);
+        member.wakeup();
+        PowerMock.expectLastCall();
+        member.ensureActive();
+        PowerMock.expectLastCall();
+        member.poll(EasyMock.anyInt());
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.tick();
+        FutureCallback<Void> callback = new FutureCallback<>();
+        herder.restartTask(TASK0, callback);
+        herder.tick();
+
+        try {
+            callback.get(1000L, TimeUnit.MILLISECONDS);
+            fail("Expected NotLeaderException to be raised");
+        } catch (ExecutionException e) {
+            assertTrue(e.getCause() instanceof NotAssignedException);
+            NotAssignedException notAssignedException = (NotAssignedException) e.getCause();
+            assertEquals(ownerUrl, notAssignedException.forwardUrl());
+        }
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
     public void testConnectorConfigAdded() {
         // If a connector was added, we need to rebalance
         EasyMock.expect(member.memberId()).andStubReturn("member");
@@ -336,7 +686,8 @@
         expectRebalance(Collections.<String>emptyList(), Collections.<ConnectorTaskId>emptyList(),
                 ConnectProtocol.Assignment.NO_ERROR, 1, Arrays.asList(CONN1), Collections.<ConnectorTaskId>emptyList());
         worker.startConnector(EasyMock.<ConnectorConfig>anyObject(), EasyMock.<ConnectorContext>anyObject(),
-                EasyMock.eq(herder));
+                EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
+        EasyMock.expect(worker.isRunning(CONN1)).andReturn(true);
         PowerMock.expectLastCall();
         EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS);
         member.poll(EasyMock.anyInt());
@@ -345,7 +696,7 @@
         PowerMock.replayAll();
 
         herder.tick(); // join
-        connectorConfigCallback.onCompletion(null, CONN1); // read updated config
+        configUpdateListener.onConnectorConfigUpdate(CONN1); // read updated config
         herder.tick(); // apply config
         herder.tick(); // do rebalance
 
@@ -353,7 +704,7 @@
     }
 
     @Test
-    public void testConnectorConfigUpdate() {
+    public void testConnectorConfigUpdate() throws Exception {
         // Connector config can be applied without any rebalance
 
         EasyMock.expect(member.memberId()).andStubReturn("member");
@@ -363,8 +714,9 @@
         expectRebalance(1, Arrays.asList(CONN1), Collections.<ConnectorTaskId>emptyList());
         expectPostRebalanceCatchup(SNAPSHOT);
         worker.startConnector(EasyMock.<ConnectorConfig>anyObject(), EasyMock.<ConnectorContext>anyObject(),
-                EasyMock.eq(herder));
+                EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
         PowerMock.expectLastCall();
+        EasyMock.expect(worker.isRunning(CONN1)).andReturn(true);
         EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS);
         member.poll(EasyMock.anyInt());
         PowerMock.expectLastCall();
@@ -377,8 +729,9 @@
         worker.stopConnector(CONN1);
         PowerMock.expectLastCall();
         worker.startConnector(EasyMock.<ConnectorConfig>anyObject(), EasyMock.<ConnectorContext>anyObject(),
-                EasyMock.eq(herder));
+                EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
         PowerMock.expectLastCall();
+        EasyMock.expect(worker.isRunning(CONN1)).andReturn(true);
         EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS);
         member.poll(EasyMock.anyInt());
         PowerMock.expectLastCall();
@@ -386,7 +739,7 @@
         PowerMock.replayAll();
 
         herder.tick(); // join
-        connectorConfigCallback.onCompletion(null, CONN1); // read updated config
+        configUpdateListener.onConnectorConfigUpdate(CONN1); // read updated config
         herder.tick(); // apply config
 
         PowerMock.verifyAll();
@@ -414,7 +767,7 @@
         expectRebalance(Collections.<String>emptyList(), Collections.<ConnectorTaskId>emptyList(),
                 ConnectProtocol.Assignment.NO_ERROR, 1, Collections.<String>emptyList(),
                 Arrays.asList(TASK0));
-        worker.startTask(EasyMock.eq(TASK0), EasyMock.<TaskConfig>anyObject(), EasyMock.eq(herder));
+        worker.startTask(EasyMock.eq(TASK0), EasyMock.<TaskConfig>anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
         PowerMock.expectLastCall();
         member.poll(EasyMock.anyInt());
         PowerMock.expectLastCall();
@@ -422,7 +775,7 @@
         PowerMock.replayAll();
 
         herder.tick(); // join
-        taskConfigCallback.onCompletion(null, Arrays.asList(TASK0, TASK1, TASK2)); // read updated config
+        configUpdateListener.onTaskConfigUpdate(Arrays.asList(TASK0, TASK1, TASK2)); // read updated config
         herder.tick(); // apply config
         herder.tick(); // do rebalance
 
@@ -437,9 +790,8 @@
                 ConnectProtocol.Assignment.CONFIG_MISMATCH, 1, Collections.<String>emptyList(),
                 Collections.<ConnectorTaskId>emptyList());
         // Reading to end of log times out
-        TestFuture<Void> readToEndFuture = new TestFuture<>();
-        readToEndFuture.resolveOnGet(new TimeoutException());
-        EasyMock.expect(configStorage.readToEnd()).andReturn(readToEndFuture);
+        configStorage.refresh(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class));
+        EasyMock.expectLastCall().andThrow(new TimeoutException());
         member.maybeLeaveGroup();
         EasyMock.expectLastCall();
         PowerMock.expectPrivate(herder, "backoff", DistributedConfig.WORKER_UNSYNC_BACKOFF_MS_DEFAULT);
@@ -450,11 +802,12 @@
         expectPostRebalanceCatchup(SNAPSHOT);
 
         worker.startConnector(EasyMock.<ConnectorConfig>anyObject(), EasyMock.<ConnectorContext>anyObject(),
-                EasyMock.eq(herder));
+                EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
         PowerMock.expectLastCall();
         EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS);
-        worker.startTask(EasyMock.eq(TASK1), EasyMock.<TaskConfig>anyObject(), EasyMock.eq(herder));
+        worker.startTask(EasyMock.eq(TASK1), EasyMock.<TaskConfig>anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
         PowerMock.expectLastCall();
+        EasyMock.expect(worker.isRunning(CONN1)).andReturn(true);
         member.poll(EasyMock.anyInt());
         PowerMock.expectLastCall();
 
@@ -515,8 +868,9 @@
         expectRebalance(1, Arrays.asList(CONN1), Collections.<ConnectorTaskId>emptyList());
         expectPostRebalanceCatchup(SNAPSHOT);
         worker.startConnector(EasyMock.<ConnectorConfig>anyObject(), EasyMock.<ConnectorContext>anyObject(),
-                EasyMock.eq(herder));
+                EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
         PowerMock.expectLastCall();
+        EasyMock.expect(worker.isRunning(CONN1)).andReturn(true);
         EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS);
 
         // list connectors, get connector info, get connector config, get task configs
@@ -533,7 +887,7 @@
             @Override
             public Object answer() throws Throwable {
                 // Simulate response to writing config + waiting until end of log to be read
-                connectorConfigCallback.onCompletion(null, CONN1);
+                configUpdateListener.onConnectorConfigUpdate(CONN1);
                 return null;
             }
         });
@@ -544,8 +898,9 @@
         PowerMock.expectLastCall();
         Capture<ConnectorConfig> capturedUpdatedConfig = EasyMock.newCapture();
         worker.startConnector(EasyMock.capture(capturedUpdatedConfig), EasyMock.<ConnectorContext>anyObject(),
-                EasyMock.eq(herder));
+                EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
         PowerMock.expectLastCall();
+        EasyMock.expect(worker.isRunning(CONN1)).andReturn(true);
         EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS);
 
         member.poll(EasyMock.anyInt());
@@ -642,10 +997,9 @@
         PowerMock.expectLastCall();
     }
 
-    private void expectPostRebalanceCatchup(final ClusterConfigState readToEndSnapshot) {
-        TestFuture<Void> readToEndFuture = new TestFuture<>();
-        readToEndFuture.resolveOnGet((Void) null);
-        EasyMock.expect(configStorage.readToEnd()).andReturn(readToEndFuture);
+    private void expectPostRebalanceCatchup(final ClusterConfigState readToEndSnapshot) throws TimeoutException {
+        configStorage.refresh(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class));
+        EasyMock.expectLastCall();
         EasyMock.expect(configStorage.snapshot()).andReturn(readToEndSnapshot);
     }
 
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java
index bf33cb3..f7423ec 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java
@@ -31,7 +31,8 @@
 import org.apache.kafka.common.requests.SyncGroupRequest;
 import org.apache.kafka.common.requests.SyncGroupResponse;
 import org.apache.kafka.common.utils.MockTime;
-import org.apache.kafka.connect.storage.KafkaConfigStorage;
+import org.apache.kafka.connect.runtime.TargetState;
+import org.apache.kafka.connect.storage.KafkaConfigBackingStore;
 import org.apache.kafka.connect.util.ConnectorTaskId;
 import org.apache.kafka.test.TestUtils;
 import org.easymock.EasyMock;
@@ -76,7 +77,7 @@
     private Metrics metrics;
     private ConsumerNetworkClient consumerClient;
     private MockRebalanceListener rebalanceListener;
-    @Mock private KafkaConfigStorage configStorage;
+    @Mock private KafkaConfigBackingStore configStorage;
     private WorkerCoordinator coordinator;
 
     private ClusterConfigState configState1;
@@ -91,7 +92,7 @@
         this.consumerClient = new ConsumerNetworkClient(client, metadata, time, 100, 1000);
         this.metrics = new Metrics(time);
         this.rebalanceListener = new MockRebalanceListener();
-        this.configStorage = PowerMock.createMock(KafkaConfigStorage.class);
+        this.configStorage = PowerMock.createMock(KafkaConfigBackingStore.class);
 
         client.setNode(node);
 
@@ -110,6 +111,7 @@
         configState1 = new ClusterConfigState(
                 1L, Collections.singletonMap(connectorId, 1),
                 Collections.singletonMap(connectorId, (Map<String, String>) new HashMap<String, String>()),
+                Collections.singletonMap(connectorId, TargetState.STARTED),
                 Collections.singletonMap(taskId0, (Map<String, String>) new HashMap<String, String>()),
                 Collections.<String>emptySet()
         );
@@ -119,6 +121,9 @@
         Map<String, Map<String, String>> configState2ConnectorConfigs = new HashMap<>();
         configState2ConnectorConfigs.put(connectorId, new HashMap<String, String>());
         configState2ConnectorConfigs.put(connectorId2, new HashMap<String, String>());
+        Map<String, TargetState> targetStates = new HashMap<>();
+        targetStates.put(connectorId, TargetState.STARTED);
+        targetStates.put(connectorId2, TargetState.STARTED);
         Map<ConnectorTaskId, Map<String, String>> configState2TaskConfigs = new HashMap<>();
         configState2TaskConfigs.put(taskId0, new HashMap<String, String>());
         configState2TaskConfigs.put(taskId1, new HashMap<String, String>());
@@ -126,6 +131,7 @@
         configState2 = new ClusterConfigState(
                 2L, configState2ConnectorTaskCounts,
                 configState2ConnectorConfigs,
+                targetStates,
                 configState2TaskConfigs,
                 Collections.<String>emptySet()
         );
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java
index 8e9d52b..64d5b5e 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java
@@ -31,11 +31,6 @@
 import org.powermock.api.easymock.annotation.MockStrict;
 import org.powermock.modules.junit4.PowerMockRunner;
 
-import javax.ws.rs.client.Client;
-import javax.ws.rs.client.ClientBuilder;
-import javax.ws.rs.client.Invocation;
-import javax.ws.rs.client.WebTarget;
-import javax.ws.rs.core.Response;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.Arrays;
@@ -43,6 +38,12 @@
 import java.util.HashMap;
 import java.util.Map;
 
+import javax.ws.rs.client.Client;
+import javax.ws.rs.client.ClientBuilder;
+import javax.ws.rs.client.Invocation;
+import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.core.Response;
+
 import static org.junit.Assert.assertEquals;
 
 @RunWith(PowerMockRunner.class)
@@ -71,15 +72,15 @@
 
     @Test
     public void testCORSEnabled() {
-        checkCORSRequest("*", "http://bar.com", "http://bar.com");
+        checkCORSRequest("*", "http://bar.com", "http://bar.com", "PUT");
     }
 
     @Test
     public void testCORSDisabled() {
-        checkCORSRequest("", "http://bar.com", null);
+        checkCORSRequest("", "http://bar.com", null, null);
     }
 
-    public void checkCORSRequest(String corsDomain, String origin, String expectedHeader) {
+    public void checkCORSRequest(String corsDomain, String origin, String expectedHeader, String method) {
         // To be able to set the Origin, we need to toggle this flag
         System.setProperty("sun.net.http.allowRestrictedHeaders", "true");
 
@@ -92,10 +93,12 @@
                 return null;
             }
         });
+
         PowerMock.replayAll();
 
         Map<String, String> workerProps = baseWorkerProps();
         workerProps.put(WorkerConfig.ACCESS_CONTROL_ALLOW_ORIGIN_CONFIG, corsDomain);
+        workerProps.put(WorkerConfig.ACCESS_CONTROL_ALLOW_METHODS_CONFIG, method);
         WorkerConfig workerConfig = new StandaloneConfig(workerProps);
         server = new RestServer(workerConfig);
         server.start(herder);
@@ -107,6 +110,15 @@
         assertEquals(200, response.getStatus());
 
         assertEquals(expectedHeader, response.getHeaderString("Access-Control-Allow-Origin"));
+
+        response = request("/connector-plugins/FileStreamSource/validate")
+            .header("Referer", origin + "/page")
+            .header("Origin", origin)
+            .header("Access-Control-Request-Method", method)
+            .options();
+        assertEquals(404, response.getStatus());
+        assertEquals(expectedHeader, response.getHeaderString("Access-Control-Allow-Origin"));
+        assertEquals(method, response.getHeaderString("Access-Control-Allow-Methods"));
         PowerMock.verifyAll();
     }
 
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java
index 1049e7e..241d331 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java
@@ -22,7 +22,9 @@
 import org.apache.kafka.common.config.Config;
 import org.apache.kafka.common.config.ConfigDef;
 import org.apache.kafka.common.config.ConfigDef.Importance;
+import org.apache.kafka.common.config.ConfigDef.Recommender;
 import org.apache.kafka.common.config.ConfigDef.Type;
+import org.apache.kafka.common.config.ConfigDef.Width;
 import org.apache.kafka.connect.connector.Connector;
 import org.apache.kafka.connect.connector.Task;
 import org.apache.kafka.connect.runtime.AbstractHerder;
@@ -48,6 +50,7 @@
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -68,7 +71,8 @@
     private static Map<String, String> props = new HashMap<>();
     static {
         props.put("test.string.config", "testString");
-        props.put("test.int.config", "10");
+        props.put("test.int.config", "1");
+        props.put("test.list.config", "a,b");
     }
 
     private static final ConfigInfos CONFIG_INFOS;
@@ -76,22 +80,27 @@
     static {
         List<ConfigInfo> configs = new LinkedList<>();
 
-        ConfigKeyInfo configKeyInfo = new ConfigKeyInfo("test.string.config", "STRING", true, "", "HIGH", "Test configuration for string type.", null, -1, "NONE", "test.string.config", new LinkedList<String>());
-        ConfigValueInfo configValueInfo = new ConfigValueInfo("test.string.config", "testString", Collections.<Object>emptyList(), Collections.<String>emptyList(), true);
+        ConfigKeyInfo configKeyInfo = new ConfigKeyInfo("test.string.config", "STRING", true, "", "HIGH", "Test configuration for string type.", null, -1, "NONE", "test.string.config", Collections.<String>emptyList());
+        ConfigValueInfo configValueInfo = new ConfigValueInfo("test.string.config", "testString", Collections.<String>emptyList(), Collections.<String>emptyList(), true);
         ConfigInfo configInfo = new ConfigInfo(configKeyInfo, configValueInfo);
         configs.add(configInfo);
 
-        configKeyInfo = new ConfigKeyInfo("test.int.config", "INT", true, "", "MEDIUM", "Test configuration for integer type.", null, -1, "NONE", "test.int.config", new LinkedList<String>());
-        configValueInfo = new ConfigValueInfo("test.int.config", 10, Collections.<Object>emptyList(), Collections.<String>emptyList(), true);
+        configKeyInfo = new ConfigKeyInfo("test.int.config", "INT", true, "", "MEDIUM", "Test configuration for integer type.", "Test", 1, "MEDIUM", "test.int.config", Collections.<String>emptyList());
+        configValueInfo = new ConfigValueInfo("test.int.config", "1", Arrays.asList("1", "2", "3"), Collections.<String>emptyList(), true);
         configInfo = new ConfigInfo(configKeyInfo, configValueInfo);
         configs.add(configInfo);
 
-        configKeyInfo = new ConfigKeyInfo("test.string.config.default", "STRING", false, "", "LOW", "Test configuration with default value.", null, -1, "NONE", "test.string.config.default", new LinkedList<String>());
-        configValueInfo = new ConfigValueInfo("test.string.config.default", "", Collections.<Object>emptyList(), Collections.<String>emptyList(), true);
+        configKeyInfo = new ConfigKeyInfo("test.string.config.default", "STRING", false, "", "LOW", "Test configuration with default value.", null, -1, "NONE", "test.string.config.default", Collections.<String>emptyList());
+        configValueInfo = new ConfigValueInfo("test.string.config.default", "", Collections.<String>emptyList(), Collections.<String>emptyList(), true);
         configInfo = new ConfigInfo(configKeyInfo, configValueInfo);
         configs.add(configInfo);
 
-        CONFIG_INFOS = new ConfigInfos(ConnectorPluginsResourceTestConnector.class.getName(), 0, Collections.<String>emptyList(), configs);
+        configKeyInfo = new ConfigKeyInfo("test.list.config", "LIST", true, "", "HIGH", "Test configuration for list type.", "Test", 2, "LONG", "test.list.config", Collections.<String>emptyList());
+        configValueInfo = new ConfigValueInfo("test.list.config", "a,b", Arrays.asList("a", "b", "c"), Collections.<String>emptyList(), true);
+        configInfo = new ConfigInfo(configKeyInfo, configValueInfo);
+        configs.add(configInfo);
+
+        CONFIG_INFOS = new ConfigInfos(ConnectorPluginsResourceTestConnector.class.getName(), 0, Collections.singletonList("Test"), configs);
     }
 
     @Mock
@@ -140,17 +149,21 @@
         assertTrue(connectorPlugins.contains(new ConnectorPluginInfo(ConnectorPluginsResourceTestConnector.class.getCanonicalName())));
     }
 
+
     /* Name here needs to be unique as we are testing the aliasing mechanism */
     public static class ConnectorPluginsResourceTestConnector extends Connector {
 
-        public static final String TEST_STRING_CONFIG = "test.string.config";
-        public static final String TEST_INT_CONFIG = "test.int.config";
-        public static final String TEST_STRING_CONFIG_DEFAULT = "test.string.config.default";
+        private static final String TEST_STRING_CONFIG = "test.string.config";
+        private static final String TEST_INT_CONFIG = "test.int.config";
+        private static final String TEST_STRING_CONFIG_DEFAULT = "test.string.config.default";
+        private static final String TEST_LIST_CONFIG = "test.list.config";
+        private static final String GROUP = "Test";
 
         private static final ConfigDef CONFIG_DEF = new ConfigDef()
             .define(TEST_STRING_CONFIG, Type.STRING, Importance.HIGH, "Test configuration for string type.")
-            .define(TEST_INT_CONFIG, Type.INT, Importance.MEDIUM, "Test configuration for integer type.")
-            .define(TEST_STRING_CONFIG_DEFAULT, Type.STRING, "", Importance.LOW, "Test configuration with default value.");
+            .define(TEST_INT_CONFIG, Type.INT, Importance.MEDIUM, "Test configuration for integer type.", GROUP, 1, Width.MEDIUM, TEST_INT_CONFIG, new IntegerRecommender())
+            .define(TEST_STRING_CONFIG_DEFAULT, Type.STRING, "", Importance.LOW, "Test configuration with default value.")
+            .define(TEST_LIST_CONFIG, Type.LIST, Importance.HIGH, "Test configuration for list type.", GROUP, 2, Width.LONG, TEST_LIST_CONFIG, new ListRecommender());
 
         @Override
         public String version() {
@@ -182,4 +195,29 @@
             return CONFIG_DEF;
         }
     }
+
+    private static class IntegerRecommender implements Recommender {
+
+        @Override
+        public List<Object> validValues(String name, Map<String, Object> parsedConfig) {
+            return Arrays.<Object>asList(1, 2, 3);
+        }
+
+        @Override
+        public boolean visible(String name, Map<String, Object> parsedConfig) {
+            return true;
+        }
+    }
+
+    private static class ListRecommender implements Recommender {
+        @Override
+        public List<Object> validValues(String name, Map<String, Object> parsedConfig) {
+            return Arrays.<Object>asList("a", "b", "c");
+        }
+
+        @Override
+        public boolean visible(String name, Map<String, Object> parsedConfig) {
+            return true;
+        }
+    }
 }
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java
index 970f56c..aa1b9a7 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java
@@ -24,6 +24,7 @@
 import org.apache.kafka.connect.errors.NotFoundException;
 import org.apache.kafka.connect.runtime.ConnectorConfig;
 import org.apache.kafka.connect.runtime.Herder;
+import org.apache.kafka.connect.runtime.distributed.NotAssignedException;
 import org.apache.kafka.connect.runtime.distributed.NotLeaderException;
 import org.apache.kafka.connect.runtime.rest.RestServer;
 import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo;
@@ -43,6 +44,7 @@
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
+import javax.ws.rs.BadRequestException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -64,6 +66,7 @@
     private static final String LEADER_URL = "http://leader:8083/";
     private static final String CONNECTOR_NAME = "test";
     private static final String CONNECTOR2_NAME = "test2";
+    private static final Boolean FORWARD = true;
     private static final Map<String, String> CONNECTOR_CONFIG = new HashMap<>();
     static {
         CONNECTOR_CONFIG.put("name", CONNECTOR_NAME);
@@ -103,7 +106,7 @@
 
         PowerMock.replayAll();
 
-        Collection<String> connectors = connectorsResource.listConnectors();
+        Collection<String> connectors = connectorsResource.listConnectors(FORWARD);
         // Ordering isn't guaranteed, compare sets
         assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_NAME, CONNECTOR2_NAME)), new HashSet<>(connectors));
 
@@ -116,13 +119,13 @@
         herder.connectors(EasyMock.capture(cb));
         expectAndCallbackNotLeaderException(cb);
         // Should forward request
-        EasyMock.expect(RestServer.httpRequest(EasyMock.eq("http://leader:8083/connectors"), EasyMock.eq("GET"),
+        EasyMock.expect(RestServer.httpRequest(EasyMock.eq("http://leader:8083/connectors?forward=false"), EasyMock.eq("GET"),
                 EasyMock.isNull(), EasyMock.anyObject(TypeReference.class)))
                 .andReturn(new RestServer.HttpResponse<>(200, new HashMap<String, List<String>>(), Arrays.asList(CONNECTOR2_NAME, CONNECTOR_NAME)));
 
         PowerMock.replayAll();
 
-        Collection<String> connectors = connectorsResource.listConnectors();
+        Collection<String> connectors = connectorsResource.listConnectors(FORWARD);
         // Ordering isn't guaranteed, compare sets
         assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_NAME, CONNECTOR2_NAME)), new HashSet<>(connectors));
 
@@ -138,7 +141,7 @@
         PowerMock.replayAll();
 
         // throws
-        connectorsResource.listConnectors();
+        connectorsResource.listConnectors(FORWARD);
     }
 
     @Test
@@ -151,7 +154,7 @@
 
         PowerMock.replayAll();
 
-        connectorsResource.createConnector(body);
+        connectorsResource.createConnector(FORWARD, body);
 
         PowerMock.verifyAll();
     }
@@ -164,12 +167,12 @@
         herder.putConnectorConfig(EasyMock.eq(CONNECTOR_NAME), EasyMock.eq(body.config()), EasyMock.eq(false), EasyMock.capture(cb));
         expectAndCallbackNotLeaderException(cb);
         // Should forward request
-        EasyMock.expect(RestServer.httpRequest(EasyMock.eq("http://leader:8083/connectors"), EasyMock.eq("POST"), EasyMock.eq(body), EasyMock.<TypeReference>anyObject()))
+        EasyMock.expect(RestServer.httpRequest(EasyMock.eq("http://leader:8083/connectors?forward=false"), EasyMock.eq("POST"), EasyMock.eq(body), EasyMock.<TypeReference>anyObject()))
                 .andReturn(new RestServer.HttpResponse<>(201, new HashMap<String, List<String>>(), new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG, CONNECTOR_TASK_NAMES)));
 
         PowerMock.replayAll();
 
-        connectorsResource.createConnector(body);
+        connectorsResource.createConnector(FORWARD, body);
 
         PowerMock.verifyAll();
 
@@ -186,7 +189,7 @@
 
         PowerMock.replayAll();
 
-        connectorsResource.createConnector(body);
+        connectorsResource.createConnector(FORWARD, body);
 
         PowerMock.verifyAll();
     }
@@ -199,7 +202,7 @@
 
         PowerMock.replayAll();
 
-        connectorsResource.destroyConnector(CONNECTOR_NAME);
+        connectorsResource.destroyConnector(CONNECTOR_NAME, FORWARD);
 
         PowerMock.verifyAll();
     }
@@ -210,12 +213,12 @@
         herder.putConnectorConfig(EasyMock.eq(CONNECTOR_NAME), EasyMock.<Map<String, String>>isNull(), EasyMock.eq(true), EasyMock.capture(cb));
         expectAndCallbackNotLeaderException(cb);
         // Should forward request
-        EasyMock.expect(RestServer.httpRequest("http://leader:8083/connectors/" + CONNECTOR_NAME, "DELETE", null, null))
+        EasyMock.expect(RestServer.httpRequest("http://leader:8083/connectors/" + CONNECTOR_NAME + "?forward=false", "DELETE", null, null))
                 .andReturn(new RestServer.HttpResponse<>(204, new HashMap<String, List<String>>(), null));
 
         PowerMock.replayAll();
 
-        connectorsResource.destroyConnector(CONNECTOR_NAME);
+        connectorsResource.destroyConnector(CONNECTOR_NAME, FORWARD);
 
         PowerMock.verifyAll();
     }
@@ -229,7 +232,7 @@
 
         PowerMock.replayAll();
 
-        connectorsResource.destroyConnector(CONNECTOR_NAME);
+        connectorsResource.destroyConnector(CONNECTOR_NAME, FORWARD);
 
         PowerMock.verifyAll();
     }
@@ -242,7 +245,7 @@
 
         PowerMock.replayAll();
 
-        ConnectorInfo connInfo = connectorsResource.getConnector(CONNECTOR_NAME);
+        ConnectorInfo connInfo = connectorsResource.getConnector(CONNECTOR_NAME, FORWARD);
         assertEquals(new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG, CONNECTOR_TASK_NAMES), connInfo);
 
         PowerMock.verifyAll();
@@ -256,7 +259,7 @@
 
         PowerMock.replayAll();
 
-        Map<String, String> connConfig = connectorsResource.getConnectorConfig(CONNECTOR_NAME);
+        Map<String, String> connConfig = connectorsResource.getConnectorConfig(CONNECTOR_NAME, FORWARD);
         assertEquals(CONNECTOR_CONFIG, connConfig);
 
         PowerMock.verifyAll();
@@ -270,7 +273,7 @@
 
         PowerMock.replayAll();
 
-        connectorsResource.getConnectorConfig(CONNECTOR_NAME);
+        connectorsResource.getConnectorConfig(CONNECTOR_NAME, FORWARD);
 
         PowerMock.verifyAll();
     }
@@ -283,11 +286,18 @@
 
         PowerMock.replayAll();
 
-        connectorsResource.putConnectorConfig(CONNECTOR_NAME, CONNECTOR_CONFIG);
+        connectorsResource.putConnectorConfig(CONNECTOR_NAME, FORWARD, CONNECTOR_CONFIG);
 
         PowerMock.verifyAll();
     }
 
+    @Test(expected = BadRequestException.class)
+    public void testPutConnectorConfigNameMismatch() throws Throwable {
+        Map<String, String> connConfig = new HashMap<>(CONNECTOR_CONFIG);
+        connConfig.put(ConnectorConfig.NAME_CONFIG, "mismatched-name");
+        connectorsResource.putConnectorConfig(CONNECTOR_NAME, FORWARD, connConfig);
+    }
+
     @Test
     public void testGetConnectorTaskConfigs() throws Throwable {
         final Capture<Callback<List<TaskInfo>>> cb = Capture.newInstance();
@@ -296,7 +306,7 @@
 
         PowerMock.replayAll();
 
-        List<TaskInfo> taskInfos = connectorsResource.getTaskConfigs(CONNECTOR_NAME);
+        List<TaskInfo> taskInfos = connectorsResource.getTaskConfigs(CONNECTOR_NAME, FORWARD);
         assertEquals(TASK_INFOS, taskInfos);
 
         PowerMock.verifyAll();
@@ -310,7 +320,7 @@
 
         PowerMock.replayAll();
 
-        connectorsResource.getTaskConfigs(CONNECTOR_NAME);
+        connectorsResource.getTaskConfigs(CONNECTOR_NAME, FORWARD);
 
         PowerMock.verifyAll();
     }
@@ -323,7 +333,7 @@
 
         PowerMock.replayAll();
 
-        connectorsResource.putTaskConfigs(CONNECTOR_NAME, TASK_CONFIGS);
+        connectorsResource.putTaskConfigs(CONNECTOR_NAME, FORWARD, TASK_CONFIGS);
 
         PowerMock.verifyAll();
     }
@@ -336,7 +346,108 @@
 
         PowerMock.replayAll();
 
-        connectorsResource.putTaskConfigs(CONNECTOR_NAME, TASK_CONFIGS);
+        connectorsResource.putTaskConfigs(CONNECTOR_NAME, FORWARD, TASK_CONFIGS);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test(expected = NotFoundException.class)
+    public void testRestartConnectorNotFound() throws Throwable {
+        final Capture<Callback<Void>> cb = Capture.newInstance();
+        herder.restartConnector(EasyMock.eq(CONNECTOR_NAME), EasyMock.capture(cb));
+        expectAndCallbackException(cb, new NotFoundException("not found"));
+
+        PowerMock.replayAll();
+
+        connectorsResource.restartConnector(CONNECTOR_NAME, FORWARD);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testRestartConnectorLeaderRedirect() throws Throwable {
+        final Capture<Callback<Void>> cb = Capture.newInstance();
+        herder.restartConnector(EasyMock.eq(CONNECTOR_NAME), EasyMock.capture(cb));
+        expectAndCallbackNotLeaderException(cb);
+
+        EasyMock.expect(RestServer.httpRequest(EasyMock.eq("http://leader:8083/connectors/" + CONNECTOR_NAME + "/restart?forward=true"),
+                EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.<TypeReference>anyObject()))
+                .andReturn(new RestServer.HttpResponse<>(202, new HashMap<String, List<String>>(), null));
+
+        PowerMock.replayAll();
+
+        connectorsResource.restartConnector(CONNECTOR_NAME, null);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testRestartConnectorOwnerRedirect() throws Throwable {
+        final Capture<Callback<Void>> cb = Capture.newInstance();
+        herder.restartConnector(EasyMock.eq(CONNECTOR_NAME), EasyMock.capture(cb));
+        String ownerUrl = "http://owner:8083";
+        expectAndCallbackException(cb, new NotAssignedException("not owner test", ownerUrl));
+
+        EasyMock.expect(RestServer.httpRequest(EasyMock.eq("http://owner:8083/connectors/" + CONNECTOR_NAME + "/restart?forward=false"),
+                EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.<TypeReference>anyObject()))
+                .andReturn(new RestServer.HttpResponse<>(202, new HashMap<String, List<String>>(), null));
+
+        PowerMock.replayAll();
+
+        connectorsResource.restartConnector(CONNECTOR_NAME, true);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test(expected = NotFoundException.class)
+    public void testRestartTaskNotFound() throws Throwable {
+        ConnectorTaskId taskId = new ConnectorTaskId(CONNECTOR_NAME, 0);
+        final Capture<Callback<Void>> cb = Capture.newInstance();
+        herder.restartTask(EasyMock.eq(taskId), EasyMock.capture(cb));
+        expectAndCallbackException(cb, new NotFoundException("not found"));
+
+        PowerMock.replayAll();
+
+        connectorsResource.restartTask(CONNECTOR_NAME, 0, FORWARD);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testRestartTaskLeaderRedirect() throws Throwable {
+        ConnectorTaskId taskId = new ConnectorTaskId(CONNECTOR_NAME, 0);
+
+        final Capture<Callback<Void>> cb = Capture.newInstance();
+        herder.restartTask(EasyMock.eq(taskId), EasyMock.capture(cb));
+        expectAndCallbackNotLeaderException(cb);
+
+        EasyMock.expect(RestServer.httpRequest(EasyMock.eq("http://leader:8083/connectors/" + CONNECTOR_NAME + "/tasks/0/restart?forward=true"),
+                EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.<TypeReference>anyObject()))
+                .andReturn(new RestServer.HttpResponse<>(202, new HashMap<String, List<String>>(), null));
+
+        PowerMock.replayAll();
+
+        connectorsResource.restartTask(CONNECTOR_NAME, 0, null);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testRestartTaskOwnerRedirect() throws Throwable {
+        ConnectorTaskId taskId = new ConnectorTaskId(CONNECTOR_NAME, 0);
+
+        final Capture<Callback<Void>> cb = Capture.newInstance();
+        herder.restartTask(EasyMock.eq(taskId), EasyMock.capture(cb));
+        String ownerUrl = "http://owner:8083";
+        expectAndCallbackException(cb, new NotAssignedException("not owner test", ownerUrl));
+
+        EasyMock.expect(RestServer.httpRequest(EasyMock.eq("http://owner:8083/connectors/" + CONNECTOR_NAME + "/tasks/0/restart?forward=false"),
+                EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.<TypeReference>anyObject()))
+                .andReturn(new RestServer.HttpResponse<>(202, new HashMap<String, List<String>>(), null));
+
+        PowerMock.replayAll();
+
+        connectorsResource.restartTask(CONNECTOR_NAME, 0, true);
 
         PowerMock.verifyAll();
     }
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java
index 3959ff8..10e5194 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java
@@ -27,6 +27,7 @@
 import org.apache.kafka.connect.runtime.ConnectorStatus;
 import org.apache.kafka.connect.runtime.Herder;
 import org.apache.kafka.connect.runtime.HerderConnectorContext;
+import org.apache.kafka.connect.runtime.TargetState;
 import org.apache.kafka.connect.runtime.TaskConfig;
 import org.apache.kafka.connect.runtime.TaskStatus;
 import org.apache.kafka.connect.runtime.Worker;
@@ -36,6 +37,7 @@
 import org.apache.kafka.connect.sink.SinkTask;
 import org.apache.kafka.connect.source.SourceConnector;
 import org.apache.kafka.connect.source.SourceTask;
+import org.apache.kafka.connect.storage.MemoryConfigBackingStore;
 import org.apache.kafka.connect.storage.StatusBackingStore;
 import org.apache.kafka.connect.util.Callback;
 import org.apache.kafka.connect.util.ConnectorTaskId;
@@ -80,7 +82,7 @@
 
     @Before
     public void setup() {
-        herder = new StandaloneHerder(WORKER_ID, worker, statusBackingStore);
+        herder = new StandaloneHerder(worker, WORKER_ID, statusBackingStore, new MemoryConfigBackingStore());
     }
 
     @Test
@@ -156,12 +158,171 @@
     }
 
     @Test
+    public void testRestartConnector() throws Exception {
+        expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false);
+
+        worker.stopConnector(CONNECTOR_NAME);
+        EasyMock.expectLastCall();
+
+        worker.startConnector(EasyMock.eq(new ConnectorConfig(connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class))),
+                EasyMock.anyObject(HerderConnectorContext.class), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
+        EasyMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class), false, createCallback);
+
+        FutureCallback<Void> cb = new FutureCallback<>();
+        herder.restartConnector(CONNECTOR_NAME, cb);
+        cb.get(1000L, TimeUnit.MILLISECONDS);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testRestartConnectorFailureOnStop() throws Exception {
+        expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false);
+
+        RuntimeException e = new RuntimeException();
+        worker.stopConnector(CONNECTOR_NAME);
+        EasyMock.expectLastCall().andThrow(e);
+
+        // the connector will not be started after the failure in start
+
+        PowerMock.replayAll();
+
+        herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class), false, createCallback);
+
+        FutureCallback<Void> cb = new FutureCallback<>();
+        herder.restartConnector(CONNECTOR_NAME, cb);
+        try {
+            cb.get(1000L, TimeUnit.MILLISECONDS);
+            fail();
+        } catch (ExecutionException exception) {
+            assertEquals(e, exception.getCause());
+        }
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testRestartConnectorFailureOnStart() throws Exception {
+        expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false);
+
+        worker.stopConnector(CONNECTOR_NAME);
+        EasyMock.expectLastCall();
+
+        RuntimeException e = new RuntimeException();
+        worker.startConnector(EasyMock.eq(new ConnectorConfig(connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class))),
+                EasyMock.anyObject(HerderConnectorContext.class), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
+        EasyMock.expectLastCall().andThrow(e);
+
+        PowerMock.replayAll();
+
+        herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class), false, createCallback);
+
+        FutureCallback<Void> cb = new FutureCallback<>();
+        herder.restartConnector(CONNECTOR_NAME, cb);
+        try {
+            cb.get(1000L, TimeUnit.MILLISECONDS);
+            fail();
+        } catch (ExecutionException exception) {
+            assertEquals(e, exception.getCause());
+        }
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testRestartTask() throws Exception {
+        ConnectorTaskId taskId = new ConnectorTaskId(CONNECTOR_NAME, 0);
+        expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false);
+
+        worker.stopAndAwaitTask(taskId);
+        EasyMock.expectLastCall();
+
+        Map<String, String> generatedTaskProps = taskConfig(BogusSourceTask.class, false);
+        worker.startTask(taskId, new TaskConfig(generatedTaskProps), herder, TargetState.STARTED);
+        EasyMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class), false, createCallback);
+
+        FutureCallback<Void> cb = new FutureCallback<>();
+        herder.restartTask(taskId, cb);
+        cb.get(1000L, TimeUnit.MILLISECONDS);
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testRestartTaskFailureOnStop() throws Exception {
+        ConnectorTaskId taskId = new ConnectorTaskId(CONNECTOR_NAME, 0);
+        expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false);
+
+        RuntimeException e = new RuntimeException();
+        worker.stopAndAwaitTask(taskId);
+        EasyMock.expectLastCall().andThrow(e);
+
+        // task will not be started after the failure in stop
+
+        PowerMock.replayAll();
+
+        herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class), false, createCallback);
+
+        FutureCallback<Void> cb = new FutureCallback<>();
+        herder.restartTask(taskId, cb);
+        try {
+            cb.get(1000L, TimeUnit.MILLISECONDS);
+            fail("Expected restart callback to raise an exception");
+        } catch (ExecutionException exception) {
+            assertEquals(e, exception.getCause());
+        }
+        PowerMock.verifyAll();
+    }
+
+    @Test
+    public void testRestartTaskFailureOnStart() throws Exception {
+        ConnectorTaskId taskId = new ConnectorTaskId(CONNECTOR_NAME, 0);
+        expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false);
+
+        worker.stopAndAwaitTask(taskId);
+        EasyMock.expectLastCall();
+
+        RuntimeException e = new RuntimeException();
+        Map<String, String> generatedTaskProps = taskConfig(BogusSourceTask.class, false);
+        worker.startTask(taskId, new TaskConfig(generatedTaskProps), herder, TargetState.STARTED);
+        EasyMock.expectLastCall().andThrow(e);
+
+        PowerMock.replayAll();
+
+        herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class), false, createCallback);
+
+        FutureCallback<Void> cb = new FutureCallback<>();
+        herder.restartTask(taskId, cb);
+        try {
+            cb.get(1000L, TimeUnit.MILLISECONDS);
+            fail("Expected restart callback to raise an exception");
+        } catch (ExecutionException exception) {
+            assertEquals(e, exception.getCause());
+        }
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
     public void testCreateAndStop() throws Exception {
         connector = PowerMock.createMock(BogusSourceConnector.class);
         expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false);
         // herder.stop() should stop any running connectors and tasks even if destroyConnector was not invoked
         expectStop();
 
+        statusBackingStore.stop();
+        EasyMock.expectLastCall();
+        worker.stop();
+        EasyMock.expectLastCall();
+
         PowerMock.replayAll();
 
         herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class), false, createCallback);
@@ -180,7 +341,7 @@
         Callback<List<TaskInfo>> taskConfigsCb = PowerMock.createMock(Callback.class);
 
         // Check accessors with empty worker
-        listConnectorsCb.onCompletion(null, Collections.EMPTY_LIST);
+        listConnectorsCb.onCompletion(null, Collections.EMPTY_SET);
         EasyMock.expectLastCall();
         connectorInfoCb.onCompletion(EasyMock.<NotFoundException>anyObject(), EasyMock.<ConnectorInfo>isNull());
         EasyMock.expectLastCall();
@@ -195,7 +356,7 @@
         expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false);
 
         // Validate accessors with 1 connector
-        listConnectorsCb.onCompletion(null, Arrays.asList(CONNECTOR_NAME));
+        listConnectorsCb.onCompletion(null, Collections.singleton(CONNECTOR_NAME));
         EasyMock.expectLastCall();
         ConnectorInfo connInfo = new ConnectorInfo(CONNECTOR_NAME, connConfig, Arrays.asList(new ConnectorTaskId(CONNECTOR_NAME, 0)));
         connectorInfoCb.onCompletion(null, connInfo);
@@ -245,8 +406,9 @@
         EasyMock.expectLastCall();
         Capture<ConnectorConfig> capturedConfig = EasyMock.newCapture();
         worker.startConnector(EasyMock.capture(capturedConfig), EasyMock.<ConnectorContext>anyObject(),
-                EasyMock.eq(herder));
+                EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
         EasyMock.expectLastCall();
+        EasyMock.expect(worker.isRunning(CONNECTOR_NAME)).andReturn(true);
         // Generate same task config, which should result in no additional action to restart tasks
         EasyMock.expect(worker.connectorTaskConfigs(CONNECTOR_NAME, DEFAULT_MAX_TASKS, TOPICS_LIST))
                 .andReturn(Collections.singletonList(taskConfig(BogusSourceTask.class, false)));
@@ -257,7 +419,6 @@
         connectorConfigCb.onCompletion(null, newConnConfig);
         EasyMock.expectLastCall();
 
-
         PowerMock.replayAll();
 
         herder.putConnectorConfig(CONNECTOR_NAME, connConfig, false, createCallback);
@@ -290,8 +451,9 @@
         Map<String, String> connectorProps = connectorConfig(name, connClass);
 
         worker.startConnector(EasyMock.eq(new ConnectorConfig(connectorProps)), EasyMock.anyObject(HerderConnectorContext.class),
-                EasyMock.eq(herder));
+                EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED));
         EasyMock.expectLastCall();
+        EasyMock.expect(worker.isRunning(name)).andReturn(true);
 
         ConnectorInfo connInfo = new ConnectorInfo(CONNECTOR_NAME, connectorProps, Arrays.asList(new ConnectorTaskId(CONNECTOR_NAME, 0)));
         createCallback.onCompletion(null, new Herder.Created<>(true, connInfo));
@@ -303,15 +465,15 @@
         EasyMock.expect(worker.connectorTaskConfigs(CONNECTOR_NAME, DEFAULT_MAX_TASKS, TOPICS_LIST))
                 .andReturn(Collections.singletonList(generatedTaskProps));
 
-        worker.startTask(new ConnectorTaskId(CONNECTOR_NAME, 0), new TaskConfig(generatedTaskProps), herder);
+        worker.startTask(new ConnectorTaskId(CONNECTOR_NAME, 0), new TaskConfig(generatedTaskProps), herder, TargetState.STARTED);
         EasyMock.expectLastCall();
     }
 
     private void expectStop() {
         ConnectorTaskId task = new ConnectorTaskId(CONNECTOR_NAME, 0);
-        worker.stopTasks(Collections.singleton(task));
+        worker.stopTasks(Collections.singletonList(task));
         EasyMock.expectLastCall();
-        worker.awaitStopTasks(Collections.singleton(task));
+        worker.awaitStopTasks(Collections.singletonList(task));
         EasyMock.expectLastCall();
         worker.stopConnector(CONNECTOR_NAME);
         EasyMock.expectLastCall();
@@ -321,7 +483,6 @@
         expectStop();
     }
 
-
     private static HashMap<String, String> connectorConfig(String name, Class<? extends Connector> connClass) {
         HashMap<String, String> connectorProps = new HashMap<>();
         connectorProps.put(ConnectorConfig.NAME_CONFIG, name);
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigStorageTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java
similarity index 73%
rename from connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigStorageTest.java
rename to connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java
index 5e79a8d..617177e 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigStorageTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java
@@ -21,7 +21,6 @@
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.connect.data.Field;
 import org.apache.kafka.connect.data.Schema;
@@ -57,13 +56,12 @@
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
-import static org.junit.Assert.fail;
 
 @RunWith(PowerMockRunner.class)
-@PrepareForTest(KafkaConfigStorage.class)
+@PrepareForTest(KafkaConfigBackingStore.class)
 @PowerMockIgnore("javax.management.*")
 @SuppressWarnings("unchecked")
-public class KafkaConfigStorageTest {
+public class KafkaConfigBackingStoreTest {
     private static final String TOPIC = "connect-configs";
     private static final Map<String, String> DEFAULT_CONFIG_STORAGE_PROPS = new HashMap<>();
     private static final DistributedConfig DEFAULT_DISTRIBUTED_CONFIG;
@@ -84,6 +82,7 @@
     private static final List<String> CONNECTOR_IDS = Arrays.asList("connector1", "connector2");
     private static final List<String> CONNECTOR_CONFIG_KEYS = Arrays.asList("connector-connector1", "connector-connector2");
     private static final List<String> COMMIT_TASKS_CONFIG_KEYS = Arrays.asList("commit-connector1", "commit-connector2");
+    private static final List<String> TARGET_STATE_KEYS =  Arrays.asList("target-state-connector1", "target-state-connector2");
 
     // Need a) connector with multiple tasks and b) multiple connectors
     private static final List<ConnectorTaskId> TASK_IDS = Arrays.asList(
@@ -100,17 +99,20 @@
             Collections.singletonMap("config-key-three", "config-value-three")
     );
     private static final List<Struct> CONNECTOR_CONFIG_STRUCTS = Arrays.asList(
-            new Struct(KafkaConfigStorage.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(0)),
-            new Struct(KafkaConfigStorage.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(1)),
-            new Struct(KafkaConfigStorage.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(2))
+            new Struct(KafkaConfigBackingStore.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(0)),
+            new Struct(KafkaConfigBackingStore.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(1)),
+            new Struct(KafkaConfigBackingStore.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(2))
     );
     private static final List<Struct> TASK_CONFIG_STRUCTS = Arrays.asList(
-            new Struct(KafkaConfigStorage.TASK_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(0)),
-            new Struct(KafkaConfigStorage.TASK_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(1))
+            new Struct(KafkaConfigBackingStore.TASK_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(0)),
+            new Struct(KafkaConfigBackingStore.TASK_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(1))
     );
 
     private static final Struct TASKS_COMMIT_STRUCT_TWO_TASK_CONNECTOR
-            = new Struct(KafkaConfigStorage.CONNECTOR_TASKS_COMMIT_V0).put("tasks", 2);
+            = new Struct(KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0).put("tasks", 2);
+
+    private static final Struct TASKS_COMMIT_STRUCT_ZERO_TASK_CONNECTOR
+            = new Struct(KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0).put("tasks", 0);
 
     // The exact format doesn't matter here since both conversions are mocked
     private static final List<byte[]> CONFIGS_SERIALIZED = Arrays.asList(
@@ -122,12 +124,10 @@
     @Mock
     private Converter converter;
     @Mock
-    private Callback<String> connectorReconfiguredCallback;
-    @Mock
-    private Callback<List<ConnectorTaskId>> tasksReconfiguredCallback;
+    private ConfigBackingStore.UpdateListener configUpdateListener;
     @Mock
     KafkaBasedLog<String, byte[]> storeLog;
-    private KafkaConfigStorage configStorage;
+    private KafkaConfigBackingStore configStorage;
 
     private Capture<String> capturedTopic = EasyMock.newCapture();
     private Capture<Map<String, Object>> capturedProducerProps = EasyMock.newCapture();
@@ -138,8 +138,8 @@
 
     @Before
     public void setUp() {
-        configStorage = PowerMock.createPartialMock(KafkaConfigStorage.class, new String[]{"createKafkaBasedLog"},
-                converter, connectorReconfiguredCallback, tasksReconfiguredCallback);
+        configStorage = PowerMock.createPartialMock(KafkaConfigBackingStore.class, new String[]{"createKafkaBasedLog"}, converter);
+        configStorage.setUpdateListener(configUpdateListener);
     }
 
     @Test
@@ -169,23 +169,27 @@
         expectStart(Collections.EMPTY_LIST, Collections.EMPTY_MAP);
 
         expectConvertWriteAndRead(
-                CONNECTOR_CONFIG_KEYS.get(0), KafkaConfigStorage.CONNECTOR_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(0),
+                CONNECTOR_CONFIG_KEYS.get(0), KafkaConfigBackingStore.CONNECTOR_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(0),
                 "properties", SAMPLE_CONFIGS.get(0));
-        connectorReconfiguredCallback.onCompletion(null, CONNECTOR_IDS.get(0));
+        configUpdateListener.onConnectorConfigUpdate(CONNECTOR_IDS.get(0));
         EasyMock.expectLastCall();
 
         expectConvertWriteAndRead(
-                CONNECTOR_CONFIG_KEYS.get(1), KafkaConfigStorage.CONNECTOR_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(1),
+                CONNECTOR_CONFIG_KEYS.get(1), KafkaConfigBackingStore.CONNECTOR_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(1),
                 "properties", SAMPLE_CONFIGS.get(1));
-        connectorReconfiguredCallback.onCompletion(null, CONNECTOR_IDS.get(1));
+        configUpdateListener.onConnectorConfigUpdate(CONNECTOR_IDS.get(1));
         EasyMock.expectLastCall();
 
         // Config deletion
         expectConvertWriteAndRead(
-                CONNECTOR_CONFIG_KEYS.get(1), KafkaConfigStorage.CONNECTOR_CONFIGURATION_V0, null, null, null);
-        connectorReconfiguredCallback.onCompletion(null, CONNECTOR_IDS.get(1));
+                CONNECTOR_CONFIG_KEYS.get(1), KafkaConfigBackingStore.CONNECTOR_CONFIGURATION_V0, null, null, null);
+        configUpdateListener.onConnectorConfigRemove(CONNECTOR_IDS.get(1));
         EasyMock.expectLastCall();
 
+        // Target state deletion
+        storeLog.send(TARGET_STATE_KEYS.get(1), null);
+        PowerMock.expectLastCall();
+
         expectStop();
 
         PowerMock.replayAll();
@@ -214,7 +218,7 @@
         assertEquals(SAMPLE_CONFIGS.get(1), configState.connectorConfig(CONNECTOR_IDS.get(1)));
 
         // Deletion should remove the second one we added
-        configStorage.putConnectorConfig(CONNECTOR_IDS.get(1), null);
+        configStorage.removeConnectorConfig(CONNECTOR_IDS.get(1));
         configState = configStorage.snapshot();
         assertEquals(3, configState.offset());
         assertEquals(SAMPLE_CONFIGS.get(0), configState.connectorConfig(CONNECTOR_IDS.get(0)));
@@ -233,17 +237,17 @@
         // Task configs should read to end, write to the log, read to end, write root, then read to end again
         expectReadToEnd(new LinkedHashMap<String, byte[]>());
         expectConvertWriteRead(
-                TASK_CONFIG_KEYS.get(0), KafkaConfigStorage.TASK_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(0),
+                TASK_CONFIG_KEYS.get(0), KafkaConfigBackingStore.TASK_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(0),
                 "properties", SAMPLE_CONFIGS.get(0));
         expectConvertWriteRead(
-                TASK_CONFIG_KEYS.get(1), KafkaConfigStorage.TASK_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(1),
+                TASK_CONFIG_KEYS.get(1), KafkaConfigBackingStore.TASK_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(1),
                 "properties", SAMPLE_CONFIGS.get(1));
         expectReadToEnd(new LinkedHashMap<String, byte[]>());
         expectConvertWriteRead(
-                COMMIT_TASKS_CONFIG_KEYS.get(0), KafkaConfigStorage.CONNECTOR_TASKS_COMMIT_V0, CONFIGS_SERIALIZED.get(2),
+                COMMIT_TASKS_CONFIG_KEYS.get(0), KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0, CONFIGS_SERIALIZED.get(2),
                 "tasks", 2); // Starts with 0 tasks, after update has 2
         // As soon as root is rewritten, we should see a callback notifying us that we reconfigured some tasks
-        tasksReconfiguredCallback.onCompletion(null, Arrays.asList(TASK_IDS.get(0), TASK_IDS.get(1)));
+        configUpdateListener.onTaskConfigUpdate(Arrays.asList(TASK_IDS.get(0), TASK_IDS.get(1)));
         EasyMock.expectLastCall();
 
         // Records to be read by consumer as it reads to the end of the log
@@ -272,10 +276,8 @@
 
         // Writing task task configs should block until all the writes have been performed and the root record update
         // has completed
-        Map<ConnectorTaskId, Map<String, String>> taskConfigs = new HashMap<>();
-        taskConfigs.put(TASK_IDS.get(0), SAMPLE_CONFIGS.get(0));
-        taskConfigs.put(TASK_IDS.get(1), SAMPLE_CONFIGS.get(1));
-        configStorage.putTaskConfigs(taskConfigs);
+        List<Map<String, String>> taskConfigs = Arrays.asList(SAMPLE_CONFIGS.get(0), SAMPLE_CONFIGS.get(1));
+        configStorage.putTaskConfigs("connector1", taskConfigs);
 
         // Validate root config by listing all connectors and tasks
         configState = configStorage.snapshot();
@@ -293,6 +295,57 @@
     }
 
     @Test
+    public void testPutTaskConfigsZeroTasks() throws Exception {
+        expectConfigure();
+        expectStart(Collections.EMPTY_LIST, Collections.EMPTY_MAP);
+
+        // Task configs should read to end, write to the log, read to end, write root.
+        expectReadToEnd(new LinkedHashMap<String, byte[]>());
+        expectConvertWriteRead(
+            COMMIT_TASKS_CONFIG_KEYS.get(0), KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0, CONFIGS_SERIALIZED.get(0),
+            "tasks", 0); // We have 0 tasks
+        // As soon as root is rewritten, we should see a callback notifying us that we reconfigured some tasks
+        configUpdateListener.onTaskConfigUpdate(Collections.<ConnectorTaskId>emptyList());
+        EasyMock.expectLastCall();
+
+        // Records to be read by consumer as it reads to the end of the log
+        LinkedHashMap<String, byte[]> serializedConfigs = new LinkedHashMap<>();
+        serializedConfigs.put(COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0));
+        expectReadToEnd(serializedConfigs);
+
+        expectStop();
+
+        PowerMock.replayAll();
+
+        configStorage.configure(DEFAULT_DISTRIBUTED_CONFIG);
+        configStorage.start();
+
+        // Bootstrap as if we had already added the connector, but no tasks had been added yet
+        whiteboxAddConnector(CONNECTOR_IDS.get(0), SAMPLE_CONFIGS.get(0), Collections.EMPTY_LIST);
+
+        // Null before writing
+        ClusterConfigState configState = configStorage.snapshot();
+        assertEquals(-1, configState.offset());
+
+        // Writing task task configs should block until all the writes have been performed and the root record update
+        // has completed
+        List<Map<String, String>> taskConfigs = Collections.emptyList();
+        configStorage.putTaskConfigs("connector1", taskConfigs);
+
+        // Validate root config by listing all connectors and tasks
+        configState = configStorage.snapshot();
+        assertEquals(1, configState.offset());
+        String connectorName = CONNECTOR_IDS.get(0);
+        assertEquals(Arrays.asList(connectorName), new ArrayList<>(configState.connectors()));
+        assertEquals(Collections.emptyList(), configState.tasks(connectorName));
+        assertEquals(Collections.EMPTY_SET, configState.inconsistentConnectors());
+
+        configStorage.stop();
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
     public void testRestore() throws Exception {
         // Restoring data should notify only of the latest values after loading is complete. This also validates
         // that inconsistent state is ignored.
@@ -347,10 +400,63 @@
     }
 
     @Test
+    public void testRestoreZeroTasks() throws Exception {
+        // Restoring data should notify only of the latest values after loading is complete. This also validates
+        // that inconsistent state is ignored.
+        expectConfigure();
+        // Overwrite each type at least once to ensure we see the latest data after loading
+        List<ConsumerRecord<String, byte[]>> existingRecords = Arrays.asList(
+            new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)),
+            new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(1)),
+            new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)),
+            new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(3)),
+            new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(4)),
+            // Connector after root update should make it through, task update shouldn't
+            new ConsumerRecord<>(TOPIC, 0, 5, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5)),
+            new ConsumerRecord<>(TOPIC, 0, 6, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(6)),
+            new ConsumerRecord<>(TOPIC, 0, 7, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(7)));
+        LinkedHashMap<byte[], Struct> deserialized = new LinkedHashMap();
+        deserialized.put(CONFIGS_SERIALIZED.get(0), CONNECTOR_CONFIG_STRUCTS.get(0));
+        deserialized.put(CONFIGS_SERIALIZED.get(1), TASK_CONFIG_STRUCTS.get(0));
+        deserialized.put(CONFIGS_SERIALIZED.get(2), TASK_CONFIG_STRUCTS.get(0));
+        deserialized.put(CONFIGS_SERIALIZED.get(3), CONNECTOR_CONFIG_STRUCTS.get(1));
+        deserialized.put(CONFIGS_SERIALIZED.get(4), TASKS_COMMIT_STRUCT_TWO_TASK_CONNECTOR);
+        deserialized.put(CONFIGS_SERIALIZED.get(5), CONNECTOR_CONFIG_STRUCTS.get(2));
+        deserialized.put(CONFIGS_SERIALIZED.get(6), TASK_CONFIG_STRUCTS.get(1));
+        deserialized.put(CONFIGS_SERIALIZED.get(7), TASKS_COMMIT_STRUCT_ZERO_TASK_CONNECTOR);
+        logOffset = 8;
+        expectStart(existingRecords, deserialized);
+
+        // Shouldn't see any callbacks since this is during startup
+
+        expectStop();
+
+        PowerMock.replayAll();
+
+        configStorage.configure(DEFAULT_DISTRIBUTED_CONFIG);
+        configStorage.start();
+
+        // Should see a single connector and its config should be the last one seen anywhere in the log
+        ClusterConfigState configState = configStorage.snapshot();
+        assertEquals(8, configState.offset()); // Should always be next to be read, even if uncommitted
+        assertEquals(Arrays.asList(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors()));
+        // CONNECTOR_CONFIG_STRUCTS[2] -> SAMPLE_CONFIGS[2]
+        assertEquals(SAMPLE_CONFIGS.get(2), configState.connectorConfig(CONNECTOR_IDS.get(0)));
+        // Should see 0 tasks for that connector.
+        assertEquals(Collections.emptyList(), configState.tasks(CONNECTOR_IDS.get(0)));
+        // Both TASK_CONFIG_STRUCTS[0] -> SAMPLE_CONFIGS[0]
+        assertEquals(Collections.EMPTY_SET, configState.inconsistentConnectors());
+
+        configStorage.stop();
+
+        PowerMock.verifyAll();
+    }
+
+    @Test
     public void testPutTaskConfigsDoesNotResolveAllInconsistencies() throws Exception {
         // Test a case where a failure and compaction has left us in an inconsistent state when reading the log.
-        // We start out by loading an initial configuration where we started to write a task update and failed before
-        // writing an the commit, and then compaction cleaned up the earlier record.
+        // We start out by loading an initial configuration where we started to write a task update, and then
+        // compaction cleaned up the earlier record.
 
         expectConfigure();
         List<ConsumerRecord<String, byte[]>> existingRecords = Arrays.asList(
@@ -368,20 +474,17 @@
         logOffset = 6;
         expectStart(existingRecords, deserialized);
 
-        // One failed attempt to write new task configs
-        expectReadToEnd(new LinkedHashMap<String, byte[]>());
-
         // Successful attempt to write new task config
         expectReadToEnd(new LinkedHashMap<String, byte[]>());
         expectConvertWriteRead(
-                TASK_CONFIG_KEYS.get(0), KafkaConfigStorage.TASK_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(0),
+                TASK_CONFIG_KEYS.get(0), KafkaConfigBackingStore.TASK_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(0),
                 "properties", SAMPLE_CONFIGS.get(0));
         expectReadToEnd(new LinkedHashMap<String, byte[]>());
         expectConvertWriteRead(
-                COMMIT_TASKS_CONFIG_KEYS.get(0), KafkaConfigStorage.CONNECTOR_TASKS_COMMIT_V0, CONFIGS_SERIALIZED.get(2),
+                COMMIT_TASKS_CONFIG_KEYS.get(0), KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0, CONFIGS_SERIALIZED.get(2),
                 "tasks", 1); // Updated to just 1 task
         // As soon as root is rewritten, we should see a callback notifying us that we reconfigured some tasks
-        tasksReconfiguredCallback.onCompletion(null, Arrays.asList(TASK_IDS.get(0)));
+        configUpdateListener.onTaskConfigUpdate(Arrays.asList(TASK_IDS.get(0)));
         EasyMock.expectLastCall();
         // Records to be read by consumer as it reads to the end of the log
         LinkedHashMap<String, byte[]> serializedConfigs = new LinkedHashMap<>();
@@ -389,7 +492,6 @@
         serializedConfigs.put(COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(2));
         expectReadToEnd(serializedConfigs);
 
-
         expectStop();
 
         PowerMock.replayAll();
@@ -407,17 +509,9 @@
         assertNull(configState.taskConfig(TASK_IDS.get(1)));
         assertEquals(Collections.singleton(CONNECTOR_IDS.get(0)), configState.inconsistentConnectors());
 
-        // First try sending an invalid set of configs (can't possibly represent a valid config set for the tasks)
-        try {
-            configStorage.putTaskConfigs(Collections.singletonMap(TASK_IDS.get(1), SAMPLE_CONFIGS.get(2)));
-            fail("Should have failed due to incomplete task set.");
-        } catch (KafkaException e) {
-            // expected
-        }
-
         // Next, issue a write that has everything that is needed and it should be accepted. Note that in this case
         // we are going to shrink the number of tasks to 1
-        configStorage.putTaskConfigs(Collections.singletonMap(TASK_IDS.get(0), SAMPLE_CONFIGS.get(0)));
+        configStorage.putTaskConfigs("connector1", Collections.singletonList(SAMPLE_CONFIGS.get(0)));
         // Validate updated config
         configState = configStorage.snapshot();
         // This is only two more ahead of the last one because multiple calls fail, and so their configs are not written
@@ -504,7 +598,7 @@
 
 
     private void expectConvertWriteAndRead(final String configKey, final Schema valueSchema, final byte[] serialized,
-                                                                 final String dataFieldName, final Object dataFieldValue) {
+                                           final String dataFieldName, final Object dataFieldValue) {
         expectConvertWriteRead(configKey, valueSchema, serialized, dataFieldName, dataFieldValue);
         LinkedHashMap<String, byte[]> recordsToRead = new LinkedHashMap<>();
         recordsToRead.put(configKey, serialized);
diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala
index 24174be..a8a282e 100644
--- a/core/src/main/scala/kafka/admin/AdminUtils.scala
+++ b/core/src/main/scala/kafka/admin/AdminUtils.scala
@@ -605,10 +605,10 @@
             new MetadataResponse.PartitionMetadata(Errors.forException(e), partition, leaderInfo, replicaInfo.asJava, isrInfo.asJava)
         }
       }
-      new MetadataResponse.TopicMetadata(Errors.NONE, topic, partitionMetadata.toList.asJava)
+      new MetadataResponse.TopicMetadata(Errors.NONE, topic, Topic.isInternal(topic), partitionMetadata.asJava)
     } else {
       // topic doesn't exist, send appropriate error code
-      new MetadataResponse.TopicMetadata(Errors.UNKNOWN_TOPIC_OR_PARTITION, topic, java.util.Collections.emptyList())
+      new MetadataResponse.TopicMetadata(Errors.UNKNOWN_TOPIC_OR_PARTITION, topic, Topic.isInternal(topic), java.util.Collections.emptyList())
     }
   }
 
diff --git a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala
index 446ab9f..1bf351a 100755
--- a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala
+++ b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala
@@ -39,7 +39,7 @@
     CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.zkConnectOpt)
 
     val zkConnect = opts.options.valueOf(opts.zkConnectOpt)
-    val zkUtils = ZkUtils(zkConnect, 
+    val zkUtils = ZkUtils(zkConnect,
                           30000,
                           30000,
                           JaasUtils.isZkSecurityEnabled())
@@ -93,8 +93,8 @@
     val topicsToMoveJsonString = Utils.readFileAsString(topicsToMoveJsonFile)
     val disableRackAware = opts.options.has(opts.disableRackAware)
     val (proposedAssignments, currentAssignments) = generateAssignment(zkUtils, brokerListToReassign, topicsToMoveJsonString, disableRackAware)
-    println("Current partition replica assignment\n\n%s".format(zkUtils.getPartitionReassignmentZkData(currentAssignments)))
-    println("Proposed partition reassignment configuration\n\n%s".format(zkUtils.getPartitionReassignmentZkData(proposedAssignments)))
+    println("Current partition replica assignment\n\n%s".format(zkUtils.formatAsReassignmentJson(currentAssignments)))
+    println("Proposed partition reassignment configuration\n\n%s".format(zkUtils.formatAsReassignmentJson(proposedAssignments)))
   }
 
   def generateAssignment(zkUtils: ZkUtils, brokerListToReassign: Seq[Int], topicsToMoveJsonString: String, disableRackAware: Boolean): (Map[TopicAndPartition, Seq[Int]], Map[TopicAndPartition, Seq[Int]]) = {
@@ -125,9 +125,14 @@
       CommandLineUtils.printUsageAndDie(opts.parser, "If --execute option is used, command must include --reassignment-json-file that was output " + "during the --generate option")
     val reassignmentJsonFile =  opts.options.valueOf(opts.reassignmentJsonFileOpt)
     val reassignmentJsonString = Utils.readFileAsString(reassignmentJsonFile)
+    executeAssignment(zkUtils, reassignmentJsonString)
+  }
+
+  def executeAssignment(zkUtils: ZkUtils,reassignmentJsonString: String){
+
     val partitionsToBeReassigned = zkUtils.parsePartitionReassignmentDataWithoutDedup(reassignmentJsonString)
     if (partitionsToBeReassigned.isEmpty)
-      throw new AdminCommandFailedException("Partition reassignment data file %s is empty".format(reassignmentJsonFile))
+      throw new AdminCommandFailedException("Partition reassignment data file is empty")
     val duplicateReassignedPartitions = CoreUtils.duplicates(partitionsToBeReassigned.map{ case(tp,replicas) => tp})
     if (duplicateReassignedPartitions.nonEmpty)
       throw new AdminCommandFailedException("Partition reassignment contains duplicate topic partitions: %s".format(duplicateReassignedPartitions.mkString(",")))
@@ -144,10 +149,10 @@
     // before starting assignment, output the current replica assignment to facilitate rollback
     val currentPartitionReplicaAssignment = zkUtils.getReplicaAssignmentForTopics(partitionsToBeReassigned.map(_._1.topic))
     println("Current partition replica assignment\n\n%s\n\nSave this to use as the --reassignment-json-file option during rollback"
-      .format(zkUtils.getPartitionReassignmentZkData(currentPartitionReplicaAssignment)))
+      .format(zkUtils.formatAsReassignmentJson(currentPartitionReplicaAssignment)))
     // start the reassignment
     if(reassignPartitionsCommand.reassignPartitions())
-      println("Successfully started reassignment of partitions %s".format(zkUtils.getPartitionReassignmentZkData(partitionsToBeReassigned.toMap)))
+      println("Successfully started reassignment of partitions %s".format(zkUtils.formatAsReassignmentJson(partitionsToBeReassigned.toMap)))
     else
       println("Failed to reassign partitions %s".format(partitionsToBeReassigned))
   }
@@ -228,7 +233,7 @@
         false
       }
       else {
-        val jsonReassignmentData = zkUtils.getPartitionReassignmentZkData(validPartitions)
+        val jsonReassignmentData = zkUtils.formatAsReassignmentJson(validPartitions)
         zkUtils.createPersistentPath(ZkUtils.ReassignPartitionsPath, jsonReassignmentData)
         true
       }
diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala
index dd4ea88..9f1014f 100755
--- a/core/src/main/scala/kafka/admin/TopicCommand.scala
+++ b/core/src/main/scala/kafka/admin/TopicCommand.scala
@@ -170,7 +170,7 @@
     }
     topics.foreach { topic =>
       try {
-        if (TopicConstants.INTERNAL_TOPICS.contains(topic)) {
+        if (Topic.isInternal(topic)) {
           throw new AdminOperationException("Topic %s is a kafka internal topic and is not allowed to be marked for deletion.".format(topic))
         } else {
           zkUtils.createPersistentPath(getDeleteTopicPath(topic))
@@ -379,7 +379,7 @@
   def shortMessageSizeWarning(maxMessageBytes: Int): String = {
     "\n\n" +
       "*****************************************************************************************************\n" +
-      "*** WARNING: you are creating a topic where the the max.message.bytes is greater than the consumer ***\n" +
+      "*** WARNING: you are creating a topic where the max.message.bytes is greater than the consumer ***\n" +
       "*** default. This operation is potentially dangerous. Consumers will get failures if their        ***\n" +
       "*** fetch.message.max.bytes < the value you are using.                                            ***\n" +
       "*****************************************************************************************************\n" +
diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala
index 77b85e0..61290c1 100755
--- a/core/src/main/scala/kafka/cluster/Broker.scala
+++ b/core/src/main/scala/kafka/cluster/Broker.scala
@@ -5,7 +5,7 @@
  * The ASF licenses this file to You under the Apache License, Version 2.0
  * (the "License"); you may not use this file except in compliance with
  * the License.  You may obtain a copy of the License at
- * 
+ *
  *    http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
@@ -123,7 +123,7 @@
   def getNode(protocolType: SecurityProtocol): Node = {
     val endpoint = endPoints.getOrElse(protocolType,
       throw new BrokerEndPointNotAvailableException(s"End point with security protocol $protocolType not found for broker $id"))
-    new Node(id, endpoint.host, endpoint.port)
+    new Node(id, endpoint.host, endpoint.port, rack.orNull)
   }
 
   def getBrokerEndPoint(protocolType: SecurityProtocol): BrokerEndPoint = {
diff --git a/core/src/main/scala/kafka/common/ErrorMapping.scala b/core/src/main/scala/kafka/common/ErrorMapping.scala
index 9708c4e..91a1d75 100644
--- a/core/src/main/scala/kafka/common/ErrorMapping.scala
+++ b/core/src/main/scala/kafka/common/ErrorMapping.scala
@@ -63,6 +63,8 @@
   val GroupAuthorizationCode: Short = 30
   val ClusterAuthorizationCode: Short = 31
   // 32: INVALID_TIMESTAMP
+  // 33: UNSUPPORTED_SASL_MECHANISM
+  // 34: ILLEGAL_SASL_STATE
 
   private val exceptionToCode =
     Map[Class[Throwable], Short](
diff --git a/core/src/main/scala/kafka/common/Topic.scala b/core/src/main/scala/kafka/common/Topic.scala
index 6067712..054c5eb 100644
--- a/core/src/main/scala/kafka/common/Topic.scala
+++ b/core/src/main/scala/kafka/common/Topic.scala
@@ -18,7 +18,7 @@
 package kafka.common
 
 import util.matching.Regex
-import kafka.coordinator.GroupCoordinator
+import org.apache.kafka.common.internals.TopicConstants.INTERNAL_TOPICS
 
 object Topic {
   val legalChars = "[a-zA-Z0-9\\._\\-]"
@@ -62,4 +62,7 @@
     topicA.replace('.', '_') == topicB.replace('.', '_')
   }
 
+  def isInternal(topic: String): Boolean =
+    INTERNAL_TOPICS.contains(topic)
+
 }
diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala
index 07e974c..e1c792d 100644
--- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala
+++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala
@@ -21,6 +21,7 @@
 import kafka.api.OffsetRequest
 import kafka.utils._
 import kafka.common.{InvalidConfigException, Config}
+import java.util.Locale
 
 object ConsumerConfig extends Config {
   val RefreshMetadataBackoffMs = 200
@@ -163,7 +164,7 @@
   val offsetsCommitMaxRetries = props.getInt("offsets.commit.max.retries", OffsetsCommitMaxRetries)
 
   /** Specify whether offsets should be committed to "zookeeper" (default) or "kafka" */
-  val offsetsStorage = props.getString("offsets.storage", OffsetsStorage).toLowerCase
+  val offsetsStorage = props.getString("offsets.storage", OffsetsStorage).toLowerCase(Locale.ROOT)
 
   /** If you are using "kafka" as offsets.storage, you can dual commit offsets to ZooKeeper (in addition to Kafka). This
     * is required during migration from zookeeper-based offset storage to kafka-based offset storage. With respect to any
diff --git a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala
index 0735651..df98db7 100755
--- a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala
+++ b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala
@@ -104,10 +104,10 @@
                            currentTopicInfo.partitionId,
                            item.message,
                            item.offset,
-                           item.message.timestamp,
-                           item.message.timestampType,
                            keyDecoder,
-                           valueDecoder)
+                           valueDecoder,
+                           item.message.timestamp,
+                           item.message.timestampType)
   }
 
   def clearCurrentChunk() {
diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
index e9731fd..018946e 100755
--- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
+++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
@@ -89,6 +89,14 @@
     val brokerEndPoint = broker.getBrokerEndPoint(config.interBrokerSecurityProtocol)
     val brokerNode = new Node(broker.id, brokerEndPoint.host, brokerEndPoint.port)
     val networkClient = {
+      val channelBuilder = ChannelBuilders.create(
+        config.interBrokerSecurityProtocol,
+        Mode.CLIENT,
+        LoginType.SERVER,
+        config.values,
+        config.saslMechanismInterBrokerProtocol,
+        config.saslInterBrokerHandshakeRequestEnable
+      )
       val selector = new Selector(
         NetworkReceive.UNLIMITED,
         config.connectionsMaxIdleMs,
@@ -97,7 +105,7 @@
         "controller-channel",
         Map("broker-id" -> broker.id.toString).asJava,
         false,
-        ChannelBuilders.create(config.interBrokerSecurityProtocol, Mode.CLIENT, LoginType.SERVER, config.values)
+        channelBuilder
       )
       new NetworkClient(
         selector,
diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala
index 103f6cf..6c503a5 100755
--- a/core/src/main/scala/kafka/controller/KafkaController.scala
+++ b/core/src/main/scala/kafka/controller/KafkaController.scala
@@ -23,15 +23,16 @@
 import org.apache.kafka.common.requests.{AbstractRequest, AbstractRequestResponse}
 
 import scala.collection._
-import com.yammer.metrics.core.Gauge
+import com.yammer.metrics.core.{Gauge, Meter}
 import java.util.concurrent.TimeUnit
+
 import kafka.admin.AdminUtils
 import kafka.admin.PreferredReplicaLeaderElectionCommand
 import kafka.api._
 import kafka.cluster.Broker
 import kafka.common._
 import kafka.log.LogConfig
-import kafka.metrics.{KafkaTimer, KafkaMetricsGroup}
+import kafka.metrics.{KafkaMetricsGroup, KafkaTimer}
 import kafka.utils.ZkUtils._
 import kafka.utils._
 import kafka.utils.CoreUtils._
@@ -39,8 +40,9 @@
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.utils.Time
 import org.I0Itec.zkclient.{IZkChildListener, IZkDataListener, IZkStateListener, ZkClient, ZkConnection}
-import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException}
+import org.I0Itec.zkclient.exception.{ZkNoNodeException, ZkNodeExistsException}
 import java.util.concurrent.locks.ReentrantLock
+
 import kafka.server._
 import kafka.common.TopicAndPartition
 
@@ -1468,6 +1470,13 @@
 }
 
 object ControllerStats extends KafkaMetricsGroup {
-  val uncleanLeaderElectionRate = newMeter("UncleanLeaderElectionsPerSec", "elections", TimeUnit.SECONDS)
-  val leaderElectionTimer = new KafkaTimer(newTimer("LeaderElectionRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS))
+
+  private val _uncleanLeaderElectionRate = newMeter("UncleanLeaderElectionsPerSec", "elections", TimeUnit.SECONDS)
+  private val _leaderElectionTimer = new KafkaTimer(newTimer("LeaderElectionRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS))
+
+  // KafkaServer needs to initialize controller metrics during startup. We perform initialization
+  // through method calls to avoid Scala compiler warnings.
+  def uncleanLeaderElectionRate: Meter = _uncleanLeaderElectionRate
+
+  def leaderElectionTimer: KafkaTimer = _leaderElectionTimer
 }
diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
index 30a3a78..fb71254 100644
--- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
+++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
@@ -51,6 +51,8 @@
                        val groupConfig: GroupConfig,
                        val offsetConfig: OffsetConfig,
                        val groupManager: GroupMetadataManager,
+                       val heartbeatPurgatory: DelayedOperationPurgatory[DelayedHeartbeat],
+                       val joinPurgatory: DelayedOperationPurgatory[DelayedJoin],
                        time: Time) extends Logging {
   type JoinCallback = JoinGroupResult => Unit
   type SyncCallback = (Array[Byte], Short) => Unit
@@ -59,9 +61,6 @@
 
   private val isActive = new AtomicBoolean(false)
 
-  private var heartbeatPurgatory: DelayedOperationPurgatory[DelayedHeartbeat] = null
-  private var joinPurgatory: DelayedOperationPurgatory[DelayedJoin] = null
-
   def offsetsTopicConfigs: Properties = {
     val props = new Properties
     props.put(LogConfig.CleanupPolicyProp, LogConfig.Compact)
@@ -80,8 +79,6 @@
    */
   def startup() {
     info("Starting up.")
-    heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", brokerId)
-    joinPurgatory = new DelayedOperationPurgatory[DelayedJoin]("Rebalance", brokerId)
     isActive.set(true)
     info("Startup complete.")
   }
@@ -414,6 +411,8 @@
           } else if (generationId != group.generationId) {
             responseCallback(offsetMetadata.mapValues(_ => Errors.ILLEGAL_GENERATION.code))
           } else {
+            val member = group.get(memberId)
+            completeAndScheduleNextHeartbeatExpiration(group, member)
             delayedOffsetStore = Some(groupManager.prepareStoreOffsets(groupId, memberId, generationId,
               offsetMetadata, responseCallback))
           }
@@ -729,6 +728,17 @@
             zkUtils: ZkUtils,
             replicaManager: ReplicaManager,
             time: Time): GroupCoordinator = {
+    val heartbeatPurgatory = DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", config.brokerId)
+    val joinPurgatory = DelayedOperationPurgatory[DelayedJoin]("Rebalance", config.brokerId)
+    apply(config, zkUtils, replicaManager, heartbeatPurgatory, joinPurgatory, time)
+  }
+
+  def apply(config: KafkaConfig,
+            zkUtils: ZkUtils,
+            replicaManager: ReplicaManager,
+            heartbeatPurgatory: DelayedOperationPurgatory[DelayedHeartbeat],
+            joinPurgatory: DelayedOperationPurgatory[DelayedJoin],
+            time: Time): GroupCoordinator = {
     val offsetConfig = OffsetConfig(maxMetadataSize = config.offsetMetadataMaxSize,
       loadBufferSize = config.offsetsLoadBufferSize,
       offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L,
@@ -741,7 +751,7 @@
       groupMaxSessionTimeoutMs = config.groupMaxSessionTimeoutMs)
 
     val groupManager = new GroupMetadataManager(config.brokerId, offsetConfig, replicaManager, zkUtils, time)
-    new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, groupManager, time)
+    new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, groupManager, heartbeatPurgatory, joinPurgatory, time)
   }
 
 }
diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala
index 698464e..d5e06fa 100755
--- a/core/src/main/scala/kafka/log/LogConfig.scala
+++ b/core/src/main/scala/kafka/log/LogConfig.scala
@@ -27,6 +27,7 @@
 import org.apache.kafka.common.config.{AbstractConfig, ConfigDef}
 import org.apache.kafka.common.record.TimestampType
 import org.apache.kafka.common.utils.Utils
+import java.util.Locale
 
 object Defaults {
   val SegmentSize = kafka.server.Defaults.LogSegmentBytes
@@ -70,10 +71,10 @@
   val fileDeleteDelayMs = getLong(LogConfig.FileDeleteDelayMsProp)
   val deleteRetentionMs = getLong(LogConfig.DeleteRetentionMsProp)
   val minCleanableRatio = getDouble(LogConfig.MinCleanableDirtyRatioProp)
-  val compact = getString(LogConfig.CleanupPolicyProp).toLowerCase != LogConfig.Delete
+  val compact = getString(LogConfig.CleanupPolicyProp).toLowerCase(Locale.ROOT) != LogConfig.Delete
   val uncleanLeaderElectionEnable = getBoolean(LogConfig.UncleanLeaderElectionEnableProp)
   val minInSyncReplicas = getInt(LogConfig.MinInSyncReplicasProp)
-  val compressionType = getString(LogConfig.CompressionTypeProp).toLowerCase
+  val compressionType = getString(LogConfig.CompressionTypeProp).toLowerCase(Locale.ROOT)
   val preallocate = getBoolean(LogConfig.PreAllocateEnableProp)
   val messageFormatVersion = ApiVersion(getString(LogConfig.MessageFormatVersionProp))
   val messageTimestampType = TimestampType.forName(getString(LogConfig.MessageTimestampTypeProp))
diff --git a/core/src/main/scala/kafka/message/CompressionCodec.scala b/core/src/main/scala/kafka/message/CompressionCodec.scala
index 4d7ce17..a485271 100644
--- a/core/src/main/scala/kafka/message/CompressionCodec.scala
+++ b/core/src/main/scala/kafka/message/CompressionCodec.scala
@@ -17,6 +17,8 @@
 
 package kafka.message
 
+import java.util.Locale
+
 object CompressionCodec {
   def getCompressionCodec(codec: Int): CompressionCodec = {
     codec match {
@@ -28,7 +30,7 @@
     }
   }
   def getCompressionCodec(name: String): CompressionCodec = {
-    name.toLowerCase match {
+    name.toLowerCase(Locale.ROOT) match {
       case NoCompressionCodec.name => NoCompressionCodec
       case GZIPCompressionCodec.name => GZIPCompressionCodec
       case SnappyCompressionCodec.name => SnappyCompressionCodec
@@ -43,10 +45,10 @@
   val brokerCompressionCodecs = List(UncompressedCodec, SnappyCompressionCodec, LZ4CompressionCodec, GZIPCompressionCodec, ProducerCompressionCodec)
   val brokerCompressionOptions = brokerCompressionCodecs.map(codec => codec.name)
 
-  def isValid(compressionType: String): Boolean = brokerCompressionOptions.contains(compressionType.toLowerCase())
+  def isValid(compressionType: String): Boolean = brokerCompressionOptions.contains(compressionType.toLowerCase(Locale.ROOT))
 
   def getCompressionCodec(compressionType: String): CompressionCodec = {
-    compressionType.toLowerCase match {
+    compressionType.toLowerCase(Locale.ROOT) match {
       case UncompressedCodec.name => NoCompressionCodec
       case _ => CompressionCodec.getCompressionCodec(compressionType)
     }
diff --git a/core/src/main/scala/kafka/message/MessageAndMetadata.scala b/core/src/main/scala/kafka/message/MessageAndMetadata.scala
index ac9ef77..5c09caf 100755
--- a/core/src/main/scala/kafka/message/MessageAndMetadata.scala
+++ b/core/src/main/scala/kafka/message/MessageAndMetadata.scala
@@ -25,10 +25,10 @@
                                     partition: Int,
                                     private val rawMessage: Message,
                                     offset: Long,
+                                    keyDecoder: Decoder[K], valueDecoder: Decoder[V],
                                     timestamp: Long = Message.NoTimestamp,
-                                    timestampType: TimestampType = TimestampType.CREATE_TIME,
-                                    keyDecoder: Decoder[K], valueDecoder: Decoder[V]) {
-  
+                                    timestampType: TimestampType = TimestampType.CREATE_TIME) {
+
   /**
    * Return the decoded message key and payload
    */
diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala
index f1ec2ef..b757abd 100644
--- a/core/src/main/scala/kafka/network/SocketServer.scala
+++ b/core/src/main/scala/kafka/network/SocketServer.scala
@@ -399,7 +399,7 @@
     "socket-server",
     metricTags,
     false,
-    ChannelBuilders.create(protocol, Mode.SERVER, LoginType.SERVER, channelConfigs))
+    ChannelBuilders.create(protocol, Mode.SERVER, LoginType.SERVER, channelConfigs, null, true))
 
   override def run() {
     startupComplete()
diff --git a/core/src/main/scala/kafka/server/DelayedOperation.scala b/core/src/main/scala/kafka/server/DelayedOperation.scala
index 0b53532..2205568 100644
--- a/core/src/main/scala/kafka/server/DelayedOperation.scala
+++ b/core/src/main/scala/kafka/server/DelayedOperation.scala
@@ -47,9 +47,7 @@
  *
  * A subclass of DelayedOperation needs to provide an implementation of both onComplete() and tryComplete().
  */
-abstract class DelayedOperation(delayMs: Long) extends TimerTask with Logging {
-
-  override val expirationMs = delayMs + System.currentTimeMillis()
+abstract class DelayedOperation(override val delayMs: Long) extends TimerTask with Logging {
 
   private val completed = new AtomicBoolean(false)
 
@@ -110,19 +108,27 @@
   }
 }
 
+object DelayedOperationPurgatory {
+
+  def apply[T <: DelayedOperation](purgatoryName: String,
+                                   brokerId: Int = 0,
+                                   purgeInterval: Int = 1000): DelayedOperationPurgatory[T] = {
+    val timer = new SystemTimer(purgatoryName)
+    new DelayedOperationPurgatory[T](purgatoryName, timer, brokerId, purgeInterval)
+  }
+
+}
+
 /**
  * A helper purgatory class for bookkeeping delayed operations with a timeout, and expiring timed out operations.
  */
-class DelayedOperationPurgatory[T <: DelayedOperation](purgatoryName: String, brokerId: Int = 0, purgeInterval: Int = 1000)
+class DelayedOperationPurgatory[T <: DelayedOperation](purgatoryName: String,
+                                                       timeoutTimer: Timer,
+                                                       brokerId: Int = 0,
+                                                       purgeInterval: Int = 1000,
+                                                       reaperEnabled: Boolean = true)
         extends Logging with KafkaMetricsGroup {
 
-  // timeout timer
-  private[this] val executor = Executors.newFixedThreadPool(1, new ThreadFactory() {
-    def newThread(runnable: Runnable): Thread =
-      Utils.newThread("executor-"+purgatoryName, runnable, false)
-  })
-  private[this] val timeoutTimer = new Timer(executor)
-
   /* a list of operation watching keys */
   private val watchersForKey = new Pool[Any, Watchers](Some((key: Any) => new Watchers(key)))
 
@@ -152,7 +158,8 @@
     metricsTags
   )
 
-  expirationReaper.start()
+  if (reaperEnabled)
+    expirationReaper.start()
 
   /**
    * Check if the operation can be completed, if not watch it based on the given watch keys
@@ -275,8 +282,9 @@
    * Shutdown the expire reaper thread
    */
   def shutdown() {
-    expirationReaper.shutdown()
-    executor.shutdown()
+    if (reaperEnabled)
+      expirationReaper.shutdown()
+    timeoutTimer.shutdown()
   }
 
   /**
@@ -338,6 +346,23 @@
     }
   }
 
+  def advanceClock(timeoutMs: Long) {
+    timeoutTimer.advanceClock(timeoutMs)
+
+    // Trigger a purge if the number of completed but still being watched operations is larger than
+    // the purge threshold. That number is computed by the difference btw the estimated total number of
+    // operations and the number of pending delayed operations.
+    if (estimatedTotalOperations.get - delayed > purgeInterval) {
+      // now set estimatedTotalOperations to delayed (the number of pending operations) since we are going to
+      // clean up watchers. Note that, if more operations are completed during the clean up, we may end up with
+      // a little overestimated total number of operations.
+      estimatedTotalOperations.getAndSet(delayed)
+      debug("Begin purging watch lists")
+      val purged = allWatchers.map(_.purgeCompleted()).sum
+      debug("Purged %d elements from watch lists.".format(purged))
+    }
+  }
+
   /**
    * A background reaper to expire delayed operations that have timed out
    */
@@ -346,20 +371,7 @@
     false) {
 
     override def doWork() {
-      timeoutTimer.advanceClock(200L)
-
-      // Trigger a purge if the number of completed but still being watched operations is larger than
-      // the purge threshold. That number is computed by the difference btw the estimated total number of
-      // operations and the number of pending delayed operations.
-      if (estimatedTotalOperations.get - delayed > purgeInterval) {
-        // now set estimatedTotalOperations to delayed (the number of pending operations) since we are going to
-        // clean up watchers. Note that, if more operations are completed during the clean up, we may end up with
-        // a little overestimated total number of operations.
-        estimatedTotalOperations.getAndSet(delayed)
-        debug("Begin purging watch lists")
-        val purged = allWatchers.map(_.purgeCompleted()).sum
-        debug("Purged %d elements from watch lists.".format(purged))
-      }
+      advanceClock(200L)
     }
   }
 }
diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala
index 4f77d30..cf7814e 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -21,34 +21,31 @@
 import java.lang.{Long => JLong, Short => JShort}
 import java.util.Properties
 
-import kafka.admin.{RackAwareMode, AdminUtils}
+import kafka.admin.{AdminUtils, RackAwareMode}
 import kafka.api._
 import kafka.cluster.Partition
+import kafka.common
 import kafka.common._
 import kafka.controller.KafkaController
 import kafka.coordinator.{GroupCoordinator, JoinGroupResult}
 import kafka.log._
 import kafka.message.{ByteBufferMessageSet, Message, MessageSet}
 import kafka.network._
-import kafka.network.RequestChannel.{Session, Response}
-import kafka.security.auth.{Authorizer, ClusterAction, Group, Create, Describe, Operation, Read, Resource, Topic, Write}
+import kafka.network.RequestChannel.{Response, Session}
+import kafka.security.auth.{Authorizer, ClusterAction, Create, Describe, Group, Operation, Read, Resource, Topic, Write}
 import kafka.utils.{Logging, SystemTime, ZKGroupTopicDirs, ZkUtils}
-import org.apache.kafka.common.errors.{InvalidTopicException, NotLeaderForPartitionException, UnknownTopicOrPartitionException,
-ClusterAuthorizationException}
+import org.apache.kafka.common.errors.{ClusterAuthorizationException, InvalidTopicException, NotLeaderForPartitionException, UnknownTopicOrPartitionException}
 import org.apache.kafka.common.metrics.Metrics
-import org.apache.kafka.common.protocol.{ApiKeys, Errors, SecurityProtocol}
-import org.apache.kafka.common.requests.{ListOffsetRequest, ListOffsetResponse, GroupCoordinatorRequest, GroupCoordinatorResponse, ListGroupsResponse,
-DescribeGroupsRequest, DescribeGroupsResponse, HeartbeatRequest, HeartbeatResponse, JoinGroupRequest, JoinGroupResponse,
-LeaveGroupRequest, LeaveGroupResponse, ResponseHeader, ResponseSend, SyncGroupRequest, SyncGroupResponse, LeaderAndIsrRequest, LeaderAndIsrResponse,
-StopReplicaRequest, StopReplicaResponse, ProduceRequest, ProduceResponse, UpdateMetadataRequest, UpdateMetadataResponse,
-MetadataRequest, MetadataResponse, OffsetCommitRequest, OffsetCommitResponse, OffsetFetchRequest, OffsetFetchResponse}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors, Protocol, SecurityProtocol}
+import org.apache.kafka.common.requests.{ApiVersionsResponse, DescribeGroupsRequest, DescribeGroupsResponse, GroupCoordinatorRequest, GroupCoordinatorResponse, HeartbeatRequest, HeartbeatResponse, JoinGroupRequest, JoinGroupResponse, LeaderAndIsrRequest, LeaderAndIsrResponse, LeaveGroupRequest, LeaveGroupResponse, ListGroupsResponse, ListOffsetRequest, ListOffsetResponse, MetadataRequest, MetadataResponse, OffsetCommitRequest, OffsetCommitResponse, OffsetFetchRequest, OffsetFetchResponse, ProduceRequest, ProduceResponse, ResponseHeader, ResponseSend, StopReplicaRequest, StopReplicaResponse, SyncGroupRequest, SyncGroupResponse, UpdateMetadataRequest, UpdateMetadataResponse}
 import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
 import org.apache.kafka.common.utils.Utils
-import org.apache.kafka.common.{TopicPartition, Node}
+import org.apache.kafka.common.{Node, TopicPartition}
 import org.apache.kafka.common.internals.TopicConstants
 
 import scala.collection._
 import scala.collection.JavaConverters._
+import org.apache.kafka.common.requests.SaslHandshakeResponse
 
 /**
  * Logic to handle the various Kafka requests
@@ -72,7 +69,7 @@
    * Top-level method that handles all requests and multiplexes to the right api
    */
   def handle(request: RequestChannel.Request) {
-    try{
+    try {
       trace("Handling request:%s from connection %s;securityProtocol:%s,principal:%s".
         format(request.requestObj, request.connectionId, request.securityProtocol, request.session.principal))
       ApiKeys.forId(request.requestId) match {
@@ -93,6 +90,8 @@
         case ApiKeys.SYNC_GROUP => handleSyncGroupRequest(request)
         case ApiKeys.DESCRIBE_GROUPS => handleDescribeGroupRequest(request)
         case ApiKeys.LIST_GROUPS => handleListGroupsRequest(request)
+        case ApiKeys.SASL_HANDSHAKE => handleSaslHandshakeRequest(request)
+        case ApiKeys.API_VERSIONS => handleApiVersionsRequest(request)
         case requestId => throw new KafkaException("Unknown api code " + requestId)
       }
     } catch {
@@ -140,7 +139,7 @@
       }
 
       val responseHeader = new ResponseHeader(correlationId)
-      val leaderAndIsrResponse=
+      val leaderAndIsrResponse =
         if (authorize(request.session, ClusterAction, Resource.ClusterResource)) {
           val result = replicaManager.becomeLeaderOrFollower(correlationId, leaderAndIsrRequest, metadataCache, onLeadershipChange)
           new LeaderAndIsrResponse(result.errorCode, result.responseMap.mapValues(new JShort(_)).asJava)
@@ -231,7 +230,7 @@
       }
       val filteredRequestInfo = offsetCommitRequest.offsetData.asScala.toMap -- invalidRequestsInfo.keys
 
-      val (authorizedRequestInfo, unauthorizedRequestInfo) =  filteredRequestInfo.partition {
+      val (authorizedRequestInfo, unauthorizedRequestInfo) = filteredRequestInfo.partition {
         case (topicPartition, offsetMetadata) => authorize(request.session, Read, new Resource(Topic, topicPartition.topic))
       }
 
@@ -248,7 +247,7 @@
         val combinedCommitStatus = mergedCommitStatus.mapValues(new JShort(_)) ++ invalidRequestsInfo.map(_._1 -> new JShort(Errors.UNKNOWN_TOPIC_OR_PARTITION.code))
 
         val responseHeader = new ResponseHeader(header.correlationId)
-        val responseBody =  new OffsetCommitResponse(combinedCommitStatus.asJava)
+        val responseBody = new OffsetCommitResponse(combinedCommitStatus.asJava)
         requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, responseHeader, responseBody)))
       }
 
@@ -373,7 +372,7 @@
           val respHeader = new ResponseHeader(request.header.correlationId)
           val respBody = request.header.apiVersion match {
             case 0 => new ProduceResponse(mergedResponseStatus.asJava)
-            case version@ (1 | 2) => new ProduceResponse(mergedResponseStatus.asJava, delayTimeMs, version)
+            case version@(1 | 2) => new ProduceResponse(mergedResponseStatus.asJava, delayTimeMs, version)
             // This case shouldn't happen unless a new version of ProducerRequest is added without
             // updating this part of the code to handle it properly.
             case version => throw new IllegalArgumentException(s"Version `$version` of ProduceRequest is not handled. Code must be updated.")
@@ -423,7 +422,7 @@
   def handleFetchRequest(request: RequestChannel.Request) {
     val fetchRequest = request.requestObj.asInstanceOf[FetchRequest]
 
-    val (authorizedRequestInfo, unauthorizedRequestInfo) =  fetchRequest.requestInfo.partition {
+    val (authorizedRequestInfo, unauthorizedRequestInfo) = fetchRequest.requestInfo.partition {
       case (topicAndPartition, _) => authorize(request.session, Read, new Resource(Topic, topicAndPartition.topic))
     }
 
@@ -549,14 +548,14 @@
         case utpe: UnknownTopicOrPartitionException =>
           debug("Offset request with correlation id %d from client %s on partition %s failed due to %s".format(
                correlationId, clientId, topicPartition, utpe.getMessage))
-          (topicPartition,  new ListOffsetResponse.PartitionData(Errors.forException(utpe).code, List[JLong]().asJava))
+          (topicPartition, new ListOffsetResponse.PartitionData(Errors.forException(utpe).code, List[JLong]().asJava))
         case nle: NotLeaderForPartitionException =>
           debug("Offset request with correlation id %d from client %s on partition %s failed due to %s".format(
                correlationId, clientId, topicPartition,nle.getMessage))
-          (topicPartition,  new ListOffsetResponse.PartitionData(Errors.forException(nle).code, List[JLong]().asJava))
+          (topicPartition, new ListOffsetResponse.PartitionData(Errors.forException(nle).code, List[JLong]().asJava))
         case e: Throwable =>
           error("Error while responding to offset request", e)
-          (topicPartition,  new ListOffsetResponse.PartitionData(Errors.forException(e).code, List[JLong]().asJava))
+          (topicPartition, new ListOffsetResponse.PartitionData(Errors.forException(e).code, List[JLong]().asJava))
       }
     })
 
@@ -588,7 +587,7 @@
     else
       offsetTimeArray = new Array[(Long, Long)](segsArray.length)
 
-    for(i <- 0 until segsArray.length)
+    for (i <- 0 until segsArray.length)
       offsetTimeArray(i) = (segsArray(i).baseOffset, segsArray(i).lastModified)
     if (segsArray.last.size > 0)
       offsetTimeArray(segsArray.length) = (log.logEndOffset, SystemTime.milliseconds)
@@ -607,18 +606,18 @@
           if (offsetTimeArray(startIndex)._2 <= timestamp)
             isFound = true
           else
-            startIndex -=1
+            startIndex -= 1
         }
     }
 
     val retSize = maxNumOffsets.min(startIndex + 1)
     val ret = new Array[Long](retSize)
-    for(j <- 0 until retSize) {
+    for (j <- 0 until retSize) {
       ret(j) = offsetTimeArray(startIndex)._1
       startIndex -= 1
     }
     // ensure that the returned seq is in descending order of offsets
-    ret.toSeq.sortBy(- _)
+    ret.toSeq.sortBy(-_)
   }
 
   private def createTopic(topic: String,
@@ -629,12 +628,15 @@
       AdminUtils.createTopic(zkUtils, topic, numPartitions, replicationFactor, properties, RackAwareMode.Safe)
       info("Auto creation of topic %s with %d partitions and replication factor %d is successful"
         .format(topic, numPartitions, replicationFactor))
-      new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, topic, java.util.Collections.emptyList())
+      new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, topic, common.Topic.isInternal(topic),
+        java.util.Collections.emptyList())
     } catch {
       case e: TopicExistsException => // let it go, possibly another broker created this topic
-        new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, topic, java.util.Collections.emptyList())
+        new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, topic, common.Topic.isInternal(topic),
+          java.util.Collections.emptyList())
       case itex: InvalidTopicException =>
-        new MetadataResponse.TopicMetadata(Errors.INVALID_TOPIC_EXCEPTION, topic, java.util.Collections.emptyList())
+        new MetadataResponse.TopicMetadata(Errors.INVALID_TOPIC_EXCEPTION, topic, common.Topic.isInternal(topic),
+          java.util.Collections.emptyList())
     }
   }
 
@@ -654,8 +656,8 @@
     topicMetadata.headOption.getOrElse(createGroupMetadataTopic())
   }
 
-  private def getTopicMetadata(topics: Set[String], securityProtocol: SecurityProtocol): Seq[MetadataResponse.TopicMetadata] = {
-    val topicResponses = metadataCache.getTopicMetadata(topics, securityProtocol)
+  private def getTopicMetadata(topics: Set[String], securityProtocol: SecurityProtocol, errorUnavailableEndpoints: Boolean): Seq[MetadataResponse.TopicMetadata] = {
+    val topicResponses = metadataCache.getTopicMetadata(topics, securityProtocol, errorUnavailableEndpoints)
     if (topics.isEmpty || topicResponses.size == topics.size) {
       topicResponses
     } else {
@@ -666,7 +668,8 @@
         } else if (config.autoCreateTopicsEnable) {
           createTopic(topic, config.numPartitions, config.defaultReplicationFactor)
         } else {
-          new MetadataResponse.TopicMetadata(Errors.UNKNOWN_TOPIC_OR_PARTITION, topic, java.util.Collections.emptyList())
+          new MetadataResponse.TopicMetadata(Errors.UNKNOWN_TOPIC_OR_PARTITION, topic, common.Topic.isInternal(topic),
+            java.util.Collections.emptyList())
         }
       }
       topicResponses ++ responsesForNonExistentTopics
@@ -678,16 +681,24 @@
    */
   def handleTopicMetadataRequest(request: RequestChannel.Request) {
     val metadataRequest = request.body.asInstanceOf[MetadataRequest]
+    val requestVersion = request.header.apiVersion()
 
-    val topics = metadataRequest.topics.asScala.toSet
-    var (authorizedTopics, unauthorizedTopics) = if (metadataRequest.topics.isEmpty) {
-      //if topics is empty -> fetch all topics metadata but filter out the topic response that are not authorized
-      val authorized = metadataCache.getAllTopics()
-        .filter(topic => authorize(request.session, Describe, new Resource(Topic, topic)))
-      (authorized, mutable.Set[String]())
-    } else {
+    val topics =
+      // Handle old metadata request logic. Version 0 has no way to specify "no topics".
+      if (requestVersion == 0) {
+        if (metadataRequest.topics() == null || metadataRequest.topics().isEmpty)
+          metadataCache.getAllTopics()
+        else
+          metadataRequest.topics.asScala.toSet
+      } else {
+        if (metadataRequest.isAllTopics)
+          metadataCache.getAllTopics()
+        else
+          metadataRequest.topics.asScala.toSet
+      }
+
+    var (authorizedTopics, unauthorizedTopics) =
       topics.partition(topic => authorize(request.session, Describe, new Resource(Topic, topic)))
-    }
 
     if (authorizedTopics.nonEmpty) {
       val nonExistingTopics = metadataCache.getNonExistingTopics(authorizedTopics)
@@ -702,22 +713,32 @@
     }
 
     val unauthorizedTopicMetadata = unauthorizedTopics.map(topic =>
-      new MetadataResponse.TopicMetadata(Errors.TOPIC_AUTHORIZATION_FAILED, topic, java.util.Collections.emptyList()))
+      new MetadataResponse.TopicMetadata(Errors.TOPIC_AUTHORIZATION_FAILED, topic, common.Topic.isInternal(topic),
+        java.util.Collections.emptyList()))
 
-    val topicMetadata = if (authorizedTopics.isEmpty)
-      Seq.empty[MetadataResponse.TopicMetadata]
-    else
-      getTopicMetadata(authorizedTopics, request.securityProtocol)
+    // In version 0, we returned an error when brokers with replicas were unavailable,
+    // while in higher versions we simply don't include the broker in the returned broker list
+    val errorUnavailableEndpoints = requestVersion == 0
+    val topicMetadata =
+      if (authorizedTopics.isEmpty)
+        Seq.empty[MetadataResponse.TopicMetadata]
+      else
+        getTopicMetadata(authorizedTopics, request.securityProtocol, errorUnavailableEndpoints)
+
+    val completeTopicMetadata = topicMetadata ++ unauthorizedTopicMetadata
 
     val brokers = metadataCache.getAliveBrokers
 
-    trace("Sending topic metadata %s and brokers %s for correlation id %d to client %s".format(topicMetadata.mkString(","),
+    trace("Sending topic metadata %s and brokers %s for correlation id %d to client %s".format(completeTopicMetadata.mkString(","),
       brokers.mkString(","), request.header.correlationId, request.header.clientId))
 
     val responseHeader = new ResponseHeader(request.header.correlationId)
+
     val responseBody = new MetadataResponse(
       brokers.map(_.getNode(request.securityProtocol)).asJava,
-      (topicMetadata ++ unauthorizedTopicMetadata).asJava
+      metadataCache.getControllerId.getOrElse(MetadataResponse.NO_CONTROLLER_ID),
+      completeTopicMetadata.asJava,
+      requestVersion
     )
     requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, responseHeader, responseBody)))
   }
@@ -846,7 +867,7 @@
       ListGroupsResponse.fromError(Errors.CLUSTER_AUTHORIZATION_FAILED)
     } else {
       val (error, groups) = coordinator.handleListGroups()
-      val allGroups = groups.map{ group => new ListGroupsResponse.Group(group.groupId, group.protocolType) }
+      val allGroups = groups.map { group => new ListGroupsResponse.Group(group.groupId, group.protocolType) }
       new ListGroupsResponse(error.code, allGroups.asJava)
     }
     requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, responseHeader, responseBody)))
@@ -993,6 +1014,29 @@
     }
   }
 
+  def handleSaslHandshakeRequest(request: RequestChannel.Request) {
+    val respHeader = new ResponseHeader(request.header.correlationId)
+    val response = new SaslHandshakeResponse(Errors.ILLEGAL_SASL_STATE.code, config.saslEnabledMechanisms)
+    requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, respHeader, response)))
+  }
+
+  def handleApiVersionsRequest(request: RequestChannel.Request) {
+    // Note that broker returns its full list of supported ApiKeys and versions regardless of current
+    // authentication state (e.g., before SASL authentication on an SASL listener, do note that no
+    // Kafka protocol requests may take place on a SSL listener before the SSL handshake is finished).
+    // If this is considered to leak information about the broker version a workaround is to use SSL
+    // with client authentication which is performed at an earlier stage of the connection where the
+    // ApiVersionRequest is not available.
+    val responseHeader = new ResponseHeader(request.header.correlationId)
+    val isApiVersionsRequestVersionSupported = request.header.apiVersion <= Protocol.CURR_VERSION(ApiKeys.API_VERSIONS.id) &&
+                                              request.header.apiVersion >= Protocol.MIN_VERSIONS(ApiKeys.API_VERSIONS.id)
+    val responseBody = if (isApiVersionsRequestVersionSupported)
+      ApiVersionsResponse.apiVersionsResponse
+    else
+      ApiVersionsResponse.fromError(Errors.UNSUPPORTED_VERSION)
+    requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, responseHeader, responseBody)))
+  }
+
   def close() {
     quotaManagers.foreach { case (apiKey, quotaManager) =>
       quotaManager.shutdown()
@@ -1004,4 +1048,4 @@
     if (!authorize(request.session, ClusterAction, Resource.ClusterResource))
       throw new ClusterAuthorizationException(s"Request $request is not authorized.")
   }
-}
+}
\ No newline at end of file
diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala
index dc2a0a0..c5da55d 100755
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -19,7 +19,7 @@
 
 import java.util.Properties
 
-import kafka.api.ApiVersion
+import kafka.api.{ApiVersion, KAFKA_0_10_0_IV0}
 import kafka.cluster.EndPoint
 import kafka.consumer.ConsumerConfig
 import kafka.coordinator.OffsetConfig
@@ -130,9 +130,9 @@
   val ControlledShutdownRetryBackoffMs = 5000
   val ControlledShutdownEnable = true
 
-  /** ********* Consumer coordinator configuration ***********/
-  val ConsumerMinSessionTimeoutMs = 6000
-  val ConsumerMaxSessionTimeoutMs = 30000
+  /** ********* Group coordinator configuration ***********/
+  val GroupMinSessionTimeoutMs = 6000
+  val GroupMaxSessionTimeoutMs = 300000
 
   /** ********* Offset management configuration ***********/
   val OffsetMetadataMaxSize = OffsetConfig.DefaultMaxMetadataSize
@@ -175,6 +175,8 @@
   val SslClientAuth = SslClientAuthNone
 
   /** ********* Sasl configuration ***********/
+  val SaslMechanismInterBrokerProtocol = SaslConfigs.DEFAULT_SASL_MECHANISM
+  val SaslEnabledMechanisms = SaslConfigs.DEFAULT_SASL_ENABLED_MECHANISMS
   val SaslKerberosKinitCmd = SaslConfigs.DEFAULT_KERBEROS_KINIT_CMD
   val SaslKerberosTicketRenewWindowFactor = SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_WINDOW_FACTOR
   val SaslKerberosTicketRenewJitter = SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_JITTER
@@ -337,6 +339,8 @@
   val SslClientAuthProp = SslConfigs.SSL_CLIENT_AUTH_CONFIG
 
   /** ********* SASL Configuration ****************/
+  val SaslMechanismInterBrokerProtocolProp = "sasl.mechanism.inter.broker.protocol"
+  val SaslEnabledMechanismsProp = SaslConfigs.SASL_ENABLED_MECHANISMS
   val SaslKerberosServiceNameProp = SaslConfigs.SASL_KERBEROS_SERVICE_NAME
   val SaslKerberosKinitCmdProp = SaslConfigs.SASL_KERBEROS_KINIT_CMD
   val SaslKerberosTicketRenewWindowFactorProp = SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR
@@ -367,7 +371,7 @@
   val AuthorizerClassNameDoc = "The authorizer class that should be used for authorization"
   /** ********* Socket Server Configuration ***********/
   val PortDoc = "DEPRECATED: only used when `listeners` is not set. " +
-  "Use `listeners` instead. \n" + 
+  "Use `listeners` instead. \n" +
   "the port to listen and accept connections on"
   val HostNameDoc = "DEPRECATED: only used when `listeners` is not set. " +
   "Use `listeners` instead. \n" +
@@ -456,7 +460,7 @@
   " the leader will remove the follower from isr"
   val ReplicaSocketTimeoutMsDoc = "The socket timeout for network requests. Its value should be at least replica.fetch.wait.max.ms"
   val ReplicaSocketReceiveBufferBytesDoc = "The socket receive buffer for network requests"
-  val ReplicaFetchMaxBytesDoc = "The number of byes of messages to attempt to fetch"
+  val ReplicaFetchMaxBytesDoc = "The number of bytes of messages to attempt to fetch"
   val ReplicaFetchWaitMaxMsDoc = "max wait time for each fetcher request issued by follower replicas. This value should always be less than the " +
   "replica.lag.time.max.ms at all times to prevent frequent shrinking of ISR for low throughput topics"
   val ReplicaFetchMinBytesDoc = "Minimum bytes expected for each fetch response. If not enough bytes, wait up to replicaMaxWaitTimeMs"
@@ -480,8 +484,8 @@
   val ControlledShutdownRetryBackoffMsDoc = "Before each retry, the system needs time to recover from the state that caused the previous failure (Controller fail over, replica lag etc). This config determines the amount of time to wait before retrying."
   val ControlledShutdownEnableDoc = "Enable controlled shutdown of the server"
   /** ********* Consumer coordinator configuration ***********/
-  val ConsumerMinSessionTimeoutMsDoc = "The minimum allowed session timeout for registered consumers. Shorter timeouts leader to quicker failure detection at the cost of more frequent consumer heartbeating, which can overwhelm broker resources."
-  val ConsumerMaxSessionTimeoutMsDoc = "The maximum allowed session timeout for registered consumers. Longer timeouts give consumers more time to process messages in between heartbeats at the cost of a longer time to detect failures."
+  val GroupMinSessionTimeoutMsDoc = "The minimum allowed session timeout for registered consumers. Shorter timeouts leader to quicker failure detection at the cost of more frequent consumer heartbeating, which can overwhelm broker resources."
+  val GroupMaxSessionTimeoutMsDoc = "The maximum allowed session timeout for registered consumers. Longer timeouts give consumers more time to process messages in between heartbeats at the cost of a longer time to detect failures."
   /** ********* Offset management configuration ***********/
   val OffsetMetadataMaxSizeDoc = "The maximum size for a metadata entry associated with an offset commit"
   val OffsetsLoadBufferSizeDoc = "Batch size for reading from the offsets segments when loading offsets into the cache."
@@ -533,6 +537,8 @@
   val SslClientAuthDoc = SslConfigs.SSL_CLIENT_AUTH_DOC
 
   /** ********* Sasl Configuration ****************/
+  val SaslMechanismInterBrokerProtocolDoc = "SASL mechanism used for inter-broker communication. Default is GSSAPI."
+  val SaslEnabledMechanismsDoc = SaslConfigs.SASL_ENABLED_MECHANISMS_DOC
   val SaslKerberosServiceNameDoc = SaslConfigs.SASL_KERBEROS_SERVICE_NAME_DOC
   val SaslKerberosKinitCmdDoc = SaslConfigs.SASL_KERBEROS_KINIT_CMD_DOC
   val SaslKerberosTicketRenewWindowFactorDoc = SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_DOC
@@ -655,9 +661,9 @@
       .define(ControlledShutdownRetryBackoffMsProp, LONG, Defaults.ControlledShutdownRetryBackoffMs, MEDIUM, ControlledShutdownRetryBackoffMsDoc)
       .define(ControlledShutdownEnableProp, BOOLEAN, Defaults.ControlledShutdownEnable, MEDIUM, ControlledShutdownEnableDoc)
 
-      /** ********* Consumer coordinator configuration ***********/
-      .define(GroupMinSessionTimeoutMsProp, INT, Defaults.ConsumerMinSessionTimeoutMs, MEDIUM, ConsumerMinSessionTimeoutMsDoc)
-      .define(GroupMaxSessionTimeoutMsProp, INT, Defaults.ConsumerMaxSessionTimeoutMs, MEDIUM, ConsumerMaxSessionTimeoutMsDoc)
+      /** ********* Group coordinator configuration ***********/
+      .define(GroupMinSessionTimeoutMsProp, INT, Defaults.GroupMinSessionTimeoutMs, MEDIUM, GroupMinSessionTimeoutMsDoc)
+      .define(GroupMaxSessionTimeoutMsProp, INT, Defaults.GroupMaxSessionTimeoutMs, MEDIUM, GroupMaxSessionTimeoutMsDoc)
 
       /** ********* Offset management configuration ***********/
       .define(OffsetMetadataMaxSizeProp, INT, Defaults.OffsetMetadataMaxSize, HIGH, OffsetMetadataMaxSizeDoc)
@@ -704,6 +710,8 @@
       .define(SslCipherSuitesProp, LIST, null, MEDIUM, SslCipherSuitesDoc)
 
       /** ********* Sasl Configuration ****************/
+      .define(SaslMechanismInterBrokerProtocolProp, STRING, Defaults.SaslMechanismInterBrokerProtocol, MEDIUM, SaslMechanismInterBrokerProtocolDoc)
+      .define(SaslEnabledMechanismsProp, LIST, Defaults.SaslEnabledMechanisms, MEDIUM, SaslEnabledMechanismsDoc)
       .define(SaslKerberosServiceNameProp, STRING, null, MEDIUM, SaslKerberosServiceNameDoc)
       .define(SaslKerberosKinitCmdProp, STRING, Defaults.SaslKerberosKinitCmd, MEDIUM, SaslKerberosKinitCmdDoc)
       .define(SaslKerberosTicketRenewWindowFactorProp, DOUBLE, Defaults.SaslKerberosTicketRenewWindowFactor, MEDIUM, SaslKerberosTicketRenewWindowFactorDoc)
@@ -894,12 +902,15 @@
   val sslCipher = getList(KafkaConfig.SslCipherSuitesProp)
 
   /** ********* Sasl Configuration **************/
+  val saslMechanismInterBrokerProtocol = getString(KafkaConfig.SaslMechanismInterBrokerProtocolProp)
+  val saslEnabledMechanisms = getList(KafkaConfig.SaslEnabledMechanismsProp)
   val saslKerberosServiceName = getString(KafkaConfig.SaslKerberosServiceNameProp)
   val saslKerberosKinitCmd = getString(KafkaConfig.SaslKerberosKinitCmdProp)
   val saslKerberosTicketRenewWindowFactor = getDouble(KafkaConfig.SaslKerberosTicketRenewWindowFactorProp)
   val saslKerberosTicketRenewJitter = getDouble(KafkaConfig.SaslKerberosTicketRenewJitterProp)
   val saslKerberosMinTimeBeforeRelogin = getLong(KafkaConfig.SaslKerberosMinTimeBeforeReloginProp)
   val saslKerberosPrincipalToLocalRules = getList(KafkaConfig.SaslKerberosPrincipalToLocalRulesProp)
+  val saslInterBrokerHandshakeRequestEnable = interBrokerProtocolVersion >= KAFKA_0_10_0_IV0
 
   /** ********* Quota Configuration **************/
   val producerQuotaBytesPerSecondDefault = getLong(KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp)
@@ -1009,5 +1020,10 @@
     )
     require(interBrokerProtocolVersion >= logMessageFormatVersion,
       s"log.message.format.version $logMessageFormatVersionString cannot be used when inter.broker.protocol.version is set to $interBrokerProtocolVersionString")
+    val interBrokerUsesSasl = interBrokerSecurityProtocol == SecurityProtocol.SASL_PLAINTEXT || interBrokerSecurityProtocol == SecurityProtocol.SASL_SSL
+    require(!interBrokerUsesSasl || saslInterBrokerHandshakeRequestEnable || saslMechanismInterBrokerProtocol == SaslConfigs.GSSAPI_MECHANISM,
+      s"Only GSSAPI mechanism is supported for inter-broker communication with SASL when inter.broker.protocol.version is set to $interBrokerProtocolVersionString")
+    require(!interBrokerUsesSasl || saslEnabledMechanisms.contains(saslMechanismInterBrokerProtocol),
+      s"${KafkaConfig.SaslMechanismInterBrokerProtocolProp} must be included in ${KafkaConfig.SaslEnabledMechanismsProp} when SASL is used for inter-broker communication")
   }
 }
diff --git a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala
index 2598e6d..4e3fc29 100644
--- a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala
+++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala
@@ -30,7 +30,7 @@
 /**
  * This class registers the broker in zookeeper to allow 
  * other brokers and consumers to detect failures. It uses an ephemeral znode with the path:
- *   /brokers/[0...N] --> advertisedHost:advertisedPort
+ *   /brokers/ids/[0...N] --> advertisedHost:advertisedPort
  *   
  * Right now our definition of health is fairly naive. If we register in zk we are healthy, otherwise
  * we are dead.
diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala
index f998d82..36b52fd 100755
--- a/core/src/main/scala/kafka/server/KafkaServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaServer.scala
@@ -323,6 +323,13 @@
     def networkClientControlledShutdown(retries: Int): Boolean = {
       val metadataUpdater = new ManualMetadataUpdater()
       val networkClient = {
+        val channelBuilder = ChannelBuilders.create(
+          config.interBrokerSecurityProtocol,
+          Mode.CLIENT,
+          LoginType.SERVER,
+          config.values,
+          config.saslMechanismInterBrokerProtocol,
+          config.saslInterBrokerHandshakeRequestEnable)
         val selector = new Selector(
           NetworkReceive.UNLIMITED,
           config.connectionsMaxIdleMs,
@@ -331,7 +338,7 @@
           "kafka-server-controlled-shutdown",
           Map.empty.asJava,
           false,
-          ChannelBuilders.create(config.interBrokerSecurityProtocol, Mode.CLIENT, LoginType.SERVER, config.values)
+          channelBuilder
         )
         new NetworkClient(
           selector,
diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala
index 06fae42..b387f2e 100755
--- a/core/src/main/scala/kafka/server/MetadataCache.scala
+++ b/core/src/main/scala/kafka/server/MetadataCache.scala
@@ -24,11 +24,11 @@
 import scala.collection.JavaConverters._
 import kafka.cluster.{Broker, EndPoint}
 import kafka.api._
-import kafka.common.{BrokerEndPointNotAvailableException, TopicAndPartition}
+import kafka.common.{BrokerEndPointNotAvailableException, Topic, TopicAndPartition}
 import kafka.controller.{KafkaController, LeaderIsrAndControllerEpoch}
 import kafka.utils.CoreUtils._
 import kafka.utils.Logging
-import org.apache.kafka.common.{Node, TopicPartition}
+import org.apache.kafka.common.Node
 import org.apache.kafka.common.protocol.{Errors, SecurityProtocol}
 import org.apache.kafka.common.requests.UpdateMetadataRequest.PartitionState
 import org.apache.kafka.common.requests.{MetadataResponse, UpdateMetadataRequest}
@@ -40,16 +40,24 @@
 private[server] class MetadataCache(brokerId: Int) extends Logging {
   private val stateChangeLogger = KafkaController.stateChangeLogger
   private val cache = mutable.Map[String, mutable.Map[Int, PartitionStateInfo]]()
+  private var controllerId: Option[Int] = None
   private val aliveBrokers = mutable.Map[Int, Broker]()
   private val aliveNodes = mutable.Map[Int, collection.Map[SecurityProtocol, Node]]()
   private val partitionMetadataLock = new ReentrantReadWriteLock()
 
   this.logIdent = s"[Kafka Metadata Cache on broker $brokerId] "
 
-  private def getAliveEndpoints(brokers: Iterable[Int], protocol: SecurityProtocol): Seq[Node] = {
+  // This method is the main hotspot when it comes to the performance of metadata requests,
+  // we should be careful about adding additional logic here.
+  // filterUnavailableEndpoints exists to support v0 MetadataResponses
+  private def getEndpoints(brokers: Iterable[Int], protocol: SecurityProtocol, filterUnavailableEndpoints: Boolean): Seq[Node] = {
     val result = new mutable.ArrayBuffer[Node](math.min(aliveBrokers.size, brokers.size))
     brokers.foreach { brokerId =>
-      getAliveEndpoint(brokerId, protocol).foreach(result +=)
+      val endpoint = getAliveEndpoint(brokerId, protocol) match {
+        case None => if (!filterUnavailableEndpoints) Some(new Node(brokerId, "", -1)) else None
+        case Some(node) => Some(node)
+      }
+      endpoint.foreach(result +=)
     }
     result
   }
@@ -60,7 +68,8 @@
         throw new BrokerEndPointNotAvailableException(s"Broker `$brokerId` does not support security protocol `$protocol`"))
     }
 
-  private def getPartitionMetadata(topic: String, protocol: SecurityProtocol): Option[Iterable[MetadataResponse.PartitionMetadata]] = {
+  // errorUnavailableEndpoints exists to support v0 MetadataResponses
+  private def getPartitionMetadata(topic: String, protocol: SecurityProtocol, errorUnavailableEndpoints: Boolean): Option[Iterable[MetadataResponse.PartitionMetadata]] = {
     cache.get(topic).map { partitions =>
       partitions.map { case (partitionId, partitionState) =>
         val topicPartition = TopicAndPartition(topic, partitionId)
@@ -69,7 +78,7 @@
         val maybeLeader = getAliveEndpoint(leaderAndIsr.leader, protocol)
 
         val replicas = partitionState.allReplicas
-        val replicaInfo = getAliveEndpoints(replicas, protocol)
+        val replicaInfo = getEndpoints(replicas, protocol, errorUnavailableEndpoints)
 
         maybeLeader match {
           case None =>
@@ -79,7 +88,7 @@
 
           case Some(leader) =>
             val isr = leaderAndIsr.isr
-            val isrInfo = getAliveEndpoints(isr, protocol)
+            val isrInfo = getEndpoints(isr, protocol, errorUnavailableEndpoints)
 
             if (replicaInfo.size < replicas.size) {
               debug(s"Error while fetching metadata for $topicPartition: replica information not available for " +
@@ -101,12 +110,12 @@
     }
   }
 
-  def getTopicMetadata(topics: Set[String], protocol: SecurityProtocol): Seq[MetadataResponse.TopicMetadata] = {
+  // errorUnavailableEndpoints exists to support v0 MetadataResponses
+  def getTopicMetadata(topics: Set[String], protocol: SecurityProtocol, errorUnavailableEndpoints: Boolean = false): Seq[MetadataResponse.TopicMetadata] = {
     inReadLock(partitionMetadataLock) {
-      val topicsRequested = if (topics.isEmpty) cache.keySet else topics
-      topicsRequested.toSeq.flatMap { topic =>
-        getPartitionMetadata(topic, protocol).map { partitionMetadata =>
-          new MetadataResponse.TopicMetadata(Errors.NONE, topic, partitionMetadata.toBuffer.asJava)
+      topics.toSeq.flatMap { topic =>
+        getPartitionMetadata(topic, protocol, errorUnavailableEndpoints).map { partitionMetadata =>
+          new MetadataResponse.TopicMetadata(Errors.NONE, topic, Topic.isInternal(topic), partitionMetadata.toBuffer.asJava)
         }
       }
     }
@@ -151,8 +160,14 @@
     }
   }
 
+  def getControllerId: Option[Int] = controllerId
+
   def updateCache(correlationId: Int, updateMetadataRequest: UpdateMetadataRequest) {
     inWriteLock(partitionMetadataLock) {
+      controllerId = updateMetadataRequest.controllerId match {
+          case id if id < 0 => None
+          case id => Some(id)
+        }
       aliveNodes.clear()
       aliveBrokers.clear()
       updateMetadataRequest.liveBrokers.asScala.foreach { broker =>
diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
index 26838ca..d58f120 100644
--- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
+++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
@@ -73,6 +73,14 @@
   // as the metrics tag to avoid metric name conflicts with
   // more than one fetcher thread to the same broker
   private val networkClient = {
+    val channelBuilder = ChannelBuilders.create(
+      brokerConfig.interBrokerSecurityProtocol,
+      Mode.CLIENT,
+      LoginType.SERVER,
+      brokerConfig.values,
+      brokerConfig.saslMechanismInterBrokerProtocol,
+      brokerConfig.saslInterBrokerHandshakeRequestEnable
+    )
     val selector = new Selector(
       NetworkReceive.UNLIMITED,
       brokerConfig.connectionsMaxIdleMs,
@@ -81,7 +89,7 @@
       "replica-fetcher",
       Map("broker-id" -> sourceBroker.id.toString, "fetcher-id" -> fetcherId.toString).asJava,
       false,
-      ChannelBuilders.create(brokerConfig.interBrokerSecurityProtocol, Mode.CLIENT, LoginType.SERVER, brokerConfig.values)
+      channelBuilder
     )
     new NetworkClient(
       selector,
@@ -107,10 +115,10 @@
       val TopicAndPartition(topic, partitionId) = topicAndPartition
       val replica = replicaMgr.getReplica(topic, partitionId).get
       val messageSet = partitionData.toByteBufferMessageSet
-      warnIfMessageOversized(messageSet)
+      warnIfMessageOversized(messageSet, topicAndPartition)
 
       if (fetchOffset != replica.logEndOffset.messageOffset)
-        throw new RuntimeException("Offset mismatch: fetched offset = %d, log end offset = %d.".format(fetchOffset, replica.logEndOffset.messageOffset))
+        throw new RuntimeException("Offset mismatch for partition %s: fetched offset = %d, log end offset = %d.".format(topicAndPartition, fetchOffset, replica.logEndOffset.messageOffset))
       if (logger.isTraceEnabled)
         trace("Follower %d has replica log end offset %d for partition %s. Received %d messages and leader hw %d"
           .format(replica.brokerId, replica.logEndOffset.messageOffset, topicAndPartition, messageSet.sizeInBytes, partitionData.highWatermark))
@@ -128,15 +136,15 @@
           .format(replica.brokerId, topic, partitionId, followerHighWatermark))
     } catch {
       case e: KafkaStorageException =>
-        fatal("Disk error while replicating data.", e)
+        fatal(s"Disk error while replicating data for $topicAndPartition", e)
         Runtime.getRuntime.halt(1)
     }
   }
 
-  def warnIfMessageOversized(messageSet: ByteBufferMessageSet): Unit = {
+  def warnIfMessageOversized(messageSet: ByteBufferMessageSet, topicAndPartition: TopicAndPartition): Unit = {
     if (messageSet.sizeInBytes > 0 && messageSet.validBytes <= 0)
-      error("Replication is failing due to a message that is greater than replica.fetch.max.bytes. This " +
-        "generally occurs when the max.message.bytes has been overridden to exceed this value and a suitably large " +
+      error(s"Replication is failing due to a message that is greater than replica.fetch.max.bytes for partition $topicAndPartition. " +
+        "This generally occurs when the max.message.bytes has been overridden to exceed this value and a suitably large " +
         "message has also been sent. To fix this problem increase replica.fetch.max.bytes in your broker config to be " +
         "equal or larger than your settings for max.message.bytes, both at a broker and topic level.")
   }
diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala
index 22657f4..888912b 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -28,6 +28,7 @@
 import kafka.message.{ByteBufferMessageSet, InvalidMessageException, Message, MessageSet}
 import kafka.metrics.KafkaMetricsGroup
 import kafka.utils._
+import org.I0Itec.zkclient.IZkChildListener
 import org.apache.kafka.common.errors.{OffsetOutOfRangeException, RecordBatchTooLargeException, ReplicaNotAvailableException, RecordTooLargeException,
 InvalidTopicException, ControllerMovedException, NotLeaderForPartitionException, CorruptRecordException, UnknownTopicOrPartitionException,
 InvalidTimestampException}
@@ -39,7 +40,6 @@
 import org.apache.kafka.common.utils.{Time => JTime}
 import scala.collection._
 import scala.collection.JavaConverters._
-import org.apache.kafka.common.internals.TopicConstants
 
 /*
  * Result metadata of a log append operation on the log
@@ -122,9 +122,9 @@
   private val lastIsrChangeMs = new AtomicLong(System.currentTimeMillis())
   private val lastIsrPropagationMs = new AtomicLong(System.currentTimeMillis())
 
-  val delayedProducePurgatory = new DelayedOperationPurgatory[DelayedProduce](
+  val delayedProducePurgatory = DelayedOperationPurgatory[DelayedProduce](
     purgatoryName = "Produce", config.brokerId, config.producerPurgatoryPurgeIntervalRequests)
-  val delayedFetchPurgatory = new DelayedOperationPurgatory[DelayedFetch](
+  val delayedFetchPurgatory = DelayedOperationPurgatory[DelayedFetch](
     purgatoryName = "Fetch", config.brokerId, config.fetchPurgatoryPurgeIntervalRequests)
 
   val leaderCount = newGauge(
@@ -394,7 +394,7 @@
       BrokerTopicStats.getBrokerAllTopicsStats().totalProduceRequestRate.mark()
 
       // reject appending to internal topics if it is not allowed
-      if (TopicConstants.INTERNAL_TOPICS.contains(topicPartition.topic) && !internalTopicsAllowed) {
+      if (Topic.isInternal(topicPartition.topic) && !internalTopicsAllowed) {
         (topicPartition, LogAppendResult(
           LogAppendInfo.UnknownLogAppendInfo,
           Some(new InvalidTopicException("Cannot append to internal topic %s".format(topicPartition.topic)))))
diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala
index 50add72..8953640 100755
--- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala
+++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala
@@ -103,6 +103,10 @@
         consumer.stop()
 
         shutdownLatch.await()
+
+        if (conf.enableSystestEventsLogging) {
+          System.out.println("shutdown_complete")
+        }
       }
     })
   }
@@ -253,6 +257,9 @@
       .withRequiredArg
       .describedAs("deserializer for values")
       .ofType(classOf[String])
+    val enableSystestEventsLoggingOpt = parser.accepts("enable-systest-events",
+                                                       "Log lifecycle events of the consumer in addition to logging consumed " +
+                                                       "messages. (This is specific for system tests.)")
 
     if (args.length == 0)
       CommandLineUtils.printUsageAndDie(parser, "The console consumer is a tool that reads data from Kafka and outputs it to standard output.")
@@ -260,6 +267,7 @@
     var groupIdPassed = true
     val options: OptionSet = tryParse(parser, args)
     val useNewConsumer = options.has(useNewConsumerOpt)
+    val enableSystestEventsLogging = options.has(enableSystestEventsLoggingOpt)
 
     // If using old consumer, exactly one of whitelist/blacklist/topic is required.
     // If using new consumer, topic must be specified.
@@ -349,9 +357,9 @@
 
   override def init(props: Properties) {
     if (props.containsKey("print.timestamp"))
-      printTimestamp = props.getProperty("print.timestamp").trim.toLowerCase.equals("true")
+      printTimestamp = props.getProperty("print.timestamp").trim.equalsIgnoreCase("true")
     if (props.containsKey("print.key"))
-      printKey = props.getProperty("print.key").trim.toLowerCase.equals("true")
+      printKey = props.getProperty("print.key").trim.equalsIgnoreCase("true")
     if (props.containsKey("key.separator"))
       keySeparator = props.getProperty("key.separator").getBytes
     if (props.containsKey("line.separator"))
diff --git a/core/src/main/scala/kafka/tools/ConsoleProducer.scala b/core/src/main/scala/kafka/tools/ConsoleProducer.scala
index 0116a96..e647601 100644
--- a/core/src/main/scala/kafka/tools/ConsoleProducer.scala
+++ b/core/src/main/scala/kafka/tools/ConsoleProducer.scala
@@ -295,11 +295,11 @@
     override def init(inputStream: InputStream, props: Properties) {
       topic = props.getProperty("topic")
       if (props.containsKey("parse.key"))
-        parseKey = props.getProperty("parse.key").trim.toLowerCase.equals("true")
+        parseKey = props.getProperty("parse.key").trim.equalsIgnoreCase("true")
       if (props.containsKey("key.separator"))
         keySeparator = props.getProperty("key.separator")
       if (props.containsKey("ignore.error"))
-        ignoreError = props.getProperty("ignore.error").trim.toLowerCase.equals("true")
+        ignoreError = props.getProperty("ignore.error").trim.equalsIgnoreCase("true")
       reader = new BufferedReader(new InputStreamReader(inputStream))
     }
 
diff --git a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala
index a38c04b..6480ff5 100644
--- a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala
+++ b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala
@@ -19,8 +19,6 @@
 
 import java.util
 
-import org.apache.kafka.common.TopicPartition
-
 import scala.collection.JavaConversions._
 import java.util.concurrent.atomic.AtomicLong
 import java.nio.channels.ClosedByInterruptException
@@ -85,10 +83,9 @@
         thread.start
       for (thread <- threadList)
         thread.join
-      if(consumerTimeout.get())
-	endMs = System.currentTimeMillis - consumerConfig.consumerTimeoutMs
-      else
-	endMs = System.currentTimeMillis
+      endMs =
+        if (consumerTimeout.get()) System.currentTimeMillis - consumerConfig.consumerTimeoutMs
+        else System.currentTimeMillis
       consumerConnector.shutdown()
     }
     val elapsedSecs = (endMs - startMs) / 1000.0
@@ -279,9 +276,8 @@
       } catch {
         case _: InterruptedException =>
         case _: ClosedByInterruptException =>
-        case _: ConsumerTimeoutException => {
-          consumerTimeout.set(true);
-        }
+        case _: ConsumerTimeoutException =>
+          consumerTimeout.set(true)
         case e: Throwable => e.printStackTrace()
       }
       totalMessagesRead.addAndGet(messagesRead)
diff --git a/core/src/main/scala/kafka/tools/EndToEndLatency.scala b/core/src/main/scala/kafka/tools/EndToEndLatency.scala
index 584d4fb..1c92088 100755
--- a/core/src/main/scala/kafka/tools/EndToEndLatency.scala
+++ b/core/src/main/scala/kafka/tools/EndToEndLatency.scala
@@ -25,6 +25,7 @@
 import org.apache.kafka.common.TopicPartition
 
 import scala.collection.JavaConversions._
+import scala.util.Random
 
 
 /**
@@ -43,7 +44,7 @@
 
   def main(args: Array[String]) {
     if (args.length != 5 && args.length != 6) {
-      System.err.println("USAGE: java " + getClass.getName + " broker_list topic num_messages producer_acks message_size_bytes [optional] ssl_properties_file")
+      System.err.println("USAGE: java " + getClass.getName + " broker_list topic num_messages producer_acks message_size_bytes [optional] properties_file")
       System.exit(1)
     }
 
@@ -52,12 +53,14 @@
     val numMessages = args(2).toInt
     val producerAcks = args(3)
     val messageLen = args(4).toInt
-    val sslPropsFile = if (args.length == 6) args(5) else ""
+    val propsFile = if (args.length > 5) Some(args(5)).filter(_.nonEmpty) else None
 
     if (!List("1", "all").contains(producerAcks))
       throw new IllegalArgumentException("Latency testing requires synchronous acknowledgement. Please use 1 or all")
 
-    val consumerProps = if (sslPropsFile.equals("")) new Properties() else Utils.loadProps(sslPropsFile)
+    def loadProps: Properties = propsFile.map(Utils.loadProps).getOrElse(new Properties())
+
+    val consumerProps = loadProps
     consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
     consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, "test-group-" + System.currentTimeMillis())
     consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false")
@@ -69,7 +72,7 @@
     val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](consumerProps)
     consumer.subscribe(List(topic))
 
-    val producerProps = if (sslPropsFile.equals("")) new Properties() else Utils.loadProps(sslPropsFile)
+    val producerProps = loadProps
     producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
     producerProps.put(ProducerConfig.LINGER_MS_CONFIG, "0") //ensure writes are synchronous
     producerProps.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, Long.MaxValue.toString)
@@ -91,9 +94,10 @@
 
     var totalTime = 0.0
     val latencies = new Array[Long](numMessages)
+    val random = new Random(0)
 
     for (i <- 0 until numMessages) {
-      val message = randomBytesOfLen(messageLen)
+      val message = randomBytesOfLen(random, messageLen)
       val begin = System.nanoTime
 
       //Send message (of random bytes) synchronously then immediately poll for it
@@ -141,7 +145,7 @@
     finalise()
   }
 
-  def randomBytesOfLen(len: Int): Array[Byte] = {
-    Array.fill(len)((scala.util.Random.nextInt(26) + 65).toByte)
+  def randomBytesOfLen(random: Random, len: Int): Array[Byte] = {
+    Array.fill(len)((random.nextInt(26) + 65).toByte)
   }
 }
diff --git a/core/src/main/scala/kafka/utils/Log4jController.scala b/core/src/main/scala/kafka/utils/Log4jController.scala
index 673d84e..026fbae 100755
--- a/core/src/main/scala/kafka/utils/Log4jController.scala
+++ b/core/src/main/scala/kafka/utils/Log4jController.scala
@@ -20,6 +20,7 @@
 
 import org.apache.log4j.{Logger, Level, LogManager}
 import java.util
+import java.util.Locale
 
 
 object Log4jController {
@@ -81,7 +82,7 @@
   def setLogLevel(loggerName: String, level: String) = {
     val log = newLogger(loggerName)
     if (!loggerName.trim.isEmpty && !level.trim.isEmpty && log != null) {
-      log.setLevel(Level.toLevel(level.toUpperCase))
+      log.setLevel(Level.toLevel(level.toUpperCase(Locale.ROOT)))
       true
     }
     else false
diff --git a/core/src/main/scala/kafka/utils/Os.scala b/core/src/main/scala/kafka/utils/Os.scala
index 6574f08..0100a0a 100644
--- a/core/src/main/scala/kafka/utils/Os.scala
+++ b/core/src/main/scala/kafka/utils/Os.scala
@@ -17,7 +17,9 @@
 
 package kafka.utils
 
+import java.util.Locale
+
 object Os {
-  val name = System.getProperty("os.name").toLowerCase
+  val name = System.getProperty("os.name").toLowerCase(Locale.ROOT)
   val isWindows = name.startsWith("windows")
 }
\ No newline at end of file
diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala
index bd8ec7e..83ff517 100644
--- a/core/src/main/scala/kafka/utils/ZkUtils.scala
+++ b/core/src/main/scala/kafka/utils/ZkUtils.scala
@@ -668,7 +668,7 @@
     }
   }
 
-  // Parses without deduplicating keys so the the data can be checked before allowing reassignment to proceed
+  // Parses without deduplicating keys so the data can be checked before allowing reassignment to proceed
   def parsePartitionReassignmentDataWithoutDedup(jsonData: String): Seq[(TopicAndPartition, Seq[Int])] = {
     Json.parseFull(jsonData) match {
       case Some(m) =>
@@ -710,7 +710,7 @@
     topics
   }
 
-  def getPartitionReassignmentZkData(partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]]): String = {
+  def formatAsReassignmentJson(partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]]): String = {
     Json.encode(Map("version" -> 1, "partitions" -> partitionsToBeReassigned.map(e => Map("topic" -> e._1.topic, "partition" -> e._1.partition,
                                                                                           "replicas" -> e._2))))
   }
@@ -722,7 +722,7 @@
         deletePath(zkPath)
         info("No more partitions need to be reassigned. Deleting zk path %s".format(zkPath))
       case _ =>
-        val jsonData = getPartitionReassignmentZkData(partitionsToBeReassigned)
+        val jsonData = formatAsReassignmentJson(partitionsToBeReassigned)
         try {
           updatePersistentPath(zkPath, jsonData)
           debug("Updated partition reassignment path with %s".format(jsonData))
diff --git a/core/src/main/scala/kafka/utils/timer/Timer.scala b/core/src/main/scala/kafka/utils/timer/Timer.scala
index bdd0e75..2d78665 100644
--- a/core/src/main/scala/kafka/utils/timer/Timer.scala
+++ b/core/src/main/scala/kafka/utils/timer/Timer.scala
@@ -16,14 +16,52 @@
  */
 package kafka.utils.timer
 
-import java.util.concurrent.{DelayQueue, ExecutorService, TimeUnit}
+import java.util.concurrent.{DelayQueue, Executors, ThreadFactory, TimeUnit}
 import java.util.concurrent.atomic.AtomicInteger
 import java.util.concurrent.locks.ReentrantReadWriteLock
 
 import kafka.utils.threadsafe
+import org.apache.kafka.common.utils.Utils
+
+trait Timer {
+  /**
+    * Add a new task to this executor. It will be executed after the task's delay
+    * (beginning from the time of submission)
+    * @param timerTask the task to add
+    */
+  def add(timerTask: TimerTask): Unit
+
+  /**
+    * Advance the internal clock, executing any tasks whose expiration has been
+    * reached within the duration of the passed timeout.
+    * @param timeoutMs
+    * @return whether or not any tasks were executed
+    */
+  def advanceClock(timeoutMs: Long): Boolean
+
+  /**
+    * Get the number of tasks pending execution
+    * @return the number of tasks
+    */
+  def size: Int
+
+  /**
+    * Shutdown the timer service, leaving pending tasks unexecuted
+    */
+  def shutdown(): Unit
+}
 
 @threadsafe
-class Timer(taskExecutor: ExecutorService, tickMs: Long = 1, wheelSize: Int = 20, startMs: Long = System.currentTimeMillis) {
+class SystemTimer(executorName: String,
+                  tickMs: Long = 1,
+                  wheelSize: Int = 20,
+                  startMs: Long = System.currentTimeMillis) extends Timer {
+
+  // timeout timer
+  private[this] val taskExecutor = Executors.newFixedThreadPool(1, new ThreadFactory() {
+    def newThread(runnable: Runnable): Thread =
+      Utils.newThread("executor-"+executorName, runnable, false)
+  })
 
   private[this] val delayQueue = new DelayQueue[TimerTaskList]()
   private[this] val taskCounter = new AtomicInteger(0)
@@ -43,7 +81,7 @@
   def add(timerTask: TimerTask): Unit = {
     readLock.lock()
     try {
-      addTimerTaskEntry(new TimerTaskEntry(timerTask))
+      addTimerTaskEntry(new TimerTaskEntry(timerTask, timerTask.delayMs + System.currentTimeMillis()))
     } finally {
       readLock.unlock()
     }
@@ -82,6 +120,11 @@
     }
   }
 
-  def size(): Int = taskCounter.get
+  def size: Int = taskCounter.get
+
+  override def shutdown() {
+    taskExecutor.shutdown()
+  }
+
 }
 
diff --git a/core/src/main/scala/kafka/utils/timer/TimerTask.scala b/core/src/main/scala/kafka/utils/timer/TimerTask.scala
index d6b3a2e..6623854 100644
--- a/core/src/main/scala/kafka/utils/timer/TimerTask.scala
+++ b/core/src/main/scala/kafka/utils/timer/TimerTask.scala
@@ -18,7 +18,7 @@
 
 trait TimerTask extends Runnable {
 
-  val expirationMs: Long // timestamp in millisecond
+  val delayMs: Long // timestamp in millisecond
 
   private[this] var timerTaskEntry: TimerTaskEntry = null
 
diff --git a/core/src/main/scala/kafka/utils/timer/TimerTaskList.scala b/core/src/main/scala/kafka/utils/timer/TimerTaskList.scala
index c4aeb5d..e862f4f 100644
--- a/core/src/main/scala/kafka/utils/timer/TimerTaskList.scala
+++ b/core/src/main/scala/kafka/utils/timer/TimerTaskList.scala
@@ -29,7 +29,7 @@
   // TimerTaskList forms a doubly linked cyclic list using a dummy root entry
   // root.next points to the head
   // root.prev points to the tail
-  private[this] val root = new TimerTaskEntry(null)
+  private[this] val root = new TimerTaskEntry(null, -1)
   root.next = root
   root.prev = root
 
@@ -131,7 +131,7 @@
 
 }
 
-private[timer] class TimerTaskEntry(val timerTask: TimerTask) {
+private[timer] class TimerTaskEntry(val timerTask: TimerTask, val expirationMs: Long) extends Ordered[TimerTaskEntry] {
 
   @volatile
   var list: TimerTaskList = null
@@ -157,5 +157,8 @@
     }
   }
 
+  override def compare(that: TimerTaskEntry): Int = {
+    this.expirationMs compare that.expirationMs
+  }
 }
 
diff --git a/core/src/main/scala/kafka/utils/timer/TimingWheel.scala b/core/src/main/scala/kafka/utils/timer/TimingWheel.scala
index f5b6efe..4535f3f 100644
--- a/core/src/main/scala/kafka/utils/timer/TimingWheel.scala
+++ b/core/src/main/scala/kafka/utils/timer/TimingWheel.scala
@@ -123,7 +123,7 @@
   }
 
   def add(timerTaskEntry: TimerTaskEntry): Boolean = {
-    val expiration = timerTaskEntry.timerTask.expirationMs
+    val expiration = timerTaskEntry.expirationMs
 
     if (timerTaskEntry.cancelled) {
       // Cancelled
diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
index 1408cd9..23fcfa6 100644
--- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
+++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
@@ -13,7 +13,6 @@
 package kafka.api
 
 import java.util
-import kafka.coordinator.GroupCoordinator
 import org.apache.kafka.clients.consumer._
 import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord}
 import org.apache.kafka.common.record.TimestampType
@@ -27,6 +26,7 @@
 import scala.collection.JavaConverters._
 import scala.collection.mutable.Buffer
 import org.apache.kafka.common.internals.TopicConstants
+import org.apache.kafka.clients.producer.KafkaProducer
 
 /**
  * Integration tests for the new consumer that cover basic usage as well as server failures
@@ -78,10 +78,6 @@
     // check async commit callbacks
     val commitCallback = new CountConsumerCommitCallback()
     this.consumers(0).commitAsync(commitCallback)
-
-    // shouldn't make progress until poll is invoked
-    Thread.sleep(10)
-    assertEquals(0, commitCallback.successCount)
     awaitCommitCallback(this.consumers(0), commitCallback)
   }
 
@@ -92,6 +88,7 @@
 
     this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true")
     val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
+    consumers += consumer0
 
     val numRecords = 10000
     sendRecords(numRecords)
@@ -184,6 +181,8 @@
     this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100") // timeout quickly to avoid slow test
     this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30")
     val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
+    consumers += consumer0
+
     consumer0.subscribe(List(topic).asJava, listener)
 
     // the initial subscription should cause a callback execution
@@ -209,8 +208,6 @@
 
     // only expect one revocation since revoke is not invoked on initial membership
     assertEquals(2, listener.callsToRevoked)
-
-    consumer0.close()
   }
 
   @Test
@@ -219,20 +216,17 @@
     this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100") // timeout quickly to avoid slow test
     this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30")
     val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
+    consumers += consumer0
 
-    try {
-      val listener = new TestConsumerReassignmentListener()
-      consumer0.subscribe(List(topic).asJava, listener)
+    val listener = new TestConsumerReassignmentListener()
+    consumer0.subscribe(List(topic).asJava, listener)
 
-      // the initial subscription should cause a callback execution
-      while (listener.callsToAssigned == 0)
-        consumer0.poll(50)
+    // the initial subscription should cause a callback execution
+    while (listener.callsToAssigned == 0)
+      consumer0.poll(50)
 
-      consumer0.subscribe(List[String]().asJava)
-      assertEquals(0, consumer0.assignment.size())
-    } finally {
-      consumer0.close()
-    }
+    consumer0.subscribe(List[String]().asJava)
+    assertEquals(0, consumer0.assignment.size())
   }
 
   @Test
@@ -240,6 +234,7 @@
     this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100") // timeout quickly to avoid slow test
     this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30")
     val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
+    consumers += consumer0
 
     sendRecords(5)
     consumer0.subscribe(List(topic).asJava)
@@ -274,10 +269,14 @@
   }
 
   protected def sendRecords(numRecords: Int, tp: TopicPartition) {
+    sendRecords(this.producers(0), numRecords, tp)
+  }
+
+  protected def sendRecords(producer: KafkaProducer[Array[Byte], Array[Byte]], numRecords: Int, tp: TopicPartition) {
     (0 until numRecords).foreach { i =>
-      this.producers(0).send(new ProducerRecord(tp.topic(), tp.partition(), i.toLong, s"key $i".getBytes, s"value $i".getBytes))
+      producer.send(new ProducerRecord(tp.topic(), tp.partition(), i.toLong, s"key $i".getBytes, s"value $i".getBytes))
     }
-    this.producers(0).flush()
+    producer.flush()
   }
 
   protected def consumeAndVerifyRecords(consumer: Consumer[Array[Byte], Array[Byte]],
@@ -333,11 +332,10 @@
   protected def awaitCommitCallback[K, V](consumer: Consumer[K, V],
                                           commitCallback: CountConsumerCommitCallback,
                                           count: Int = 1): Unit = {
-    val startCount = commitCallback.successCount
     val started = System.currentTimeMillis()
-    while (commitCallback.successCount < startCount + count && System.currentTimeMillis() - started < 10000)
+    while (commitCallback.successCount < count && System.currentTimeMillis() - started < 10000)
       consumer.poll(50)
-    assertEquals(startCount + count, commitCallback.successCount)
+    assertEquals(count, commitCallback.successCount)
   }
 
   protected class CountConsumerCommitCallback extends OffsetCommitCallback {
diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala
index 49ce748..15eeb63 100644
--- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala
+++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala
@@ -41,7 +41,7 @@
     val numServers = 2
     overridingProps.put(KafkaConfig.NumPartitionsProp, 4.toString)
     TestUtils.createBrokerConfigs(numServers, zkConnect, false, interBrokerSecurityProtocol = Some(securityProtocol),
-      trustStoreFile = trustStoreFile).map(KafkaConfig.fromProps(_, overridingProps))
+      trustStoreFile = trustStoreFile, saslProperties = saslProperties).map(KafkaConfig.fromProps(_, overridingProps))
   }
 
   private var consumer1: SimpleConsumer = null
@@ -72,7 +72,7 @@
 
   private def createProducer(brokerList: String, retries: Int = 0, lingerMs: Long = 0, props: Option[Properties] = None): KafkaProducer[Array[Byte],Array[Byte]] = {
     val producer = TestUtils.createNewProducer(brokerList, securityProtocol = securityProtocol, trustStoreFile = trustStoreFile,
-      retries = retries, lingerMs = lingerMs, props = props)
+      saslProperties = saslProperties, retries = retries, lingerMs = lingerMs, props = props)
     producers += producer
     producer
   }
diff --git a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala
index e2314b3..fec96cd 100644
--- a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala
@@ -78,6 +78,9 @@
   val kafkaPrincipal: String
 
   override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks"))
+  protected def kafkaClientSaslMechanism = "GSSAPI"
+  protected def kafkaServerSaslMechanisms = List("GSSAPI")
+  override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, Some(kafkaServerSaslMechanisms)))
 
   val topicResource = new Resource(Topic, topic)
   val groupResource = new Resource(Group, group)
@@ -141,9 +144,9 @@
   override def setUp {
     securityProtocol match {
       case SecurityProtocol.SSL =>
-        startSasl(ZkSasl)
+        startSasl(ZkSasl, null, null)
       case _ =>
-        startSasl(Both)
+        startSasl(Both, List(kafkaClientSaslMechanism), kafkaServerSaslMechanisms)
     }
     super.setUp
     AclCommand.main(topicBrokerReadAclArgs)
diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
index d0680b8..de05c9c 100644
--- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
+++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
@@ -46,15 +46,15 @@
 
   override def generateConfigs() = {
     val cfgs = TestUtils.createBrokerConfigs(serverCount, zkConnect, interBrokerSecurityProtocol = Some(securityProtocol),
-      trustStoreFile = trustStoreFile)
+      trustStoreFile = trustStoreFile, saslProperties = saslProperties)
     cfgs.foreach(_.putAll(serverConfig))
     cfgs.map(KafkaConfig.fromProps)
   }
 
   @Before
   override def setUp() {
-    val producerSecurityProps = TestUtils.producerSecurityConfigs(securityProtocol, trustStoreFile)
-    val consumerSecurityProps = TestUtils.consumerSecurityConfigs(securityProtocol, trustStoreFile)
+    val producerSecurityProps = TestUtils.producerSecurityConfigs(securityProtocol, trustStoreFile, saslProperties)
+    val consumerSecurityProps = TestUtils.consumerSecurityConfigs(securityProtocol, trustStoreFile, saslProperties)
     super.setUp()
     producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArraySerializer])
     producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArraySerializer])
@@ -66,11 +66,13 @@
       producers += TestUtils.createNewProducer(brokerList,
                                                securityProtocol = this.securityProtocol,
                                                trustStoreFile = this.trustStoreFile,
+                                               saslProperties = this.saslProperties,
                                                props = Some(producerConfig))
     for (i <- 0 until consumerCount) {
       consumers += TestUtils.createNewConsumer(brokerList,
                                                securityProtocol = this.securityProtocol,
                                                trustStoreFile = this.trustStoreFile,
+                                               saslProperties = this.saslProperties,
                                                props = Some(consumerConfig))
     }
 
diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala
index ff2e63d..b22ccde 100644
--- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala
+++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala
@@ -35,6 +35,7 @@
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable.Buffer
+import java.util.Locale
 
 /* We have some tests in this class instead of `BaseConsumerTest` in order to keep the build time under control. */
 class PlaintextConsumerTest extends BaseConsumerTest {
@@ -48,6 +49,8 @@
 
     this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecords.toString)
     val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
+    consumers += consumer0
+
     consumer0.assign(List(tp).asJava)
 
     consumeAndVerifyRecords(consumer0, numRecords = numRecords, startingOffset = 0,
@@ -352,7 +355,7 @@
     producerProps.setProperty(ProducerConfig.COMPRESSION_TYPE_CONFIG, CompressionType.GZIP.name)
     producerProps.setProperty(ProducerConfig.LINGER_MS_CONFIG, Long.MaxValue.toString)
     val producer = TestUtils.createNewProducer(brokerList, securityProtocol = securityProtocol, trustStoreFile = trustStoreFile,
-        retries = 0, lingerMs = Long.MaxValue, props = Some(producerProps))
+        saslProperties = saslProperties, retries = 0, lingerMs = Long.MaxValue, props = Some(producerProps))
     (0 until numRecords).foreach { i =>
       producer.send(new ProducerRecord(tp.topic, tp.partition, i.toLong, s"key $i".getBytes, s"value $i".getBytes))
     }
@@ -405,6 +408,7 @@
   def testFetchInvalidOffset() {
     this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none");
     val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
+    consumers += consumer0
 
     // produce one record
     val totalRecords = 2
@@ -426,8 +430,6 @@
     assertNotNull(outOfRangePartitions)
     assertEquals(1, outOfRangePartitions.size)
     assertEquals(outOfRangePos.toLong, outOfRangePartitions.get(tp))
-
-    consumer0.close()
   }
 
   @Test
@@ -435,6 +437,7 @@
     val maxFetchBytes = 10 * 1024
     this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, maxFetchBytes.toString)
     val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
+    consumers += consumer0
 
     // produce a record that is larger than the configured fetch size
     val record = new ProducerRecord[Array[Byte], Array[Byte]](tp.topic(), tp.partition(), "key".getBytes, new Array[Byte](maxFetchBytes + 1))
@@ -450,8 +453,6 @@
     assertEquals(1, oversizedPartitions.size)
     // the oversized message is at offset 0
     assertEquals(0L, oversizedPartitions.get(tp))
-
-    consumer0.close()
   }
 
   @Test
@@ -460,6 +461,7 @@
     this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "roundrobin-group")
     this.consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, classOf[RoundRobinAssignor].getName)
     val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
+    consumers += consumer0
 
     // create two new topics, each having 2 partitions
     val topic1 = "topic1"
@@ -512,13 +514,13 @@
     val (rrConsumers, consumerPollers) = createConsumerGroupAndWaitForAssignment(10, List(topic1, topic2), subscriptions)
 
     // add one more consumer and validate re-assignment
-    addConsumersToGroupAndWaitForGroupAssignment(1, rrConsumers, consumerPollers, List(topic1, topic2), subscriptions)
+    addConsumersToGroupAndWaitForGroupAssignment(1, consumers, consumerPollers, List(topic1, topic2), subscriptions)
 
     // done with pollers and consumers
     for (poller <- consumerPollers)
       poller.shutdown()
 
-    for (consumer <- rrConsumers)
+    for (consumer <- consumers)
       consumer.unsubscribe()
   }
 
@@ -605,7 +607,7 @@
     for (i <- 0 until numRecords) {
       val record = records.get(i)
       assertEquals(s"key $i", new String(record.key()))
-      assertEquals(s"value $i$appendStr".toUpperCase, new String(record.value()))
+      assertEquals(s"value $i$appendStr".toUpperCase(Locale.ROOT), new String(record.value()))
     }
 
     // commit sync and verify onCommit is called
@@ -688,6 +690,7 @@
     producerProps.put(ProducerConfig.INTERCEPTOR_CLASSES_CONFIG, "org.apache.kafka.test.MockProducerInterceptor")
     producerProps.put("mock.interceptor.append", appendStr)
     val testProducer = new KafkaProducer[Array[Byte], Array[Byte]](producerProps, new ByteArraySerializer(), new ByteArraySerializer())
+    producers += testProducer
 
     // producing records should succeed
     testProducer.send(new ProducerRecord(tp.topic(), tp.partition(), s"key".getBytes, s"value will not be modified".getBytes))
@@ -695,6 +698,8 @@
     // create consumer with interceptor that has different key and value types from the consumer
     this.consumerConfig.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, "org.apache.kafka.test.MockConsumerInterceptor")
     val testConsumer = new KafkaConsumer[Array[Byte], Array[Byte]](this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
+    consumers += testConsumer
+
     testConsumer.assign(List(tp).asJava)
     testConsumer.seek(tp, 0)
 
@@ -702,9 +707,6 @@
     val records = consumeRecords(testConsumer, 1)
     val record = records.get(0)
     assertEquals(s"value will not be modified", new String(record.value()))
-
-    testConsumer.close()
-    testProducer.close()
   }
 
   def testConsumeMessagesWithCreateTime() {
@@ -762,12 +764,14 @@
 
     // create one more consumer and add it to the group; we will timeout this consumer
     val timeoutConsumer = new KafkaConsumer[Array[Byte], Array[Byte]](this.consumerConfig)
-    val expandedConsumers = consumers ++ Buffer[KafkaConsumer[Array[Byte], Array[Byte]]](timeoutConsumer)
+    // Close the consumer on test teardown, unless this test will manually
+    if(!closeConsumer)
+      consumers += timeoutConsumer
     val timeoutPoller = subscribeConsumerAndStartPolling(timeoutConsumer, List(topic, topic1))
-    val expandedPollers = consumerPollers ++ Buffer[ConsumerAssignmentPoller](timeoutPoller)
+    consumerPollers += timeoutPoller
 
     // validate the initial assignment
-    validateGroupAssignment(expandedPollers, subscriptions, s"Did not get valid initial assignment for partitions ${subscriptions.asJava}")
+    validateGroupAssignment(consumerPollers, subscriptions, s"Did not get valid initial assignment for partitions ${subscriptions.asJava}")
 
     // stop polling and close one of the consumers, should trigger partition re-assignment among alive consumers
     timeoutPoller.shutdown()
@@ -859,6 +863,7 @@
     val consumerGroup = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]()
     for (i <- 0 until consumerCount)
       consumerGroup += new KafkaConsumer[Array[Byte], Array[Byte]](this.consumerConfig)
+    consumers ++= consumerGroup
 
     // create consumer pollers, wait for assignment and validate it
     val consumerPollers = subscribeConsumersAndWaitForAssignment(consumerGroup, topicsToSubscribe, subscriptions)
diff --git a/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala
new file mode 100644
index 0000000..fc79c60
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala
@@ -0,0 +1,86 @@
+/**
+  * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+  * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+  * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+  * License. You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+  * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+  * specific language governing permissions and limitations under the License.
+  */
+package kafka.api
+
+import java.io.File
+import org.apache.kafka.common.protocol.SecurityProtocol
+import kafka.server.KafkaConfig
+import org.junit.Test
+import kafka.utils.TestUtils
+import scala.collection.JavaConverters._
+
+class SaslMultiMechanismConsumerTest extends BaseConsumerTest with SaslTestHarness {
+  override protected val zkSaslEnabled = true
+  override protected val kafkaClientSaslMechanism = "PLAIN"
+  override protected val kafkaServerSaslMechanisms = List("GSSAPI", "PLAIN")
+  override protected def allKafkaClientSaslMechanisms = List("PLAIN", "GSSAPI")
+  this.serverConfig.setProperty(KafkaConfig.ZkEnableSecureAclsProp, "true")
+  override protected def securityProtocol = SecurityProtocol.SASL_SSL
+  override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks"))
+  override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, Some(kafkaServerSaslMechanisms)))
+
+  @Test
+  def testMultipleBrokerMechanisms() {
+
+    val plainSaslProducer = producers(0)
+    val plainSaslConsumer = consumers(0)
+
+    val gssapiSaslProperties = kafkaSaslProperties("GSSAPI")
+    val gssapiSaslProducer = TestUtils.createNewProducer(brokerList,
+                                                         securityProtocol = this.securityProtocol,
+                                                         trustStoreFile = this.trustStoreFile,
+                                                         saslProperties = Some(gssapiSaslProperties))
+    producers += gssapiSaslProducer
+    val gssapiSaslConsumer = TestUtils.createNewConsumer(brokerList,
+                                                         securityProtocol = this.securityProtocol,
+                                                         trustStoreFile = this.trustStoreFile,
+                                                         saslProperties = Some(gssapiSaslProperties))
+    consumers += gssapiSaslConsumer
+    val numRecords = 1000
+    var startingOffset = 0
+
+    // Test SASL/PLAIN producer and consumer
+    sendRecords(plainSaslProducer, numRecords, tp)
+    plainSaslConsumer.assign(List(tp).asJava)
+    plainSaslConsumer.seek(tp, 0)
+    consumeAndVerifyRecords(consumer = plainSaslConsumer, numRecords = numRecords, startingOffset = startingOffset)
+    val plainCommitCallback = new CountConsumerCommitCallback()
+    plainSaslConsumer.commitAsync(plainCommitCallback)
+    awaitCommitCallback(plainSaslConsumer, plainCommitCallback)
+    startingOffset += numRecords
+
+    // Test SASL/GSSAPI producer and consumer
+    sendRecords(gssapiSaslProducer, numRecords, tp)
+    gssapiSaslConsumer.assign(List(tp).asJava)
+    gssapiSaslConsumer.seek(tp, startingOffset)
+    consumeAndVerifyRecords(consumer = gssapiSaslConsumer, numRecords = numRecords, startingOffset = startingOffset)
+    val gssapiCommitCallback = new CountConsumerCommitCallback()
+    gssapiSaslConsumer.commitAsync(gssapiCommitCallback)
+    awaitCommitCallback(gssapiSaslConsumer, gssapiCommitCallback)
+    startingOffset += numRecords
+
+    // Test SASL/PLAIN producer and SASL/GSSAPI consumer
+    sendRecords(plainSaslProducer, numRecords, tp)
+    gssapiSaslConsumer.assign(List(tp).asJava)
+    gssapiSaslConsumer.seek(tp, startingOffset)
+    consumeAndVerifyRecords(consumer = gssapiSaslConsumer, numRecords = numRecords, startingOffset = startingOffset)
+    startingOffset += numRecords
+
+    // Test SASL/GSSAPI producer and SASL/PLAIN consumer
+    sendRecords(gssapiSaslProducer, numRecords, tp)
+    plainSaslConsumer.assign(List(tp).asJava)
+    plainSaslConsumer.seek(tp, startingOffset)
+    consumeAndVerifyRecords(consumer = plainSaslConsumer, numRecords = numRecords, startingOffset = startingOffset)
+
+  }
+}
diff --git a/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala
new file mode 100644
index 0000000..bdca577
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala
@@ -0,0 +1,27 @@
+/**
+  * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+  * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+  * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+  * License. You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+  * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+  * specific language governing permissions and limitations under the License.
+  */
+package kafka.api
+
+import java.io.File
+import org.apache.kafka.common.protocol.SecurityProtocol
+import kafka.server.KafkaConfig
+
+class SaslPlainPlaintextConsumerTest extends BaseConsumerTest with SaslTestHarness {
+  override protected val zkSaslEnabled = true
+  override protected val kafkaClientSaslMechanism = "PLAIN"
+  override protected val kafkaServerSaslMechanisms = List(kafkaClientSaslMechanism)
+  this.serverConfig.setProperty(KafkaConfig.ZkEnableSecureAclsProp, "true")
+  override protected def securityProtocol = SecurityProtocol.SASL_PLAINTEXT
+  override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks"))
+  override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, Some(kafkaServerSaslMechanisms)))
+}
diff --git a/core/src/test/scala/integration/kafka/api/SaslPlainSslEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/SaslPlainSslEndToEndAuthorizationTest.scala
new file mode 100644
index 0000000..63636c0
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/api/SaslPlainSslEndToEndAuthorizationTest.scala
@@ -0,0 +1,28 @@
+/**
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+package kafka.api
+
+import kafka.server.KafkaConfig
+import org.apache.kafka.common.protocol.SecurityProtocol
+
+class SaslPlainSslEndToEndAuthorizationTest extends EndToEndAuthorizationTest {
+  override protected def securityProtocol = SecurityProtocol.SASL_SSL
+  override protected def kafkaClientSaslMechanism = "PLAIN"
+  override protected def kafkaServerSaslMechanisms = List("PLAIN")
+  override val clientPrincipal = "testuser"
+  override val kafkaPrincipal = "admin"
+}
diff --git a/core/src/test/scala/integration/kafka/api/SaslSetup.scala b/core/src/test/scala/integration/kafka/api/SaslSetup.scala
index 967cae1..765f191 100644
--- a/core/src/test/scala/integration/kafka/api/SaslSetup.scala
+++ b/core/src/test/scala/integration/kafka/api/SaslSetup.scala
@@ -18,12 +18,14 @@
 package kafka.api
 
 import java.io.File
+import java.util.Properties
 import javax.security.auth.login.Configuration
-
 import kafka.security.minikdc.MiniKdc
+import kafka.server.KafkaConfig
 import kafka.utils.{JaasTestUtils, TestUtils}
 import org.apache.kafka.common.security.JaasUtils
-import org.apache.kafka.common.security.kerberos.LoginManager
+import org.apache.kafka.common.security.authenticator.LoginManager
+import org.apache.kafka.common.config.SaslConfigs
 
 /*
  * Implements an enumeration for the modes enabled here:
@@ -40,39 +42,56 @@
 trait SaslSetup {
   private val workDir = TestUtils.tempDir()
   private val kdcConf = MiniKdc.createConfig
-  private val kdc = new MiniKdc(kdcConf, workDir)
+  private var kdc: MiniKdc = null
 
-  def startSasl(mode: SaslSetupMode = Both) {
+  def startSasl(mode: SaslSetupMode = Both, kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanisms: List[String]) {
     // Important if tests leak consumers, producers or brokers
     LoginManager.closeAll()
-    val (serverKeytabFile, clientKeytabFile) = createKeytabsAndSetConfiguration(mode)
-    kdc.start()
-    kdc.createPrincipal(serverKeytabFile, "kafka/localhost")
-    kdc.createPrincipal(clientKeytabFile, "client")
+    val hasKerberos = mode != ZkSasl && (kafkaClientSaslMechanisms.contains("GSSAPI") || kafkaServerSaslMechanisms.contains("GSSAPI"))
+    if (hasKerberos) {
+      val serverKeytabFile = TestUtils.tempFile()
+      val clientKeytabFile = TestUtils.tempFile()
+      setJaasConfiguration(mode, kafkaServerSaslMechanisms, kafkaClientSaslMechanisms, Some(serverKeytabFile), Some(clientKeytabFile))
+      kdc = new MiniKdc(kdcConf, workDir)
+      kdc.start()
+      kdc.createPrincipal(serverKeytabFile, "kafka/localhost")
+      kdc.createPrincipal(clientKeytabFile, "client")
+    } else {
+      setJaasConfiguration(mode, kafkaServerSaslMechanisms, kafkaClientSaslMechanisms)
+    }
     if (mode == Both || mode == ZkSasl)
       System.setProperty("zookeeper.authProvider.1", "org.apache.zookeeper.server.auth.SASLAuthenticationProvider")
   }
 
-  protected def createKeytabsAndSetConfiguration(mode: SaslSetupMode): (File, File) = {
-    val serverKeytabFile = TestUtils.tempFile()
-    val clientKeytabFile = TestUtils.tempFile()
+  protected def setJaasConfiguration(mode: SaslSetupMode, kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanisms: List[String], 
+      serverKeytabFile: Option[File] = None, clientKeytabFile: Option[File] = None) {
     val jaasFile = mode match {
       case ZkSasl => JaasTestUtils.writeZkFile()
-      case KafkaSasl => JaasTestUtils.writeKafkaFile(serverKeytabFile, clientKeytabFile)
-      case Both => JaasTestUtils.writeZkAndKafkaFiles(serverKeytabFile, clientKeytabFile)
+      case KafkaSasl => JaasTestUtils.writeKafkaFile(kafkaServerSaslMechanisms, kafkaClientSaslMechanisms, serverKeytabFile, clientKeytabFile)
+      case Both => JaasTestUtils.writeZkAndKafkaFiles(kafkaServerSaslMechanisms, kafkaClientSaslMechanisms, serverKeytabFile, clientKeytabFile)
     }
     // This will cause a reload of the Configuration singleton when `getConfiguration` is called
     Configuration.setConfiguration(null)
     System.setProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM, jaasFile)
-    (serverKeytabFile, clientKeytabFile)
   }
 
   def closeSasl() {
-    kdc.stop()
+    if (kdc != null)
+      kdc.stop()
     // Important if tests leak consumers, producers or brokers
     LoginManager.closeAll()
     System.clearProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)
     System.clearProperty("zookeeper.authProvider.1")
     Configuration.setConfiguration(null)
   }
+
+  def kafkaSaslProperties(clientSaslMechanism: String, serverSaslMechanisms: Option[Seq[String]] = None) = {
+    val props = new Properties
+    props.put(SaslConfigs.SASL_MECHANISM, clientSaslMechanism)
+    serverSaslMechanisms.foreach { serverMechanisms =>
+        props.put(KafkaConfig.SaslMechanismInterBrokerProtocolProp, clientSaslMechanism)
+        props.put(SaslConfigs.SASL_ENABLED_MECHANISMS, serverMechanisms.mkString(","))
+    }
+    props
+  }
 }
diff --git a/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala
index b4ae74f..8fd3eb4 100644
--- a/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala
+++ b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala
@@ -17,13 +17,19 @@
 
 trait SaslTestHarness extends ZooKeeperTestHarness with SaslSetup {
   protected val zkSaslEnabled: Boolean
+  protected val kafkaClientSaslMechanism = "GSSAPI"
+  protected val kafkaServerSaslMechanisms = List(kafkaClientSaslMechanism)
+
+  // Override this list to enable client login modules for multiple mechanisms for testing
+  // of multi-mechanism brokers with clients using different mechanisms in a single JVM
+  protected def allKafkaClientSaslMechanisms = List(kafkaClientSaslMechanism)
 
   @Before
   override def setUp() {
     if (zkSaslEnabled)
-      startSasl(Both)
+      startSasl(Both, kafkaServerSaslMechanisms, allKafkaClientSaslMechanisms)
     else
-      startSasl(KafkaSasl)
+      startSasl(KafkaSasl, kafkaServerSaslMechanisms, allKafkaClientSaslMechanisms)
     super.setUp
   }
 
@@ -32,5 +38,4 @@
     super.tearDown
     closeSasl()
   }
-
 }
diff --git a/core/src/test/scala/other/kafka/TestPurgatoryPerformance.scala b/core/src/test/scala/other/kafka/TestPurgatoryPerformance.scala
index 744be3b..ba89fc8 100644
--- a/core/src/test/scala/other/kafka/TestPurgatoryPerformance.scala
+++ b/core/src/test/scala/other/kafka/TestPurgatoryPerformance.scala
@@ -95,7 +95,7 @@
     val latencySamples = new LatencySamples(1000000, pct75, pct50)
     val intervalSamples = new IntervalSamples(1000000, requestRate)
 
-    val purgatory = new DelayedOperationPurgatory[FakeOperation]("fake purgatory")
+    val purgatory = DelayedOperationPurgatory[FakeOperation]("fake purgatory")
     val queue = new CompletionQueue()
 
     val gcNames = gcMXBeans.map(_.getName)
diff --git a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala
new file mode 100644
index 0000000..ac2c1ae
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala
@@ -0,0 +1,112 @@
+/**
+  * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+  * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+  * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+  * License. You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+  * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+  * specific language governing permissions and limitations under the License.
+  */
+package unit.kafka.admin
+
+import kafka.admin.ReassignPartitionsCommand
+import kafka.server.{KafkaConfig, KafkaServer}
+import kafka.utils.TestUtils._
+import kafka.utils.ZkUtils._
+import kafka.utils.{CoreUtils, Logging}
+import kafka.zk.ZooKeeperTestHarness
+import org.junit.{After, Before, Test}
+import org.junit.Assert.assertEquals
+import scala.collection.Seq
+
+
+class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
+  val partitionId = 0
+  var servers: Seq[KafkaServer] = null
+  val topicName = "my-topic"
+
+  @Before
+  override def setUp() {
+    super.setUp()
+  }
+
+  def startBrokers(brokerIds: Seq[Int]) {
+    servers = brokerIds.map(i => createBrokerConfig(i, zkConnect))
+      .map(c => createServer(KafkaConfig.fromProps(c)))
+  }
+
+  @After
+  override def tearDown() {
+    servers.foreach(_.shutdown())
+    servers.foreach(server => CoreUtils.delete(server.config.logDirs))
+    super.tearDown()
+  }
+
+  @Test
+  def shouldMoveSinglePartition {
+    //Given a single replica on server 100
+    startBrokers(Seq(100, 101))
+    val partition = 0
+    createTopic(zkUtils, topicName, Map(partition -> Seq(100)), servers = servers)
+
+    //When we move the replica on 100 to broker 101
+    ReassignPartitionsCommand.executeAssignment(zkUtils, s"""{"version":1,"partitions":[{"topic":"$topicName","partition":0,"replicas":[101]}]}""")
+    waitForReasignmentToComplete()
+
+    //Then the replica should be on 101
+    assertEquals(zkUtils.getPartitionAssignmentForTopics(Seq(topicName)).get(topicName).get(partition), Seq(101))
+  }
+
+  @Test
+  def shouldExpandCluster() {
+    //Given partitions on 2 of 3 brokers
+    val brokers = Array(100, 101, 102)
+    startBrokers(brokers)
+    createTopic(zkUtils, topicName, Map(
+      0 -> Seq(100, 101),
+      1 -> Seq(100, 101),
+      2 -> Seq(100, 101)
+    ), servers = servers)
+
+    //When rebalancing
+    val newAssignment = ReassignPartitionsCommand.generateAssignment(zkUtils, brokers, json(topicName), true)._1
+    ReassignPartitionsCommand.executeAssignment(zkUtils, zkUtils.formatAsReassignmentJson(newAssignment))
+    waitForReasignmentToComplete()
+
+    //Then the replicas should span all three brokers
+    val actual = zkUtils.getPartitionAssignmentForTopics(Seq(topicName))(topicName)
+    assertEquals(actual.values.flatten.toSeq.distinct.sorted, Seq(100, 101, 102))
+  }
+
+  @Test
+  def shouldShrinkCluster() {
+    //Given partitions on 3 of 3 brokers
+    val brokers = Array(100, 101, 102)
+    startBrokers(brokers)
+    createTopic(zkUtils, topicName, Map(
+      0 -> Seq(100, 101),
+      1 -> Seq(101, 102),
+      2 -> Seq(102, 100)
+    ), servers = servers)
+
+    //When rebalancing
+    val newAssignment = ReassignPartitionsCommand.generateAssignment(zkUtils, Array(100, 101), json(topicName), true)._1
+    ReassignPartitionsCommand.executeAssignment(zkUtils, zkUtils.formatAsReassignmentJson(newAssignment))
+    waitForReasignmentToComplete()
+
+    //Then replicas should only span the first two brokers
+    val actual = zkUtils.getPartitionAssignmentForTopics(Seq(topicName))(topicName)
+    assertEquals(actual.values.flatten.toSeq.distinct.sorted, Seq(100, 101))
+  }
+
+  def waitForReasignmentToComplete() {
+    waitUntilTrue(() => !zkUtils.pathExists(ReassignPartitionsPath), s"Znode $zkUtils.ReassignPartitionsPath wasn't deleted")
+  }
+
+  def json(topic: String): String = {
+    s"""{"topics": [{"topic": "$topic"}],"version":1}"""
+  }
+}
diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
index acdb660..beab1b5 100644
--- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
@@ -17,14 +17,14 @@
 
 package kafka.coordinator
 
+import kafka.utils.timer.MockTimer
 import org.apache.kafka.common.record.Record
 import org.junit.Assert._
-import kafka.common.{OffsetAndMetadata, TopicAndPartition}
+import kafka.common.OffsetAndMetadata
 import kafka.message.{Message, MessageSet}
-import kafka.server.{ReplicaManager, KafkaConfig}
+import kafka.server.{DelayedOperationPurgatory, ReplicaManager, KafkaConfig}
 import kafka.utils._
-import org.apache.kafka.common.utils.SystemTime
-import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.{utils, TopicPartition}
 import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.requests.{OffsetCommitRequest, JoinGroupRequest}
 import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
@@ -56,6 +56,7 @@
   val ConsumerMinSessionTimeout = 10
   val ConsumerMaxSessionTimeout = 1000
   val DefaultSessionTimeout = 500
+  var timer: MockTimer = null
   var groupCoordinator: GroupCoordinator = null
   var replicaManager: ReplicaManager = null
   var scheduler: KafkaScheduler = null
@@ -87,7 +88,14 @@
     EasyMock.expect(zkUtils.getPartitionAssignmentForTopics(Seq(TopicConstants.GROUP_METADATA_TOPIC_NAME))).andReturn(ret)
     EasyMock.replay(zkUtils)
 
-    groupCoordinator = GroupCoordinator(KafkaConfig.fromProps(props), zkUtils, replicaManager, new SystemTime)
+    timer = new MockTimer
+
+    val config = KafkaConfig.fromProps(props)
+
+    val heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", timer, config.brokerId, reaperEnabled = false)
+    val joinPurgatory = new DelayedOperationPurgatory[DelayedJoin]("Rebalance", timer, config.brokerId, reaperEnabled = false)
+
+    groupCoordinator = GroupCoordinator(config, zkUtils, replicaManager, heartbeatPurgatory, joinPurgatory, timer.time)
     groupCoordinator.startup()
 
     // add the partition into the owned partition list
@@ -284,6 +292,90 @@
   }
 
   @Test
+  def testSessionTimeout() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+
+    val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols)
+    val assignedConsumerId = joinGroupResult.memberId
+    val generationId = joinGroupResult.generationId
+    val joinGroupErrorCode = joinGroupResult.errorCode
+    assertEquals(Errors.NONE.code, joinGroupErrorCode)
+
+    EasyMock.reset(replicaManager)
+    val (_, syncGroupErrorCode) = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()))
+    assertEquals(Errors.NONE.code, syncGroupErrorCode)
+
+    EasyMock.reset(replicaManager)
+    EasyMock.expect(replicaManager.getPartition(TopicConstants.GROUP_METADATA_TOPIC_NAME, groupPartitionId)).andReturn(None)
+    EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes()
+    EasyMock.replay(replicaManager)
+
+    timer.advanceClock(DefaultSessionTimeout + 100)
+
+    EasyMock.reset(replicaManager)
+    val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1)
+    assertEquals(Errors.UNKNOWN_MEMBER_ID.code, heartbeatResult)
+  }
+
+  @Test
+  def testHeartbeatMaintainsSession() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val sessionTimeout = 1000
+
+    val joinGroupResult = joinGroup(groupId, memberId, sessionTimeout, protocolType, protocols)
+    val assignedConsumerId = joinGroupResult.memberId
+    val generationId = joinGroupResult.generationId
+    val joinGroupErrorCode = joinGroupResult.errorCode
+    assertEquals(Errors.NONE.code, joinGroupErrorCode)
+
+    EasyMock.reset(replicaManager)
+    val (_, syncGroupErrorCode) = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()))
+    assertEquals(Errors.NONE.code, syncGroupErrorCode)
+
+    timer.advanceClock(sessionTimeout / 2)
+
+    EasyMock.reset(replicaManager)
+    var heartbeatResult = heartbeat(groupId, assignedConsumerId, 1)
+    assertEquals(Errors.NONE.code, heartbeatResult)
+
+    timer.advanceClock(sessionTimeout / 2 + 100)
+
+    EasyMock.reset(replicaManager)
+    heartbeatResult = heartbeat(groupId, assignedConsumerId, 1)
+    assertEquals(Errors.NONE.code, heartbeatResult)
+  }
+
+  @Test
+  def testCommitMaintainsSession() {
+    val sessionTimeout = 1000
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val tp = new TopicPartition("topic", 0)
+    val offset = OffsetAndMetadata(0)
+
+    val joinGroupResult = joinGroup(groupId, memberId, sessionTimeout, protocolType, protocols)
+    val assignedConsumerId = joinGroupResult.memberId
+    val generationId = joinGroupResult.generationId
+    val joinGroupErrorCode = joinGroupResult.errorCode
+    assertEquals(Errors.NONE.code, joinGroupErrorCode)
+
+    EasyMock.reset(replicaManager)
+    val (_, syncGroupErrorCode) = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()))
+    assertEquals(Errors.NONE.code, syncGroupErrorCode)
+
+    timer.advanceClock(sessionTimeout / 2)
+
+    EasyMock.reset(replicaManager)
+    val commitOffsetResult = commitOffsets(groupId, assignedConsumerId, generationId, immutable.Map(tp -> offset))
+    assertEquals(Errors.NONE.code, commitOffsetResult(tp))
+
+    timer.advanceClock(sessionTimeout / 2 + 100)
+
+    EasyMock.reset(replicaManager)
+    val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1)
+    assertEquals(Errors.NONE.code, heartbeatResult)
+  }
+
+  @Test
   def testSyncGroupEmptyAssignment() {
     val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
 
@@ -459,7 +551,10 @@
     // with no leader SyncGroup, the follower's request should failure with an error indicating
     // that it should rejoin
     EasyMock.reset(replicaManager)
-    val followerSyncFuture= sendSyncGroupFollower(groupId, nextGenerationId, otherJoinResult.memberId)
+    val followerSyncFuture = sendSyncGroupFollower(groupId, nextGenerationId, otherJoinResult.memberId)
+
+    timer.advanceClock(DefaultSessionTimeout + 100)
+
     val followerSyncResult = await(followerSyncFuture, DefaultSessionTimeout+100)
     assertEquals(Errors.REBALANCE_IN_PROGRESS.code, followerSyncResult._2)
   }
@@ -628,17 +723,20 @@
 
   @Test
   def testGenerationIdIncrementsOnRebalance() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-    val otherMemberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-
-    val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols)
+    val joinGroupResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols)
     val initialGenerationId = joinGroupResult.generationId
     val joinGroupErrorCode = joinGroupResult.errorCode
+    val memberId = joinGroupResult.memberId
     assertEquals(1, initialGenerationId)
     assertEquals(Errors.NONE.code, joinGroupErrorCode)
 
     EasyMock.reset(replicaManager)
-    val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, protocolType, protocols)
+    val syncGroupResult = syncGroupLeader(groupId, initialGenerationId, memberId, Map(memberId -> Array[Byte]()))
+    val syncGroupErrorCode = syncGroupResult._2
+    assertEquals(Errors.NONE.code, syncGroupErrorCode)
+
+    EasyMock.reset(replicaManager)
+    val otherJoinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols)
     val nextGenerationId = otherJoinGroupResult.generationId
     val otherJoinGroupErrorCode = otherJoinGroupResult.errorCode
     assertEquals(2, nextGenerationId)
@@ -860,6 +958,7 @@
                         protocolType: String,
                         protocols: List[(String, Array[Byte])]): JoinGroupResult = {
     val responseFuture = sendJoinGroup(groupId, memberId, sessionTimeout, protocolType, protocols)
+    timer.advanceClock(10)
     // should only have to wait as long as session timeout, but allow some extra time in case of an unexpected delay
     Await.result(responseFuture, Duration(sessionTimeout+100, TimeUnit.MILLISECONDS))
   }
diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
index 2ca64f2..8e8ae8b 100755
--- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
+++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
@@ -19,7 +19,6 @@
 
 import java.io.File
 import java.util.Arrays
-
 import kafka.common.KafkaException
 import kafka.server._
 import kafka.utils.{CoreUtils, TestUtils}
@@ -27,8 +26,8 @@
 import org.apache.kafka.common.protocol.SecurityProtocol
 import org.apache.kafka.common.security.auth.KafkaPrincipal
 import org.junit.{After, Before}
-
 import scala.collection.mutable.Buffer
+import java.util.Properties
 
 /**
  * A test harness that brings up some number of broker nodes
@@ -57,6 +56,7 @@
 
   protected def securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT
   protected def trustStoreFile: Option[File] = None
+  protected def saslProperties: Option[Properties] = None
 
   @Before
   override def setUp() {
diff --git a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala
deleted file mode 100755
index 5221855..0000000
--- a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala
+++ /dev/null
@@ -1,95 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package kafka.integration
-
-import org.junit.{Test, After, Before}
-import kafka.zk.ZooKeeperTestHarness
-import kafka.utils.TestUtils._
-import org.junit.Assert._
-import kafka.utils.{CoreUtils, TestUtils}
-import kafka.server.{KafkaConfig, KafkaServer}
-
-class RollingBounceTest extends ZooKeeperTestHarness {
-
-  val partitionId = 0
-  var servers: Seq[KafkaServer] = null
-
-  @Before
-  override def setUp() {
-    super.setUp()
-    // controlled.shutdown.enable is true by default
-    val configs = (0 until 4).map(i => TestUtils.createBrokerConfig(i, zkConnect))
-    configs(3).put("controlled.shutdown.retry.backoff.ms", "100")
- 
-    // start all the servers
-    servers = configs.map(c => TestUtils.createServer(KafkaConfig.fromProps(c)))
-  }
-
-  @After
-  override def tearDown() {
-    servers.foreach(_.shutdown())
-    servers.foreach(server => CoreUtils.delete(server.config.logDirs))
-    super.tearDown()
-  }
-
-  @Test
-  def testRollingBounce {
-    // start all the brokers
-    val topic1 = "new-topic1"
-    val topic2 = "new-topic2"
-    val topic3 = "new-topic3"
-    val topic4 = "new-topic4"
-
-    // create topics with 1 partition, 2 replicas, one on each broker
-    createTopic(zkUtils, topic1, partitionReplicaAssignment = Map(0->Seq(0,1)), servers = servers)
-    createTopic(zkUtils, topic2, partitionReplicaAssignment = Map(0->Seq(1,2)), servers = servers)
-    createTopic(zkUtils, topic3, partitionReplicaAssignment = Map(0->Seq(2,3)), servers = servers)
-    createTopic(zkUtils, topic4, partitionReplicaAssignment = Map(0->Seq(0,3)), servers = servers)
-
-    // Do a rolling bounce and check if leader transitions happen correctly
-
-    // Bring down the leader for the first topic
-    bounceServer(topic1, 0)
-
-    // Bring down the leader for the second topic
-    bounceServer(topic2, 1)
-
-    // Bring down the leader for the third topic
-    bounceServer(topic3, 2)
-
-    // Bring down the leader for the fourth topic
-    bounceServer(topic4, 3)
-  }
-
-  private def bounceServer(topic: String, startIndex: Int) {
-    var prevLeader = 0
-    if (isLeaderLocalOnBroker(topic, partitionId, servers(startIndex))) {
-      servers(startIndex).shutdown()
-      prevLeader = startIndex
-    }
-    else {
-      servers((startIndex + 1) % 4).shutdown()
-      prevLeader = (startIndex + 1) % 4
-    }
-    var newleader = waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partitionId)
-    // Ensure the new leader is different from the old
-    assertTrue("Leader transition did not happen for " + topic, newleader.getOrElse(-1) != -1 && (newleader.getOrElse(-1) != prevLeader))
-    // Start the server back up again
-    servers(prevLeader).startup()
-  }
-}
diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala
index 3f6a275..8c973a4 100755
--- a/core/src/test/scala/unit/kafka/log/LogTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogTest.scala
@@ -228,7 +228,7 @@
   /**
    * Test reading at the boundary of the log, specifically
    * - reading from the logEndOffset should give an empty message set
-   * - reading from the the maxOffset should give an empty message set
+   * - reading from the maxOffset should give an empty message set
    * - reading beyond the log end offset should throw an OffsetOutOfRangeException
    */
   @Test
diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala
new file mode 100644
index 0000000..8bf4d73
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala
@@ -0,0 +1,55 @@
+/**
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *    http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+
+package kafka.server
+
+import org.apache.kafka.common.protocol.ApiKeys
+import org.apache.kafka.common.requests.ApiVersionsResponse.ApiVersion
+import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse}
+import org.junit.Assert._
+import org.junit.Test
+
+import scala.collection.JavaConversions._
+
+object ApiVersionsRequestTest {
+  def validateApiVersionsResponse(apiVersionsResponse: ApiVersionsResponse) {
+    assertEquals("API keys in ApiVersionsResponse must match API keys supported by broker.", ApiKeys.values.length, apiVersionsResponse.apiVersions.size)
+    for (expectedApiVersion: ApiVersion <- ApiVersionsResponse.apiVersionsResponse.apiVersions) {
+      val actualApiVersion = apiVersionsResponse.apiVersion(expectedApiVersion.apiKey)
+      assertNotNull(s"API key ${actualApiVersion.apiKey} is supported by broker, but not received in ApiVersionsResponse.", actualApiVersion)
+      assertEquals("API key must be supported by the broker.", expectedApiVersion.apiKey, actualApiVersion.apiKey)
+      assertEquals(s"Received unexpected min version for API key ${actualApiVersion.apiKey}.", expectedApiVersion.minVersion, actualApiVersion.minVersion)
+      assertEquals(s"Received unexpected max version for API key ${actualApiVersion.apiKey}.", expectedApiVersion.maxVersion, actualApiVersion.maxVersion)
+    }
+  }
+}
+
+class ApiVersionsRequestTest extends BaseRequestTest {
+
+  override def numBrokers: Int = 1
+
+  @Test
+  def testApiVersionsRequest() {
+    val apiVersionsResponse = sendApiVersionsRequest(new ApiVersionsRequest, 0)
+    ApiVersionsRequestTest.validateApiVersionsResponse(apiVersionsResponse)
+  }
+
+  private def sendApiVersionsRequest(request: ApiVersionsRequest, version: Short): ApiVersionsResponse = {
+    val response = send(request, ApiKeys.API_VERSIONS, version)
+    ApiVersionsResponse.parse(response)
+  }
+}
\ No newline at end of file
diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsTest.scala
new file mode 100644
index 0000000..177b509
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/server/ApiVersionsTest.scala
@@ -0,0 +1,51 @@
+/**
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *    http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+
+package unit.kafka.server
+
+import org.apache.kafka.common.requests.ApiVersionsResponse
+import org.apache.kafka.common.protocol.{Protocol, ApiKeys}
+import org.junit.Assert._
+import org.junit.Test
+
+class ApiVersionsTest {
+
+  @Test
+  def testApiVersions {
+    val apiVersions = ApiVersionsResponse.apiVersionsResponse.apiVersions
+    assertEquals("API versions for all API keys must be maintained.", apiVersions.size, ApiKeys.values().length)
+
+    for (key <- ApiKeys.values) {
+      val version = ApiVersionsResponse.apiVersionsResponse.apiVersion(key.id)
+      assertNotNull(s"Could not find ApiVersion for API ${key.name}", version)
+      assertEquals(s"Incorrect min version for Api ${key.name}.", version.minVersion, Protocol.MIN_VERSIONS(key.id))
+      assertEquals(s"Incorrect max version for Api ${key.name}.", version.maxVersion, Protocol.CURR_VERSION(key.id))
+
+      // Check if versions less than min version are indeed set as null, i.e., deprecated.
+      for (i <- 0 until version.minVersion) {
+        assertNull(s"Request version $i for API ${version.apiKey} must be null.", Protocol.REQUESTS(version.apiKey)(i))
+        assertNull(s"Response version $i for API ${version.apiKey} must be null.", Protocol.RESPONSES(version.apiKey)(i))
+      }
+
+      // Check if versions between min and max versions are non null, i.e., valid.
+      for (i <- version.minVersion.toInt to version.maxVersion) {
+        assertNotNull(s"Request version $i for API ${version.apiKey} must not be null.", Protocol.REQUESTS(version.apiKey)(i))
+        assertNotNull(s"Response version $i for API ${version.apiKey} must not be null.", Protocol.RESPONSES(version.apiKey)(i))
+      }
+    }
+  }
+}
\ No newline at end of file
diff --git a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala
new file mode 100644
index 0000000..906c4b2
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala
@@ -0,0 +1,114 @@
+/**
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *    http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+
+package kafka.server
+
+import java.io.{DataInputStream, DataOutputStream}
+import java.net.Socket
+import java.nio.ByteBuffer
+import java.util.Properties
+
+import kafka.integration.KafkaServerTestHarness
+import kafka.network.SocketServer
+import kafka.utils._
+import org.apache.kafka.common.protocol.{ApiKeys, SecurityProtocol}
+import org.apache.kafka.common.requests.{AbstractRequest, RequestHeader, ResponseHeader}
+import org.junit.Before
+
+abstract class BaseRequestTest extends KafkaServerTestHarness {
+  private var correlationId = 0
+
+  // If required, set number of brokers
+  protected def numBrokers: Int = 3
+
+  // If required, override properties by mutating the passed Properties object
+  protected def propertyOverrides(properties: Properties) {}
+
+  def generateConfigs() = {
+    val props = TestUtils.createBrokerConfigs(numBrokers, zkConnect, enableControlledShutdown = false,
+      interBrokerSecurityProtocol = Some(securityProtocol),
+      trustStoreFile = trustStoreFile, saslProperties = saslProperties)
+    props.foreach(propertyOverrides)
+    props.map(KafkaConfig.fromProps)
+  }
+
+  @Before
+  override def setUp() {
+    super.setUp()
+    TestUtils.waitUntilTrue(() => servers.head.metadataCache.getAliveBrokers.size == numBrokers, "Wait for cache to update")
+  }
+
+  def socketServer = {
+    servers.find { server =>
+      val state = server.brokerState.currentState
+      state != NotRunning.state && state != BrokerShuttingDown.state
+    }.map(_.socketServer).getOrElse(throw new IllegalStateException("No live broker is available"))
+  }
+
+  def connect(s: SocketServer = socketServer, protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): Socket = {
+    new Socket("localhost", s.boundPort(protocol))
+  }
+
+  private def sendRequest(socket: Socket, request: Array[Byte]) {
+    val outgoing = new DataOutputStream(socket.getOutputStream)
+    outgoing.writeInt(request.length)
+    outgoing.write(request)
+    outgoing.flush()
+  }
+
+  private def receiveResponse(socket: Socket): Array[Byte] = {
+    val incoming = new DataInputStream(socket.getInputStream)
+    val len = incoming.readInt()
+    val response = new Array[Byte](len)
+    incoming.readFully(response)
+    response
+  }
+
+  def requestAndReceive(socket: Socket, request: Array[Byte]): Array[Byte] = {
+    sendRequest(socket, request)
+    receiveResponse(socket)
+  }
+
+  def send(request: AbstractRequest, apiKey: ApiKeys, version: Short): ByteBuffer = {
+    val socket = connect()
+    try {
+      send(socket, request, apiKey, version)
+    } finally {
+      socket.close()
+    }
+  }
+
+  /**
+    * Serializes and send the request to the given api. A ByteBuffer containing the response is returned.
+    */
+  def send(socket: Socket, request: AbstractRequest, apiKey: ApiKeys, version: Short): ByteBuffer = {
+    correlationId += 1
+    val serializedBytes = {
+      val header = new RequestHeader(apiKey.id, version, "", correlationId)
+      val byteBuffer = ByteBuffer.allocate(header.sizeOf() + request.sizeOf)
+      header.writeTo(byteBuffer)
+      request.writeTo(byteBuffer)
+      byteBuffer.array()
+    }
+
+    val response = requestAndReceive(socket, serializedBytes)
+
+    val responseBuffer = ByteBuffer.wrap(response)
+    ResponseHeader.parse(responseBuffer) // Parse the header to ensure its valid and move the buffer forward
+    responseBuffer
+  }
+}
diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala
index 193acfd2b..69e83c0 100644
--- a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala
@@ -104,7 +104,7 @@
       assertEquals(10, numCallbacks)
       time.sleep(sleepTime)
 
-      // Callback can only be triggered after the the delay time passes
+      // Callback can only be triggered after the delay time passes
       clientMetrics.throttledRequestReaper.doWork()
       assertEquals(0, queueSizeMetric.value().toInt)
       assertEquals(11, numCallbacks)
diff --git a/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala b/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala
index df8d5b1..2c70137 100644
--- a/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala
+++ b/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala
@@ -26,7 +26,7 @@
 
   @Before
   def setUp() {
-    purgatory = new DelayedOperationPurgatory[MockDelayedOperation](purgatoryName = "mock")
+    purgatory = DelayedOperationPurgatory[MockDelayedOperation](purgatoryName = "mock")
   }
 
   @After
diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
index fa240d2..f8476cd 100755
--- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
@@ -550,6 +550,8 @@
         case KafkaConfig.SslCipherSuitesProp => // ignore string
 
         //Sasl Configs
+        case KafkaConfig.SaslMechanismInterBrokerProtocolProp => // ignore
+        case KafkaConfig.SaslEnabledMechanismsProp =>
         case KafkaConfig.SaslKerberosServiceNameProp => // ignore string
         case KafkaConfig.SaslKerberosKinitCmdProp =>
         case KafkaConfig.SaslKerberosTicketRenewWindowFactorProp =>
diff --git a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala
index 017faea..770513c 100644
--- a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala
+++ b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala
@@ -158,7 +158,8 @@
     val updateMetadataRequest = new UpdateMetadataRequest(controllerId, controllerEpoch, partitionStates.asJava, brokers.asJava)
     cache.updateCache(15, updateMetadataRequest)
 
-    val topicMetadatas = cache.getTopicMetadata(Set(topic), SecurityProtocol.PLAINTEXT)
+    // Validate errorUnavailableEndpoints = false
+    val topicMetadatas = cache.getTopicMetadata(Set(topic), SecurityProtocol.PLAINTEXT, errorUnavailableEndpoints = false)
     assertEquals(1, topicMetadatas.size)
 
     val topicMetadata = topicMetadatas.head
@@ -169,9 +170,25 @@
 
     val partitionMetadata = partitionMetadatas.get(0)
     assertEquals(0, partitionMetadata.partition)
-    assertEquals(Errors.REPLICA_NOT_AVAILABLE, partitionMetadata.error)
-    assertEquals(Set(0), partitionMetadata.replicas.asScala.map(_.id).toSet)
+    assertEquals(Errors.NONE, partitionMetadata.error)
+    assertEquals(Set(0, 1), partitionMetadata.replicas.asScala.map(_.id).toSet)
     assertEquals(Set(0), partitionMetadata.isr.asScala.map(_.id).toSet)
+
+    // Validate errorUnavailableEndpoints = true
+    val topicMetadatasWithError = cache.getTopicMetadata(Set(topic), SecurityProtocol.PLAINTEXT, errorUnavailableEndpoints = true)
+    assertEquals(1, topicMetadatasWithError.size)
+
+    val topicMetadataWithError = topicMetadatasWithError.head
+    assertEquals(Errors.NONE, topicMetadataWithError.error)
+
+    val partitionMetadatasWithError = topicMetadataWithError.partitionMetadata
+    assertEquals(1, partitionMetadatasWithError.size)
+
+    val partitionMetadataWithError = partitionMetadatasWithError.get(0)
+    assertEquals(0, partitionMetadataWithError.partition)
+    assertEquals(Errors.REPLICA_NOT_AVAILABLE, partitionMetadataWithError.error)
+    assertEquals(Set(0), partitionMetadataWithError.replicas.asScala.map(_.id).toSet)
+    assertEquals(Set(0), partitionMetadataWithError.isr.asScala.map(_.id).toSet)
   }
 
   @Test
@@ -197,7 +214,8 @@
     val updateMetadataRequest = new UpdateMetadataRequest(controllerId, controllerEpoch, partitionStates.asJava, brokers.asJava)
     cache.updateCache(15, updateMetadataRequest)
 
-    val topicMetadatas = cache.getTopicMetadata(Set(topic), SecurityProtocol.PLAINTEXT)
+    // Validate errorUnavailableEndpoints = false
+    val topicMetadatas = cache.getTopicMetadata(Set(topic), SecurityProtocol.PLAINTEXT, errorUnavailableEndpoints = false)
     assertEquals(1, topicMetadatas.size)
 
     val topicMetadata = topicMetadatas.head
@@ -208,9 +226,25 @@
 
     val partitionMetadata = partitionMetadatas.get(0)
     assertEquals(0, partitionMetadata.partition)
-    assertEquals(Errors.REPLICA_NOT_AVAILABLE, partitionMetadata.error)
+    assertEquals(Errors.NONE, partitionMetadata.error)
     assertEquals(Set(0), partitionMetadata.replicas.asScala.map(_.id).toSet)
-    assertEquals(Set(0), partitionMetadata.isr.asScala.map(_.id).toSet)
+    assertEquals(Set(0, 1), partitionMetadata.isr.asScala.map(_.id).toSet)
+
+    // Validate errorUnavailableEndpoints = true
+    val topicMetadatasWithError = cache.getTopicMetadata(Set(topic), SecurityProtocol.PLAINTEXT, errorUnavailableEndpoints = true)
+    assertEquals(1, topicMetadatasWithError.size)
+
+    val topicMetadataWithError = topicMetadatasWithError.head
+    assertEquals(Errors.NONE, topicMetadataWithError.error)
+
+    val partitionMetadatasWithError = topicMetadataWithError.partitionMetadata
+    assertEquals(1, partitionMetadatasWithError.size)
+
+    val partitionMetadataWithError = partitionMetadatasWithError.get(0)
+    assertEquals(0, partitionMetadataWithError.partition)
+    assertEquals(Errors.REPLICA_NOT_AVAILABLE, partitionMetadataWithError.error)
+    assertEquals(Set(0), partitionMetadataWithError.replicas.asScala.map(_.id).toSet)
+    assertEquals(Set(0), partitionMetadataWithError.isr.asScala.map(_.id).toSet)
   }
 
   @Test
diff --git a/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala
new file mode 100644
index 0000000..3d4b40c
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala
@@ -0,0 +1,168 @@
+/**
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *    http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+
+package kafka.server
+
+import java.util.Properties
+
+import kafka.utils.TestUtils
+import org.apache.kafka.common.internals.TopicConstants
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.{MetadataRequest, MetadataResponse}
+import org.junit.Assert._
+import org.junit.Test
+
+import scala.collection.JavaConverters._
+
+class MetadataRequestTest extends BaseRequestTest {
+
+  override def propertyOverrides(properties: Properties) {
+    properties.setProperty(KafkaConfig.RackProp, s"rack/${properties.getProperty(KafkaConfig.BrokerIdProp)}")
+  }
+
+  @Test
+  def testControllerId() {
+    val controllerServer = servers.find(_.kafkaController.isActive()).get
+    val controllerId = controllerServer.config.brokerId
+    val metadataResponse = sendMetadataRequest(MetadataRequest.allTopics(), 1)
+
+    assertEquals("Controller id should match the active controller",
+      controllerId, metadataResponse.controller.id)
+
+    // Fail over the controller
+    controllerServer.shutdown()
+    controllerServer.startup()
+
+    val controllerServer2 = servers.find(_.kafkaController.isActive()).get
+    val controllerId2 = controllerServer2.config.brokerId
+    assertNotEquals("Controller id should switch to a new broker", controllerId, controllerId2)
+    TestUtils.waitUntilTrue(() => {
+      val metadataResponse2 = sendMetadataRequest(MetadataRequest.allTopics(), 1)
+      controllerServer2.apis.brokerId == metadataResponse2.controller.id
+    }, "Controller id should match the active controller after failover", 5000)
+  }
+
+  @Test
+  def testRack() {
+    val metadataResponse = sendMetadataRequest(MetadataRequest.allTopics(), 1)
+    // Validate rack matches what's set in generateConfigs() above
+    metadataResponse.brokers.asScala.foreach { broker =>
+      assertEquals("Rack information should match config", s"rack/${broker.id}", broker.rack)
+    }
+  }
+
+  @Test
+  def testIsInternal() {
+    val internalTopic = TopicConstants.GROUP_METADATA_TOPIC_NAME
+    val notInternalTopic = "notInternal"
+    // create the topics
+    TestUtils.createTopic(zkUtils, internalTopic, 3, 2, servers)
+    TestUtils.createTopic(zkUtils, notInternalTopic, 3, 2, servers)
+
+    val metadataResponse = sendMetadataRequest(MetadataRequest.allTopics(), 1)
+    assertTrue("Response should have no errors", metadataResponse.errors.isEmpty)
+
+    val topicMetadata = metadataResponse.topicMetadata.asScala
+    val internalTopicMetadata = topicMetadata.find(_.topic == internalTopic).get
+    val notInternalTopicMetadata = topicMetadata.find(_.topic == notInternalTopic).get
+
+    assertTrue("internalTopic should show isInternal", internalTopicMetadata.isInternal)
+    assertFalse("notInternalTopic topic not should show isInternal", notInternalTopicMetadata.isInternal)
+  }
+
+  @Test
+  def testNoTopicsRequest() {
+    // create some topics
+    TestUtils.createTopic(zkUtils, "t1", 3, 2, servers)
+    TestUtils.createTopic(zkUtils, "t2", 3, 2, servers)
+
+    // v0, Doesn't support a "no topics" request
+    // v1, Empty list represents "no topics"
+    val metadataResponse = sendMetadataRequest(new MetadataRequest(List[String]().asJava), 1)
+    assertTrue("Response should have no errors", metadataResponse.errors.isEmpty)
+    assertTrue("Response should have no topics", metadataResponse.topicMetadata.isEmpty)
+  }
+
+  @Test
+  def testAllTopicsRequest() {
+    // create some topics
+    TestUtils.createTopic(zkUtils, "t1", 3, 2, servers)
+    TestUtils.createTopic(zkUtils, "t2", 3, 2, servers)
+
+    // v0, Empty list represents all topics
+    val metadataResponseV0 = sendMetadataRequest(new MetadataRequest(List[String]().asJava), 0)
+    assertTrue("V0 Response should have no errors", metadataResponseV0.errors.isEmpty)
+    assertEquals("V0 Response should have 2 (all) topics", 2, metadataResponseV0.topicMetadata.size())
+
+    // v1, Null represents all topics
+    val metadataResponseV1 = sendMetadataRequest(MetadataRequest.allTopics(), 1)
+    assertTrue("V1 Response should have no errors", metadataResponseV1.errors.isEmpty)
+    assertEquals("V1 Response should have 2 (all) topics", 2, metadataResponseV1.topicMetadata.size())
+  }
+
+  @Test
+  def testReplicaDownResponse() {
+    val replicaDownTopic = "replicaDown"
+    val replicaCount = 3
+
+    // create a topic with 3 replicas
+    TestUtils.createTopic(zkUtils, replicaDownTopic, 1, replicaCount, servers)
+
+    // Kill a replica node that is not the leader
+    val metadataResponse = sendMetadataRequest(new MetadataRequest(List(replicaDownTopic).asJava), 1)
+    val partitionMetadata = metadataResponse.topicMetadata.asScala.head.partitionMetadata.asScala.head
+    val downNode = servers.find { server =>
+      val serverId = server.apis.brokerId
+      val leaderId = partitionMetadata.leader.id
+      val replicaIds = partitionMetadata.replicas.asScala.map(_.id)
+      serverId != leaderId && replicaIds.contains(serverId)
+    }.get
+    downNode.shutdown()
+
+    TestUtils.waitUntilTrue(() => {
+      val response = sendMetadataRequest(new MetadataRequest(List(replicaDownTopic).asJava), 1)
+      val metadata = response.topicMetadata.asScala.head.partitionMetadata.asScala.head
+      val replica = metadata.replicas.asScala.find(_.id == downNode.apis.brokerId).get
+      replica.host == "" & replica.port == -1
+    }, "Replica was not found down", 5000)
+
+    // Validate version 0 still filters unavailable replicas and contains error
+    val v0MetadataResponse = sendMetadataRequest(new MetadataRequest(List(replicaDownTopic).asJava), 0)
+    val v0BrokerIds = v0MetadataResponse.brokers().asScala.map(_.id).toSeq
+    assertTrue("Response should have no errors", v0MetadataResponse.errors.isEmpty)
+    assertFalse(s"The downed broker should not be in the brokers list", v0BrokerIds.contains(downNode))
+    assertTrue("Response should have one topic", v0MetadataResponse.topicMetadata.size == 1)
+    val v0PartitionMetadata = v0MetadataResponse.topicMetadata.asScala.head.partitionMetadata.asScala.head
+    assertTrue("PartitionMetadata should have an error", v0PartitionMetadata.error == Errors.REPLICA_NOT_AVAILABLE)
+    assertTrue(s"Response should have ${replicaCount - 1} replicas", v0PartitionMetadata.replicas.size == replicaCount - 1)
+
+    // Validate version 1 returns unavailable replicas with no error
+    val v1MetadataResponse = sendMetadataRequest(new MetadataRequest(List(replicaDownTopic).asJava), 1)
+    val v1BrokerIds = v1MetadataResponse.brokers().asScala.map(_.id).toSeq
+    assertTrue("Response should have no errors", v1MetadataResponse.errors.isEmpty)
+    assertFalse(s"The downed broker should not be in the brokers list", v1BrokerIds.contains(downNode))
+    assertEquals("Response should have one topic", 1, v1MetadataResponse.topicMetadata.size)
+    val v1PartitionMetadata = v1MetadataResponse.topicMetadata.asScala.head.partitionMetadata.asScala.head
+    assertEquals("PartitionMetadata should have no errors", Errors.NONE, v1PartitionMetadata.error)
+    assertEquals(s"Response should have $replicaCount replicas", replicaCount, v1PartitionMetadata.replicas.size)
+  }
+
+  private def sendMetadataRequest(request: MetadataRequest, version: Short): MetadataResponse = {
+    val response = send(request, ApiKeys.METADATA, version)
+    MetadataResponse.parse(response, version)
+  }
+}
diff --git a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala
new file mode 100644
index 0000000..632665a
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala
@@ -0,0 +1,78 @@
+/**
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+package kafka.server
+
+import java.io.IOException
+import java.net.Socket
+import java.util.Collections
+import org.apache.kafka.common.protocol.{ApiKeys, SecurityProtocol}
+import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse}
+import org.apache.kafka.common.requests.SaslHandshakeRequest
+import org.apache.kafka.common.requests.SaslHandshakeResponse
+import org.apache.kafka.common.protocol.Errors
+import org.junit.Test
+import org.junit.Assert._
+import kafka.api.SaslTestHarness
+
+class SaslApiVersionsRequestTest extends BaseRequestTest with SaslTestHarness {
+  override protected def securityProtocol = SecurityProtocol.SASL_PLAINTEXT
+  override protected val kafkaClientSaslMechanism = "PLAIN"
+  override protected val kafkaServerSaslMechanisms = List("PLAIN")
+  override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, Some(kafkaServerSaslMechanisms)))
+  override protected val zkSaslEnabled = false
+  override def numBrokers = 1
+
+  @Test
+  def testApiVersionsRequestBeforeSaslHandshakeRequest() {
+    val plaintextSocket = connect(protocol = securityProtocol)
+    try {
+      val apiVersionsResponse = sendApiVersionsRequest(plaintextSocket, new ApiVersionsRequest, 0)
+      ApiVersionsRequestTest.validateApiVersionsResponse(apiVersionsResponse)
+      sendSaslHandshakeRequestValidateResponse(plaintextSocket)
+    } finally {
+      plaintextSocket.close()
+    }
+  }
+
+  @Test
+  def testApiVersionsRequestAfterSaslHandshakeRequest() {
+    val plaintextSocket = connect(protocol = securityProtocol)
+    try {
+      sendSaslHandshakeRequestValidateResponse(plaintextSocket)
+      try {
+        sendApiVersionsRequest(plaintextSocket, new ApiVersionsRequest, 0)
+        fail("Versions Request during Sasl handshake did not fail")
+      } catch {
+        case ioe: IOException => // expected exception
+      }
+    } finally {
+      plaintextSocket.close()
+    }
+  }
+
+  private def sendApiVersionsRequest(socket: Socket, request: ApiVersionsRequest, version: Short): ApiVersionsResponse = {
+    val response = send(socket, request, ApiKeys.API_VERSIONS, version)
+    ApiVersionsResponse.parse(response)
+  }
+
+  private def sendSaslHandshakeRequestValidateResponse(socket: Socket) {
+    val response = send(socket, new SaslHandshakeRequest("PLAIN"), ApiKeys.SASL_HANDSHAKE, 0)
+    val handshakeResponse = SaslHandshakeResponse.parse(response)
+    assertEquals(Errors.NONE.code, handshakeResponse.errorCode())
+    assertEquals(Collections.singletonList("PLAIN"), handshakeResponse.enabledMechanisms())
+  }
+}
diff --git a/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala b/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala
index a14cd3f..7c4b951 100644
--- a/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala
@@ -20,16 +20,14 @@
 
 object JaasTestUtils {
 
-  case class Krb5LoginModule(contextName: String,
-                             useKeyTab: Boolean,
+  case class Krb5LoginModule(useKeyTab: Boolean,
                              storeKey: Boolean,
                              keyTab: String,
                              principal: String,
                              debug: Boolean,
                              serviceName: Option[String]) {
-    def toJaasSection: JaasSection = {
-      JaasSection(
-        contextName,
+    def toJaasModule: JaasModule = {
+      JaasModule(
         "com.sun.security.auth.module.Krb5LoginModule",
         debug = debug,
         entries = Map(
@@ -42,15 +40,38 @@
     }
   }
 
-  case class JaasSection(contextName: String,
-                         moduleName: String,
-                         debug: Boolean,
-                         entries: Map[String, String]) {
+  case class PlainLoginModule(username: String,
+                              password: String,
+                              debug: Boolean = false,
+                              validUsers: Map[String, String] = Map.empty) {
+    def toJaasModule: JaasModule = {
+      JaasModule(
+        "org.apache.kafka.common.security.plain.PlainLoginModule",
+        debug = debug,
+        entries = Map(
+          "username" -> username,
+          "password" -> password
+        ) ++ validUsers.map { case (user, pass) => (s"user_$user"-> pass)}
+      )
+    }
+  }
+
+  case class JaasModule(moduleName: String,
+                        debug: Boolean,
+                        entries: Map[String, String]) {
     override def toString: String = {
-      s"""|$contextName {
-          |  $moduleName required
+      s"""$moduleName required
           |  debug=$debug
           |  ${entries.map { case (k, v) => s"""$k="$v"""" }.mkString("", "\n|  ", ";")}
+          |"""
+    }
+  }
+
+  class JaasSection(contextName: String,
+                    jaasModule: Seq[JaasModule]) {
+    override def toString: String = {
+      s"""|$contextName {
+          |  ${jaasModule.mkString("\n  ")}
           |};
           |""".stripMargin
     }
@@ -67,6 +88,11 @@
   private val KafkaServerPrincipal = "kafka/localhost@EXAMPLE.COM"
   private val KafkaClientContextName = "KafkaClient"
   private val KafkaClientPrincipal = "client@EXAMPLE.COM"
+  
+  private val KafkaPlainUser = "testuser"
+  private val KafkaPlainPassword = "testuser-secret"
+  private val KafkaPlainAdmin = "admin"
+  private val KafkaPlainAdminPassword = "admin-secret"
 
   def writeZkFile(): String = {
     val jaasFile = TestUtils.tempFile()
@@ -74,43 +100,65 @@
     jaasFile.getCanonicalPath
   }
 
-  def writeKafkaFile(serverKeyTabLocation: File, clientKeyTabLocation: File): String = {
+  def writeKafkaFile(kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanisms: List[String], serverKeyTabLocation: Option[File], clientKeyTabLocation: Option[File]): String = {
     val jaasFile = TestUtils.tempFile()
-    writeToFile(jaasFile, kafkaSections(serverKeyTabLocation, clientKeyTabLocation))
+    val kafkaSections = Seq(kafkaServerSection(kafkaServerSaslMechanisms, serverKeyTabLocation), kafkaClientSection(kafkaClientSaslMechanisms, clientKeyTabLocation))
+    writeToFile(jaasFile, kafkaSections)
     jaasFile.getCanonicalPath
   }
 
-  def writeZkAndKafkaFiles(serverKeyTabLocation: File, clientKeyTabLocation: File): String = {
+  def writeZkAndKafkaFiles(kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanisms: List[String], serverKeyTabLocation: Option[File], clientKeyTabLocation: Option[File]): String = {
     val jaasFile = TestUtils.tempFile()
-    writeToFile(jaasFile, kafkaSections(serverKeyTabLocation, clientKeyTabLocation) ++ zkSections)
+    val kafkaSections = Seq(kafkaServerSection(kafkaServerSaslMechanisms, serverKeyTabLocation), kafkaClientSection(kafkaClientSaslMechanisms, clientKeyTabLocation))
+    writeToFile(jaasFile, kafkaSections ++ zkSections)
     jaasFile.getCanonicalPath
   }
 
   private def zkSections: Seq[JaasSection] = Seq(
-    JaasSection(ZkServerContextName, ZkModule, false, Map("user_super" -> ZkUserSuperPasswd, s"user_$ZkUser" -> ZkUserPassword)),
-    JaasSection(ZkClientContextName, ZkModule, false, Map("username" -> ZkUser, "password" -> ZkUserPassword))
+    new JaasSection(ZkServerContextName, Seq(JaasModule(ZkModule, false, Map("user_super" -> ZkUserSuperPasswd, s"user_$ZkUser" -> ZkUserPassword)))),
+    new JaasSection(ZkClientContextName, Seq(JaasModule(ZkModule, false, Map("username" -> ZkUser, "password" -> ZkUserPassword))))
   )
 
-  private def kafkaSections(serverKeytabLocation: File, clientKeytabLocation: File): Seq[JaasSection] = {
-    Seq(
-      Krb5LoginModule(
-        KafkaServerContextName,
-        useKeyTab = true,
-        storeKey = true,
-        keyTab = serverKeytabLocation.getAbsolutePath,
-        principal = KafkaServerPrincipal,
-        debug = true,
-        serviceName = Some("kafka")),
-      Krb5LoginModule(
-        KafkaClientContextName,
-        useKeyTab = true,
-        storeKey = true,
-        keyTab = clientKeytabLocation.getAbsolutePath,
-        principal = KafkaClientPrincipal,
-        debug = true,
-        serviceName = Some("kafka")
-      )
-    ).map(_.toJaasSection)
+  private def kafkaServerSection(mechanisms: List[String], keytabLocation: Option[File]): JaasSection = {
+    val modules = mechanisms.map {
+      case "GSSAPI" =>
+        Krb5LoginModule(
+          useKeyTab = true,
+          storeKey = true,
+          keyTab = keytabLocation.getOrElse(throw new IllegalArgumentException("Keytab location not specified for GSSAPI")).getAbsolutePath,
+          principal = KafkaServerPrincipal,
+          debug = true,
+          serviceName = Some("kafka")).toJaasModule
+      case "PLAIN" =>
+        PlainLoginModule(
+          KafkaPlainAdmin,
+          KafkaPlainAdminPassword,
+          debug = false,
+          Map(KafkaPlainAdmin -> KafkaPlainAdminPassword, KafkaPlainUser -> KafkaPlainPassword)).toJaasModule
+      case mechanism => throw new IllegalArgumentException("Unsupported server mechanism " + mechanism)
+    }
+    new JaasSection(KafkaServerContextName, modules)
+  }
+
+  private def kafkaClientSection(mechanisms: List[String], keytabLocation: Option[File]): JaasSection = {
+    val modules = mechanisms.map {
+      case "GSSAPI" =>
+        Krb5LoginModule(
+          useKeyTab = true,
+          storeKey = true,
+          keyTab = keytabLocation.getOrElse(throw new IllegalArgumentException("Keytab location not specified for GSSAPI")).getAbsolutePath,
+          principal = KafkaClientPrincipal,
+          debug = true,
+          serviceName = Some("kafka")
+        ).toJaasModule
+      case "PLAIN" =>
+        PlainLoginModule(
+          KafkaPlainUser,
+          KafkaPlainPassword
+        ).toJaasModule
+      case mechanism => throw new IllegalArgumentException("Unsupported client mechanism " + mechanism)
+    }
+    new JaasSection(KafkaClientContextName, modules)
   }
 
   private def jaasSectionsToString(jaasSections: Seq[JaasSection]): String =
diff --git a/core/src/test/scala/unit/kafka/utils/MockTime.scala b/core/src/test/scala/unit/kafka/utils/MockTime.scala
index ee65748..0858e04 100644
--- a/core/src/test/scala/unit/kafka/utils/MockTime.scala
+++ b/core/src/test/scala/unit/kafka/utils/MockTime.scala
@@ -19,6 +19,8 @@
 
 import java.util.concurrent._
 
+import org.apache.kafka.common.utils
+
 /**
  * A class used for unit testing things which depend on the Time interface.
  * 
@@ -47,3 +49,13 @@
   override def toString() = "MockTime(%d)".format(milliseconds)
 
 }
+
+object MockTime {
+  implicit def toCommonTime(time: MockTime): utils.Time = new utils.Time {
+    override def nanoseconds(): Long = time.nanoseconds
+
+    override def milliseconds(): Long = time.milliseconds
+
+    override def sleep(ms: Long): Unit = time.sleep(ms)
+  }
+}
\ No newline at end of file
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index be7741d..7df87fc 100755
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -26,12 +26,10 @@
 import java.security.cert.X509Certificate
 import javax.net.ssl.X509TrustManager
 import charset.Charset
-
 import kafka.security.auth.{Acl, Authorizer, Resource}
 import org.apache.kafka.common.protocol.SecurityProtocol
 import org.apache.kafka.common.utils.Utils._
 import org.apache.kafka.test.TestSslUtils
-
 import scala.collection.mutable.{ArrayBuffer, ListBuffer}
 import kafka.server._
 import kafka.producer._
@@ -95,15 +93,8 @@
   def tempRelativeDir(parent: String): File = {
     val parentFile = new File(parent)
     parentFile.mkdirs()
-    val f = Files.createTempDirectory(parentFile.toPath, "kafka-").toFile
-    f.deleteOnExit()
 
-    Runtime.getRuntime().addShutdownHook(new Thread() {
-      override def run() = {
-        Utils.delete(f)
-      }
-    })
-    f
+    org.apache.kafka.test.TestUtils.tempDirectory(parentFile.toPath, "kafka-");
   }
 
   /**
@@ -153,6 +144,7 @@
     enableDeleteTopic: Boolean = false,
     interBrokerSecurityProtocol: Option[SecurityProtocol] = None,
     trustStoreFile: Option[File] = None,
+    saslProperties: Option[Properties] = None,
     enablePlaintext: Boolean = true,
     enableSsl: Boolean = false,
     enableSaslPlaintext: Boolean = false,
@@ -160,7 +152,7 @@
     rackInfo: Map[Int, String] = Map()): Seq[Properties] = {
     (0 until numConfigs).map { node =>
       createBrokerConfig(node, zkConnect, enableControlledShutdown, enableDeleteTopic, RandomPort,
-        interBrokerSecurityProtocol, trustStoreFile, enablePlaintext = enablePlaintext, enableSsl = enableSsl,
+        interBrokerSecurityProtocol, trustStoreFile, saslProperties, enablePlaintext = enablePlaintext, enableSsl = enableSsl,
         enableSaslPlaintext = enableSaslPlaintext, enableSaslSsl = enableSaslSsl, rack = rackInfo.get(node))
     }
   }
@@ -180,6 +172,7 @@
     port: Int = RandomPort,
     interBrokerSecurityProtocol: Option[SecurityProtocol] = None,
     trustStoreFile: Option[File] = None,
+    saslProperties: Option[Properties] = None,
     enablePlaintext: Boolean = true,
     enableSaslPlaintext: Boolean = false, saslPlaintextPort: Int = RandomPort,
     enableSsl: Boolean = false, sslPort: Int = RandomPort,
@@ -218,6 +211,9 @@
     if (protocolAndPorts.exists { case (protocol, _) => usesSslTransportLayer(protocol) })
       props.putAll(sslConfigs(Mode.SERVER, false, trustStoreFile, s"server$nodeId"))
 
+    if (protocolAndPorts.exists { case (protocol, _) => usesSaslTransportLayer(protocol) })
+      props.putAll(saslConfigs(saslProperties))
+
     interBrokerSecurityProtocol.foreach { protocol =>
       props.put(KafkaConfig.InterBrokerSecurityProtocolProp, protocol.name)
     }
@@ -447,16 +443,19 @@
   private def securityConfigs(mode: Mode,
                               securityProtocol: SecurityProtocol,
                               trustStoreFile: Option[File],
-                              certAlias: String): Properties = {
+                              certAlias: String,
+                              saslProperties: Option[Properties]): Properties = {
     val props = new Properties
     if (usesSslTransportLayer(securityProtocol))
       props.putAll(sslConfigs(mode, securityProtocol == SecurityProtocol.SSL, trustStoreFile, certAlias))
+    if (usesSaslTransportLayer(securityProtocol))
+      props.putAll(saslConfigs(saslProperties))
     props.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, securityProtocol.name)
     props
   }
 
-  def producerSecurityConfigs(securityProtocol: SecurityProtocol, trustStoreFile: Option[File]): Properties =
-    securityConfigs(Mode.CLIENT, securityProtocol, trustStoreFile, "producer")
+  def producerSecurityConfigs(securityProtocol: SecurityProtocol, trustStoreFile: Option[File], saslProperties: Option[Properties]): Properties =
+    securityConfigs(Mode.CLIENT, securityProtocol, trustStoreFile, "producer", saslProperties)
 
   /**
    * Create a (new) producer with a few pre-configured properties.
@@ -470,6 +469,7 @@
                         requestTimeoutMs: Long = 10 * 1024L,
                         securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT,
                         trustStoreFile: Option[File] = None,
+                        saslProperties: Option[Properties] = None,
                         keySerializer: Serializer[K] = new ByteArraySerializer,
                         valueSerializer: Serializer[V] = new ByteArraySerializer,
                         props: Option[Properties] = None): KafkaProducer[K, V] = {
@@ -500,7 +500,7 @@
      * SSL client auth fails.
      */
     if (!producerProps.containsKey(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG))
-      producerProps.putAll(producerSecurityConfigs(securityProtocol, trustStoreFile))
+      producerProps.putAll(producerSecurityConfigs(securityProtocol, trustStoreFile, saslProperties))
 
     new KafkaProducer[K, V](producerProps, keySerializer, valueSerializer)
   }
@@ -510,8 +510,13 @@
     case _ => false
   }
 
-  def consumerSecurityConfigs(securityProtocol: SecurityProtocol, trustStoreFile: Option[File]): Properties =
-    securityConfigs(Mode.CLIENT, securityProtocol, trustStoreFile, "consumer")
+  private def usesSaslTransportLayer(securityProtocol: SecurityProtocol): Boolean = securityProtocol match {
+    case SecurityProtocol.SASL_PLAINTEXT | SecurityProtocol.SASL_SSL => true
+    case _ => false
+  }
+
+  def consumerSecurityConfigs(securityProtocol: SecurityProtocol, trustStoreFile: Option[File], saslProperties: Option[Properties]): Properties =
+    securityConfigs(Mode.CLIENT, securityProtocol, trustStoreFile, "consumer", saslProperties)
 
   /**
    * Create a new consumer with a few pre-configured properties.
@@ -524,6 +529,7 @@
                         sessionTimeout: Int = 30000,
                         securityProtocol: SecurityProtocol,
                         trustStoreFile: Option[File] = None,
+                        saslProperties: Option[Properties] = None,
                         props: Option[Properties] = None) : KafkaConsumer[Array[Byte],Array[Byte]] = {
     import org.apache.kafka.clients.consumer.ConsumerConfig
 
@@ -552,7 +558,7 @@
      * SSL client auth fails.
      */
     if(!consumerProps.containsKey(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG))
-      consumerProps.putAll(consumerSecurityConfigs(securityProtocol, trustStoreFile))
+      consumerProps.putAll(consumerSecurityConfigs(securityProtocol, trustStoreFile, saslProperties))
 
     new KafkaConsumer[Array[Byte],Array[Byte]](consumerProps)
   }
@@ -1065,6 +1071,13 @@
     sslProps
   }
 
+  def saslConfigs(saslProperties: Option[Properties]): Properties = {
+    saslProperties match {
+      case Some(properties) => properties
+      case None => new Properties
+    }
+  }
+
   // a X509TrustManager to trust self-signed certs for unit tests.
   def trustAllCerts: X509TrustManager = {
     val trustManager = new X509TrustManager() {
diff --git a/core/src/test/scala/unit/kafka/utils/timer/MockTimer.scala b/core/src/test/scala/unit/kafka/utils/timer/MockTimer.scala
new file mode 100644
index 0000000..d18a060
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/utils/timer/MockTimer.scala
@@ -0,0 +1,57 @@
+/**
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+package kafka.utils.timer
+
+import kafka.utils.MockTime
+
+import scala.collection.mutable
+
+class MockTimer extends Timer {
+
+  val time = new MockTime
+  private val taskQueue = mutable.PriorityQueue[TimerTaskEntry]()
+
+  def add(timerTask: TimerTask) {
+    if (timerTask.delayMs <= 0)
+      timerTask.run()
+    else
+      taskQueue.enqueue(new TimerTaskEntry(timerTask, timerTask.delayMs + time.milliseconds))
+  }
+
+  def advanceClock(timeoutMs: Long): Boolean = {
+    time.sleep(timeoutMs)
+
+    var executed = false
+    val now = time.milliseconds
+
+    while (taskQueue.nonEmpty && now > taskQueue.head.expirationMs) {
+      val taskEntry = taskQueue.dequeue()
+      if (!taskEntry.cancelled) {
+        val task = taskEntry.timerTask
+        task.run()
+        executed = true
+      }
+    }
+
+    executed
+  }
+
+  def size: Int = taskQueue.size
+
+  override def shutdown(): Unit = {}
+
+}
diff --git a/core/src/test/scala/unit/kafka/utils/timer/TimerTaskListTest.scala b/core/src/test/scala/unit/kafka/utils/timer/TimerTaskListTest.scala
index a018dde..29c9067 100644
--- a/core/src/test/scala/unit/kafka/utils/timer/TimerTaskListTest.scala
+++ b/core/src/test/scala/unit/kafka/utils/timer/TimerTaskListTest.scala
@@ -18,11 +18,11 @@
 
 import org.junit.Assert._
 import java.util.concurrent.atomic._
-import org.junit.{Test, After, Before}
+import org.junit.Test
 
 class TimerTaskListTest {
 
-  private class TestTask(val expirationMs: Long) extends TimerTask {
+  private class TestTask(val delayMs: Long) extends TimerTask {
     def run(): Unit = { }
   }
 
@@ -42,8 +42,8 @@
     val list3 = new TimerTaskList(sharedCounter)
 
     val tasks = (1 to 10).map { i =>
-      val task = new TestTask(10L)
-      list1.add(new TimerTaskEntry(task))
+      val task = new TestTask(0L)
+      list1.add(new TimerTaskEntry(task, 10L))
       assertEquals(i, sharedCounter.get)
       task
     }.toSeq
@@ -54,7 +54,7 @@
     tasks.take(4).foreach { task =>
       val prevCount = sharedCounter.get
       // new TimerTaskEntry(task) will remove the existing entry from the list
-      list2.add(new TimerTaskEntry(task))
+      list2.add(new TimerTaskEntry(task, 10L))
       assertEquals(prevCount, sharedCounter.get)
     }
     assertEquals(10 - 4, size(list1))
@@ -66,7 +66,7 @@
     tasks.drop(4).foreach { task =>
       val prevCount = sharedCounter.get
       // new TimerTaskEntry(task) will remove the existing entry from the list
-      list3.add(new TimerTaskEntry(task))
+      list3.add(new TimerTaskEntry(task, 10L))
       assertEquals(prevCount, sharedCounter.get)
     }
     assertEquals(0, size(list1))
diff --git a/core/src/test/scala/unit/kafka/utils/timer/TimerTest.scala b/core/src/test/scala/unit/kafka/utils/timer/TimerTest.scala
index 95de378..54b73b8 100644
--- a/core/src/test/scala/unit/kafka/utils/timer/TimerTest.scala
+++ b/core/src/test/scala/unit/kafka/utils/timer/TimerTest.scala
@@ -27,7 +27,7 @@
 
 class TimerTest {
 
-  private class TestTask(override val expirationMs: Long, id: Int, latch: CountDownLatch, output: ArrayBuffer[Int]) extends TimerTask {
+  private class TestTask(override val delayMs: Long, id: Int, latch: CountDownLatch, output: ArrayBuffer[Int]) extends TimerTask {
     private[this] val completed = new AtomicBoolean(false)
     def run(): Unit = {
       if (completed.compareAndSet(false, true)) {
@@ -37,32 +37,31 @@
     }
   }
 
-  private[this] var executor: ExecutorService = null
+  private[this] var timer: Timer = null
 
   @Before
   def setup() {
-    executor = Executors.newSingleThreadExecutor()
+    timer = new SystemTimer("test", tickMs = 1, wheelSize = 3)
   }
 
   @After
   def teardown(): Unit = {
-    executor.shutdown()
-    executor = null
+    timer.shutdown()
   }
 
   @Test
   def testAlreadyExpiredTask(): Unit = {
-    val startTime = System.currentTimeMillis()
-    val timer = new Timer(taskExecutor = executor, tickMs = 1, wheelSize = 3, startMs = startTime)
     val output = new ArrayBuffer[Int]()
 
 
     val latches = (-5 until 0).map { i =>
       val latch = new CountDownLatch(1)
-      timer.add(new TestTask(startTime + i, i, latch, output))
+      timer.add(new TestTask(i, i, latch, output))
       latch
     }
 
+    timer.advanceClock(0)
+
     latches.take(5).foreach { latch =>
       assertEquals("already expired tasks should run immediately", true, latch.await(3, TimeUnit.SECONDS))
     }
@@ -72,8 +71,6 @@
 
   @Test
   def testTaskExpiration(): Unit = {
-    val startTime = System.currentTimeMillis()
-    val timer = new Timer(taskExecutor = executor, tickMs = 1, wheelSize = 3, startMs = startTime)
     val output = new ArrayBuffer[Int]()
 
     val tasks = new ArrayBuffer[TestTask]()
@@ -82,27 +79,27 @@
     val latches =
       (0 until 5).map { i =>
         val latch = new CountDownLatch(1)
-        tasks += new TestTask(startTime + i, i, latch, output)
+        tasks += new TestTask(i, i, latch, output)
         ids += i
         latch
       } ++ (10 until 100).map { i =>
         val latch = new CountDownLatch(2)
-        tasks += new TestTask(startTime + i, i, latch, output)
-        tasks += new TestTask(startTime + i, i, latch, output)
+        tasks += new TestTask(i, i, latch, output)
+        tasks += new TestTask(i, i, latch, output)
         ids += i
         ids += i
         latch
       } ++ (100 until 500).map { i =>
         val latch = new CountDownLatch(1)
-        tasks += new TestTask(startTime + i, i, latch, output)
+        tasks += new TestTask(i, i, latch, output)
         ids += i
         latch
       }
 
     // randomly submit requests
-    Random.shuffle(tasks.toSeq).foreach { task => timer.add(task) }
+    tasks.foreach { task => timer.add(task) }
 
-    while (timer.advanceClock(1000)) {}
+    while (timer.advanceClock(2000)) {}
 
     latches.foreach { latch => latch.await() }
 
diff --git a/docs/api.html b/docs/api.html
index d303244..8d5be9b 100644
--- a/docs/api.html
+++ b/docs/api.html
@@ -15,7 +15,7 @@
  limitations under the License.
 -->
 
-Apache Kafka includes new java clients (in the org.apache.kafka.clients package). These are meant to supplant the older Scala clients, but for compatability they will co-exist for some time. These clients are available in a separate jar with minimal dependencies, while the old Scala clients remain packaged with the server.
+Apache Kafka includes new java clients (in the org.apache.kafka.clients package). These are meant to supplant the older Scala clients, but for compatibility they will co-exist for some time. These clients are available in a separate jar with minimal dependencies, while the old Scala clients remain packaged with the server.
 
 <h3><a id="producerapi" href="#producerapi">2.1 Producer API</a></h3>
 
diff --git a/docs/configuration.html b/docs/configuration.html
index e5280a5..f9bd1e4 100644
--- a/docs/configuration.html
+++ b/docs/configuration.html
@@ -207,7 +207,7 @@
     <tr>
       <td>fetch.message.max.bytes</td>
       <td nowrap>1024 * 1024</td>
-      <td>The number of byes of messages to attempt to fetch for each topic-partition in each fetch request. These bytes will be read into memory for each partition, so this helps control the memory used by the consumer. The fetch request size must be at least as large as the maximum message size the server allows or else it is possible for the producer to send messages larger than the consumer can fetch.</td>
+      <td>The number of bytes of messages to attempt to fetch for each topic-partition in each fetch request. These bytes will be read into memory for each partition, so this helps control the memory used by the consumer. The fetch request size must be at least as large as the maximum message size the server allows or else it is possible for the producer to send messages larger than the consumer can fetch.</td>
     </tr>
      <tr>
       <td>num.consumer.fetchers</td>
diff --git a/docs/connect.html b/docs/connect.html
index dc6ad6e..5cd4130 100644
--- a/docs/connect.html
+++ b/docs/connect.html
@@ -25,7 +25,7 @@
     <li><b>Distributed and standalone modes</b> - scale up to a large, centrally managed service supporting an entire organization or scale down to development, testing, and small production deployments</li>
     <li><b>REST interface</b> - submit and manage connectors to your Kafka Connect cluster via an easy to use REST API</li>
     <li><b>Automatic offset management</b> - with just a little information from connectors, Kafka Connect can manage the offset commit process automatically so connector developers do not need to worry about this error prone part of connector development</li>
-    <li><b>Distributed and scalable by default</b> - Kafka Connect builds on the existing </li>
+    <li><b>Distributed and scalable by default</b> - Kafka Connect builds on the existing group management protocol. More workers can be added to scale up a Kafka Connect cluster.</li>
     <li><b>Streaming/batch integration</b> - leveraging Kafka's existing capabilities, Kafka Connect is an ideal solution for bridging streaming and batch data systems</li>
 </ul>
 
@@ -76,6 +76,8 @@
     <li><code>tasks.max</code> - The maximum number of tasks that should be created for this connector. The connector may create fewer tasks if it cannot achieve this level of parallelism.</li>
 </ul>
 
+The <code>connector.class</code> config supports several formats: the full name or alias of the class for this connector. If the connector is org.apache.kafka.connect.file.FileStreamSinkConnector, you can either specify this full name or use FileStreamSink or FileStreamSinkConnector to make the configuration a bit shorter.
+
 Sink connectors also have one additional option to control their input:
 <ul>
     <li><code>topics</code> - A list of topics to use as input for this connector</li>
@@ -83,10 +85,9 @@
 
 For any other options, you should consult the documentation for the connector.
 
-
 <h4><a id="connect_rest" href="#connect_rest">REST API</a></h4>
 
-Since Kafka Connect is intended to be run as a service, it also supports a REST API for managing connectors. By default this service runs on port 8083. The following are the currently supported endpoints:
+Since Kafka Connect is intended to be run as a service, it also provides a REST API for managing connectors. By default this service runs on port 8083. The following are the currently supported endpoints:
 
 <ul>
     <li><code>GET /connectors</code> - return a list of active connectors</li>
@@ -98,6 +99,13 @@
     <li><code>DELETE /connectors/{name}</code> - delete a connector, halting all tasks and deleting its configuration</li>
 </ul>
 
+Kafka Connect also provides a REST API for getting information about connector plugins:
+
+<ul>
+    <li><code>GET /connector-plugins</code>- return a list of connector plugins installed in the Kafka Connect cluster. Note that the API only checks for connectors on the worker that handles the request, which means you may see inconsistent results, especially during a rolling upgrade if you add new connector jars</li>
+    <li><code>PUT /connector-plugins/{connector-type}/config/validate</code> - validate the provided configuration values against the configuration definition. This API performs per config validation, returns suggested values and error messages during validation.</li>
+</ul>
+
 <h3><a id="connect_development" href="#connect_development">8.3 Connector Development Guide</a></h3>
 
 This guide describes how developers can write new connectors for Kafka Connect to move data between Kafka and other systems. It briefly reviews a few key concepts and then describes how to create a simple connector.
@@ -108,7 +116,7 @@
 
 To copy data between Kafka and another system, users create a <code>Connector</code> for the system they want to pull data from or push data to. Connectors come in two flavors: <code>SourceConnectors</code> import data from another system (e.g. <code>JDBCSourceConnector</code> would import a relational database into Kafka) and <code>SinkConnectors</code> export data (e.g. <code>HDFSSinkConnector</code> would export the contents of a Kafka topic to an HDFS file).
 
-<code>Connectors</code> do not perform any data copying themselves: their configuration describes the data to be copied, and the <code>Connector</code> is responsible for breaking that job into a set of <code>Tasks</code> that can be distributed to workers. These <code>Tasks</code> also come in two corresponding flavors: <code>SourceTask</code>and <code>SinkTask</code>.
+<code>Connectors</code> do not perform any data copying themselves: their configuration describes the data to be copied, and the <code>Connector</code> is responsible for breaking that job into a set of <code>Tasks</code> that can be distributed to workers. These <code>Tasks</code> also come in two corresponding flavors: <code>SourceTask</code> and <code>SinkTask</code>.
 
 With an assignment in hand, each <code>Task</code> must copy its subset of the data to or from Kafka. In Kafka Connect, it should always be possible to frame these assignments as a set of input and output streams consisting of records with consistent schemas. Sometimes this mapping is obvious: each file in a set of log files can be considered a stream with each parsed line forming a record using the same schema and offsets stored as byte offsets in the file. In other cases it may require more effort to map to this model: a JDBC connector can map each table to a stream, but the offset is less clear. One possible mapping uses a timestamp column to generate queries incrementally returning new data, and the last queried timestamp can be used as the offset.
 
@@ -183,6 +191,9 @@
 }
 </pre>
 
+Although not used in the example, <code>SourceTask</code> also provides two APIs to commit offsets in the source system: <code>commit</code> and <code>commitSourceRecord</code>. The APIs are provided for source systems which have an acknowledgement mechanism for messages. Overriding these methods allows the source connector to acknowledge messages in the source system, either in bulk or individually, once they have been written to Kafka.
+The <code>commit<code> API stores the offsets in the source system, up to the offsets that have been returned by <code>poll</code>. The implementation of this API should block until the commit is complete. The <code>commitSourceRecord</code> API saves the offset in the source system for each <code>SourceRecord</code> after it is written to Kafka. As Kafka Connect will record offsets automatically, <code>SourceTask<code>s are not required to implement them. In cases where a connector does need to acknowledge messages in the source system, only one of the APIs is typically required.
+
 Even with multiple tasks, this method implementation is usually pretty simple. It just has to determine the number of input tasks, which may require contacting the remote service it is pulling data from, and then divvy them up. Because some patterns for splitting work among tasks are so common, some utilities are provided in <code>ConnectorUtils</code> to simplify these cases.
 
 Note that this simple example does not include dynamic input. See the discussion in the next section for how to trigger updates to task configs.
@@ -242,11 +253,11 @@
 
 Again, we've omitted some details, but we can see the important steps: the <code>poll()</code> method is going to be called repeatedly, and for each call it will loop trying to read records from the file. For each line it reads, it also tracks the file offset. It uses this information to create an output <code>SourceRecord</code> with four pieces of information: the source partition (there is only one, the single file being read), source offset (byte offset in the file), output topic name, and output value (the line, and we include a schema indicating this value will always be a string). Other variants of the <code>SourceRecord</code> constructor can also include a specific output partition and a key.
 
-Note that this implementation uses the normal Java <code>InputStream</code>interface and may sleep if data is not available. This is acceptable because Kafka Connect provides each task with a dedicated thread. While task implementations have to conform to the basic <code>poll()</code>interface, they have a lot of flexibility in how they are implemented. In this case, an NIO-based implementation would be more efficient, but this simple approach works, is quick to implement, and is compatible with older versions of Java.
+Note that this implementation uses the normal Java <code>InputStream</code> interface and may sleep if data is not available. This is acceptable because Kafka Connect provides each task with a dedicated thread. While task implementations have to conform to the basic <code>poll()</code> interface, they have a lot of flexibility in how they are implemented. In this case, an NIO-based implementation would be more efficient, but this simple approach works, is quick to implement, and is compatible with older versions of Java.
 
 <h5><a id="connect_sinktasks" href="#connect_sinktasks">Sink Tasks</a></h5>
 
-The previous section described how to implement a simple <code>SourceTask</code>. Unlike <code>SourceConnector</code>and <code>SinkConnector</code>, <code>SourceTask</code>and <code>SinkTask</code>have very different interfaces because <code>SourceTask</code>uses a pull interface and <code>SinkTask</code>uses a push interface. Both share the common lifecycle methods, but the <code>SinkTask</code>interface is quite different:
+The previous section described how to implement a simple <code>SourceTask</code>. Unlike <code>SourceConnector</code> and <code>SinkConnector</code>, <code>SourceTask</code> and <code>SinkTask</code> have very different interfaces because <code>SourceTask</code> uses a pull interface and <code>SinkTask</code> uses a push interface. Both share the common lifecycle methods, but the <code>SinkTask</code> interface is quite different:
 
 <pre>
 public abstract class SinkTask implements Task {
@@ -257,17 +268,17 @@
 public abstract void flush(Map&lt;TopicPartition, Long&gt; offsets);
 </pre>
 
-The <code>SinkTask</code> documentation contains full details, but this interface is nearly as simple as the the <code>SourceTask</code>. The <code>put()</code>method should contain most of the implementation, accepting sets of <code>SinkRecords</code>, performing any required translation, and storing them in the destination system. This method does not need to ensure the data has been fully written to the destination system before returning. In fact, in many cases internal buffering will be useful so an entire batch of records can be sent at once, reducing the overhead of inserting events into the downstream data store. The <code>SinkRecords</code>contain essentially the same information as <code>SourceRecords</code>: Kafka topic, partition, offset and the event key and value.
+The <code>SinkTask</code> documentation contains full details, but this interface is nearly as simple as the <code>SourceTask</code>. The <code>put()</code> method should contain most of the implementation, accepting sets of <code>SinkRecords</code>, performing any required translation, and storing them in the destination system. This method does not need to ensure the data has been fully written to the destination system before returning. In fact, in many cases internal buffering will be useful so an entire batch of records can be sent at once, reducing the overhead of inserting events into the downstream data store. The <code>SinkRecords</code> contain essentially the same information as <code>SourceRecords</code>: Kafka topic, partition, offset and the event key and value.
 
-The <code>flush()</code>method is used during the offset commit process, which allows tasks to recover from failures and resume from a safe point such that no events will be missed. The method should push any outstanding data to the destination system and then block until the write has been acknowledged. The <code>offsets</code>parameter can often be ignored, but is useful in some cases where implementations want to store offset information in the destination store to provide exactly-once
-delivery. For example, an HDFS connector could do this and use atomic move operations to make sure the <code>flush()</code>operation atomically commits the data and offsets to a final location in HDFS.
+The <code>flush()</code> method is used during the offset commit process, which allows tasks to recover from failures and resume from a safe point such that no events will be missed. The method should push any outstanding data to the destination system and then block until the write has been acknowledged. The <code>offsets</code> parameter can often be ignored, but is useful in some cases where implementations want to store offset information in the destination store to provide exactly-once
+delivery. For example, an HDFS connector could do this and use atomic move operations to make sure the <code>flush()</code> operation atomically commits the data and offsets to a final location in HDFS.
 
 
 <h5><a id="connect_resuming" href="#connect_resuming">Resuming from Previous Offsets</a></h5>
 
-The <code>SourceTask</code>implementation included a stream ID (the input filename) and offset (position in the file) with each record. The framework uses this to commit offsets periodically so that in the case of a failure, the task can recover and minimize the number of events that are reprocessed and possibly duplicated (or to resume from the most recent offset if Kafka Connect was stopped gracefully, e.g. in standalone mode or due to a job reconfiguration). This commit process is completely automated by the framework, but only the connector knows how to seek back to the right position in the input stream to resume from that location.
+The <code>SourceTask</code> implementation included a stream ID (the input filename) and offset (position in the file) with each record. The framework uses this to commit offsets periodically so that in the case of a failure, the task can recover and minimize the number of events that are reprocessed and possibly duplicated (or to resume from the most recent offset if Kafka Connect was stopped gracefully, e.g. in standalone mode or due to a job reconfiguration). This commit process is completely automated by the framework, but only the connector knows how to seek back to the right position in the input stream to resume from that location.
 
-To correctly resume upon startup, the task can use the <code>SourceContext</code>passed into its <code>initialize()</code>method to access the offset data. In <code>initialize()</code>, we would add a bit more code to read the offset (if it exists) and seek to that position:
+To correctly resume upon startup, the task can use the <code>SourceContext</code> passed into its <code>initialize()</code> method to access the offset data. In <code>initialize()</code>, we would add a bit more code to read the offset (if it exists) and seek to that position:
 
 <pre>
     stream = new FileInputStream(filename);
@@ -285,19 +296,18 @@
 
 Kafka Connect is intended to define bulk data copying jobs, such as copying an entire database rather than creating many jobs to copy each table individually. One consequence of this design is that the set of input or output streams for a connector can vary over time.
 
-Source connectors need to monitor the source system for changes, e.g. table additions/deletions in a database. When they pick up changes, they should notify the framework via the <code>ConnectorContext</code>object that reconfiguration is necessary. For example, in a <code>SourceConnector</code>:
-
+Source connectors need to monitor the source system for changes, e.g. table additions/deletions in a database. When they pick up changes, they should notify the framework via the <code>ConnectorContext</code> object that reconfiguration is necessary. For example, in a <code>SourceConnector</code>:
 
 <pre>
 if (inputsChanged())
     this.context.requestTaskReconfiguration();
 </pre>
 
-The framework will promptly request new configuration information and update the tasks, allowing them to gracefully commit their progress before reconfiguring them. Note that in the <code>SourceConnector</code>this monitoring is currently left up to the connector implementation. If an extra thread is required to perform this monitoring, the connector must allocate it itself.
+The framework will promptly request new configuration information and update the tasks, allowing them to gracefully commit their progress before reconfiguring them. Note that in the <code>SourceConnector</code> this monitoring is currently left up to the connector implementation. If an extra thread is required to perform this monitoring, the connector must allocate it itself.
 
-Ideally this code for monitoring changes would be isolated to the <code>Connector</code>and tasks would not need to worry about them. However, changes can also affect tasks, most commonly when one of their input streams is destroyed in the input system, e.g. if a table is dropped from a database. If the <code>Task</code>encounters the issue before the <code>Connector</code>, which will be common if the <code>Connector</code>needs to poll for changes, the <code>Task</code>will need to handle the subsequent error. Thankfully, this can usually be handled simply by catching and handling the appropriate exception.
+Ideally this code for monitoring changes would be isolated to the <code>Connector</code> and tasks would not need to worry about them. However, changes can also affect tasks, most commonly when one of their input streams is destroyed in the input system, e.g. if a table is dropped from a database. If the <code>Task</code> encounters the issue before the <code>Connector</code>, which will be common if the <code>Connector</code> needs to poll for changes, the <code>Task</code> will need to handle the subsequent error. Thankfully, this can usually be handled simply by catching and handling the appropriate exception.
 
-<code>SinkConnectors</code> usually only have to handle the addition of streams, which may translate to new entries in their outputs (e.g., a new database table). The framework manages any changes to the Kafka input, such as when the set of input topics changes because of a regex subscription. <code>SinkTasks</code>should expect new input streams, which may require creating new resources in the downstream system, such as a new table in a database. The trickiest situation to handle in these cases may be conflicts between multiple <code>SinkTasks</code>seeing a new input stream for the first time and simultaneoulsy trying to create the new resource. <code>SinkConnectors</code>, on the other hand, will generally require no special code for handling a dynamic set of streams.
+<code>SinkConnectors</code> usually only have to handle the addition of streams, which may translate to new entries in their outputs (e.g., a new database table). The framework manages any changes to the Kafka input, such as when the set of input topics changes because of a regex subscription. <code>SinkTasks</code> should expect new input streams, which may require creating new resources in the downstream system, such as a new table in a database. The trickiest situation to handle in these cases may be conflicts between multiple <code>SinkTasks</code> seeing a new input stream for the first time and simultaneously trying to create the new resource. <code>SinkConnectors</code>, on the other hand, will generally require no special code for handling a dynamic set of streams.
 
 <h4><a id="connect_schemas" href="#connect_schemas">Working with Schemas</a></h4>
 
@@ -305,24 +315,24 @@
 
 To create more complex data, you'll need to work with the Kafka Connect <code>data</code> API. Most structured records will need to interact with two classes in addition to primitive types: <code>Schema</code> and <code>Struct</code>.
 
-The API documentation provides a complete reference, but here is a simple example creating a <code>Schema</code>and <code>Struct</code>:
+The API documentation provides a complete reference, but here is a simple example creating a <code>Schema</code> and <code>Struct</code>:
 
 <pre>
 Schema schema = SchemaBuilder.struct().name(NAME)
-                    .field("name", Schema.STRING_SCHEMA)
-                    .field("age", Schema.INT_SCHEMA)
-                    .field("admin", new SchemaBuilder.boolean().defaultValue(false).build())
-                    .build();
+    .field("name", Schema.STRING_SCHEMA)
+    .field("age", Schema.INT_SCHEMA)
+    .field("admin", new SchemaBuilder.boolean().defaultValue(false).build())
+    .build();
 
 Struct struct = new Struct(schema)
-                           .put("name", "Barbara Liskov")
-                           .put("age", 75)
-                           .build();
+    .put("name", "Barbara Liskov")
+    .put("age", 75)
+    .build();
 </pre>
 
 If you are implementing a source connector, you'll need to decide when and how to create schemas. Where possible, you should avoid recomputing them as much as possible. For example, if your connector is guaranteed to have a fixed schema, create it statically and reuse a single instance.
 
-However, many connectors will have dynamic schemas. One simple example of this is a database connector. Considering even just a single table, the schema will not be predefined for the entire connector (as it varies from table to table). But it also may not be fixed for a single table over the lifetime of the connector since the user may execute an <code>ALTER TABLE</code>command. The connector must be able to detect these changes and react appropriately.
+However, many connectors will have dynamic schemas. One simple example of this is a database connector. Considering even just a single table, the schema will not be predefined for the entire connector (as it varies from table to table). But it also may not be fixed for a single table over the lifetime of the connector since the user may execute an <code>ALTER TABLE</code> command. The connector must be able to detect these changes and react appropriately.
 
 Sink connectors are usually simpler because they are consuming data and therefore do not need to create schemas. However, they should take just as much care to validate that the schemas they receive have the expected format. When the schema does not match -- usually indicating the upstream producer is generating invalid data that cannot be correctly translated to the destination system -- sink connectors should throw an exception to indicate this error to the system.
 
diff --git a/docs/implementation.html b/docs/implementation.html
index ecd99e7..be81227 100644
--- a/docs/implementation.html
+++ b/docs/implementation.html
@@ -90,7 +90,7 @@
    * Get a list of valid offsets (up to maxSize) before the given time.
    * The result is a list of offsets, in descending order.
    * @param time: time in millisecs,
-   *              if set to OffsetRequest$.MODULE$.LATIEST_TIME(), get from the latest offset available.
+   *              if set to OffsetRequest$.MODULE$.LATEST_TIME(), get from the latest offset available.
    *              if set to OffsetRequest$.MODULE$.EARLIEST_TIME(), get from the earliest offset available.
    */
   public long[] getOffsetsBefore(String topic, int partition, long time, int maxNumOffsets);
@@ -292,7 +292,7 @@
 </p>
 <h4><a id="impl_zktopic" href="#impl_zktopic">Broker Topic Registry</a></h4>
 <pre>
-/brokers/topics/[topic]/[0...N] --> nPartions (ephemeral node)
+/brokers/topics/[topic]/[0...N] --> nPartitions (ephemeral node)
 </pre>
 
 <p>
diff --git a/docs/migration.html b/docs/migration.html
index 2da6a7e..5240d86 100644
--- a/docs/migration.html
+++ b/docs/migration.html
@@ -27,7 +27,7 @@
     <li>Use the 0.7 to 0.8 <a href="tools.html">migration tool</a> to mirror data from the 0.7 cluster into the 0.8 cluster.
     <li>When the 0.8 cluster is fully caught up, redeploy all data <i>consumers</i> running the 0.8 client and reading from the 0.8 cluster.
     <li>Finally migrate all 0.7 producers to 0.8 client publishing data to the 0.8 cluster.
-    <li>Decomission the 0.7 cluster.
+    <li>Decommission the 0.7 cluster.
     <li>Drink.
 </ol>
 
diff --git a/docs/ops.html b/docs/ops.html
index b239a0e..8b1cc23 100644
--- a/docs/ops.html
+++ b/docs/ops.html
@@ -70,7 +70,7 @@
 
 <h4><a id="basic_ops_restarting" href="#basic_ops_restarting">Graceful shutdown</a></h4>
 
-The Kafka cluster will automatically detect any broker shutdown or failure and elect new leaders for the partitions on that machine. This will occur whether a server fails or it is brought down intentionally for maintenance or configuration changes. For the latter cases Kafka supports a more graceful mechanism for stoping a server than just killing it.
+The Kafka cluster will automatically detect any broker shutdown or failure and elect new leaders for the partitions on that machine. This will occur whether a server fails or it is brought down intentionally for maintenance or configuration changes. For the latter cases Kafka supports a more graceful mechanism for stopping a server than just killing it.
 
 When a server is stopped gracefully it has two optimizations it will take advantage of:
 <ol>
@@ -138,7 +138,7 @@
 
 <h4><a id="basic_ops_consumer_group" href="#basic_ops_consumer_group">Managing Consumer Groups</a></h4>
 
-With the ConumserGroupCommand tool, we can list, delete, or describe consumer groups. For example, to list all consumer groups across all topics:
+With the ConsumerGroupCommand tool, we can list, delete, or describe consumer groups. For example, to list all consumer groups across all topics:
 
 <pre>
  &gt; bin/kafka-consumer-groups.sh --zookeeper localhost:2181 --list
@@ -156,7 +156,7 @@
 </pre>
 
 
-When youre using the <a href="https://cwiki.apache.org/confluence/display/KAFKA/Consumer+Client+Re-Design">new consumer-groups API</a> where the broker handles coordination of partition handling and rebalance, you can manage the groups with the "--new-consumer" flags:
+When you're using the <a href="https://cwiki.apache.org/confluence/display/KAFKA/Consumer+Client+Re-Design">new consumer-groups API</a> where the broker handles coordination of partition handling and rebalance, you can manage the groups with the "--new-consumer" flags:
 
 <pre>
  &gt; bin/kafka-consumer-groups.sh --new-consumer --bootstrap-server broker1:9092 --list
diff --git a/docs/streams.html b/docs/streams.html
index 9b94bb3..91fda36 100644
--- a/docs/streams.html
+++ b/docs/streams.html
@@ -64,7 +64,7 @@
 <h5><a id="streams_time" href="#streams_time">Time</a></h5>
 
 <p>
-A critical aspect in stream processing is the the notion of <b>time</b>, and how it is modeled and integrated.
+A critical aspect in stream processing is the notion of <b>time</b>, and how it is modeled and integrated.
 For example, some operations such as <b>windowing</b> are defined based on time boundaries.
 </p>
 <p>
diff --git a/examples/src/main/java/kafka/examples/KafkaConsumerProducerDemo.java b/examples/src/main/java/kafka/examples/KafkaConsumerProducerDemo.java
index 414a6f7..c1541bf 100644
--- a/examples/src/main/java/kafka/examples/KafkaConsumerProducerDemo.java
+++ b/examples/src/main/java/kafka/examples/KafkaConsumerProducerDemo.java
@@ -18,7 +18,7 @@
 
 public class KafkaConsumerProducerDemo {
     public static void main(String[] args) {
-        boolean isAsync = args.length == 0 || !args[0].trim().toLowerCase().equals("sync");
+        boolean isAsync = args.length == 0 || !args[0].trim().equalsIgnoreCase("sync");
         Producer producerThread = new Producer(KafkaProperties.TOPIC, isAsync);
         producerThread.start();
 
diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle
index f035838..8b292e8 100644
--- a/gradle/dependencies.gradle
+++ b/gradle/dependencies.gradle
@@ -27,7 +27,6 @@
   apacheds: "2.0.0-M21",
   argparse4j: "0.5.0",
   bcpkix: "1.54",
-  hadoop: "2.7.2",
   easymock: "3.4",
   jackson: "2.6.3",
   jetty: "9.2.15.v20160210",
@@ -38,12 +37,12 @@
   metrics: "2.2.0",
   powermock: "1.6.4",
   reflections: "0.9.10",
-  rocksDB: "4.1.0",
+  rocksDB: "4.4.1",
   scalaTest: "2.2.6",
   scalaParserCombinators: "1.0.4",
   scoverage: "1.1.1",
-  slf4j: "1.7.18",
-  snappy: "1.1.2.1",
+  slf4j: "1.7.21",
+  snappy: "1.1.2.4",
   zkclient: "0.8",
   zookeeper: "3.4.6",
 ]
diff --git a/gradlew b/gradlew
index 9d82f78..27309d9 100755
--- a/gradlew
+++ b/gradlew
@@ -6,12 +6,30 @@
 ##
 ##############################################################################
 
-# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script.
-DEFAULT_JVM_OPTS=""
+# Attempt to set APP_HOME
+# Resolve links: $0 may be a link
+PRG="$0"
+# Need this for relative symlinks.
+while [ -h "$PRG" ] ; do
+    ls=`ls -ld "$PRG"`
+    link=`expr "$ls" : '.*-> \(.*\)$'`
+    if expr "$link" : '/.*' > /dev/null; then
+        PRG="$link"
+    else
+        PRG=`dirname "$PRG"`"/$link"
+    fi
+done
+SAVED="`pwd`"
+cd "`dirname \"$PRG\"`/" >/dev/null
+APP_HOME="`pwd -P`"
+cd "$SAVED" >/dev/null
 
 APP_NAME="Gradle"
 APP_BASE_NAME=`basename "$0"`
 
+# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script.
+DEFAULT_JVM_OPTS=""
+
 # Use the maximum available, or set MAX_FD != -1 to use that value.
 MAX_FD="maximum"
 
@@ -30,6 +48,7 @@
 cygwin=false
 msys=false
 darwin=false
+nonstop=false
 case "`uname`" in
   CYGWIN* )
     cygwin=true
@@ -40,26 +59,11 @@
   MINGW* )
     msys=true
     ;;
+  NONSTOP* )
+    nonstop=true
+    ;;
 esac
 
-# Attempt to set APP_HOME
-# Resolve links: $0 may be a link
-PRG="$0"
-# Need this for relative symlinks.
-while [ -h "$PRG" ] ; do
-    ls=`ls -ld "$PRG"`
-    link=`expr "$ls" : '.*-> \(.*\)$'`
-    if expr "$link" : '/.*' > /dev/null; then
-        PRG="$link"
-    else
-        PRG=`dirname "$PRG"`"/$link"
-    fi
-done
-SAVED="`pwd`"
-cd "`dirname \"$PRG\"`/" >/dev/null
-APP_HOME="`pwd -P`"
-cd "$SAVED" >/dev/null
-
 CLASSPATH=$APP_HOME/gradle/wrapper/gradle-wrapper.jar
 
 # Determine the Java command to use to start the JVM.
@@ -85,7 +89,7 @@
 fi
 
 # Increase the maximum file descriptors if we can.
-if [ "$cygwin" = "false" -a "$darwin" = "false" ] ; then
+if [ "$cygwin" = "false" -a "$darwin" = "false" -a "$nonstop" = "false" ] ; then
     MAX_FD_LIMIT=`ulimit -H -n`
     if [ $? -eq 0 ] ; then
         if [ "$MAX_FD" = "maximum" -o "$MAX_FD" = "max" ] ; then
diff --git a/gradlew.bat b/gradlew.bat
index 72d362d..f6d5974 100644
--- a/gradlew.bat
+++ b/gradlew.bat
@@ -8,14 +8,14 @@
 @rem Set local scope for the variables with windows NT shell

 if "%OS%"=="Windows_NT" setlocal

 

-@rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script.

-set DEFAULT_JVM_OPTS=

-

 set DIRNAME=%~dp0

 if "%DIRNAME%" == "" set DIRNAME=.

 set APP_BASE_NAME=%~n0

 set APP_HOME=%DIRNAME%

 

+@rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script.

+set DEFAULT_JVM_OPTS=

+

 @rem Find java.exe

 if defined JAVA_HOME goto findJavaFromJavaHome

 

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 4124b32..e53b037 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
@@ -24,11 +24,11 @@
 import org.apache.kafka.common.serialization.Serializer;
 import org.apache.kafka.streams.KafkaStreams;
 import org.apache.kafka.streams.KeyValue;
-import org.apache.kafka.streams.kstream.HoppingWindows;
 import org.apache.kafka.streams.kstream.KStreamBuilder;
 import org.apache.kafka.streams.kstream.KStream;
 import org.apache.kafka.streams.kstream.KTable;
 import org.apache.kafka.streams.kstream.KeyValueMapper;
+import org.apache.kafka.streams.kstream.TimeWindows;
 import org.apache.kafka.streams.kstream.ValueJoiner;
 import org.apache.kafka.streams.kstream.Windowed;
 import org.apache.kafka.streams.StreamsConfig;
@@ -160,7 +160,7 @@
                         return new KeyValue<>(viewRegion.region, viewRegion);
                     }
                 })
-                .countByKey(HoppingWindows.of("GeoPageViewsWindow").with(7 * 24 * 60 * 60 * 1000), Serdes.String())
+                .countByKey(TimeWindows.of("GeoPageViewsWindow", 7 * 24 * 60 * 60 * 1000L).advanceBy(1000), Serdes.String())
                 // TODO: we can merge ths toStream().map(...) with a single toStream(...)
                 .toStream()
                 .map(new KeyValueMapper<Windowed<String>, Long, KeyValue<WindowedPageViewByRegion, RegionCount>>() {
@@ -168,10 +168,10 @@
                     public KeyValue<WindowedPageViewByRegion, RegionCount> apply(Windowed<String> key, Long value) {
                         WindowedPageViewByRegion wViewByRegion = new WindowedPageViewByRegion();
                         wViewByRegion.windowStart = key.window().start();
-                        wViewByRegion.region = key.value();
+                        wViewByRegion.region = key.key();
 
                         RegionCount rCount = new RegionCount();
-                        rCount.region = key.value();
+                        rCount.region = key.key();
                         rCount.count = value;
 
                         return new KeyValue<>(wViewByRegion, rCount);
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 e61842f..8a0af6c 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
@@ -30,11 +30,11 @@
 import org.apache.kafka.streams.KafkaStreams;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.StreamsConfig;
-import org.apache.kafka.streams.kstream.HoppingWindows;
 import org.apache.kafka.streams.kstream.KStreamBuilder;
 import org.apache.kafka.streams.kstream.KStream;
 import org.apache.kafka.streams.kstream.KTable;
 import org.apache.kafka.streams.kstream.KeyValueMapper;
+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.kstream.Windowed;
@@ -99,7 +99,7 @@
                         return new KeyValue<>(viewRegion.get("region").textValue(), viewRegion);
                     }
                 })
-                .countByKey(HoppingWindows.of("GeoPageViewsWindow").with(7 * 24 * 60 * 60 * 1000), Serdes.String())
+                .countByKey(TimeWindows.of("GeoPageViewsWindow", 7 * 24 * 60 * 60 * 1000L).advanceBy(1000), Serdes.String())
                 // TODO: we can merge ths toStream().map(...) with a single toStream(...)
                 .toStream()
                 .map(new KeyValueMapper<Windowed<String>, Long, KeyValue<JsonNode, JsonNode>>() {
@@ -107,7 +107,7 @@
                     public KeyValue<JsonNode, JsonNode> apply(Windowed<String> key, Long value) {
                         ObjectNode keyNode = JsonNodeFactory.instance.objectNode();
                         keyNode.put("window-start", key.window().start())
-                                .put("region", key.value());
+                                .put("region", key.key());
 
                         ObjectNode valueNode = JsonNodeFactory.instance.objectNode();
                         valueNode.put("count", value);
diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java
index c12977f..12395f9 100644
--- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java
+++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java
@@ -29,6 +29,7 @@
 import org.apache.kafka.streams.kstream.ValueMapper;
 
 import java.util.Arrays;
+import java.util.Locale;
 import java.util.Properties;
 
 /**
@@ -63,12 +64,12 @@
                 .flatMapValues(new ValueMapper<String, Iterable<String>>() {
                     @Override
                     public Iterable<String> apply(String value) {
-                        return Arrays.asList(value.toLowerCase().split(" "));
+                        return Arrays.asList(value.toLowerCase(Locale.getDefault()).split(" "));
                     }
                 }).map(new KeyValueMapper<String, String, KeyValue<String, String>>() {
                     @Override
                     public KeyValue<String, String> apply(String key, String value) {
-                        return new KeyValue<String, String>(value, value);
+                        return new KeyValue<>(value, value);
                     }
                 })
                 .countByKey("Counts");
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 a5cddfd..34c35b7 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
@@ -30,6 +30,7 @@
 import org.apache.kafka.streams.state.KeyValueStore;
 import org.apache.kafka.streams.state.Stores;
 
+import java.util.Locale;
 import java.util.Properties;
 
 /**
@@ -63,7 +64,7 @@
 
                 @Override
                 public void process(String dummy, String line) {
-                    String[] words = line.toLowerCase().split(" ");
+                    String[] words = line.toLowerCase(Locale.getDefault()).split(" ");
 
                     for (String word : words) {
                         Integer oldValue = this.kvStore.get(word);
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 e8fda10..45024f2 100644
--- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
+++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
@@ -97,6 +97,12 @@
     // usage only and should not be exposed to users at all.
     private final UUID processId;
 
+    /**
+     * Construct the stream instance.
+     *
+     * @param builder  the processor topology builder specifying the computational logic
+     * @param props    properties for the {@link StreamsConfig}
+     */
     public KafkaStreams(TopologyBuilder builder, Properties props) {
         this(builder, new StreamsConfig(props));
     }
@@ -104,8 +110,8 @@
     /**
      * Construct the stream instance.
      *
-     * @param builder The processor topology builder specifying the computational logic
-     * @param config The stream configs
+     * @param builder  the processor topology builder specifying the computational logic
+     * @param config   the stream configs
      */
     public KafkaStreams(TopologyBuilder builder, StreamsConfig config) {
         // create the metrics
@@ -138,6 +144,7 @@
 
     /**
      * Start the stream instance by starting all its threads.
+     * @throws IllegalStateException if process was already started
      */
     public synchronized void start() {
         log.debug("Starting Kafka Stream process");
@@ -157,6 +164,7 @@
     /**
      * Shutdown this stream instance by signaling all the threads to stop,
      * and then wait for them to join.
+     * @throws IllegalStateException if process has not started yet
      */
     public synchronized void close() {
         log.debug("Stopping Kafka Stream process");
diff --git a/streams/src/main/java/org/apache/kafka/streams/KeyValue.java b/streams/src/main/java/org/apache/kafka/streams/KeyValue.java
index ca86fc4..58f2083 100644
--- a/streams/src/main/java/org/apache/kafka/streams/KeyValue.java
+++ b/streams/src/main/java/org/apache/kafka/streams/KeyValue.java
@@ -29,14 +29,31 @@
  */
 public class KeyValue<K, V> {
 
+    /** The key of the key-value pair. */
     public final K key;
+    /** The value of the key-value pair. */
     public final V value;
 
+    /**
+     * Create a new key-value pair.
+     *
+     * @param key    the key
+     * @param value  the value
+     */
     public KeyValue(K key, V value) {
         this.key = key;
         this.value = value;
     }
 
+    /**
+     * Create a new key-value pair.
+     *
+     * @param key    the key
+     * @param value  the value
+     * @param <K>    the type of the key
+     * @param <V>    the type of the value
+     * @return       a new key value pair
+     */
     public static <K, V> KeyValue<K, V> pair(K key, V value) {
         return new KeyValue<>(key, value);
     }
diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
index 3e0f955..99eb58f 100644
--- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
+++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
@@ -63,7 +63,7 @@
     public static final String NUM_STREAM_THREADS_CONFIG = "num.stream.threads";
     private static final String NUM_STREAM_THREADS_DOC = "The number of threads to execute stream processing.";
 
-    /** <code>num.stream.threads</code> */
+    /** <code>num.standby.replicas</code> */
     public static final String NUM_STANDBY_REPLICAS_CONFIG = "num.standby.replicas";
     private static final String NUM_STANDBY_REPLICAS_DOC = "The number of standby replicas for each task.";
 
@@ -91,11 +91,11 @@
     public static final String REPLICATION_FACTOR_CONFIG = "replication.factor";
     public static final String REPLICATION_FACTOR_DOC = "The replication factor for change log topics and repartition topics created by the stream processing application.";
 
-    /** <code>replication.factor</code> */
+    /** <code>key.serde</code> */
     public static final String KEY_SERDE_CLASS_CONFIG = "key.serde";
     public static final String KEY_SERDE_CLASS_DOC = "Serializer / deserializer class for key that implements the <code>Serde</code> interface.";
 
-    /** <code>replication.factor</code> */
+    /** <code>value.serde</code> */
     public static final String VALUE_SERDE_CLASS_CONFIG = "value.serde";
     public static final String VALUE_SERDE_CLASS_DOC = "Serializer / deserializer class for value that implements the <code>Serde</code> interface.";
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java
index 9ec9f96..989d89f 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java
@@ -18,7 +18,7 @@
 package org.apache.kafka.streams.kstream;
 
 /**
- * The Aggregator interface for aggregating values of the given key.
+ * The {@link Aggregator} interface for aggregating values of the given key.
  *
  * @param <K>   key type
  * @param <V>   original value type
@@ -26,5 +26,13 @@
  */
 public interface Aggregator<K, V, T> {
 
+    /**
+     * Compute a new aggregate from the key and value of a record and the current aggregate of the same key.
+     *
+     * @param aggKey     the key of the record
+     * @param value      the value of the record
+     * @param aggregate  the current aggregate value
+     * @return           the new aggregate value
+     */
     T apply(K aggKey, V value, T aggregate);
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java
index 83064e8..b3e3169 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java
@@ -18,9 +18,8 @@
 package org.apache.kafka.streams.kstream;
 
 
-
 /**
- * The ForeachAction interface for performing an action on a key-value pair.
+ * The {@link ForeachAction} interface for performing an action on a key-value pair.
  * Note that this action is stateless. If stateful processing is required, consider
  * using {@link KStream#transform(TransformerSupplier, String...)} or
  * {@link KStream#process(ProcessorSupplier, String...)} instead.
@@ -29,6 +28,13 @@
  * @param <V>   original value type
  */
 public interface ForeachAction<K, V> {
+
+    /**
+     * Perform an action for each record of a stream.
+     *
+     * @param key    the key of the record
+     * @param value  the value of the record
+     */
     void apply(K key, V value);
 }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java
deleted file mode 100644
index aa866e4..0000000
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.streams.kstream;
-
-import org.apache.kafka.streams.kstream.internals.HoppingWindow;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * The hopping window specifications used for aggregations.
- */
-public class HoppingWindows extends Windows<HoppingWindow> {
-
-    private static final long DEFAULT_SIZE_MS = 1000L;
-
-    public final long size;
-
-    public final long period;
-
-    private HoppingWindows(String name, long size, long period) {
-        super(name);
-
-        this.size = size;
-        this.period = period;
-    }
-
-    /**
-     * Returns a half-interval hopping window definition with the window size in milliseconds
-     * of the form &#91; N &#42; default_size, N &#42; default_size + default_size &#41;
-     */
-    public static HoppingWindows of(String name) {
-        return new HoppingWindows(name, DEFAULT_SIZE_MS, DEFAULT_SIZE_MS);
-    }
-
-    /**
-     * Returns a new hopping window definition with the original size but reassign the window
-     * period in milliseconds of the form &#91; N &#42; period, N &#42; period + size &#41;
-     */
-    public HoppingWindows with(long size) {
-        return new HoppingWindows(this.name, size, this.period);
-    }
-
-    /**
-     * Returns a new hopping window definition with the original size but reassign the window
-     * period in milliseconds of the form &#91; N &#42; period, N &#42; period + size &#41;
-     */
-    public HoppingWindows every(long period) {
-        return new HoppingWindows(this.name, this.size, period);
-    }
-
-    @Override
-    public Map<Long, HoppingWindow> windowsFor(long timestamp) {
-        long enclosed = (size - 1) / period;
-
-        long windowStart = Math.max(0, timestamp - timestamp % period - enclosed * period);
-
-        Map<Long, HoppingWindow> windows = new HashMap<>();
-        while (windowStart <= timestamp) {
-            // add the window
-            HoppingWindow window = new HoppingWindow(windowStart, windowStart + this.size);
-            windows.put(windowStart, window);
-
-            // advance the step period
-            windowStart += this.period;
-        }
-
-        return windows;
-    }
-
-    @Override
-    public boolean equalTo(Windows other) {
-        if (!other.getClass().equals(HoppingWindows.class))
-            return false;
-
-        HoppingWindows otherWindows = (HoppingWindows) other;
-
-        return this.size == otherWindows.size && this.period == otherWindows.period;
-    }
-}
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Initializer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Initializer.java
index 67c1c21..39bc40d 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/Initializer.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Initializer.java
@@ -18,11 +18,16 @@
 package org.apache.kafka.streams.kstream;
 
 /**
- * The Initializer interface for creating an initial value in aggregations.
+ * The {@link Initializer} interface for creating an initial value in aggregations.
  *
  * @param <T>   aggregate value type
  */
 public interface Initializer<T> {
 
+    /**
+     * Return the initial value for an aggregation.
+     *
+     * @return  the initial value for an aggregation
+     */
     T apply();
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java
index 24dbdd3..f45c064 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java
@@ -17,17 +17,36 @@
 
 package org.apache.kafka.streams.kstream;
 
-
-import org.apache.kafka.streams.kstream.internals.TumblingWindow;
+import org.apache.kafka.streams.kstream.internals.TimeWindow;
 
 import java.util.Map;
 
 /**
  * The window specifications used for joins.
+ * <p>
+ * A {@link JoinWindows} instance defines a join over two stream on the same key and a maximum time difference.
+ * In SQL-style you would express this join as
+ * <pre>
+ *     SELECT * FROM stream1, stream2
+ *     WHERE
+ *       stream1.key = stream2.key
+ *       AND
+ *       stream2.ts - before <= stream1.ts <= stream2.ts + after
+ * </pre>
+ * There are three different window configuration supported:
+ * <ul>
+ *     <li>before = after = time-difference</li>
+ *     <li>before = 0 and after = time-difference</li>
+ *     <li>before = time-difference and after = 0</li>
+ * </ul>
+ * A join is symmetric in the sense, that a join specification on the first stream returns the same result record as
+ * a join specification on the second stream with flipped before and after values.
  */
-public class JoinWindows extends Windows<TumblingWindow> {
+public class JoinWindows extends Windows<TimeWindow> {
 
+    /** Maximum time difference for tuples that are before the join tuple. */
     public final long before;
+    /** Maximum time difference for tuples that are after the join tuple. */
     public final long after;
 
     private JoinWindows(String name, long before, long after) {
@@ -42,51 +61,62 @@
     }
 
     /**
-     * Specifies that records of the same key are joinable if their timestamp stamps are within
-     * timeDifference.
+     * Specifies that records of the same key are joinable if their timestamps are within {@code timeDifference}.
      *
-     * @param timeDifference    join window interval in milliseconds
+     * @param timeDifference    join window interval
      */
     public JoinWindows within(long timeDifference) {
         return new JoinWindows(this.name, timeDifference, timeDifference);
     }
 
     /**
-     * Specifies that records of the same key are joinable if their timestamp stamps are within
+     * Specifies that records of the same key are joinable if their timestamps are within
      * the join window interval, and if the timestamp of a record from the secondary stream is
      * earlier than or equal to the timestamp of a record from the first stream.
      *
-     * @param timeDifference    join window interval in milliseconds
+     * @param timeDifference    join window interval
      */
     public JoinWindows before(long timeDifference) {
         return new JoinWindows(this.name, timeDifference, this.after);
     }
 
     /**
-     * Specifies that records of the same key are joinable if their timestamp stamps are within
+     * Specifies that records of the same key are joinable if their timestamps are within
      * the join window interval, and if the timestamp of a record from the secondary stream
      * is later than or equal to the timestamp of a record from the first stream.
      *
-     * @param timeDifference    join window interval in milliseconds
+     * @param timeDifference    join window interval
      */
     public JoinWindows after(long timeDifference) {
         return new JoinWindows(this.name, this.before, timeDifference);
     }
 
+    /**
+     * Not supported by {@link JoinWindows}. Throws {@link UnsupportedOperationException}.
+     */
     @Override
-    public Map<Long, TumblingWindow> windowsFor(long timestamp) {
-        // this function should never be called
+    public Map<Long, TimeWindow> windowsFor(long timestamp) {
         throw new UnsupportedOperationException("windowsFor() is not supported in JoinWindows");
     }
 
     @Override
-    public boolean equalTo(Windows other) {
-        if (!other.getClass().equals(JoinWindows.class))
+    public final boolean equals(Object o) {
+        if (o == this) {
+            return true;
+        }
+        if (!(o instanceof JoinWindows)) {
             return false;
+        }
 
-        JoinWindows otherWindows = (JoinWindows) other;
+        JoinWindows other = (JoinWindows) o;
+        return this.before == other.before && this.after == other.after;
+    }
 
-        return this.before == otherWindows.before && this.after == otherWindows.after;
+    @Override
+    public int hashCode() {
+        int result = (int) (before ^ (before >>> 32));
+        result = 31 * result + (int) (after ^ (after >>> 32));
+        return result;
     }
 
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java
new file mode 100644
index 0000000..2ebad87
--- /dev/null
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java
@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.streams.kstream;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+import org.apache.kafka.common.serialization.Serde;
+
+/**
+ * {@link KGroupedTable} is an abstraction of a <i>grouped changelog stream</i> from a primary-keyed table,
+ * usually on a different grouping key than the original primary key.
+ * <p>
+ * It is an intermediate representation after a re-grouping of a {@link KTable} before an aggregation is applied
+ * to the new partitions resulting in a new {@link KTable}.
+ *
+ * @param <K> Type of primary keys
+ * @param <V> Type of value changes
+ */
+@InterfaceStability.Unstable
+public interface KGroupedTable<K, V> {
+
+    /**
+     * Combine updating values of this stream by the selected key into a new instance of {@link KTable}.
+     *
+     * @param adder         the instance of {@link Reducer} for addition
+     * @param subtractor    the instance of {@link Reducer} for subtraction
+     * @param name          the name of the resulted {@link KTable}
+     * @return a {@link KTable} with the same key and value types as this {@link KGroupedTable},
+     *         containing aggregated values for each key
+     */
+    KTable<K, V> reduce(Reducer<V> adder,
+                        Reducer<V> subtractor,
+                        String name);
+
+    /**
+     * Aggregate updating values of this stream by the selected key into a new instance of {@link KTable}.
+     *
+     * @param initializer   the instance of {@link Initializer}
+     * @param adder         the instance of {@link Aggregator} for addition
+     * @param substractor   the instance of {@link Aggregator} for subtraction
+     * @param aggValueSerde value serdes for materializing the aggregated table,
+     *                      if not specified the default serdes defined in the configs will be used
+     * @param name          the name of the resulted table
+     * @param <T>           the value type of the aggregated {@link KTable}
+     * @return a {@link KTable} with same key and aggregated value type {@code T},
+     *         containing aggregated values for each key
+     */
+    <T> KTable<K, T> aggregate(Initializer<T> initializer,
+                               Aggregator<K, V, T> adder,
+                               Aggregator<K, V, T> substractor,
+                               Serde<T> aggValueSerde,
+                               String name);
+
+    /**
+     * Aggregate updating values of this stream by the selected key into a new instance of {@link KTable}
+     * using default serializers and deserializers.
+     *
+     * @param initializer   the instance of {@link Initializer}
+     * @param adder         the instance of {@link Aggregator} for addition
+     * @param substractor   the instance of {@link Aggregator} for subtraction
+     * @param name          the name of the resulted {@link KTable}
+     * @param <T>           the value type of the aggregated {@link KTable}
+     * @return a {@link KTable} with same key and aggregated value type {@code T},
+     *         containing aggregated values for each key
+     */
+    <T> KTable<K, T> aggregate(Initializer<T> initializer,
+                               Aggregator<K, V, T> adder,
+                               Aggregator<K, V, T> substractor,
+                               String name);
+
+    /**
+     * Count number of records of this stream by the selected key into a new instance of {@link KTable}.
+     *
+     * @param name          the name of the resulted {@link KTable}
+     * @return a {@link KTable} with same key and {@link Long} value type as this {@link KGroupedTable},
+     *         containing the number of values for each key
+     */
+    KTable<K, Long> count(String name);
+
+}
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
index 27475aa..a1ecfa4 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
@@ -24,10 +24,18 @@
 import org.apache.kafka.streams.processor.StreamPartitioner;
 
 /**
- * KStream is an abstraction of a <i>record stream</i> of key-value pairs.
+ * {@link KStream} is an abstraction of a <i>record stream</i> of key-value pairs.
+ * <p>
+ * A {@link KStream} is either defined from one or multiple Kafka topics that are consumed message by message or
+ * the result of a {@link KStream} transformation. A {@link KTable} can also be converted into a {@link KStream}.
+ * <p>
+ * A {@link KStream} can be transformed record by record, joined with another {@link KStream} or {@link KTable}, or
+ * can be aggregated into a {@link KTable}.
  *
  * @param <K> Type of keys
  * @param <V> Type of values
+ *
+ * @see KTable
  */
 @InterfaceStability.Unstable
 public interface KStream<K, V> {
@@ -36,6 +44,8 @@
      * Create a new instance of {@link KStream} that consists of all elements of this stream which satisfy a predicate.
      *
      * @param predicate     the instance of {@link Predicate}
+     *
+     * @return a {@link KStream} that contains only those records that satisfy the given predicate
      */
     KStream<K, V> filter(Predicate<K, V> predicate);
 
@@ -43,15 +53,30 @@
      * Create a new instance of {@link KStream} that consists all elements of this stream which do not satisfy a predicate.
      *
      * @param predicate     the instance of {@link Predicate}
+     *
+     * @return a {@link KStream} that contains only those records that do not satisfy the given predicate
      */
     KStream<K, V> filterNot(Predicate<K, V> predicate);
 
+
+    /**
+     * Create a new key from the current key and value.
+     *
+     * @param mapper  the instance of {@link KeyValueMapper}
+     * @param <K1>   the new key type on the stream
+     *
+     * @return a {@link KStream} that contains records with different key type and same value type
+     */
+    <K1> KStream<K1, V> selectKey(KeyValueMapper<K, V, K1> mapper);
+
     /**
      * Create a new instance of {@link KStream} by transforming each element in this stream into a different element in the new stream.
      *
      * @param mapper        the instance of {@link KeyValueMapper}
      * @param <K1>          the key type of the new stream
      * @param <V1>          the value type of the new stream
+     *
+     * @return a {@link KStream} that contains records with new key and value type
      */
     <K1, V1> KStream<K1, V1> map(KeyValueMapper<K, V, KeyValue<K1, V1>> mapper);
 
@@ -60,6 +85,8 @@
      *
      * @param mapper        the instance of {@link ValueMapper}
      * @param <V1>          the value type of the new stream
+     *
+     * @return a {@link KStream} that contains records with unmodified keys and new values of different type
      */
     <V1> KStream<K, V1> mapValues(ValueMapper<V, V1> mapper);
 
@@ -115,6 +142,8 @@
      * @param mapper        the instance of {@link KeyValueMapper}
      * @param <K1>          the key type of the new stream
      * @param <V1>          the value type of the new stream
+     *
+     * @return a {@link KStream} that contains more or less records with new key and value type
      */
     <K1, V1> KStream<K1, V1> flatMap(KeyValueMapper<K, V, Iterable<KeyValue<K1, V1>>> mapper);
 
@@ -123,6 +152,8 @@
      *
      * @param processor     the instance of {@link ValueMapper}
      * @param <V1>          the value type of the new stream
+     *
+     * @return a {@link KStream} that contains more or less records with unmodified keys and new values of different type
      */
     <V1> KStream<K, V1> flatMapValues(ValueMapper<V, Iterable<V1>> processor);
 
@@ -134,6 +165,8 @@
      * assigned to this stream only. An element will be dropped if none of the predicates evaluate to true.
      *
      * @param predicates    the ordered list of {@link Predicate} instances
+     *
+     * @return multiple distinct substreams of this {@link KStream}
      */
     KStream<K, V>[] branch(Predicate<K, V>... predicates);
 
@@ -143,6 +176,8 @@
      * This is equivalent to calling {@link #to(String)} and {@link org.apache.kafka.streams.kstream.KStreamBuilder#stream(String...)}.
      *
      * @param topic     the topic name
+     *
+     * @return a {@link KStream} that contains the exact same records as this {@link KStream}
      */
     KStream<K, V> through(String topic);
 
@@ -150,7 +185,7 @@
      * Perform an action on each element of {@link KStream}.
      * Note that this is a terminal operation that returns void.
      *
-     * @param action An action to perform on each element
+     * @param action an action to perform on each element
      */
     void foreach(ForeachAction<K, V> action);
 
@@ -162,6 +197,8 @@
      * @param partitioner  the function used to determine how records are distributed among partitions of the topic,
      *                     if not specified producer's {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} will be used
      * @param topic        the topic name
+     *
+     * @return a {@link KStream} that contains the exact same records as this {@link KStream}
      */
     KStream<K, V> through(StreamPartitioner<K, V> partitioner, String topic);
 
@@ -178,6 +215,8 @@
      * @param valSerde     value serde used to send key-value pairs,
      *                     if not specified the default value serde defined in the configuration will be used
      * @param topic        the topic name
+     *
+     * @return a {@link KStream} that contains the exact same records as this {@link KStream}
      */
     KStream<K, V> through(Serde<K> keySerde, Serde<V> valSerde, String topic);
 
@@ -196,6 +235,8 @@
      *                     {@link org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner} will be used
      *                     &mdash; otherwise {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} will be used
      * @param topic        the topic name
+     *
+     * @return a {@link KStream} that contains the exact same records as this {@link KStream}
      */
     KStream<K, V> through(Serde<K> keySerde, Serde<V> valSerde, StreamPartitioner<K, V> partitioner, String topic);
 
@@ -251,6 +292,8 @@
      *
      * @param transformerSupplier   the instance of {@link TransformerSupplier} that generates {@link org.apache.kafka.streams.kstream.Transformer}
      * @param stateStoreNames       the names of the state store used by the processor
+     *
+     * @return a new {@link KStream} with transformed key and value types
      */
     <K1, V1> KStream<K1, V1> transform(TransformerSupplier<K, V, KeyValue<K1, V1>> transformerSupplier, String... stateStoreNames);
 
@@ -259,6 +302,8 @@
      *
      * @param valueTransformerSupplier  the instance of {@link ValueTransformerSupplier} that generates {@link org.apache.kafka.streams.kstream.ValueTransformer}
      * @param stateStoreNames           the names of the state store used by the processor
+     *
+     * @return a {@link KStream} that contains records with unmodified keys and transformed values with type {@code R}
      */
     <R> KStream<K, R> transformValues(ValueTransformerSupplier<V, R> valueTransformerSupplier, String... stateStoreNames);
 
@@ -284,6 +329,9 @@
      *                          if not specified the default serdes defined in the configs will be used
      * @param <V1>              the value type of the other stream
      * @param <R>               the value type of the new stream
+     *
+     * @return a {@link KStream} that contains join-records for each key and values computed by the given {@link ValueJoiner},
+     *         one for each matched record-pair with the same key and within the joining window intervals
      */
     <V1, R> KStream<K, R> join(
             KStream<K, V1> otherStream,
@@ -302,6 +350,9 @@
      * @param windows       the specification of the {@link JoinWindows}
      * @param <V1>          the value type of the other stream
      * @param <R>           the value type of the new stream
+     *
+     * @return a {@link KStream} that contains join-records for each key and values computed by the given {@link ValueJoiner},
+     *         one for each matched record-pair with the same key and within the joining window intervals
      */
     <V1, R> KStream<K, R> join(
             KStream<K, V1> otherStream,
@@ -322,6 +373,9 @@
      *                          if not specified the default serdes defined in the configs will be used
      * @param <V1>              the value type of the other stream
      * @param <R>               the value type of the new stream
+     *
+     * @return a {@link KStream} that contains join-records for each key and values computed by the given {@link ValueJoiner},
+     *         one for each matched record-pair with the same key and within the joining window intervals
      */
     <V1, R> KStream<K, R> outerJoin(
             KStream<K, V1> otherStream,
@@ -340,6 +394,9 @@
      * @param windows       the specification of the {@link JoinWindows}
      * @param <V1>          the value type of the other stream
      * @param <R>           the value type of the new stream
+     *
+     * @return a {@link KStream} that contains join-records for each key and values computed by the given {@link ValueJoiner},
+     *         one for each matched record-pair with the same key and within the joining window intervals
      */
     <V1, R> KStream<K, R> outerJoin(
             KStream<K, V1> otherStream,
@@ -358,6 +415,9 @@
      *                          if not specified the default serdes defined in the configs will be used
      * @param <V1>              the value type of the other stream
      * @param <R>               the value type of the new stream
+     *
+     * @return a {@link KStream} that contains join-records for each key and values computed by the given {@link ValueJoiner},
+     *         one for each matched record-pair with the same key and within the joining window intervals
      */
     <V1, R> KStream<K, R> leftJoin(
             KStream<K, V1> otherStream,
@@ -375,6 +435,9 @@
      * @param windows       the specification of the {@link JoinWindows}
      * @param <V1>          the value type of the other stream
      * @param <R>           the value type of the new stream
+     *
+     * @return a {@link KStream} that contains join-records for each key and values computed by the given {@link ValueJoiner},
+     *         one for each matched record-pair with the same key and within the joining window intervals
      */
     <V1, R> KStream<K, R> leftJoin(
             KStream<K, V1> otherStream,
@@ -388,6 +451,9 @@
      * @param joiner    the instance of {@link ValueJoiner}
      * @param <V1>      the value type of the table
      * @param <V2>      the value type of the new stream
+     *
+     * @return a {@link KStream} that contains join-records for each key and values computed by the given {@link ValueJoiner},
+     *         one for each matched record-pair with the same key and within the joining window intervals
      */
     <V1, V2> KStream<K, V2> leftJoin(KTable<K, V1> table, ValueJoiner<V, V1, V2> joiner);
 
@@ -400,6 +466,10 @@
      *                          if not specified the default serdes defined in the configs will be used
      * @param valueSerde        value serdes for materializing the aggregated table,
      *                          if not specified the default serdes defined in the configs will be used
+     *
+     * @return a windowed {@link KTable} which can be treated as a list of {@code KTable}s
+     *         where each table contains records with unmodified keys and values
+     *         that represent the latest (rolling) aggregate for each key within that window
      */
     <W extends Window> KTable<Windowed<K>, V> reduceByKey(Reducer<V> reducer,
                                                           Windows<W> windows,
@@ -412,6 +482,10 @@
      *
      * @param reducer the instance of {@link Reducer}
      * @param windows the specification of the aggregation {@link Windows}
+     *
+     * @return a windowed {@link KTable} which can be treated as a list of {@code KTable}s
+     *         where each table contains records with unmodified keys and values
+     *         that represent the latest (rolling) aggregate for each key within that window
      */
     <W extends Window> KTable<Windowed<K>, V> reduceByKey(Reducer<V> reducer, Windows<W> windows);
 
@@ -424,6 +498,8 @@
      * @param valueSerde        value serdes for materializing the aggregated table,
      *                          if not specified the default serdes defined in the configs will be used
      * @param name              the name of the resulted {@link KTable}
+     *
+     * @return a {@link KTable} that contains records with unmodified keys and values that represent the latest (rolling) aggregate for each key
      */
     KTable<K, V> reduceByKey(Reducer<V> reducer,
                              Serde<K> keySerde,
@@ -435,6 +511,8 @@
      *
      * @param reducer the instance of {@link Reducer}
      * @param name    the name of the resulted {@link KTable}
+     *
+     * @return a {@link KTable} that contains records with unmodified keys and values that represent the latest (rolling) aggregate for each key
      */
     KTable<K, V> reduceByKey(Reducer<V> reducer, String name);
 
@@ -449,6 +527,10 @@
      * @param aggValueSerde aggregate value serdes for materializing the aggregated table,
      *                      if not specified the default serdes defined in the configs will be used
      * @param <T>           the value type of the resulted {@link KTable}
+     *
+     * @return a windowed {@link KTable} which can be treated as a list of {@code KTable}s
+     *         where each table contains records with unmodified keys and values with type {@code T}
+     *         that represent the latest (rolling) aggregate for each key within that window
      */
     <T, W extends Window> KTable<Windowed<K>, T> aggregateByKey(Initializer<T> initializer,
                                                                 Aggregator<K, V, T> aggregator,
@@ -464,6 +546,10 @@
      * @param aggregator    the instance of {@link Aggregator}
      * @param windows       the specification of the aggregation {@link Windows}
      * @param <T>           the value type of the resulted {@link KTable}
+     *
+     * @return a windowed {@link KTable} which can be treated as a list of {@code KTable}s
+     *         where each table contains records with unmodified keys and values with type {@code T}
+     *         that represent the latest (rolling) aggregate for each key within that window
      */
     <T, W extends Window> KTable<Windowed<K>, T> aggregateByKey(Initializer<T> initializer,
                                                                 Aggregator<K, V, T> aggregator,
@@ -480,6 +566,8 @@
      *                      if not specified the default serdes defined in the configs will be used
      * @param name          the name of the resulted {@link KTable}
      * @param <T>           the value type of the resulted {@link KTable}
+     *
+     * @return a {@link KTable} that contains records with unmodified keys and values (of different type) that represent the latest (rolling) aggregate for each key
      */
     <T> KTable<K, T> aggregateByKey(Initializer<T> initializer,
                                     Aggregator<K, V, T> aggregator,
@@ -495,42 +583,56 @@
      * @param aggregator    the class of {@link Aggregator}
      * @param name          the name of the resulted {@link KTable}
      * @param <T>           the value type of the resulted {@link KTable}
+     *
+     * @return a {@link KTable} that contains records with unmodified keys and values (of different type) that represent the latest (rolling) aggregate for each key
      */
     <T> KTable<K, T> aggregateByKey(Initializer<T> initializer,
                                     Aggregator<K, V, T> aggregator,
                                     String name);
 
     /**
-     * Count number of messages of this stream by key on a window basis into a new instance of windowed {@link KTable}.
+     * Count number of records of this stream by key on a window basis into a new instance of windowed {@link KTable}.
      *
      * @param windows       the specification of the aggregation {@link Windows}
      * @param keySerde      key serdes for materializing the counting table,
      *                      if not specified the default serdes defined in the configs will be used
+     *
+     * @return a windowed {@link KTable} which can be treated as a list of {@code KTable}s
+     *         where each table contains records with unmodified keys and values
+     *         that represent the latest (rolling) count (i.e., number of records) for each key within that window
      */
     <W extends Window> KTable<Windowed<K>, Long> countByKey(Windows<W> windows, Serde<K> keySerde);
 
     /**
-     * Count number of messages of this stream by key on a window basis into a new instance of windowed {@link KTable}
+     * Count number of records of this stream by key on a window basis into a new instance of windowed {@link KTable}
      * with default serializers and deserializers.
      *
      * @param windows       the specification of the aggregation {@link Windows}
+     *
+     * @return a windowed {@link KTable} which can be treated as a list of {@code KTable}s
+     *         where each table contains records with unmodified keys and values
+     *         that represent the latest (rolling) count (i.e., number of records) for each key within that window
      */
     <W extends Window> KTable<Windowed<K>, Long> countByKey(Windows<W> windows);
 
     /**
-     * Count number of messages of this stream by key into a new instance of ever-updating {@link KTable}.
+     * Count number of records of this stream by key into a new instance of ever-updating {@link KTable}.
      *
      * @param keySerde      key serdes for materializing the counting table,
      *                      if not specified the default serdes defined in the configs will be used
      * @param name          the name of the resulted {@link KTable}
+     *
+     * @return a {@link KTable} that contains records with unmodified keys and values that represent the latest (rolling) count (i.e., number of records) for each key
      */
     KTable<K, Long> countByKey(Serde<K> keySerde, String name);
 
     /**
-     * Count number of messages of this stream by key into a new instance of ever-updating {@link KTable}
+     * Count number of records of this stream by key into a new instance of ever-updating {@link KTable}
      * with default serializers and deserializers.
      *
      * @param name          the name of the resulted {@link KTable}
+     *
+     * @return a {@link KTable} that contains records with unmodified keys and values that represent the latest (rolling) count (i.e., number of records) for each key
      */
     KTable<K, Long> countByKey(String name);
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java
index 6b770b4..9d90ba0 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java
@@ -28,35 +28,44 @@
 import java.util.concurrent.atomic.AtomicInteger;
 
 /**
- * KStreamBuilder is a subclass of {@link TopologyBuilder} that provides the Kafka Streams DSL
+ * {@link KStreamBuilder} is a subclass of {@link TopologyBuilder} that provides the Kafka Streams DSL
  * for users to specify computational logic and translates the given logic to a {@link org.apache.kafka.streams.processor.internals.ProcessorTopology}.
  */
 public class KStreamBuilder extends TopologyBuilder {
 
     private final AtomicInteger index = new AtomicInteger(0);
 
+    /**
+     * Create a new {@link KStreamBuilder} instance.
+     */
     public KStreamBuilder() {
         super();
     }
 
     /**
-     * Creates a {@link KStream} instance from the specified topics.
+     * Create a {@link KStream} instance from the specified topics.
      * The default deserializers specified in the config are used.
+     * <p>
+     * If multiple topics are specified there are nor ordering guaranteed for records from different topics.
      *
      * @param topics    the topic names; must contain at least one topic name
+     * @return a {@link KStream} for the specified topics
      */
     public <K, V> KStream<K, V> stream(String... topics) {
         return stream(null, null, topics);
     }
 
     /**
-     * Creates a {@link KStream} instance for the specified topics.
+     * Create a {@link KStream} instance from the specified topics.
+     * <p>
+     * If multiple topics are specified there are nor ordering guaranteed for records from different topics.
      *
      * @param keySerde  key serde used to read this source {@link KStream},
      *                  if not specified the default serde defined in the configs will be used
      * @param valSerde  value serde used to read this source {@link KStream},
      *                  if not specified the default serde defined in the configs will be used
      * @param topics    the topic names; must contain at least one topic name
+     * @return a {@link KStream} for the specified topics
      */
     public <K, V> KStream<K, V> stream(Serde<K> keySerde, Serde<V> valSerde, String... topics) {
         String name = newName(KStreamImpl.SOURCE_NAME);
@@ -67,23 +76,25 @@
     }
 
     /**
-     * Creates a {@link KTable} instance for the specified topic.
+     * Create a {@link KTable} instance for the specified topic.
      * The default deserializers specified in the config are used.
      *
      * @param topic     the topic name; cannot be null
+     * @return a {@link KTable} for the specified topics
      */
     public <K, V> KTable<K, V> table(String topic) {
         return table(null, null, topic);
     }
 
     /**
-     * Creates a {@link KTable} instance for the specified topic.
+     * Create a {@link KTable} instance for the specified topic.
      *
      * @param keySerde   key serde used to send key-value pairs,
      *                   if not specified the default key serde defined in the configuration will be used
      * @param valSerde   value serde used to send key-value pairs,
      *                   if not specified the default value serde defined in the configuration will be used
      * @param topic      the topic name; cannot be null
+     * @return a {@link KTable} for the specified topics
      */
     public <K, V> KTable<K, V> table(Serde<K> keySerde, Serde<V> valSerde, String topic) {
         String source = newName(KStreamImpl.SOURCE_NAME);
@@ -98,9 +109,12 @@
     }
 
     /**
-     * Creates a new instance of {@link KStream} by merging the given streams
+     * Create a new instance of {@link KStream} by merging the given streams.
+     * <p>
+     * There are nor ordering guaranteed for records from different streams.
      *
      * @param streams   the instances of {@link KStream} to be merged
+     * @return a {@link KStream} containing all records of the given streams
      */
     public <K, V> KStream<K, V> merge(KStream<K, V>... streams) {
         return KStreamImpl.merge(this, streams);
@@ -111,6 +125,7 @@
      * This function is only for internal usage.
      *
      * @param prefix    processor name prefix
+     * @return a new unique name
      */
     public String newName(String prefix) {
         return prefix + String.format("%010d", index.getAndIncrement());
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java
index bb6878f..cc5a521 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java
@@ -23,10 +23,19 @@
 import org.apache.kafka.streams.processor.StreamPartitioner;
 
 /**
- * KTable is an abstraction of a <i>changelog stream</i> from a primary-keyed table.
+ * {@link KTable} is an abstraction of a <i>changelog stream</i> from a primary-keyed table.
+ * Each record in this stream is an update on the primary-keyed table with the record key as the primary key.
+ * <p>
+ * A {@link KTable} is either defined from one or multiple Kafka topics that are consumed message by message or
+ * the result of a {@link KTable} transformation. An aggregation of a {@link KStream} also yields a {@link KTable}.
+ * <p>
+ * A {@link KTable} can be transformed record by record, joined with another {@link KTable} or {@link KStream}, or
+ * can be re-partitioned and aggregated into a new {@link KTable}.
  *
  * @param <K> Type of primary keys
  * @param <V> Type of value changes
+ *
+ * @see KStream
  */
 @InterfaceStability.Unstable
 public interface KTable<K, V> {
@@ -35,13 +44,17 @@
      * Create a new instance of {@link KTable} that consists of all elements of this stream which satisfy a predicate.
      *
      * @param predicate     the instance of {@link Predicate}
+     *
+     * @return a {@link KTable} that contains only those records that satisfy the given predicate
      */
     KTable<K, V> filter(Predicate<K, V> predicate);
 
     /**
-     * Create a new instance of {@link KTable} that consists all elements of this stream which do not satisfy a predicate
+     * Create a new instance of {@link KTable} that consists all elements of this stream which do not satisfy a predicate.
      *
      * @param predicate     the instance of {@link Predicate}
+     *
+     * @return a {@link KTable} that contains only those records that do not satisfy the given predicate
      */
     KTable<K, V> filterNot(Predicate<K, V> predicate);
 
@@ -50,12 +63,14 @@
      *
      * @param mapper        the instance of {@link ValueMapper}
      * @param <V1>          the value type of the new stream
+     *
+     * @return a {@link KTable} that contains records with unmodified keys and new values of different type
      */
     <V1> KTable<K, V1> mapValues(ValueMapper<V, V1> mapper);
 
 
     /**
-     * Print the elements of this stream to System.out
+     * Print the elements of this stream to {@code System.out}
      *
      * Implementors will need to override toString for keys and values that are not of
      * type String, Integer etc to get meaningful information.
@@ -63,7 +78,7 @@
     void print();
 
     /**
-     * Print the elements of this stream to System.out
+     * Print the elements of this stream to {@code System.out}
      * @param keySerde key serde used to send key-value pairs,
      *                 if not specified the default serde defined in the configs will be used
      * @param valSerde value serde used to send key-value pairs,
@@ -75,15 +90,16 @@
     void print(Serde<K> keySerde, Serde<V> valSerde);
 
     /**
-     * Write the elements of this stream to a file at the given path.
+     * Write the elements of this stream to a file at the given path using default serializers and deserializers.
      * @param filePath name of file to write to
      *
-     * Implementors will need to override toString for keys and values that are not of
-     * type String, Integer etc to get meaningful information.
+     * Implementors will need to override {@code toString} for keys and values that are not of
+     * type {@link String}, {@link Integer} etc. to get meaningful information.
      */
     void writeAsText(String filePath);
 
     /**
+     * Write the elements of this stream to a file at the given path.
      *
      * @param filePath name of file to write to
      * @param keySerde key serde used to send key-value pairs,
@@ -91,8 +107,8 @@
      * @param valSerde value serde used to send key-value pairs,
      *                 if not specified the default serde defined in the configs will be used
      *
-     * Implementors will need to override toString for keys and values that are not of
-     * type String, Integer etc to get meaningful information.
+     * Implementors will need to override {@code toString} for keys and values that are not of
+     * type {@link String}, {@link Integer} etc. to get meaningful information.
      */
     void  writeAsText(String filePath, Serde<K> keySerde, Serde<V> valSerde);
 
@@ -102,6 +118,8 @@
      * This is equivalent to calling {@link #to(String)} and {@link org.apache.kafka.streams.kstream.KStreamBuilder#table(String)}.
      *
      * @param topic         the topic name
+     *
+     * @return a new {@link KTable} that contains the exact same records as this {@link KTable}
      */
     KTable<K, V> through(String topic);
 
@@ -113,6 +131,8 @@
      * @param partitioner  the function used to determine how records are distributed among partitions of the topic,
      *                     if not specified producer's {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} will be used
      * @param topic        the topic name
+     *
+     * @return a new {@link KTable} that contains the exact same records as this {@link KTable}
      */
     KTable<K, V> through(StreamPartitioner<K, V> partitioner, String topic);
 
@@ -129,6 +149,8 @@
      * @param valSerde     value serde used to send key-value pairs,
      *                     if not specified the default value serde defined in the configuration will be used
      * @param topic        the topic name
+     *
+     * @return a new {@link KTable} that contains the exact same records as this {@link KTable}
      */
     KTable<K, V> through(Serde<K> keySerde, Serde<V> valSerde, String topic);
 
@@ -147,6 +169,8 @@
      *                     {@link org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner} will be used
      *                     &mdash; otherwise {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} will be used
      * @param topic        the topic name
+     *
+     * @return a new {@link KTable} that contains the exact same records as this {@link KTable}
      */
     KTable<K, V> through(Serde<K> keySerde, Serde<V> valSerde, StreamPartitioner<K, V> partitioner, String topic);
 
@@ -199,16 +223,37 @@
 
     /**
      * Convert this stream to a new instance of {@link KStream}.
+     *
+     * @return a {@link KStream} that contains the same records as this {@link KTable};
+     *         the records are no longer treated as updates on a primary-keyed table,
+     *         but rather as normal key-value pairs in a record stream
      */
     KStream<K, V> toStream();
 
     /**
+     *  Convert this stream to a new instance of {@link KStream} using the given {@link KeyValueMapper} to select
+     *  the new key.
+     *
+     * @param mapper  @param mapper  the instance of {@link KeyValueMapper}
+     * @param <K1> the new key type
+     *
+     * @return a {@link KStream} that contains records with new keys of different type for each update of this {@link KTable}
+     * @return a {@link KStream} that contains the transformed records from this {@link KTable};
+     *         the records are no longer treated as updates on a primary-keyed table,
+     *         but rather as normal key-value pairs in a record stream
+     */
+    <K1> KStream<K1, V> toStream(KeyValueMapper<K, V, K1> mapper);
+
+    /**
      * Combine values of this stream with another {@link KTable} stream's elements of the same key using Inner Join.
      *
      * @param other         the instance of {@link KTable} joined with this stream
      * @param joiner        the instance of {@link ValueJoiner}
      * @param <V1>          the value type of the other stream
      * @param <R>           the value type of the new stream
+     *
+     * @return a {@link KTable} that contains join-records for each key and values computed by the given {@link ValueJoiner},
+     *         one for each matched record-pair with the same key
      */
     <V1, R> KTable<K, R> join(KTable<K, V1> other, ValueJoiner<V, V1, R> joiner);
 
@@ -219,6 +264,9 @@
      * @param joiner        the instance of {@link ValueJoiner}
      * @param <V1>          the value type of the other stream
      * @param <R>           the value type of the new stream
+     *
+     * @return a {@link KTable} that contains join-records for each key and values computed by the given {@link ValueJoiner},
+     *         one for each matched record-pair with the same key
      */
     <V1, R> KTable<K, R> outerJoin(KTable<K, V1> other, ValueJoiner<V, V1, R> joiner);
 
@@ -229,123 +277,43 @@
      * @param joiner        the instance of {@link ValueJoiner}
      * @param <V1>          the value type of the other stream
      * @param <R>           the value type of the new stream
+     *
+     * @return a {@link KTable} that contains join-records for each key and values computed by the given {@link ValueJoiner},
+     *         one for each matched record-pair with the same key
      */
     <V1, R> KTable<K, R> leftJoin(KTable<K, V1> other, ValueJoiner<V, V1, R> joiner);
 
     /**
-     * Combine updating values of this stream by the selected key into a new instance of {@link KTable}.
-     *
-     * @param adder             the instance of {@link Reducer} for addition
-     * @param subtractor        the instance of {@link Reducer} for subtraction
-     * @param selector          the instance of {@link KeyValueMapper} that select the aggregate key
-     * @param keySerde          key serdes for materializing the aggregated table,
-     *                          if not specified the default serdes defined in the configs will be used
-     * @param valueSerde        value serdes for materializing the aggregated table,
-     *                          if not specified the default serdes defined in the configs will be used
-     * @param name              the name of the resulted {@link KTable}
-     * @param <K1>              the key type of the aggregated {@link KTable}
-     * @param <V1>              the value type of the aggregated {@link KTable}
-     */
-    <K1, V1> KTable<K1, V1> reduce(Reducer<V1> adder,
-                                   Reducer<V1> subtractor,
-                                   KeyValueMapper<K, V, KeyValue<K1, V1>> selector,
-                                   Serde<K1> keySerde,
-                                   Serde<V1> valueSerde,
-                                   String name);
-
-    /**
-     * Combine updating values of this stream by the selected key into a new instance of {@link KTable}
-     * using default serializers and deserializers.
-     *
-     * @param adder         the instance of {@link Reducer} for addition
-     * @param subtractor    the instance of {@link Reducer} for subtraction
-     * @param selector      the instance of {@link KeyValueMapper} that select the aggregate key
-     * @param name          the name of the resulted {@link KTable}
-     * @param <K1>          the key type of the aggregated {@link KTable}
-     * @param <V1>          the value type of the aggregated {@link KTable}
-     */
-    <K1, V1> KTable<K1, V1> reduce(Reducer<V1> adder,
-                                   Reducer<V1> subtractor,
-                                   KeyValueMapper<K, V, KeyValue<K1, V1>> selector,
-                                   String name);
-
-    /**
-     * Aggregate updating values of this stream by the selected key into a new instance of {@link KTable}.
-     *
-     * @param initializer   the instance of {@link Initializer}
-     * @param adder         the instance of {@link Aggregator} for addition
-     * @param substractor   the instance of {@link Aggregator} for subtraction
-     * @param selector      the instance of {@link KeyValueMapper} that select the aggregate key
-     * @param keySerde      key serdes for materializing this stream and the aggregated table,
-     *                      if not specified the default serdes defined in the configs will be used
-     * @param valueSerde    value serdes for materializing this stream,
-     *                      if not specified the default serdes defined in the configs will be used
-     * @param aggValueSerde value serdes for materializing the aggregated table,
-     *                      if not specified the default serdes defined in the configs will be used
-     * @param name          the name of the resulted table
-     * @param <K1>          the key type of this {@link KTable}
-     * @param <V1>          the value type of this {@link KTable}
-     * @param <T>           the value type of the aggregated {@link KTable}
-     */
-    <K1, V1, T> KTable<K1, T> aggregate(Initializer<T> initializer,
-                                        Aggregator<K1, V1, T> adder,
-                                        Aggregator<K1, V1, T> substractor,
-                                        KeyValueMapper<K, V, KeyValue<K1, V1>> selector,
-                                        Serde<K1> keySerde,
-                                        Serde<V1> valueSerde,
-                                        Serde<T> aggValueSerde,
-                                        String name);
-
-    /**
-     * Aggregate updating values of this stream by the selected key into a new instance of {@link KTable}
-     * using default serializers and deserializers.
-     *
-     * @param initializer   the instance of {@link Initializer}
-     * @param adder         the instance of {@link Aggregator} for addition
-     * @param substractor   the instance of {@link Aggregator} for subtraction
-     * @param selector      the instance of {@link KeyValueMapper} that select the aggregate key
-     * @param name          the name of the resulted {@link KTable}
-     * @param <K1>          the key type of the aggregated {@link KTable}
-     * @param <V1>          the value type of the aggregated {@link KTable}
-     * @param <T>           the value type of the aggregated {@link KTable}
-     */
-    <K1, V1, T> KTable<K1, T> aggregate(Initializer<T> initializer,
-                                        Aggregator<K1, V1, T> adder,
-                                        Aggregator<K1, V1, T> substractor,
-                                        KeyValueMapper<K, V, KeyValue<K1, V1>> selector,
-                                        String name);
-
-    /**
-     * Count number of records of this stream by the selected key into a new instance of {@link KTable}.
-     *
-     * @param selector      the instance of {@link KeyValueMapper} that select the aggregate key
+     * Group the records of this {@link KTable} using the provided {@link KeyValueMapper}.
+     * 
+     * @param selector      select the grouping key and value to be aggregated
      * @param keySerde      key serdes for materializing this stream,
      *                      if not specified the default serdes defined in the configs will be used
      * @param valueSerde    value serdes for materializing this stream,
      *                      if not specified the default serdes defined in the configs will be used
-     * @param name          the name of the resulted table
-     * @param <K1>          the key type of the aggregated {@link KTable}
+     * @param <K1>          the key type of the {@link KGroupedTable}
+     * @param <V1>          the value type of the {@link KGroupedTable}
+     *
+     * @return a {@link KGroupedTable} that contains the re-partitioned records of this {@link KTable}
      */
-    <K1> KTable<K1, Long> count(KeyValueMapper<K, V, K1> selector,
-                                Serde<K1> keySerde,
-                                Serde<V> valueSerde,
-                                String name);
+    <K1, V1> KGroupedTable<K1, V1> groupBy(KeyValueMapper<K, V, KeyValue<K1, V1>> selector, Serde<K1> keySerde, Serde<V1> valueSerde);
 
     /**
-     * Count number of records of this stream by the selected key into a new instance of {@link KTable}
-     * using default serializers and deserializers.
+     * Group the records of this {@link KTable} using the provided {@link KeyValueMapper} and default serializers and deserializers.
      *
-     * @param selector      the instance of {@link KeyValueMapper} that select the aggregate key
-     * @param name          the name of the resulted {@link KTable}
-     * @param <K1>          the key type of the aggregated {@link KTable}
+     * @param selector      select the grouping key and value to be aggregated
+     * @param <K1>          the key type of the {@link KGroupedTable}
+     * @param <V1>          the value type of the {@link KGroupedTable}
+     *
+     * @return a {@link KGroupedTable} that contains the re-partitioned records of this {@link KTable}
      */
-    <K1> KTable<K1, Long> count(KeyValueMapper<K, V, K1> selector, String name);
+    <K1, V1> KGroupedTable<K1, V1> groupBy(KeyValueMapper<K, V, KeyValue<K1, V1>> selector);
 
     /**
      * Perform an action on each element of {@link KTable}.
      * Note that this is a terminal operation that returns void.
      *
-     * @param action An action to perform on each element
+     * @param action an action to perform on each element
      */
     void foreach(ForeachAction<K, V> action);
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java
index a4aed91..b36ed63 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java
@@ -18,7 +18,7 @@
 package org.apache.kafka.streams.kstream;
 
 /**
- * The KeyValueMapper interface for mapping a key-value pair to a new value (could be another key-value pair).
+ * The {@link KeyValueMapper} interface for mapping a key-value pair to a new value (could be another key-value pair).
  *
  * @param <K>   original key type
  * @param <V>   original value type
@@ -26,5 +26,12 @@
  */
 public interface KeyValueMapper<K, V, R> {
 
+    /**
+     * Map a record with the given key and value to a new value.
+     *
+     * @param key    the key of the record
+     * @param value  the value of the record
+     * @return       the new value
+     */
     R apply(K key, V value);
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java
index c90554b..2df2d5fb 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java
@@ -18,12 +18,19 @@
 package org.apache.kafka.streams.kstream;
 
 /**
- * The Predicate interface represents a predicate (boolean-valued function) of a key-value pair.
+ * The {@link Predicate} interface represents a predicate (boolean-valued function) of a key-value pair.
  *
  * @param <K>   key type
  * @param <V>   value type
  */
 public interface Predicate<K, V> {
 
+    /**
+     * Test if the record with the given key and value satisfies the predicate.
+     *
+     * @param key    the key of the record
+     * @param value  the value of the record
+     * @return       return {@code true} if the key-value pair satisfies the predicate&mdash;{@code false} otherwise
+     */
     boolean test(K key, V value);
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Reducer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Reducer.java
index 551a672..e7cfa0e 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/Reducer.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Reducer.java
@@ -18,11 +18,18 @@
 package org.apache.kafka.streams.kstream;
 
 /**
- * The Reducer interface for combining two values of the same type into a new value.
+ * The {@link Reducer} interface for combining two values of the same type into a new value.
  *
  * @param <V>   value type
  */
 public interface Reducer<V> {
 
+    /**
+     * Aggregate the two given values into a single one.
+     *
+     * @param value1  the first value for the aggregation
+     * @param value2  the second value for the aggregation
+     * @return        the aggregated value
+     */
     V apply(V value1, V value2);
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java
new file mode 100644
index 0000000..e4ce883
--- /dev/null
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java
@@ -0,0 +1,133 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.streams.kstream;
+
+import org.apache.kafka.streams.kstream.internals.TimeWindow;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * The time-based window specifications used for aggregations.
+ * <p>
+ * The semantics of a time-based window are: Every T1 (advance) time-units, compute the aggregate total for T2 (size) time-units.
+ * <ul>
+ *     <li> If {@code advance < size} a hopping windows is defined: <br />
+ *          it discretize a stream into overlapping windows, which implies that a record maybe contained in one and or more "adjacent" windows.</li>
+ *     <li> If {@code advance == size} a tumbling window is defined:<br />
+ *          it discretize a stream into non-overlapping windows, which implies that a record is only ever contained in one and only one tumbling window.</li>
+ * </ul>
+ */
+public class TimeWindows extends Windows<TimeWindow> {
+
+    /**
+     * The size of the window, i.e. how long a window lasts.
+     * The window size's effective time unit is determined by the semantics of the topology's
+     * configured {@link org.apache.kafka.streams.processor.TimestampExtractor}.
+     */
+    public final long size;
+
+    /**
+     * The size of the window's advance interval, i.e. by how much a window moves forward relative
+     * to the previous one. The interval's effective time unit is determined by the semantics of
+     * the topology's configured {@link org.apache.kafka.streams.processor.TimestampExtractor}.
+     */
+    public final long advance;
+
+    private TimeWindows(String name, long size, long advance) {
+        super(name);
+        if (size <= 0) {
+            throw new IllegalArgumentException("window size must be > 0 (you provided " + size + ")");
+        }
+        this.size = size;
+        if (!(0 < advance && advance <= size)) {
+            throw new IllegalArgumentException(
+                String.format("advance interval (%d) must lie within interval (0, %d]", advance, size));
+        }
+        this.advance = advance;
+    }
+
+    /**
+     * Returns a window definition with the given window size, and with the advance interval being
+     * equal to the window size. Think: [N * size, N * size + size), with N denoting the N-th
+     * window.
+     *
+     * This provides the semantics of tumbling windows, which are fixed-sized, gap-less,
+     * non-overlapping windows. Tumbling windows are a specialization of hopping windows.
+     *
+     * @param name The name of the window. Must not be null or empty.
+     * @param size The size of the window, with the requirement that size &gt; 0.
+     *             The window size's effective time unit is determined by the semantics of the
+     *             topology's configured {@link org.apache.kafka.streams.processor.TimestampExtractor}.
+     * @return a new window definition
+     */
+    public static TimeWindows of(String name, long size) {
+        return new TimeWindows(name, size, size);
+    }
+
+    /**
+     * Returns a window definition with the original size, but advance ("hop") the window by the given
+     * interval, which specifies by how much a window moves forward relative to the previous one.
+     * Think: [N * advanceInterval, N * advanceInterval + size), with N denoting the N-th window.
+     *
+     * This provides the semantics of hopping windows, which are fixed-sized, overlapping windows.
+     *
+     * @param interval The advance interval ("hop") of the window, with the requirement that
+     *                 0 &lt; interval &le; size. The interval's effective time unit is
+     *                 determined by the semantics of the topology's configured
+     *                 {@link org.apache.kafka.streams.processor.TimestampExtractor}.
+     * @return a new window definition
+     */
+    public TimeWindows advanceBy(long interval) {
+        return new TimeWindows(this.name, this.size, interval);
+    }
+
+    @Override
+    public Map<Long, TimeWindow> windowsFor(long timestamp) {
+        long enclosed = (size - 1) / advance;
+        long windowStart = Math.max(0, timestamp - timestamp % advance - enclosed * advance);
+
+        Map<Long, TimeWindow> windows = new HashMap<>();
+        while (windowStart <= timestamp) {
+            TimeWindow window = new TimeWindow(windowStart, windowStart + this.size);
+            windows.put(windowStart, window);
+            windowStart += this.advance;
+        }
+        return windows;
+    }
+
+    @Override
+    public final boolean equals(Object o) {
+        if (o == this) {
+            return true;
+        }
+        if (!(o instanceof TimeWindows)) {
+            return false;
+        }
+        TimeWindows other = (TimeWindows) o;
+        return this.size == other.size && this.advance == other.advance;
+    }
+
+    @Override
+    public int hashCode() {
+        int result = (int) (size ^ (size >>> 32));
+        result = 31 * result + (int) (advance ^ (advance >>> 32));
+        return result;
+    }
+
+}
\ No newline at end of file
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java
index 5197e94..239854b 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java
@@ -20,7 +20,7 @@
 import org.apache.kafka.streams.processor.ProcessorContext;
 
 /**
- * A stateful Transformer interface for transform a key-value pair into a new value.
+ * A stateful {@link Transformer} interface for transform a key-value pair into a new value.
  *
  * @param <K>   key type
  * @param <V>   value type
@@ -40,10 +40,10 @@
     void init(ProcessorContext context);
 
     /**
-     * Transform the message with the given key and value.
+     * Transform the record with the given key and value.
      *
-     * @param key the key for the message
-     * @param value the value for the message
+     * @param key the key for the record
+     * @param value the value for the record
      * @return new value; if null no key-value pair will be forwarded to down stream
      */
     R transform(K key, V value);
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java
index fc7ba60..0341702 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java
@@ -18,9 +18,14 @@
 package org.apache.kafka.streams.kstream;
 
 /**
- * A transformer supplier which can create one or more {@link Transformer} instances.
+ * A {@link TransformerSupplier} interface which can create one or more {@link Transformer} instances.
  */
 public interface TransformerSupplier<K, V, R> {
 
+    /**
+     * Return a new {@link Transformer} instance.
+     *
+     * @return  a new {@link Transformer} instance
+     */
     Transformer<K, V, R> get();
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/TumblingWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/TumblingWindows.java
deleted file mode 100644
index cadedba..0000000
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/TumblingWindows.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.streams.kstream;
-
-import org.apache.kafka.streams.kstream.internals.TumblingWindow;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * The tumbling window specifications used for aggregations.
- */
-public class TumblingWindows extends Windows<TumblingWindow> {
-
-    private static final long DEFAULT_SIZE_MS = 1000L;
-
-    public final long size;
-
-    private TumblingWindows(String name, long size) {
-        super(name);
-
-        this.size = size;
-    }
-
-    /**
-     * Returns a half-interval sliding window definition with the default window size
-     */
-    public static TumblingWindows of(String name) {
-        return new TumblingWindows(name, DEFAULT_SIZE_MS);
-    }
-
-    /**
-     * Returns a half-interval sliding window definition with the window size in milliseconds
-     */
-    public TumblingWindows with(long size) {
-        return new TumblingWindows(this.name, size);
-    }
-
-    @Override
-    public Map<Long, TumblingWindow> windowsFor(long timestamp) {
-        long windowStart = timestamp - timestamp % size;
-
-        // we cannot use Collections.singleMap since it does not support remove() call
-        Map<Long, TumblingWindow> windows = new HashMap<>();
-        windows.put(windowStart, new TumblingWindow(windowStart, windowStart + size));
-
-        return windows;
-    }
-
-    @Override
-    public boolean equalTo(Windows other) {
-        if (!other.getClass().equals(TumblingWindows.class))
-            return false;
-
-        TumblingWindows otherWindows = (TumblingWindows) other;
-
-        return this.size == otherWindows.size;
-    }
-}
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java
index 7cadfb4..f45f8c5 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java
@@ -29,21 +29,31 @@
 
     private static final long DEFAULT_START_TIMESTAMP = 0L;
 
+    /** The start timestamp of the window. */
     public final long start;
 
     private UnlimitedWindows(String name, long start) {
         super(name);
 
+        if (start < 0) {
+            throw new IllegalArgumentException("start must be > 0 (you provided " + start + ")");
+        }
         this.start = start;
     }
 
     /**
-     * Returns an unlimited window definition
+     * Return an unlimited window starting at timestamp zero.
      */
     public static UnlimitedWindows of(String name) {
         return new UnlimitedWindows(name, DEFAULT_START_TIMESTAMP);
     }
 
+    /**
+     * Return a new unlimited window for the specified start timestamp.
+     *
+     * @param start  the window start time
+     * @return       a new unlimited window that starts at {@code start}
+     */
     public UnlimitedWindows startOn(long start) {
         return new UnlimitedWindows(this.name, start);
     }
@@ -52,21 +62,31 @@
     public Map<Long, UnlimitedWindow> windowsFor(long timestamp) {
         // always return the single unlimited window
 
-        // we cannot use Collections.singleMap since it does not support remove() call
+        // we cannot use Collections.singleMap since it does not support remove()
         Map<Long, UnlimitedWindow> windows = new HashMap<>();
-        windows.put(start, new UnlimitedWindow(start));
-
-
+        if (timestamp >= start) {
+            windows.put(start, new UnlimitedWindow(start));
+        }
         return windows;
     }
 
     @Override
-    public boolean equalTo(Windows other) {
-        if (!other.getClass().equals(UnlimitedWindows.class))
+    public final boolean equals(Object o) {
+        if (o == this) {
+            return true;
+        }
+
+        if (!(o instanceof UnlimitedWindows)) {
             return false;
+        }
 
-        UnlimitedWindows otherWindows = (UnlimitedWindows) other;
-
-        return this.start == otherWindows.start;
+        UnlimitedWindows other = (UnlimitedWindows) o;
+        return this.start == other.start;
     }
-}
+
+    @Override
+    public int hashCode() {
+        return (int) (start ^ (start >>> 32));
+    }
+
+}
\ No newline at end of file
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java
index 5f00a1a..8d4a8e7 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java
@@ -18,7 +18,7 @@
 package org.apache.kafka.streams.kstream;
 
 /**
- * The ValueJoiner interface for joining two values and return a the joined new value.
+ * The {@link ValueJoiner} interface for joining two values into a new value.
  *
  * @param <V1>  first value type
  * @param <V2>  second value type
@@ -26,5 +26,12 @@
  */
 public interface ValueJoiner<V1, V2, R> {
 
+    /**
+     * Return a joined value consisting of {@code value1} and {@code value2}.
+     *
+     * @param value1  the first value for joining
+     * @param value2  the second value for joining
+     * @return        the joined value
+     */
     R apply(V1 value1, V2 value2);
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java
index 6e62a55..e168e37 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java
@@ -18,12 +18,18 @@
 package org.apache.kafka.streams.kstream;
 
 /**
- * The KeyValueMapper interface for mapping an original value to a new value (could be another key-value pair).
+ * The {@link ValueMapper} interface for mapping an original value to a new value (could be another key-value pair).
  *
  * @param <V1>  original value type
  * @param <V2>  mapped value type
  */
 public interface ValueMapper<V1, V2> {
 
+    /**
+     * Map the given value to a new value.
+     *
+     * @param value  the value to be mapped
+     * @return       the new value
+     */
     V2 apply(V1 value);
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java
index 63214fd..f92d9a1 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java
@@ -20,7 +20,7 @@
 import org.apache.kafka.streams.processor.ProcessorContext;
 
 /**
- * A stateful Value Transformer interface for transform a value into a new value.
+ * A stateful {@link ValueTransformer} interface to transform a value into a new value.
  *
  * @param <V>   value type
  * @param <R>   return type
@@ -31,7 +31,7 @@
      * Initialize this transformer with the given context. The framework ensures this is called once per processor when the topology
      * that contains it is initialized.
      * <p>
-     * If this tranformer is to be {@link #punctuate(long) called periodically} by the framework, then this method should
+     * If this transformer is to be {@link #punctuate(long) called periodically} by the framework, then this method should
      * {@link ProcessorContext#schedule(long) schedule itself} with the provided context.
      *
      * @param context the context; may not be null
@@ -39,9 +39,9 @@
     void init(ProcessorContext context);
 
     /**
-     * Transform the message with the given key and value.
+     * Transform the record with the given key and value.
      *
-     * @param value the value for the message
+     * @param value the value for the record
      * @return new value
      */
     R transform(V value);
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerSupplier.java
index 6bc86bc..ecd454a 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerSupplier.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerSupplier.java
@@ -18,9 +18,14 @@
 package org.apache.kafka.streams.kstream;
 
 /**
- * A value transformer supplier which can create one or more {@link ValueTransformer} instances.
+ * A {@link ValueTransformerSupplier} interface which can create one or more {@link ValueTransformer} instances.
  */
 public interface ValueTransformerSupplier<V, R> {
 
+    /**
+     * Return a new {@link ValueTransformer} instance.
+     *
+     * @return  a new {@link ValueTransformer} instance.
+     */
     ValueTransformer<V, R> get();
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java
index f2965dc..e1ea9a0 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java
@@ -25,44 +25,53 @@
     private long start;
     private long end;
 
+    /**
+     * Create a new window for the given start time (inclusive) and end time (exclusive).
+     *
+     * @param start  the start timestamp of the window (inclusive)
+     * @param end    the end timestamp of the window (exclusive)
+     */
     public Window(long start, long end) {
         this.start = start;
         this.end = end;
     }
 
     /**
-     * Returns the start timestamp of this window, inclusive
+     * Return the start timestamp of this window, inclusive
      */
     public long start() {
         return start;
     }
 
     /**
-     * Returns the end timestamp of this window, exclusive
+     * Return the end timestamp of this window, exclusive
      */
     public long end() {
         return end;
     }
 
+    /**
+     * Check if the given window overlaps with this window.
+     *
+     * @param other  another window
+     * @return       {@code true} if {@code other} overlaps with this window&mdash;{@code false} otherwise
+     */
     public boolean overlap(Window other) {
         return this.start() < other.end() || other.start() < this.end();
     }
 
-    public boolean equalsTo(Window other) {
-        return this.start() == other.start() && this.end() == other.end();
-    }
-
     @Override
     public boolean equals(Object obj) {
-        if (obj == this)
+        if (obj == this) {
             return true;
+        }
 
-        if (!(obj instanceof Window))
+        if (getClass() != obj.getClass()) {
             return false;
+        }
 
         Window other = (Window) obj;
-
-        return this.equalsTo(other) && this.start == other.start && this.end == other.end;
+        return this.start == other.start && this.end == other.end;
     }
 
     @Override
@@ -70,4 +79,5 @@
         long n = (this.start << 32) | this.end;
         return (int) (n % 0xFFFFFFFFL);
     }
-}
\ No newline at end of file
+
+}
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java
index 3691282..feaf6a3 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java
@@ -22,30 +22,40 @@
  * i.e. {@link KStream#aggregateByKey(Initializer, Aggregator, Windows, org.apache.kafka.common.serialization.Serde,
  * org.apache.kafka.common.serialization.Serde)}
  *
- * @param <T> Type of the key
+ * @param <K> Type of the key
  */
-public class Windowed<T> {
+public class Windowed<K> {
 
-    private T value;
+    private K key;
 
     private Window window;
 
-    public Windowed(T value, Window window) {
-        this.value = value;
+    public Windowed(K key, Window window) {
+        this.key = key;
         this.window = window;
     }
 
-    public T value() {
-        return value;
+    /**
+     * Return the key of the window.
+     *
+     * @return the key of the window
+     */
+    public K key() {
+        return key;
     }
 
+    /**
+     * Return the window containing the values associated with this key.
+     *
+     * @return  the window containing the values
+     */
     public Window window() {
         return window;
     }
 
     @Override
     public String toString() {
-        return "[" + value + "@" + window.start() + "]";
+        return "[" + key + "@" + window.start() + "]";
     }
 
     @Override
@@ -58,12 +68,12 @@
 
         Windowed<?> that = (Windowed) obj;
 
-        return this.window.equals(that.window) && this.value.equals(that.value);
+        return this.window.equals(that.window) && this.key.equals(that.key);
     }
 
     @Override
     public int hashCode() {
-        long n = ((long) window.hashCode() << 32) | value.hashCode();
+        long n = ((long) window.hashCode() << 32) | key.hashCode();
         return (int) (n % 0xFFFFFFFFL);
     }
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java
index e7dc23e..06cacb4 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java
@@ -30,24 +30,22 @@
 
     private static final int DEFAULT_NUM_SEGMENTS = 3;
 
-    private static final long DEFAULT_EMIT_DURATION = 1000L;
-
     private static final long DEFAULT_MAINTAIN_DURATION = 24 * 60 * 60 * 1000L;   // one day
 
     private static final AtomicInteger NAME_INDEX = new AtomicInteger(0);
 
     protected String name;
 
-    private long emitDurationMs;
-
     private long maintainDurationMs;
 
     public int segments;
 
     protected Windows(String name) {
+        if (name == null || name.isEmpty()) {
+            throw new IllegalArgumentException("name must not be null or empty");
+        }
         this.name = name;
         this.segments = DEFAULT_NUM_SEGMENTS;
-        this.emitDurationMs = DEFAULT_EMIT_DURATION;
         this.maintainDurationMs = DEFAULT_MAINTAIN_DURATION;
     }
 
@@ -56,16 +54,9 @@
     }
 
     /**
-     * Set the window emit duration in milliseconds of system time.
-     */
-    public Windows emit(long durationMs) {
-        this.emitDurationMs = durationMs;
-
-        return this;
-    }
-
-    /**
      * Set the window maintain duration in milliseconds of system time.
+     *
+     * @return  itself
      */
     public Windows until(long durationMs) {
         this.maintainDurationMs = durationMs;
@@ -76,6 +67,8 @@
     /**
      * Specify the number of segments to be used for rolling the window store,
      * this function is not exposed to users but can be called by developers that extend this JoinWindows specs.
+     *
+     * @return  itself
      */
     protected Windows segments(int segments) {
         this.segments = segments;
@@ -83,19 +76,21 @@
         return this;
     }
 
-    public long emitEveryMs() {
-        return this.emitDurationMs;
-    }
-
+    /**
+     * Return the window maintain duration in milliseconds of system time.
+     *
+     * @return the window maintain duration in milliseconds of system time
+     */
     public long maintainMs() {
         return this.maintainDurationMs;
     }
 
-    protected String newName(String prefix) {
-        return prefix + String.format("%010d", NAME_INDEX.getAndIncrement());
-    }
-
-    public abstract boolean equalTo(Windows other);
-
+    /**
+     * Creates all windows that contain the provided timestamp.
+     *
+     * @param timestamp  the timestamp window should get created for
+     * @return  a map of {@code windowStartTimestamp -> Window} entries
+     */
     public abstract Map<Long, W> windowsFor(long timestamp);
+
 }
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 c537465..ebada92 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
@@ -36,6 +36,9 @@
         this.sourceNodes = sourceNodes;
     }
 
+    /**
+     * @throws TopologyBuilderException if the streams are not joinable
+     */
     protected Set<String> ensureJoinableWith(AbstractStream<K> other) {
         Set<String> thisSourceNodes = sourceNodes;
         Set<String> otherSourceNodes = other.sourceNodes;
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 d4c4e2d..ce9be49 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
@@ -26,12 +26,20 @@
 
     private static final int NEWFLAG_SIZE = 1;
 
-    private final Deserializer<T> inner;
+    private Deserializer<T> inner;
 
     public ChangedDeserializer(Deserializer<T> inner) {
         this.inner = inner;
     }
 
+    public Deserializer<T> inner() {
+        return inner;
+    }
+
+    public void setInner(Deserializer<T> inner) {
+        this.inner = inner;
+    }
+
     @Override
     public void configure(Map<String, ?> configs, boolean isKey) {
         // do nothing
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 5ea0791..12e06f8 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
@@ -27,17 +27,29 @@
 
     private static final int NEWFLAG_SIZE = 1;
 
-    private final Serializer<T> inner;
+    private Serializer<T> inner;
 
     public ChangedSerializer(Serializer<T> inner) {
         this.inner = inner;
     }
 
+    public Serializer<T> inner() {
+        return inner;
+    }
+
+    public void setInner(Serializer<T> inner) {
+        this.inner = inner;
+    }
+
     @Override
     public void configure(Map<String, ?> configs, boolean isKey) {
         // do nothing
     }
 
+    /**
+     * @throws StreamsException if both old and new values of data are null, or if
+     * both values are not null
+     */
     @Override
     public byte[] serialize(String topic, Change<T> data) {
         byte[] serializedKey;
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
new file mode 100644
index 0000000..f7fe4e5
--- /dev/null
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java
@@ -0,0 +1,153 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.streams.kstream.Aggregator;
+import org.apache.kafka.streams.kstream.Initializer;
+import org.apache.kafka.streams.kstream.KGroupedTable;
+import org.apache.kafka.streams.kstream.KStreamBuilder;
+import org.apache.kafka.streams.kstream.KTable;
+import org.apache.kafka.streams.kstream.Reducer;
+import org.apache.kafka.streams.processor.ProcessorSupplier;
+import org.apache.kafka.streams.processor.StateStoreSupplier;
+import org.apache.kafka.streams.state.Stores;
+
+import java.util.Collections;
+
+/**
+ * The implementation class of {@link KGroupedTable}.
+ * 
+ * @param <K> the key type
+ * @param <V> the value type
+ */
+public class KGroupedTableImpl<K, V> extends AbstractStream<K> implements KGroupedTable<K, V> {
+
+    private static final String AGGREGATE_NAME = "KTABLE-AGGREGATE-";
+
+    private static final String REDUCE_NAME = "KTABLE-REDUCE-";
+
+    private static final String REPARTITION_TOPIC_SUFFIX = "-repartition";
+
+    protected final Serde<K> keySerde;
+    protected final Serde<V> valSerde;
+
+    public KGroupedTableImpl(KStreamBuilder topology,
+                             String name,
+                             String sourceName,
+                             Serde<K> keySerde,
+                             Serde<V> valSerde) {
+        super(topology, name, Collections.singleton(sourceName));
+        this.keySerde = keySerde;
+        this.valSerde = valSerde;
+    }
+
+    @Override
+    public <T> KTable<K, T> aggregate(Initializer<T> initializer,
+                                      Aggregator<K, V, T> adder,
+                                      Aggregator<K, V, T> subtractor,
+                                      Serde<T> aggValueSerde,
+                                      String name) {
+
+        ProcessorSupplier<K, Change<V>> aggregateSupplier = new KTableAggregate<>(name, initializer, adder, subtractor);
+        return doAggregate(aggregateSupplier, aggValueSerde, AGGREGATE_NAME, name);
+    }
+
+    @Override
+    public <T> KTable<K, T> aggregate(Initializer<T> initializer,
+                            Aggregator<K, V, T> adder,
+                            Aggregator<K, V, T> substractor,
+                            String name) {
+
+        return aggregate(initializer, adder, substractor, null, name);
+    }
+
+    private <T> KTable<K, T> doAggregate(ProcessorSupplier<K, Change<V>> aggregateSupplier,
+                                         Serde<T> aggValueSerde,
+                                         String functionName,
+                                         String name) {
+        String sinkName = topology.newName(KStreamImpl.SINK_NAME);
+        String sourceName = topology.newName(KStreamImpl.SOURCE_NAME);
+        String funcName = topology.newName(functionName);
+
+        String topic = name + REPARTITION_TOPIC_SUFFIX;
+
+        Serializer<K> keySerializer = keySerde == null ? null : keySerde.serializer();
+        Deserializer<K> keyDeserializer = keySerde == null ? null : keySerde.deserializer();
+        Serializer<V> valueSerializer = valSerde == null ? null : valSerde.serializer();
+        Deserializer<V> valueDeserializer = valSerde == null ? null : valSerde.deserializer();
+
+        ChangedSerializer<V> changedValueSerializer = new ChangedSerializer<>(valueSerializer);
+        ChangedDeserializer<V> changedValueDeserializer = new ChangedDeserializer<>(valueDeserializer);
+
+        StateStoreSupplier aggregateStore = Stores.create(name)
+            .withKeys(keySerde)
+            .withValues(aggValueSerde)
+            .persistent()
+            .build();
+
+        // send the aggregate key-value pairs to the intermediate topic for partitioning
+        topology.addInternalTopic(topic);
+        topology.addSink(sinkName, topic, keySerializer, changedValueSerializer, this.name);
+
+        // read the intermediate topic
+        topology.addSource(sourceName, keyDeserializer, changedValueDeserializer, topic);
+
+        // aggregate the values with the aggregator and local store
+        topology.addProcessor(funcName, aggregateSupplier, sourceName);
+        topology.addStateStore(aggregateStore, funcName);
+
+        // return the KTable representation with the intermediate topic as the sources
+        return new KTableImpl<>(topology, funcName, aggregateSupplier, Collections.singleton(sourceName));
+    }
+
+    @Override
+    public KTable<K, V> reduce(Reducer<V> adder,
+                               Reducer<V> subtractor,
+                               String name) {
+        ProcessorSupplier<K, Change<V>> aggregateSupplier = new KTableReduce<>(name, adder, subtractor);
+        return doAggregate(aggregateSupplier, valSerde, REDUCE_NAME, name);
+    }
+
+    @Override
+    public KTable<K, Long> count(String name) {
+        return this.aggregate(
+                new Initializer<Long>() {
+                    @Override
+                    public Long apply() {
+                        return 0L;
+                    }
+                },
+                new Aggregator<K, V, Long>() {
+                    @Override
+                    public Long apply(K aggKey, V value, Long aggregate) {
+                        return aggregate + 1L;
+                    }
+                }, new Aggregator<K, V, Long>() {
+                    @Override
+                    public Long apply(K aggKey, V value, Long aggregate) {
+                        return aggregate - 1L;
+                    }
+                },
+                Serdes.Long(), name);
+    }
+
+}
\ No newline at end of file
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java
index 871a12d..b6d1492 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java
@@ -61,6 +61,9 @@
             store = (KeyValueStore<K, T>) context.getStateStore(storeName);
         }
 
+        /**
+         * @throws StreamsException if key is null
+         */
         @Override
         public void process(K key, V value) {
             // the keys should never be null
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 97a7aac..91bcef9 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
@@ -86,6 +86,8 @@
 
     private static final String REDUCE_NAME = "KSTREAM-REDUCE-";
 
+    private static final String KEY_SELECT_NAME = "KSTREAM-KEY-SELECT-";
+
     public static final String SINK_NAME = "KSTREAM-SINK-";
 
     public static final String SOURCE_NAME = "KSTREAM-SOURCE-";
@@ -121,6 +123,19 @@
     }
 
     @Override
+    @SuppressWarnings("unchecked")
+    public <K1> KStream<K1, V> selectKey(final KeyValueMapper<K, V, K1> mapper) {
+        String name = topology.newName(KEY_SELECT_NAME);
+        topology.addProcessor(name, new KStreamMap<>(new KeyValueMapper<K, V, KeyValue<K1, V>>() {
+            @Override
+            public KeyValue<K1, V> apply(K key, V value) {
+                return new KeyValue(mapper.apply(key, value), value);
+            }
+        }), this.name);
+        return new KStreamImpl<>(topology, name, sourceNodes);
+    }
+
+    @Override
     public <K1, V1> KStream<K1, V1> map(KeyValueMapper<K, V, KeyValue<K1, V1>> mapper) {
         String name = topology.newName(MAP_NAME);
 
@@ -155,6 +170,9 @@
         writeAsText(filePath, null, null);
     }
 
+    /**
+     * @throws TopologyBuilderException if file is not found
+     */
     @Override
     public void writeAsText(String filePath, Serde<K> keySerde, Serde<V> valSerde) {
         String name = topology.newName(PRINTING_NAME);
@@ -280,7 +298,7 @@
         String name = topology.newName(SINK_NAME);
 
         Serializer<K> keySerializer = keySerde == null ? null : keySerde.serializer();
-        Serializer<V> valSerializer = keySerde == null ? null : valSerde.serializer();
+        Serializer<V> valSerializer = valSerde == null ? null : valSerde.serializer();
         
         if (partitioner == null && keySerializer != null && keySerializer instanceof WindowedSerializer) {
             WindowedSerializer<Object> windowedSerializer = (WindowedSerializer<Object>) keySerializer;
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 94e0b88..864dc9c 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
@@ -28,6 +28,9 @@
 
     private final String windowName;
 
+    /**
+     * @throws TopologyBuilderException if retention period of the join window is less than expected
+     */
     KStreamJoinWindow(String windowName, long windowSizeMs, long retentionPeriodMs) {
         this.windowName = windowName;
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java
index d8caf3c..d13d112 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java
@@ -62,6 +62,9 @@
             otherWindow = (WindowStore<K, V2>) context.getStateStore(otherWindowName);
         }
 
+        /**
+         * @throws StreamsException if key is null
+         */
         @Override
         public void process(K key, V1 value) {
             // the keys should never be null
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java
index e37fe34..ed6e216 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java
@@ -58,6 +58,9 @@
             store = (KeyValueStore<K, V>) context.getStateStore(storeName);
         }
 
+        /**
+         * @throws StreamsException if key is null
+         */
         @Override
         public void process(K key, V value) {
             // the keys should never be null
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java
index f36cc8c..b4272f8 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java
@@ -163,7 +163,7 @@
         @SuppressWarnings("unchecked")
         @Override
         public T get(Windowed<K> windowedKey) {
-            K key = windowedKey.value();
+            K key = windowedKey.key();
             W window = (W) windowedKey.window();
 
             // this iterator should contain at most one element
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduce.java
index 6c05ce3..3ed1499 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduce.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduce.java
@@ -157,7 +157,7 @@
         @SuppressWarnings("unchecked")
         @Override
         public V get(Windowed<K> windowedKey) {
-            K key = windowedKey.value();
+            K key = windowedKey.key();
             W window = (W) windowedKey.window();
 
             // this iterator should only contain one element
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java
index 806c6e9..4a7c7c0 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java
@@ -63,6 +63,9 @@
             store = (KeyValueStore<K, T>) context.getStateStore(storeName);
         }
 
+        /**
+         * @throws StreamsException if key is null
+         */
         @Override
         public void process(K key, Change<V> value) {
             // the keys should never be null
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java
index ee2c931..51d4cb4 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java
@@ -18,45 +18,38 @@
 package org.apache.kafka.streams.kstream.internals;
 
 import org.apache.kafka.common.serialization.Serde;
-import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.errors.TopologyBuilderException;
-import org.apache.kafka.streams.kstream.Aggregator;
 import org.apache.kafka.streams.kstream.ForeachAction;
-import org.apache.kafka.streams.kstream.Initializer;
 import org.apache.kafka.streams.kstream.KStream;
 import org.apache.kafka.streams.kstream.KStreamBuilder;
 import org.apache.kafka.streams.kstream.KTable;
+import org.apache.kafka.streams.kstream.KGroupedTable;
 import org.apache.kafka.streams.kstream.KeyValueMapper;
 import org.apache.kafka.streams.kstream.Predicate;
-import org.apache.kafka.streams.kstream.Reducer;
 import org.apache.kafka.streams.kstream.ValueJoiner;
 import org.apache.kafka.streams.kstream.ValueMapper;
 import org.apache.kafka.streams.processor.ProcessorSupplier;
 import org.apache.kafka.streams.processor.StateStoreSupplier;
 import org.apache.kafka.streams.processor.StreamPartitioner;
-import org.apache.kafka.streams.state.Stores;
 
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.PrintStream;
-import java.util.Collections;
 import java.util.Set;
 
 /**
- * The implementation class of KTable
+ * The implementation class of {@link KTable}.
  * @param <K> the key type
  * @param <S> the source's (parent's) value type
  * @param <V> the value type
  */
 public class KTableImpl<K, S, V> extends AbstractStream<K> implements KTable<K, V> {
 
-    private static final String REPARTITION_TOPIC_SUFFIX = "-repartition";
-
-    private static final String AGGREGATE_NAME = "KTABLE-AGGREGATE-";
-
     private static final String FILTER_NAME = "KTABLE-FILTER-";
 
+    private static final String FOREACH_NAME = "KTABLE-FOREACH-";
+
     public static final String JOINTHIS_NAME = "KTABLE-JOINTHIS-";
 
     public static final String JOINOTHER_NAME = "KTABLE-JOINOTHER-";
@@ -75,16 +68,12 @@
 
     private static final String PRINTING_NAME = "KSTREAM-PRINTER-";
 
-    private static final String REDUCE_NAME = "KTABLE-REDUCE-";
-
     private static final String SELECT_NAME = "KTABLE-SELECT-";
 
     public static final String SOURCE_NAME = "KTABLE-SOURCE-";
 
     private static final String TOSTREAM_NAME = "KTABLE-TOSTREAM-";
 
-    private static final String FOREACH_NAME = "KTABLE-FOREACH-";
-
     public final ProcessorSupplier<?, ?> processorSupplier;
 
     private final Serde<K> keySerde;
@@ -157,6 +146,9 @@
         writeAsText(filePath, null, null);
     }
 
+    /**
+     * @throws TopologyBuilderException if file is not found
+     */
     @Override
     public void writeAsText(String filePath, Serde<K> keySerde, Serde<V> valSerde) {
         String name = topology.newName(PRINTING_NAME);
@@ -169,7 +161,6 @@
         }
     }
 
-
     @Override
     public KTable<K, V> through(Serde<K> keySerde,
                                 Serde<V> valSerde,
@@ -241,6 +232,11 @@
         return new KStreamImpl<>(topology, name, sourceNodes);
     }
 
+    @Override
+    public <K1> KStream<K1, V> toStream(KeyValueMapper<K, V, K1> mapper) {
+        return toStream().selectKey(mapper);
+    }
+
     @SuppressWarnings("unchecked")
     @Override
     public <V1, R> KTable<K, R> join(KTable<K, V1> other, ValueJoiner<V, V1, R> joiner) {
@@ -311,154 +307,24 @@
     }
 
     @Override
-    public <K1, V1, T> KTable<K1, T> aggregate(Initializer<T> initializer,
-                                               Aggregator<K1, V1, T> adder,
-                                               Aggregator<K1, V1, T> subtractor,
-                                               KeyValueMapper<K, V, KeyValue<K1, V1>> selector,
-                                               Serde<K1> keySerde,
-                                               Serde<V1> valueSerde,
-                                               Serde<T> aggValueSerde,
-                                               String name) {
+    public <K1, V1> KGroupedTable<K1, V1> groupBy(KeyValueMapper<K, V, KeyValue<K1, V1>> selector,
+                                                  Serde<K1> keySerde,
+                                                  Serde<V1> valueSerde) {
 
         String selectName = topology.newName(SELECT_NAME);
-        String sinkName = topology.newName(KStreamImpl.SINK_NAME);
-        String sourceName = topology.newName(KStreamImpl.SOURCE_NAME);
-        String aggregateName = topology.newName(AGGREGATE_NAME);
-
-        String topic = name + REPARTITION_TOPIC_SUFFIX;
-
-        ChangedSerializer<V1> changedValueSerializer = new ChangedSerializer<>(valueSerde.serializer());
-        ChangedDeserializer<V1> changedValueDeserializer = new ChangedDeserializer<>(valueSerde.deserializer());
 
         KTableProcessorSupplier<K, V, KeyValue<K1, V1>> selectSupplier = new KTableRepartitionMap<>(this, selector);
 
-        ProcessorSupplier<K1, Change<V1>> aggregateSupplier = new KTableAggregate<>(name, initializer, adder, subtractor);
-
-        StateStoreSupplier aggregateStore = Stores.create(name)
-                .withKeys(keySerde)
-                .withValues(aggValueSerde)
-                .persistent()
-                .build();
-
         // select the aggregate key and values (old and new), it would require parent to send old values
         topology.addProcessor(selectName, selectSupplier, this.name);
         this.enableSendingOldValues();
 
-        // send the aggregate key-value pairs to the intermediate topic for partitioning
-        topology.addInternalTopic(topic);
-        topology.addSink(sinkName, topic, keySerde.serializer(), changedValueSerializer, selectName);
-
-        // read the intermediate topic
-        topology.addSource(sourceName, keySerde.deserializer(), changedValueDeserializer, topic);
-
-        // aggregate the values with the aggregator and local store
-        topology.addProcessor(aggregateName, aggregateSupplier, sourceName);
-        topology.addStateStore(aggregateStore, aggregateName);
-
-        // return the KTable representation with the intermediate topic as the sources
-        return new KTableImpl<>(topology, aggregateName, aggregateSupplier, Collections.singleton(sourceName));
+        return new KGroupedTableImpl<>(topology, selectName, this.name, keySerde, valueSerde);
     }
 
     @Override
-    public <K1, V1, T> KTable<K1, T> aggregate(Initializer<T> initializer,
-                                               Aggregator<K1, V1, T> adder,
-                                               Aggregator<K1, V1, T> substractor,
-                                               KeyValueMapper<K, V, KeyValue<K1, V1>> selector,
-                                               String name) {
-
-        return aggregate(initializer, adder, substractor, selector, null, null, null, name);
-    }
-
-    @Override
-    public <K1> KTable<K1, Long> count(final KeyValueMapper<K, V, K1> selector,
-                                       Serde<K1> keySerde,
-                                       Serde<V> valueSerde,
-                                       String name) {
-        return this.aggregate(
-                new Initializer<Long>() {
-                    @Override
-                    public Long apply() {
-                        return 0L;
-                    }
-                },
-                new Aggregator<K1, V, Long>() {
-                    @Override
-                    public Long apply(K1 aggKey, V value, Long aggregate) {
-                        return aggregate + 1L;
-                    }
-                }, new Aggregator<K1, V, Long>() {
-                    @Override
-                    public Long apply(K1 aggKey, V value, Long aggregate) {
-                        return aggregate - 1L;
-                    }
-                }, new KeyValueMapper<K, V, KeyValue<K1, V>>() {
-                    @Override
-                    public KeyValue<K1, V> apply(K key, V value) {
-                        return new KeyValue<>(selector.apply(key, value), value);
-                    }
-                },
-                keySerde, valueSerde, Serdes.Long(), name);
-    }
-
-    @Override
-    public <K1> KTable<K1, Long> count(final KeyValueMapper<K, V, K1> selector, String name) {
-        return count(selector, null, null, name);
-    }
-
-    @Override
-    public <K1, V1> KTable<K1, V1> reduce(Reducer<V1> adder,
-                                          Reducer<V1> subtractor,
-                                          KeyValueMapper<K, V, KeyValue<K1, V1>> selector,
-                                          Serde<K1> keySerde,
-                                          Serde<V1> valueSerde,
-                                          String name) {
-
-        String selectName = topology.newName(SELECT_NAME);
-        String sinkName = topology.newName(KStreamImpl.SINK_NAME);
-        String sourceName = topology.newName(KStreamImpl.SOURCE_NAME);
-        String reduceName = topology.newName(REDUCE_NAME);
-
-        String topic = name + REPARTITION_TOPIC_SUFFIX;
-
-        ChangedSerializer<V1> changedValueSerializer = new ChangedSerializer<>(valueSerde.serializer());
-        ChangedDeserializer<V1> changedValueDeserializer = new ChangedDeserializer<>(valueSerde.deserializer());
-
-        KTableProcessorSupplier<K, V, KeyValue<K1, V1>> selectSupplier = new KTableRepartitionMap<>(this, selector);
-
-        ProcessorSupplier<K1, Change<V1>> aggregateSupplier = new KTableReduce<>(name, adder, subtractor);
-
-        StateStoreSupplier aggregateStore = Stores.create(name)
-                .withKeys(keySerde)
-                .withValues(valueSerde)
-                .persistent()
-                .build();
-
-        // select the aggregate key and values (old and new), it would require parent to send old values
-        topology.addProcessor(selectName, selectSupplier, this.name);
-        this.enableSendingOldValues();
-
-        // send the aggregate key-value pairs to the intermediate topic for partitioning
-        topology.addInternalTopic(topic);
-        topology.addSink(sinkName, topic, keySerde.serializer(), changedValueSerializer, selectName);
-
-        // read the intermediate topic
-        topology.addSource(sourceName, keySerde.deserializer(), changedValueDeserializer, topic);
-
-        // aggregate the values with the aggregator and local store
-        topology.addProcessor(reduceName, aggregateSupplier, sourceName);
-        topology.addStateStore(aggregateStore, reduceName);
-
-        // return the KTable representation with the intermediate topic as the sources
-        return new KTableImpl<>(topology, reduceName, aggregateSupplier, Collections.singleton(sourceName));
-    }
-
-    @Override
-    public <K1, V1> KTable<K1, V1> reduce(Reducer<V1> adder,
-                                          Reducer<V1> subtractor,
-                                          KeyValueMapper<K, V, KeyValue<K1, V1>> selector,
-                                          String name) {
-
-        return reduce(adder, subtractor, selector, null, null, name);
+    public <K1, V1> KGroupedTable<K1, V1> groupBy(KeyValueMapper<K, V, KeyValue<K1, V1>> selector) {
+        return this.groupBy(selector, null, null);
     }
 
     @SuppressWarnings("unchecked")
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java
index 24c8da6..36424d1 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java
@@ -60,6 +60,9 @@
             valueGetter.init(context);
         }
 
+        /**
+         * @throws StreamsException if key is null
+         */
         @Override
         public void process(K key, Change<V1> change) {
             // the keys should never be null
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java
index 4bf45ed..996ebc3 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java
@@ -60,6 +60,9 @@
             valueGetter.init(context);
         }
 
+        /**
+         * @throws StreamsException if key is null
+         */
         @Override
         public void process(K key, Change<V1> change) {
             // the keys should never be null
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java
index 49eed53..2a0d477 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java
@@ -60,6 +60,9 @@
             valueGetter.init(context);
         }
 
+        /**
+         * @throws StreamsException if key is null
+         */
         @Override
         public void process(K key, Change<V1> change) {
             // the keys should never be null
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java
index 7443d4a..fa41ed3 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java
@@ -61,6 +61,9 @@
             valueGetter.init(context);
         }
 
+        /**
+         * @throws StreamsException if key is null
+         */
         @Override
         public void process(K key, Change<V1> change) {
             // the keys should never be null
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java
index d56b3ae..bab6bf3 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java
@@ -60,6 +60,9 @@
             store = (KeyValueStore<K, V>) context.getStateStore(storeName);
         }
 
+        /**
+         * @throws StreamsException if key is null
+         */
         @Override
         public void process(K key, Change<V> value) {
             // the keys should never be null
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java
index 142a279..2a7cf1b 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java
@@ -57,6 +57,9 @@
         };
     }
 
+    /**
+     * @throws IllegalStateException since this method should never be called
+     */
     @Override
     public void enableSendingOldValues() {
         // this should never be called
@@ -74,6 +77,9 @@
 
     private class KTableMapProcessor extends AbstractProcessor<K, Change<V>> {
 
+        /**
+         * @throws StreamsException if key is null
+         */
         @Override
         public void process(K key, Change<V> change) {
             KeyValue<K1, V1> newPair = computeValue(key, change.newValue);
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/HoppingWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindow.java
similarity index 75%
rename from streams/src/main/java/org/apache/kafka/streams/kstream/internals/HoppingWindow.java
rename to streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindow.java
index 8b0b2fb..5dfb9eb 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/HoppingWindow.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindow.java
@@ -19,19 +19,15 @@
 
 import org.apache.kafka.streams.kstream.Window;
 
-public class HoppingWindow extends Window {
+public class TimeWindow extends Window {
 
-    public HoppingWindow(long start, long end) {
+    public TimeWindow(long start, long end) {
         super(start, end);
     }
 
     @Override
     public boolean overlap(Window other) {
-        return super.overlap(other) && other.getClass().equals(HoppingWindow.class);
+        return getClass() == other.getClass() && super.overlap(other);
     }
 
-    @Override
-    public boolean equalsTo(Window other) {
-        return super.equalsTo(other) && other.getClass().equals(HoppingWindow.class);
-    }
-}
+}
\ No newline at end of file
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TumblingWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TumblingWindow.java
deleted file mode 100644
index a02d4b9..0000000
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TumblingWindow.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.streams.kstream.internals;
-
-
-import org.apache.kafka.streams.kstream.Window;
-
-public class TumblingWindow extends Window {
-
-    public TumblingWindow(long start, long end) {
-        super(start, end);
-    }
-
-    @Override
-    public boolean overlap(Window other) {
-        return super.overlap(other) && other.getClass().equals(TumblingWindow.class);
-    }
-
-    @Override
-    public boolean equalsTo(Window other) {
-        return super.equalsTo(other) && other.getClass().equals(TumblingWindow.class);
-    }
-}
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindow.java
index 8ac8f70..4b93f9b 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindow.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindow.java
@@ -27,11 +27,7 @@
 
     @Override
     public boolean overlap(Window other) {
-        return super.overlap(other) && other.getClass().equals(UnlimitedWindow.class);
+        return getClass() == other.getClass() && super.overlap(other);
     }
 
-    @Override
-    public boolean equalsTo(Window other) {
-        return super.equalsTo(other) && other.getClass().equals(UnlimitedWindow.class);
-    }
-}
+}
\ No newline at end of file
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedSerializer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedSerializer.java
index 0afcad1..2e19816 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedSerializer.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedSerializer.java
@@ -40,7 +40,7 @@
 
     @Override
     public byte[] serialize(String topic, Windowed<T> data) {
-        byte[] serializedKey = inner.serialize(topic, data.value());
+        byte[] serializedKey = inner.serialize(topic, data.key());
 
         ByteBuffer buf = ByteBuffer.allocate(serializedKey.length + TIMESTAMP_SIZE);
         buf.put(serializedKey);
@@ -55,7 +55,7 @@
     }
 
     public byte[] serializeBaseKey(String topic, Windowed<T> data) {
-        return inner.serialize(topic, data.value());
+        return inner.serialize(topic, data.key());
     }
 
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitioner.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitioner.java
index 10e69cc..1e30864 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitioner.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitioner.java
@@ -29,12 +29,12 @@
     }
 
     /**
-     * WindowedStreamPartitioner determines the partition number for a message with the given windowed key and value
+     * WindowedStreamPartitioner determines the partition number for a record with the given windowed key and value
      * and the current number of partitions. The partition number id determined by the original key of the windowed key
      * using the same logic as DefaultPartitioner so that the topic is partitioned by the original key.
      *
-     * @param windowedKey the key of the message
-     * @param value the value of the message
+     * @param windowedKey the key of the record
+     * @param value the value of the record
      * @param numPartitions the total number of partitions
      * @return an integer between 0 and {@code numPartitions-1}, or {@code null} if the default partitioning logic should be used
      */
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ConsumerRecordTimestampExtractor.java b/streams/src/main/java/org/apache/kafka/streams/processor/ConsumerRecordTimestampExtractor.java
index 61b1c98..0d3424e 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/ConsumerRecordTimestampExtractor.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/ConsumerRecordTimestampExtractor.java
@@ -27,7 +27,8 @@
  * via this timestamp extractor.
  *
  * If <i>CreateTime</i> is used to define the built-in timestamps, using this extractor effectively provide
- * <i>event-time</i> semantics.
+ * <i>event-time</i> semantics. If <i>LogAppendTime</i> is used to define the built-in timestamps, using
+ * this extractor effectively provides <i>ingestion-time</i> semantics.
  *
  * If you need <i>processing-time</i> semantics, use {@link WallclockTimestampExtractor}.
  */
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 999f6a9..405ecd5 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
@@ -69,6 +69,9 @@
         return Collections.unmodifiableMap(groups);
     }
 
+    /**
+     * @throws StreamsException if no metadata can be received for a topic
+     */
     protected int maxNumPartitions(Cluster metadata, Set<String> topics) {
         int maxNumPartitions = 0;
         for (String topic : topics) {
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java
index fbd72f0..92fcf12 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java
@@ -37,10 +37,10 @@
     void init(ProcessorContext context);
 
     /**
-     * Process the message with the given key and value.
+     * Process the record with the given key and value.
      * 
-     * @param key the key for the message
-     * @param value the value for the message
+     * @param key the key for the record
+     * @param value the value for the record
      */
     void process(K key, V value);
 
@@ -53,7 +53,8 @@
     void punctuate(long timestamp);
 
     /**
-     * Close this processor and clean up any resources.
+     * Close this processor and clean up any resources. Be aware that {@link #close()} is called after an internal cleanup.
+     * Thus, it is not possible to write anything to Kafka as underlying clients are already closed.
      */
     void close();
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorSupplier.java
index 6561899..7976e16 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorSupplier.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorSupplier.java
@@ -29,5 +29,10 @@
  */
 public interface ProcessorSupplier<K, V> {
 
+    /**
+     * Return a new {@link Processor} instance.
+     *
+     * @return  a new {@link Processor} instance
+     */
     Processor<K, V> get();
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java b/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java
index b07e510..f79e6f6 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java
@@ -51,7 +51,9 @@
     void close();
 
     /**
-     * If the storage is persistent
+     * Return if the storage is persistent or not.
+     *
+     * @return  {@code true} if the storage is persistent&mdash;{@code false} otherwise
      */
     boolean persistent();
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/StateStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/processor/StateStoreSupplier.java
index 993500d..f2ae020 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/StateStoreSupplier.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/StateStoreSupplier.java
@@ -22,7 +22,17 @@
  */
 public interface StateStoreSupplier {
 
+    /**
+     * Return the name of this state store supplier.
+     *
+     * @return the name of this state store supplier
+     */
     String name();
 
+    /**
+     * Return a new {@link StateStore} instance.
+     *
+     * @return  a new {@link StateStore} instance
+     */
     StateStore get();
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/StreamPartitioner.java b/streams/src/main/java/org/apache/kafka/streams/processor/StreamPartitioner.java
index f14d9d9..fbb0378 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/StreamPartitioner.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/StreamPartitioner.java
@@ -17,21 +17,21 @@
 package org.apache.kafka.streams.processor;
 
 /**
- * Determine how messages are distributed among the partitions in a Kafka topic. If not specified, the underlying producer's
+ * Determine how records are distributed among the partitions in a Kafka topic. If not specified, the underlying producer's
  * {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} will be used to determine the partition.
  * <p>
  * Kafka topics are divided into one or more <i>partitions</i>. Since each partition must fit on the servers that host it, so
  * using multiple partitions allows the topic to scale beyond a size that will fit on a single machine. Partitions also enable you
- * to use multiple instances of your topology to process in parallel all of the messages on the topology's source topics.
+ * to use multiple instances of your topology to process in parallel all of the records on the topology's source topics.
  * <p>
  * When a topology is instantiated, each of its sources are assigned a subset of that topic's partitions. That means that only
- * those processors in that topology instance will consume the messages from those partitions. In many cases, Kafka Streams will
+ * those processors in that topology instance will consume the records from those partitions. In many cases, Kafka Streams will
  * automatically manage these instances, and adjust when new topology instances are added or removed.
  * <p>
- * Some topologies, though, need more control over which messages appear in each partition. For example, some topologies that have
- * stateful processors may want all messages within a range of keys to always be delivered to and handled by the same topology instance.
- * An upstream topology producing messages to that topic can use a custom <i>stream partitioner</i> to precisely and consistently
- * determine to which partition each message should be written.
+ * Some topologies, though, need more control over which records appear in each partition. For example, some topologies that have
+ * stateful processors may want all records within a range of keys to always be delivered to and handled by the same topology instance.
+ * An upstream topology producing records to that topic can use a custom <i>stream partitioner</i> to precisely and consistently
+ * determine to which partition each record should be written.
  * <p>
  * To do this, create a <code>StreamPartitioner</code> implementation, and when you build your topology specify that custom partitioner
  * when {@link TopologyBuilder#addSink(String, String, org.apache.kafka.common.serialization.Serializer, org.apache.kafka.common.serialization.Serializer, StreamPartitioner, String...) adding a sink}
@@ -48,10 +48,10 @@
 public interface StreamPartitioner<K, V> {
 
     /**
-     * Determine the partition number for a message with the given key and value and the current number of partitions.
+     * Determine the partition number for a record with the given key and value and the current number of partitions.
      * 
-     * @param key the key of the message
-     * @param value the value of the message
+     * @param key the key of the record
+     * @param value the value of the record
      * @param numPartitions the total number of partitions
      * @return an integer between 0 and {@code numPartitions-1}, or {@code null} if the default partitioning logic should be used
      */
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 ff21047..7fc00d1 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
@@ -25,11 +25,13 @@
 import java.nio.ByteBuffer;
 
 /**
- * The task id representation composed as topic group id plus the assigned partition id.
+ * The task ID representation composed as topic group ID plus the assigned partition ID.
  */
 public class TaskId implements Comparable<TaskId> {
 
+    /** The ID of the topic group. */
     public final int topicGroupId;
+    /** The ID of the partition. */
     public final int partition;
 
     public TaskId(int topicGroupId, int partition) {
@@ -41,6 +43,9 @@
         return topicGroupId + "_" + partition;
     }
 
+    /**
+     * @throws TaskIdFormatException if the string is not a valid {@link TaskId}
+     */
     public static TaskId parse(String string) {
         int index = string.indexOf('_');
         if (index <= 0 || index + 1 >= string.length()) throw new TaskIdFormatException(string);
@@ -55,11 +60,17 @@
         }
     }
 
+    /**
+     * @throws IOException if cannot write to output stream
+     */
     public void writeTo(DataOutputStream out) throws IOException {
         out.writeInt(topicGroupId);
         out.writeInt(partition);
     }
 
+    /**
+     * @throws IOException if cannot read from input stream
+     */
     public static TaskId readFrom(DataInputStream in) throws IOException {
         return new TaskId(in.readInt(), in.readInt());
     }
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java b/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java
index 224d580..c872fa1 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java
@@ -26,10 +26,12 @@
 public interface TimestampExtractor {
 
     /**
-     * Extracts a timestamp from a message
+     * Extracts a timestamp from a record.
+     * <p>
+     * Typically, the timestamp represents the milliseconds since midnight, January 1, 1970 UTC.
      *
-     * @param record ConsumerRecord
-     * @return timestamp
+     * @param record  a data record
+     * @return        the timestamp of the record
      */
     long extract(ConsumerRecord<Object, Object> record);
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java
index 7f5d645..5425149 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java
@@ -43,11 +43,11 @@
 /**
  * A component that is used to build a {@link ProcessorTopology}. A topology contains an acyclic graph of sources, processors,
  * and sinks. A {@link SourceNode source} is a node in the graph that consumes one or more Kafka topics and forwards them to
- * its child nodes. A {@link Processor processor} is a node in the graph that receives input messages from upstream nodes,
- * processes that message, and optionally forwarding new messages to one or all of its children. Finally, a {@link SinkNode sink}
- * is a node in the graph that receives messages from upstream nodes and writes them to a Kafka topic. This builder allows you
+ * its child nodes. A {@link Processor processor} is a node in the graph that receives input records from upstream nodes,
+ * processes that records, and optionally forwarding new records to one or all of its children. Finally, a {@link SinkNode sink}
+ * is a node in the graph that receives records from upstream nodes and writes them to a Kafka topic. This builder allows you
  * to construct an acyclic graph of these nodes, and the builder is then passed into a new {@link org.apache.kafka.streams.KafkaStreams}
- * instance that will then {@link org.apache.kafka.streams.KafkaStreams#start() begin consuming, processing, and producing messages}.
+ * instance that will then {@link org.apache.kafka.streams.KafkaStreams#start() begin consuming, processing, and producing records}.
  */
 public class TopologyBuilder {
 
@@ -193,7 +193,7 @@
     public TopologyBuilder() {}
 
     /**
-     * Add a new source that consumes the named topics and forwards the messages to child processor and/or sink nodes.
+     * Add a new source that consumes the named topics and forwards the records to child processor and/or sink nodes.
      * The source will use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key deserializer} and
      * {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value deserializer} specified in the
      * {@link org.apache.kafka.streams.StreamsConfig stream configuration}.
@@ -208,19 +208,20 @@
     }
 
     /**
-     * Add a new source that consumes the named topics and forwards the messages to child processor and/or sink nodes.
+     * Add a new source that consumes the named topics and forwards the records to child processor and/or sink nodes.
      * The source will use the specified key and value deserializers.
      *
      * @param name the unique name of the source used to reference this node when
      * {@link #addProcessor(String, ProcessorSupplier, String...) adding processor children}.
-     * @param keyDeserializer the {@link Deserializer key deserializer} used when consuming messages; may be null if the source
+     * @param keyDeserializer the {@link Deserializer key deserializer} used when consuming records; may be null if the source
      * should use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key deserializer} specified in the
      * {@link org.apache.kafka.streams.StreamsConfig stream configuration}
-     * @param valDeserializer the {@link Deserializer value deserializer} used when consuming messages; may be null if the source
+     * @param valDeserializer the {@link Deserializer value deserializer} used when consuming records; may be null if the source
      * should use the {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value deserializer} specified in the
      * {@link org.apache.kafka.streams.StreamsConfig stream configuration}
      * @param topics the name of one or more Kafka topics that this source is to consume
      * @return this builder instance so methods can be chained together; never null
+     * @throws TopologyBuilderException if processor is already added or if topics have already been registered by another source
      */
     public final TopologyBuilder addSource(String name, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) {
         if (nodeFactories.containsKey(name))
@@ -241,14 +242,14 @@
     }
 
     /**
-     * Add a new sink that forwards messages from upstream parent processor and/or source nodes to the named Kafka topic.
+     * Add a new sink that forwards records from upstream parent processor and/or source nodes to the named Kafka topic.
      * The sink will use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key serializer} and
      * {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value serializer} specified in the
      * {@link org.apache.kafka.streams.StreamsConfig stream configuration}.
      *
      * @param name the unique name of the sink
-     * @param topic the name of the Kafka topic to which this sink should write its messages
-     * @param parentNames the name of one or more source or processor nodes whose output message this sink should consume
+     * @param topic the name of the Kafka topic to which this sink should write its records
+     * @param parentNames the name of one or more source or processor nodes whose output records this sink should consume
      * and write to its topic
      * @return this builder instance so methods can be chained together; never null
      * @see #addSink(String, String, StreamPartitioner, String...)
@@ -260,22 +261,22 @@
     }
 
     /**
-     * Add a new sink that forwards messages from upstream parent processor and/or source nodes to the named Kafka topic, using
+     * Add a new sink that forwards records from upstream parent processor and/or source nodes to the named Kafka topic, using
      * the supplied partitioner.
      * The sink will use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key serializer} and
      * {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value serializer} specified in the
      * {@link org.apache.kafka.streams.StreamsConfig stream configuration}.
      * <p>
-     * The sink will also use the specified {@link StreamPartitioner} to determine how messages are distributed among
+     * The sink will also use the specified {@link StreamPartitioner} to determine how records are distributed among
      * the named Kafka topic's partitions. Such control is often useful with topologies that use
      * {@link #addStateStore(StateStoreSupplier, String...) state stores}
-     * in its processors. In most other cases, however, a partitioner need not be specified and Kafka will automatically distribute
-     * messages among partitions using Kafka's default partitioning logic.
+     * in its processors. In most other cases, however, a partitioner needs not be specified and Kafka will automatically distribute
+     * records among partitions using Kafka's default partitioning logic.
      *
      * @param name the unique name of the sink
-     * @param topic the name of the Kafka topic to which this sink should write its messages
-     * @param partitioner the function that should be used to determine the partition for each message processed by the sink
-     * @param parentNames the name of one or more source or processor nodes whose output message this sink should consume
+     * @param topic the name of the Kafka topic to which this sink should write its records
+     * @param partitioner the function that should be used to determine the partition for each record processed by the sink
+     * @param parentNames the name of one or more source or processor nodes whose output records this sink should consume
      * and write to its topic
      * @return this builder instance so methods can be chained together; never null
      * @see #addSink(String, String, String...)
@@ -287,18 +288,18 @@
     }
 
     /**
-     * Add a new sink that forwards messages from upstream parent processor and/or source nodes to the named Kafka topic.
+     * Add a new sink that forwards records from upstream parent processor and/or source nodes to the named Kafka topic.
      * The sink will use the specified key and value serializers.
      *
      * @param name the unique name of the sink
-     * @param topic the name of the Kafka topic to which this sink should write its messages
-     * @param keySerializer the {@link Serializer key serializer} used when consuming messages; may be null if the sink
+     * @param topic the name of the Kafka topic to which this sink should write its records
+     * @param keySerializer the {@link Serializer key serializer} used when consuming records; may be null if the sink
      * should use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key serializer} specified in the
      * {@link org.apache.kafka.streams.StreamsConfig stream configuration}
-     * @param valSerializer the {@link Serializer value serializer} used when consuming messages; may be null if the sink
+     * @param valSerializer the {@link Serializer value serializer} used when consuming records; may be null if the sink
      * should use the {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value serializer} specified in the
      * {@link org.apache.kafka.streams.StreamsConfig stream configuration}
-     * @param parentNames the name of one or more source or processor nodes whose output message this sink should consume
+     * @param parentNames the name of one or more source or processor nodes whose output records this sink should consume
      * and write to its topic
      * @return this builder instance so methods can be chained together; never null
      * @see #addSink(String, String, String...)
@@ -310,24 +311,25 @@
     }
 
     /**
-     * Add a new sink that forwards messages from upstream parent processor and/or source nodes to the named Kafka topic.
+     * Add a new sink that forwards records from upstream parent processor and/or source nodes to the named Kafka topic.
      * The sink will use the specified key and value serializers, and the supplied partitioner.
      *
      * @param name the unique name of the sink
-     * @param topic the name of the Kafka topic to which this sink should write its messages
-     * @param keySerializer the {@link Serializer key serializer} used when consuming messages; may be null if the sink
+     * @param topic the name of the Kafka topic to which this sink should write its records
+     * @param keySerializer the {@link Serializer key serializer} used when consuming records; may be null if the sink
      * should use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key serializer} specified in the
      * {@link org.apache.kafka.streams.StreamsConfig stream configuration}
-     * @param valSerializer the {@link Serializer value serializer} used when consuming messages; may be null if the sink
+     * @param valSerializer the {@link Serializer value serializer} used when consuming records; may be null if the sink
      * should use the {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value serializer} specified in the
      * {@link org.apache.kafka.streams.StreamsConfig stream configuration}
-     * @param partitioner the function that should be used to determine the partition for each message processed by the sink
-     * @param parentNames the name of one or more source or processor nodes whose output message this sink should consume
+     * @param partitioner the function that should be used to determine the partition for each record processed by the sink
+     * @param parentNames the name of one or more source or processor nodes whose output records this sink should consume
      * and write to its topic
      * @return this builder instance so methods can be chained together; never null
      * @see #addSink(String, String, String...)
      * @see #addSink(String, String, StreamPartitioner, String...)
      * @see #addSink(String, String, Serializer, Serializer, String...)
+     * @throws TopologyBuilderException if parent processor is not added yet, or if this processor's name is equal to the parent's name
      */
     public final <K, V> TopologyBuilder addSink(String name, String topic, Serializer<K> keySerializer, Serializer<V> valSerializer, StreamPartitioner<K, V> partitioner, String... parentNames) {
         if (nodeFactories.containsKey(name))
@@ -352,13 +354,14 @@
     }
 
     /**
-     * Add a new processor node that receives and processes messages output by one or more parent source or processor node.
-     * Any new messages output by this processor will be forwarded to its child processor or sink nodes.
+     * Add a new processor node that receives and processes records output by one or more parent source or processor node.
+     * Any new record output by this processor will be forwarded to its child processor or sink nodes.
      * @param name the unique name of the processor node
      * @param supplier the supplier used to obtain this node's {@link Processor} instance
-     * @param parentNames the name of one or more source or processor nodes whose output messages this processor should receive
+     * @param parentNames the name of one or more source or processor nodes whose output records this processor should receive
      * and process
      * @return this builder instance so methods can be chained together; never null
+     * @throws TopologyBuilderException if parent processor is not added yet, or if this processor's name is equal to the parent's name
      */
     public final TopologyBuilder addProcessor(String name, ProcessorSupplier supplier, String... parentNames) {
         if (nodeFactories.containsKey(name))
@@ -386,6 +389,7 @@
      *
      * @param supplier the supplier used to obtain this state store {@link StateStore} instance
      * @return this builder instance so methods can be chained together; never null
+     * @throws TopologyBuilderException if state store supplier is already added
      */
     public final TopologyBuilder addStateStore(StateStoreSupplier supplier, boolean isInternal, String... processorNames) {
         if (stateFactories.containsKey(supplier.name())) {
@@ -438,6 +442,7 @@
      *
      * @param processorNames the name of the processors
      * @return this builder instance so methods can be chained together; never null
+     * @throws TopologyBuilderException if less than two processors are specified, or if one of the processors is not added yet
      */
     public final TopologyBuilder connectProcessors(String... processorNames) {
         if (processorNames.length < 2)
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/WallclockTimestampExtractor.java b/streams/src/main/java/org/apache/kafka/streams/processor/WallclockTimestampExtractor.java
index 81821ce..305573b 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/WallclockTimestampExtractor.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/WallclockTimestampExtractor.java
@@ -25,9 +25,16 @@
  * Using this extractor effectively provides <i>processing-time</i> semantics.
  *
  * If you need <i>event-time</i> semantics, use {@link ConsumerRecordTimestampExtractor} with
- * built-in <i>CreateTime</i> timestamp (see KIP-32: Add timestamps to Kafka message for details).
+ * built-in <i>CreateTime</i> or <i>LogAppendTime</i> timestamp (see KIP-32: Add timestamps to Kafka message for details).
  */
 public class WallclockTimestampExtractor implements TimestampExtractor {
+
+    /**
+     * Return the current wall clock time as timestamp.
+     *
+     * @param record  a data record
+     * @return        the current wall clock time, expressed in milliseconds since midnight, January 1, 1970 UTC
+     */
     @Override
     public long extract(ConsumerRecord<Object, Object> record) {
         return System.currentTimeMillis();
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java
index b3b6537..c85ecde 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java
@@ -44,6 +44,9 @@
     protected final Set<TopicPartition> partitions;
     protected ProcessorContext processorContext;
 
+    /**
+     * @throws ProcessorStateException if the state manager cannot be created
+     */
     protected AbstractTask(TaskId id,
                            String applicationId,
                            Collection<TopicPartition> partitions,
@@ -101,6 +104,9 @@
 
     public abstract void commit();
 
+    /**
+     * @throws ProcessorStateException if there is an error while closing the state manager
+     */
     public void close() {
         try {
             stateMgr.close(recordCollectorOffsets());
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java
index 3725c4c..4477fb7 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java
@@ -36,6 +36,7 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 
 public class InternalTopicManager {
 
@@ -45,12 +46,19 @@
     private static final String ZK_TOPIC_PATH = "/brokers/topics";
     private static final String ZK_BROKER_PATH = "/brokers/ids";
     private static final String ZK_DELETE_TOPIC_PATH = "/admin/delete_topics";
+    private static final String ZK_ENTITY_CONFIG_PATH = "/config/topics";
+    // TODO: the following LogConfig dependency should be removed after KIP-4
+    private static final String CLEANUP_POLICY_PROP = "cleanup.policy";
+    private static final String COMPACT = "compact";
 
     private final ZkClient zkClient;
     private final int replicationFactor;
 
     private class ZKStringSerializer implements ZkSerializer {
 
+        /**
+         * @throws AssertionError if the byte String encoding type is not supported
+         */
         @Override
         public byte[] serialize(Object data) {
             try {
@@ -60,6 +68,9 @@
             }
         }
 
+        /**
+         * @throws AssertionError if the byte String encoding type is not supported
+         */
         @Override
         public Object deserialize(byte[] bytes) {
             try {
@@ -83,7 +94,7 @@
         this.replicationFactor = replicationFactor;
     }
 
-    public void makeReady(String topic, int numPartitions) {
+    public void makeReady(String topic, int numPartitions, boolean compactTopic) {
         boolean topicNotReady = true;
 
         while (topicNotReady) {
@@ -91,7 +102,7 @@
 
             if (topicMetadata == null) {
                 try {
-                    createTopic(topic, numPartitions, replicationFactor);
+                    createTopic(topic, numPartitions, replicationFactor, compactTopic);
                 } catch (ZkNodeExistsException e) {
                     // ignore and continue
                 }
@@ -152,9 +163,10 @@
         }
     }
 
-    private void createTopic(String topic, int numPartitions, int replicationFactor) throws ZkNodeExistsException {
+    private void createTopic(String topic, int numPartitions, int replicationFactor, boolean compactTopic) throws ZkNodeExistsException {
         log.debug("Creating topic {} with {} partitions from ZK in partition assignor.", topic, numPartitions);
-
+        Properties prop = new Properties();
+        ObjectMapper mapper = new ObjectMapper();
         List<Integer> brokers = getBrokers();
         int numBrokers = brokers.size();
         if (numBrokers < replicationFactor) {
@@ -172,14 +184,25 @@
             }
             assignment.put(i, brokerList);
         }
+        // write out config first just like in AdminUtils.scala createOrUpdateTopicPartitionAssignmentPathInZK()
+        if (compactTopic) {
+            prop.put(CLEANUP_POLICY_PROP, COMPACT);
+            try {
+                Map<String, Object> dataMap = new HashMap<>();
+                dataMap.put("version", 1);
+                dataMap.put("config", prop);
+                String data = mapper.writeValueAsString(dataMap);
+                zkClient.createPersistent(ZK_ENTITY_CONFIG_PATH + "/" + topic, data, ZooDefs.Ids.OPEN_ACL_UNSAFE);
+            } catch (JsonProcessingException e) {
+                throw new StreamsException("Error while creating topic config in ZK for internal topic " + topic, e);
+            }
+        }
 
         // try to write to ZK with open ACL
         try {
             Map<String, Object> dataMap = new HashMap<>();
             dataMap.put("version", 1);
             dataMap.put("partitions", assignment);
-
-            ObjectMapper mapper = new ObjectMapper();
             String data = mapper.writeValueAsString(dataMap);
 
             zkClient.createPersistent(ZK_TOPIC_PATH + "/" + topic, data, ZooDefs.Ids.OPEN_ACL_UNSAFE);
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java
index 717df2c..655b8b8 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java
@@ -32,6 +32,9 @@
     // record's timestamp
     private long lastKnownTime = NOT_KNOWN;
 
+    /**
+     * @throws NullPointerException if the element is null
+     */
     public void addElement(Stamped<E> elem) {
         if (elem == null) throw new NullPointerException();
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java
index ec89d47..8c6078a 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java
@@ -145,6 +145,9 @@
         return timestamp;
     }
 
+    /**
+     * @throws IllegalStateException if the record's partition does not belong to this partition group
+     */
     public int numBuffered(TopicPartition partition) {
         RecordQueue recordQueue = partitionQueues.get(partition);
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
index 1c398ac..10e7d68 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
@@ -105,6 +105,9 @@
         return metrics;
     }
 
+    /**
+     * @throws IllegalStateException if this method is called before {@link #initialized()}
+     */
     @Override
     public void register(StateStore store, boolean loggingEnabled, StateRestoreCallback stateRestoreCallback) {
         if (initialized)
@@ -113,6 +116,9 @@
         stateMgr.register(store, loggingEnabled, stateRestoreCallback);
     }
 
+    /**
+     * @throws TopologyBuilderException if an attempt is made to access this state store from an unknown node
+     */
     @Override
     public StateStore getStateStore(String name) {
         ProcessorNode node = task.node();
@@ -127,6 +133,9 @@
         return stateMgr.getStore(name);
     }
 
+    /**
+     * @throws IllegalStateException if the task's record is null
+     */
     @Override
     public String topic() {
         if (task.record() == null)
@@ -140,6 +149,9 @@
             return topic;
     }
 
+    /**
+     * @throws IllegalStateException if the task's record is null
+     */
     @Override
     public int partition() {
         if (task.record() == null)
@@ -148,6 +160,9 @@
         return task.record().partition();
     }
 
+    /**
+     * @throws IllegalStateException if the task's record is null
+     */
     @Override
     public long offset() {
         if (this.task.record() == null)
@@ -156,6 +171,9 @@
         return this.task.record().offset();
     }
 
+    /**
+     * @throws IllegalStateException if the task's record is null
+     */
     @Override
     public long timestamp() {
         if (task.record() == null)
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java
index 665d39f..0cdf44c 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java
@@ -67,6 +67,9 @@
     private final boolean isStandby;
     private final Map<String, StateRestoreCallback> restoreCallbacks; // used for standby tasks, keyed by state topic name
 
+    /**
+     * @throws IOException if any error happens while creating or locking the state directory
+     */
     public ProcessorStateManager(String applicationId, int defaultPartition, Collection<TopicPartition> sources, File baseDir, Consumer<byte[], byte[]> restoreConsumer, boolean isStandby) throws IOException {
         this.applicationId = applicationId;
         this.defaultPartition = defaultPartition;
@@ -110,6 +113,9 @@
         return applicationId + "-" + storeName + STATE_CHANGELOG_TOPIC_SUFFIX;
     }
 
+    /**
+     * @throws IOException if any error happens when locking the state directory
+     */
     public static FileLock lockStateDirectory(File stateDir) throws IOException {
         return lockStateDirectory(stateDir, 0);
     }
@@ -128,6 +134,9 @@
             retry--;
             lock = lockStateDirectory(channel);
         }
+        if (lock == null) {
+            channel.close();
+        }
         return lock;
     }
 
@@ -143,6 +152,11 @@
         return this.baseDir;
     }
 
+    /**
+     * @throws IllegalArgumentException if the store name has already been registered or if it is not a valid name
+     * (e.g., when it conflicts with the names of internal topics, like the checkpoint file name)
+     * @throws StreamsException if the store's change log does not contain the partition
+     */
     public void register(StateStore store, boolean loggingEnabled, StateRestoreCallback stateRestoreCallback) {
         if (store.name().equals(CHECKPOINT_FILE_NAME))
             throw new IllegalArgumentException("Illegal store name: " + CHECKPOINT_FILE_NAME);
@@ -313,6 +327,9 @@
         }
     }
 
+    /**
+     * @throws IOException if any error happens when flushing or closing the state stores
+     */
     public void close(Map<TopicPartition, Long> ackedOffsets) throws IOException {
         try {
             if (!stores.isEmpty()) {
@@ -354,6 +371,7 @@
         } finally {
             // release the state directory directoryLock
             directoryLock.release();
+            directoryLock.channel().close();
         }
     }
 
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 087cbd2..4e789fa 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
@@ -32,6 +32,9 @@
         return ids.containsKey(id);
     }
 
+    /**
+     * @throws NoSuchElementException if the parent of this node is null
+     */
     public T root(T id) {
         T current = id;
         T parent = ids.get(current);
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java
index 31a558b..3795916 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java
@@ -19,6 +19,7 @@
 
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.streams.kstream.internals.ChangedSerializer;
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.streams.processor.StreamPartitioner;
 
@@ -40,6 +41,9 @@
         this.partitioner = partitioner;
     }
 
+    /**
+     * @throws UnsupportedOperationException if this method adds a child to a sink node
+     */
     @Override
     public void addChild(ProcessorNode<?, ?> child) {
         throw new UnsupportedOperationException("sink node does not allow addChild");
@@ -49,8 +53,16 @@
     @Override
     public void init(ProcessorContext context) {
         this.context = context;
+
+        // if serializers are null, get the default ones from the context
         if (this.keySerializer == null) this.keySerializer = (Serializer<K>) context.keySerde().serializer();
         if (this.valSerializer == null) this.valSerializer = (Serializer<V>) context.valueSerde().serializer();
+
+        // if value serializers are for {@code Change} values, set the inner serializer when necessary
+        if (this.valSerializer instanceof ChangedSerializer &&
+                ((ChangedSerializer) this.valSerializer).inner() == null)
+            ((ChangedSerializer) this.valSerializer).setInner(context.valueSerde().serializer());
+
     }
 
     @Override
@@ -64,4 +76,9 @@
     public void close() {
         // do nothing
     }
+
+    // for test only
+    public Serializer<V> valueSerializer() {
+        return valSerializer;
+    }
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java
index 1868c1b..a550344 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java
@@ -18,6 +18,7 @@
 package org.apache.kafka.streams.processor.internals;
 
 import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.streams.kstream.internals.ChangedDeserializer;
 import org.apache.kafka.streams.processor.ProcessorContext;
 
 public class SourceNode<K, V> extends ProcessorNode<K, V> {
@@ -46,9 +47,16 @@
     public void init(ProcessorContext context) {
         this.context = context;
 
-        // if serializers are null, get the default ones from the context
-        if (this.keyDeserializer == null) this.keyDeserializer = (Deserializer<K>) context.keySerde().deserializer();
-        if (this.valDeserializer == null) this.valDeserializer = (Deserializer<V>) context.valueSerde().deserializer();
+        // if deserializers are null, get the default ones from the context
+        if (this.keyDeserializer == null)
+            this.keyDeserializer = (Deserializer<K>) context.keySerde().deserializer();
+        if (this.valDeserializer == null)
+            this.valDeserializer = (Deserializer<V>) context.valueSerde().deserializer();
+
+        // if value deserializers are for {@code Change} values, set the inner deserializer when necessary
+        if (this.valDeserializer instanceof ChangedDeserializer &&
+                ((ChangedDeserializer) this.valDeserializer).inner() == null)
+            ((ChangedDeserializer) this.valDeserializer).setInner(context.valueSerde().deserializer());
     }
 
     @Override
@@ -61,4 +69,8 @@
         // do nothing
     }
 
+    // for test only
+    public Deserializer<V> valueDeserializer() {
+        return valDeserializer;
+    }
 }
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 468fe74..d4b47e2 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
@@ -98,6 +98,9 @@
         return metrics;
     }
 
+    /**
+     * @throws IllegalStateException
+     */
     @Override
     public void register(StateStore store, boolean loggingEnabled, StateRestoreCallback stateRestoreCallback) {
         if (initialized)
@@ -106,51 +109,81 @@
         stateMgr.register(store, loggingEnabled, stateRestoreCallback);
     }
 
+    /**
+     * @throws UnsupportedOperationException
+     */
     @Override
     public StateStore getStateStore(String name) {
         throw new UnsupportedOperationException("this should not happen: getStateStore() not supported in standby tasks.");
     }
 
+    /**
+     * @throws UnsupportedOperationException
+     */
     @Override
     public String topic() {
         throw new UnsupportedOperationException("this should not happen: topic() not supported in standby tasks.");
     }
 
+    /**
+     * @throws UnsupportedOperationException
+     */
     @Override
     public int partition() {
         throw new UnsupportedOperationException("this should not happen: partition() not supported in standby tasks.");
     }
 
+    /**
+     * @throws UnsupportedOperationException
+     */
     @Override
     public long offset() {
         throw new UnsupportedOperationException("this should not happen: offset() not supported in standby tasks.");
     }
 
+    /**
+     * @throws UnsupportedOperationException
+     */
     @Override
     public long timestamp() {
         throw new UnsupportedOperationException("this should not happen: timestamp() not supported in standby tasks.");
     }
 
+    /**
+     * @throws UnsupportedOperationException
+     */
     @Override
     public <K, V> void forward(K key, V value) {
         throw new UnsupportedOperationException("this should not happen: forward() not supported in standby tasks.");
     }
 
+    /**
+     * @throws UnsupportedOperationException
+     */
     @Override
     public <K, V> void forward(K key, V value, int childIndex) {
         throw new UnsupportedOperationException("this should not happen: forward() not supported in standby tasks.");
     }
 
+    /**
+     * @throws UnsupportedOperationException
+     */
     @Override
     public <K, V> void forward(K key, V value, String childName) {
         throw new UnsupportedOperationException("this should not happen: forward() not supported in standby tasks.");
     }
 
+    /**
+     * @throws UnsupportedOperationException
+     */
     @Override
     public void commit() {
         throw new UnsupportedOperationException("this should not happen: commit() not supported in standby tasks.");
     }
 
+    /**
+     * @throws UnsupportedOperationException
+     */
     @Override
     public void schedule(long interval) {
         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/StreamPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java
index 1dd082d..341e66a 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java
@@ -96,6 +96,7 @@
      * We need to have the PartitionAssignor and its StreamThread to be mutually accessible
      * since the former needs later's cached metadata while sending subscriptions,
      * and the latter needs former's returned assignment when adding tasks.
+     * @throws KafkaException if the stream thread is not specified
      */
     @Override
     public void configure(Map<String, ?> configs) {
@@ -146,6 +147,63 @@
         return new Subscription(new ArrayList<>(topics), data.encode());
     }
 
+    /**
+     * Internal helper function that creates a Kafka topic
+     * @param topicToTaskIds Map that contains the topic names to be created
+     * @param compactTopic If true, the topic should be a compacted topic. This is used for
+     *                     change log topics usually.
+     * @param outPartitionInfo If true, compute and return all partitions created
+     * @param postPartitionPhase If true, the computation for calculating the number of partitions
+     *                           is slightly different. Set to true after the initial topic-to-partition
+     *                           assignment.
+     * @return
+     */
+    private Map<TopicPartition, PartitionInfo> prepareTopic(Map<String, Set<TaskId>> topicToTaskIds,
+                                                            boolean compactTopic,
+                                                            boolean outPartitionInfo,
+                                                            boolean postPartitionPhase) {
+        Map<TopicPartition, PartitionInfo> partitionInfos = new HashMap<>();
+        // if ZK is specified, prepare the internal source topic before calling partition grouper
+        if (internalTopicManager != null) {
+            log.debug("Starting to validate internal topics in partition assignor.");
+
+            for (Map.Entry<String, Set<TaskId>> entry : topicToTaskIds.entrySet()) {
+                String topic = entry.getKey();
+                int numPartitions = 0;
+                if (postPartitionPhase) {
+                    // the expected number of partitions is the max value of TaskId.partition + 1
+                    for (TaskId task : entry.getValue()) {
+                        if (numPartitions < task.partition + 1)
+                            numPartitions = task.partition + 1;
+                    }
+                } else {
+                    // should have size 1 only
+                    numPartitions = -1;
+                    for (TaskId task : entry.getValue()) {
+                        numPartitions = task.partition;
+                    }
+                }
+
+                internalTopicManager.makeReady(topic, numPartitions, compactTopic);
+
+                // wait until the topic metadata has been propagated to all brokers
+                List<PartitionInfo> partitions;
+                do {
+                    partitions = streamThread.restoreConsumer.partitionsFor(topic);
+                } while (partitions == null || partitions.size() != numPartitions);
+
+                if (outPartitionInfo) {
+                    for (PartitionInfo partition : partitions)
+                        partitionInfos.put(new TopicPartition(partition.topic(), partition.partition()), partition);
+                }
+            }
+
+            log.info("Completed validating internal topics in partition assignor.");
+        }
+
+        return partitionInfos;
+    }
+
     @Override
     public Map<String, Assignment> assign(Cluster metadata, Map<String, Subscription> subscriptions) {
         // This assigns tasks to consumer clients in two steps.
@@ -226,35 +284,7 @@
             }
         }
 
-        Map<TopicPartition, PartitionInfo> internalPartitionInfos = new HashMap<>();
-
-        // if ZK is specified, prepare the internal source topic before calling partition grouper
-        if (internalTopicManager != null) {
-            log.debug("Starting to validate internal source topics in partition assignor.");
-
-            for (Map.Entry<String, Set<TaskId>> entry : internalSourceTopicToTaskIds.entrySet()) {
-                String topic = entry.getKey();
-
-                // should have size 1 only
-                int numPartitions = -1;
-                for (TaskId task : entry.getValue()) {
-                    numPartitions = task.partition;
-                }
-
-                internalTopicManager.makeReady(topic, numPartitions);
-
-                // wait until the topic metadata has been propagated to all brokers
-                List<PartitionInfo> partitions;
-                do {
-                    partitions = streamThread.restoreConsumer.partitionsFor(topic);
-                } while (partitions == null || partitions.size() != numPartitions);
-
-                for (PartitionInfo partition : partitions)
-                    internalPartitionInfos.put(new TopicPartition(partition.topic(), partition.partition()), partition);
-            }
-
-            log.info("Completed validating internal source topics in partition assignor.");
-        }
+        Map<TopicPartition, PartitionInfo> internalPartitionInfos = prepareTopic(internalSourceTopicToTaskIds, false, true, false);
         internalSourceTopicToTaskIds.clear();
 
         Cluster metadataWithInternalTopics = metadata;
@@ -349,39 +379,17 @@
             }
         }
 
-        // if ZK is specified, validate the internal source topics and the state changelog topics
-        if (internalTopicManager != null) {
-            log.debug("Starting to validate changelog topics in partition assignor.");
-
-            Map<String, Set<TaskId>> topicToTaskIds = new HashMap<>();
-            topicToTaskIds.putAll(stateChangelogTopicToTaskIds);
-            topicToTaskIds.putAll(internalSourceTopicToTaskIds);
-
-            for (Map.Entry<String, Set<TaskId>> entry : topicToTaskIds.entrySet()) {
-                String topic = entry.getKey();
-
-                // the expected number of partitions is the max value of TaskId.partition + 1
-                int numPartitions = 0;
-                for (TaskId task : entry.getValue()) {
-                    if (numPartitions < task.partition + 1)
-                        numPartitions = task.partition + 1;
-                }
-
-                internalTopicManager.makeReady(topic, numPartitions);
-
-                // wait until the topic metadata has been propagated to all brokers
-                List<PartitionInfo> partitions;
-                do {
-                    partitions = streamThread.restoreConsumer.partitionsFor(topic);
-                } while (partitions == null || partitions.size() != numPartitions);
-            }
-
-            log.info("Completed validating changelog topics in partition assignor.");
-        }
+        // if ZK is specified, validate the internal topics again
+        prepareTopic(internalSourceTopicToTaskIds, false /* compactTopic */, false, true);
+        // change log topics should be compacted
+        prepareTopic(stateChangelogTopicToTaskIds, true /* compactTopic */, false, true);
 
         return assignment;
     }
 
+    /**
+     * @throws TaskAssignmentException if there is no task id for one of the partitions specified
+     */
     @Override
     public void onAssignment(Assignment assignment) {
         List<TopicPartition> partitions = new ArrayList<>(assignment.partitions());
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
index 53d0a8d..d9efb6d 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
@@ -212,6 +212,9 @@
         return punctuationQueue.mayPunctuate(timestamp, this);
     }
 
+    /**
+     * @throws IllegalStateException if the current node is not null
+     */
     @Override
     public void punctuate(ProcessorNode node, long timestamp) {
         if (currNode != null)
@@ -280,6 +283,7 @@
      * Schedules a punctuation for the processor
      *
      * @param interval  the interval in milliseconds
+     * @throws IllegalStateException if the current node is not null
      */
     public void schedule(long interval) {
         if (currNode == null)
@@ -288,6 +292,9 @@
         punctuationQueue.schedule(new PunctuationSchedule(currNode, interval));
     }
 
+    /**
+     * @throws RuntimeException if an error happens during closing of processor nodes
+     */
     @Override
     public void close() {
         this.partitionGroup.close();
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
index 38dc356..d4cb78c 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
@@ -239,6 +239,8 @@
 
     /**
      * Execute the stream processors
+     * @throws KafkaException for any Kafka-related exceptions
+     * @throws Exception for any other non-Kafka exceptions
      */
     @Override
     public void run() {
@@ -284,7 +286,7 @@
         removeStandbyTasks();
 
         // We need to first close the underlying clients before closing the state
-        // manager, for example we need to make sure producer's message sends
+        // manager, for example we need to make sure producer's record sends
         // have all been acked before the state manager records
         // changelog sent offsets
         try {
@@ -517,6 +519,7 @@
                                 if (directoryLock != null) {
                                     try {
                                         directoryLock.release();
+                                        directoryLock.channel().close();
                                     } catch (IOException e) {
                                         log.error("Failed to release the state directory lock");
                                     }
@@ -760,6 +763,9 @@
             sensor.record((endNs - startNs) / 1000000, endNs);
         }
 
+        /**
+         * @throws IllegalArgumentException if tags is not constructed in key-value pairs
+         */
         @Override
         public Sensor addLatencySensor(String scopeName, String entityName, String operationName, String... tags) {
             // extract the additional tags if there are any
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 c2175bb..0486e57 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
@@ -54,6 +54,10 @@
         this.standbyTasks = standbyTasks;
     }
 
+    /**
+     * @throws TaskAssignmentException if method fails to encode the data, e.g., if there is an
+     * IO exception during encoding
+     */
     public ByteBuffer encode() {
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream out = new DataOutputStream(baos);
@@ -96,6 +100,9 @@
         }
     }
 
+    /**
+     * @throws TaskAssignmentException if method fails to decode the data or if the data version is unknown
+     */
     public static AssignmentInfo decode(ByteBuffer data) {
         // ensure we are at the beginning of the ByteBuffer
         data.rewind();
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java
index ccd2f73..874fea8 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java
@@ -49,6 +49,9 @@
         this.standbyTasks = standbyTasks;
     }
 
+    /**
+     * @throws TaskAssignmentException if method fails to encode the data
+     */
     public ByteBuffer encode() {
         if (version == CURRENT_VERSION) {
             ByteBuffer buf = ByteBuffer.allocate(4 /* version */ + 16 /* process id */ + 4 + prevTasks.size() * 8 + 4 + standbyTasks.size() * 8);
@@ -78,6 +81,9 @@
         }
     }
 
+    /**
+     * @throws TaskAssignmentException if method fails to decode the data
+     */
     public static SubscriptionInfo decode(ByteBuffer data) {
         // ensure we are at the beginning of the ByteBuffer
         data.rewind();
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 9daac98..b19510c 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
@@ -24,13 +24,23 @@
 /**
  * Factory for creating serializers / deserializers for state stores in Kafka Streams.
  *
- * @param <K> key type of serdes
- * @param <V> value type of serdes
+ * @param <K> key type of serde
+ * @param <V> value type of serde
  */
 public final class StateSerdes<K, V> {
 
-    public static <K, V> StateSerdes<K, V> withBuiltinTypes(String topic, Class<K> keyClass, Class<V> valueClass) {
-        return new StateSerdes<>(topic, Serdes.serdeFrom(keyClass), Serdes.serdeFrom(valueClass));
+    /**
+     * Create a new instance of {@link StateSerdes} for the given state name and key-/value-type classes.
+     *
+     * @param stateName   the name of the state
+     * @param keyClass    the class of the key type
+     * @param valueClass  the class of the value type
+     * @param <K>         the key type
+     * @param <V>         the value type
+     * @return            a new instance of {@link StateSerdes}
+     */
+    public static <K, V> StateSerdes<K, V> withBuiltinTypes(String stateName, Class<K> keyClass, Class<V> valueClass) {
+        return new StateSerdes<>(stateName, Serdes.serdeFrom(keyClass), Serdes.serdeFrom(valueClass));
     }
 
     private final String stateName;
@@ -46,6 +56,7 @@
      * @param stateName     the name of the state
      * @param keySerde      the serde for keys; cannot be null
      * @param valueSerde    the serde for values; cannot be null
+     * @throws IllegalArgumentException if key or value serde is null
      */
     @SuppressWarnings("unchecked")
     public StateSerdes(String stateName,
@@ -62,46 +73,105 @@
         this.valueSerde = valueSerde;
     }
 
+    /**
+     * Return the key serde.
+     *
+     * @return the key serde
+     */
     public Serde<K> keySerde() {
         return keySerde;
     }
 
+    /**
+     * Return the value serde.
+     *
+     * @return the value serde
+     */
     public Serde<V> valueSerde() {
         return valueSerde;
     }
 
+    /**
+     * Return the key deserializer.
+     *
+     * @return the key deserializer
+     */
     public Deserializer<K> keyDeserializer() {
         return keySerde.deserializer();
     }
 
+    /**
+     * Return the key serializer.
+     *
+     * @return the key serializer
+     */
     public Serializer<K> keySerializer() {
         return keySerde.serializer();
     }
 
+    /**
+     * Return the value deserializer.
+     *
+     * @return the value deserializer
+     */
     public Deserializer<V> valueDeserializer() {
         return valueSerde.deserializer();
     }
 
+    /**
+     * Return the value serializer.
+     *
+     * @return the value serializer
+     */
     public Serializer<V> valueSerializer() {
         return valueSerde.serializer();
     }
 
-    public String topic() {
+    /**
+     * Return the name of the state.
+     *
+     * @return the name of the state
+     */
+    public String stateName() {
         return stateName;
     }
 
+    /**
+     * Deserialize the key from raw bytes.
+     *
+     * @param rawKey  the key as raw bytes
+     * @return        the key as typed object
+     */
     public K keyFrom(byte[] rawKey) {
         return keySerde.deserializer().deserialize(stateName, rawKey);
     }
 
+    /**
+     * Deserialize the value from raw bytes.
+     *
+     * @param rawValue  the value as raw bytes
+     * @return          the value as typed object
+     */
     public V valueFrom(byte[] rawValue) {
         return valueSerde.deserializer().deserialize(stateName, rawValue);
     }
 
+    /**
+     * Serialize the given key.
+     *
+     * @param key  the key to be serialized
+     * @return     the serialized key
+     */
     public byte[] rawKey(K key) {
         return keySerde.serializer().serialize(stateName, key);
     }
 
+    /**
+     * Serialize the given value.
+     *
+     * @param value  the value to be serialized
+     * @return       the serialized value
+     */
     public byte[] rawValue(V value) {
         return valueSerde.serializer().serialize(stateName, value);
     }
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java
index 4e28187..9f1e53c 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java
@@ -50,6 +50,10 @@
                                 return new InMemoryKeyValueFactory<K, V>() {
                                     private int capacity = Integer.MAX_VALUE;
 
+                                    /**
+                                     * @param capacity the maximum capacity of the in-memory cache; should be one less than a power of 2
+                                     * @throws IllegalArgumentException if the capacity of the store is zero or negative
+                                     */
                                     @Override
                                     public InMemoryKeyValueFactory<K, V> maxEntries(int capacity) {
                                         if (capacity < 1) throw new IllegalArgumentException("The capacity must be positive");
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java
index c7a882f..079a2b2 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java
@@ -23,7 +23,7 @@
 import org.apache.kafka.streams.processor.StateStore;
 
 /**
- * A windowed store interface extending {@link StateStore}
+ * A windowed store interface extending {@link StateStore}.
  *
  * @param <K> Type of keys
  * @param <V> Type of values
@@ -45,6 +45,8 @@
     /**
      * Get all the key-value pairs with the given key and the time range from all
      * the existing windows.
+     *
+     * @return an iterator over key-value pairs {@code <timestamp, value>}
      */
     WindowStoreIterator<V> fetch(K key, long timeFrom, long timeTo);
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java
index 3a5819c..a25153c 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java
@@ -35,6 +35,10 @@
 /**
  * An in-memory key-value store based on a TreeMap.
  *
+ * Note that the use of array-typed keys is discouraged because they result in incorrect ordering behavior.
+ * If you intend to work on byte arrays as key, for example, you may want to wrap them with the {@code Bytes} class,
+ * i.e. use {@code RocksDBStore<Bytes, ...>} rather than {@code RocksDBStore<byte[], ...>}.
+ *
  * @param <K> The key type
  * @param <V> The value type
  *
@@ -63,7 +67,7 @@
     }
 
     public StateStore get() {
-        return new MeteredKeyValueStore<>(new MemoryStore<K, V>(name, keySerde, valueSerde).enableLogging(), "in-memory-state", time);
+        return new MeteredKeyValueStore<>(new MemoryStore<>(name, keySerde, valueSerde).enableLogging(), "in-memory-state", time);
     }
 
     private static class MemoryStore<K, V> implements KeyValueStore<K, V> {
@@ -76,6 +80,9 @@
             this.name = name;
             this.keySerde = keySerde;
             this.valueSerde = valueSerde;
+
+            // TODO: when we have serde associated with class types, we can
+            // improve this situation by passing the comparator here.
             this.map = new TreeMap<>();
         }
 
@@ -131,12 +138,12 @@
 
         @Override
         public KeyValueIterator<K, V> range(K from, K to) {
-            return new MemoryStoreIterator<K, V>(this.map.subMap(from, true, to, false).entrySet().iterator());
+            return new MemoryStoreIterator<>(this.map.subMap(from, true, to, false).entrySet().iterator());
         }
 
         @Override
         public KeyValueIterator<K, V> all() {
-            return new MemoryStoreIterator<K, V>(this.map.entrySet().iterator());
+            return new MemoryStoreIterator<>(this.map.entrySet().iterator());
         }
 
         @Override
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 a859bd2..d410e02 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
@@ -29,6 +29,19 @@
 import java.util.Map;
 import java.util.Set;
 
+/**
+ * An in-memory LRU cache store based on HashSet and HashMap.
+ *
+ *  * Note that the use of array-typed keys is discouraged because they result in incorrect ordering behavior.
+ * If you intend to work on byte arrays as key, for example, you may want to wrap them with the {@code Bytes} class,
+ * i.e. use {@code RocksDBStore<Bytes, ...>} rather than {@code RocksDBStore<byte[], ...>}.
+
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ *
+ * @see org.apache.kafka.streams.state.Stores#create(String)
+ */
 public class MemoryLRUCache<K, V> implements KeyValueStore<K, V> {
 
     public interface EldestEntryRemovalListener<K, V> {
@@ -124,11 +137,17 @@
         return value;
     }
 
+    /**
+     * @throws UnsupportedOperationException
+     */
     @Override
     public KeyValueIterator<K, V> range(K from, K to) {
         throw new UnsupportedOperationException("MemoryLRUCache does not support range() function.");
     }
 
+    /**
+     * @throws UnsupportedOperationException
+     */
     @Override
     public KeyValueIterator<K, V> all() {
         throw new UnsupportedOperationException("MemoryLRUCache does not support all() function.");
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java
index 9808c04..5e5b54ae1 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java
@@ -137,9 +137,7 @@
     public V delete(K key) {
         long startNs = time.nanoseconds();
         try {
-            V value = this.inner.delete(key);
-
-            return value;
+            return this.inner.delete(key);
         } finally {
             this.metrics.recordLatency(this.deleteTime, startNs, time.nanoseconds());
         }
@@ -147,12 +145,12 @@
 
     @Override
     public KeyValueIterator<K, V> range(K from, K to) {
-        return new MeteredKeyValueIterator<K, V>(this.inner.range(from, to), this.rangeTime);
+        return new MeteredKeyValueIterator<>(this.inner.range(from, to), this.rangeTime);
     }
 
     @Override
     public KeyValueIterator<K, V> all() {
-        return new MeteredKeyValueIterator<K, V>(this.inner.all(), this.allTime);
+        return new MeteredKeyValueIterator<>(this.inner.all(), this.allTime);
     }
 
     @Override
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 853fc5d..ff17e68 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
@@ -55,11 +55,17 @@
     private final File file;
     private final Object lock;
 
+    /**
+     * @throws IOException
+     */
     public OffsetCheckpoint(File file) throws IOException {
         this.file = file;
         this.lock = new Object();
     }
 
+    /**
+     * @throws IOException if any file operation fails with an IO exception
+     */
     public void write(Map<TopicPartition, Long> offsets) throws IOException {
         synchronized (lock) {
             // write to temp file and then swap with the existing file
@@ -84,11 +90,17 @@
         }
     }
 
+    /**
+     * @throws IOException if file write operations failed with any IO exception
+     */
     private void writeIntLine(BufferedWriter writer, int number) throws IOException {
         writer.write(Integer.toString(number));
         writer.newLine();
     }
 
+    /**
+     * @throws IOException if file write operations failed with any IO exception
+     */
     private void writeEntry(BufferedWriter writer, TopicPartition part, long offset) throws IOException {
         writer.write(part.topic());
         writer.write(' ');
@@ -98,6 +110,11 @@
         writer.newLine();
     }
 
+
+    /**
+     * @throws IOException if any file operation fails with an IO exception
+     * @throws IllegalArgumentException if the offset checkpoint version is unknown
+     */
     public Map<TopicPartition, Long> read() throws IOException {
         synchronized (lock) {
             BufferedReader reader;
@@ -141,6 +158,9 @@
         }
     }
 
+    /**
+     * @throws IOException if file read ended prematurely
+     */
     private int readInt(BufferedReader reader) throws IOException {
         String line = reader.readLine();
         if (line == null)
@@ -148,6 +168,9 @@
         return Integer.parseInt(line);
     }
 
+    /**
+     * @throws IOException if there is any IO exception during delete
+     */
     public void delete() throws IOException {
         file.delete();
     }
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RawStoreChangeLogger.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RawStoreChangeLogger.java
deleted file mode 100644
index 4d99b59..0000000
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RawStoreChangeLogger.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.streams.state.internals;
-
-import org.apache.kafka.streams.processor.ProcessorContext;
-import org.apache.kafka.streams.state.WindowStoreUtils;
-
-import java.util.Comparator;
-import java.util.TreeSet;
-
-public class RawStoreChangeLogger extends StoreChangeLogger<byte[], byte[]> {
-
-    private class ByteArrayComparator implements Comparator<byte[]> {
-        @Override
-        public int compare(byte[] left, byte[] right) {
-            for (int i = 0, j = 0; i < left.length && j < right.length; i++, j++) {
-                int a = left[i] & 0xff;
-                int b = right[j] & 0xff;
-
-                if (a != b)
-                    return a - b;
-            }
-            return left.length - right.length;
-        }
-    }
-
-    public RawStoreChangeLogger(String storeName, ProcessorContext context) {
-        this(storeName, context, DEFAULT_WRITE_BATCH_SIZE, DEFAULT_WRITE_BATCH_SIZE);
-    }
-
-    public RawStoreChangeLogger(String storeName, ProcessorContext context, int maxDirty, int maxRemoved) {
-        super(storeName, context, context.taskId().partition, WindowStoreUtils.INNER_SERDES, maxDirty, maxRemoved);
-        init();
-    }
-
-    @Override
-    public void init() {
-        this.dirty = new TreeSet<>(new ByteArrayComparator());
-        this.removed = new TreeSet<>(new ByteArrayComparator());
-    }
-}
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 fe327f6..37609a0 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
@@ -18,6 +18,7 @@
 package org.apache.kafka.streams.state.internals;
 
 import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.utils.Bytes;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.errors.ProcessorStateException;
 import org.apache.kafka.streams.processor.ProcessorContext;
@@ -46,6 +47,18 @@
 import java.util.NoSuchElementException;
 import java.util.Set;
 
+/**
+ * A persistent key-value store based on RocksDB.
+ *
+ * Note that the use of array-typed keys is discouraged because they result in incorrect caching behavior.
+ * If you intend to work on byte arrays as key, for example, you may want to wrap them with the {@code Bytes} class,
+ * i.e. use {@code RocksDBStore<Bytes, ...>} rather than {@code RocksDBStore<byte[], ...>}.
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ *
+ * @see org.apache.kafka.streams.state.Stores#create(String)
+ */
 public class RocksDBStore<K, V> implements KeyValueStore<K, V> {
 
     private static final int TTL_NOT_USED = -1;
@@ -80,8 +93,8 @@
 
     private Set<K> cacheDirtyKeys;
     private MemoryLRUCache<K, RocksDBCacheEntry> cache;
-    private StoreChangeLogger<byte[], byte[]> changeLogger;
-    private StoreChangeLogger.ValueGetter<byte[], byte[]> getter;
+    private StoreChangeLogger<Bytes, byte[]> changeLogger;
+    private StoreChangeLogger.ValueGetter<Bytes, byte[]> getter;
 
     public KeyValueStore<K, V> enableLogging() {
         loggingEnabled = true;
@@ -156,7 +169,7 @@
         // open the DB dir
         openDB(context);
 
-        this.changeLogger = this.loggingEnabled ? new RawStoreChangeLogger(name, context) : null;
+        this.changeLogger = this.loggingEnabled ? new StoreChangeLogger<>(name, context, WindowStoreUtils.INNER_SERDES) : null;
 
         if (this.cacheSize > 0) {
             this.cache = new MemoryLRUCache<K, RocksDBCacheEntry>(name, cacheSize)
@@ -170,7 +183,6 @@
                         }
                     });
 
-
             this.cacheDirtyKeys = new HashSet<>();
         } else {
             this.cache = null;
@@ -179,10 +191,10 @@
 
         // value getter should always read directly from rocksDB
         // since it is only for values that are already flushed
-        this.getter = new StoreChangeLogger.ValueGetter<byte[], byte[]>() {
+        this.getter = new StoreChangeLogger.ValueGetter<Bytes, byte[]>() {
             @Override
-            public byte[] get(byte[] key) {
-                return getInternal(key);
+            public byte[] get(Bytes key) {
+                return getInternal(key.get());
             }
         };
 
@@ -258,7 +270,7 @@
             putInternal(rawKey, rawValue);
 
             if (loggingEnabled) {
-                changeLogger.add(rawKey);
+                changeLogger.add(Bytes.wrap(rawKey));
                 changeLogger.maybeLogChange(this.getter);
             }
         }
@@ -325,7 +337,7 @@
         if (cache != null)
             flushCache();
 
-        return new RocksDBRangeIterator<K, V>(db.newIterator(), serdes, from, to);
+        return new RocksDBRangeIterator<>(db.newIterator(), serdes, from, to);
     }
 
     @Override
@@ -336,7 +348,7 @@
 
         RocksIterator innerIter = db.newIterator();
         innerIter.seekToFirst();
-        return new RocksDbIterator<K, V>(innerIter, serdes);
+        return new RocksDbIterator<>(innerIter, serdes);
     }
 
     private void flushCache() {
@@ -348,14 +360,16 @@
             for (K key : cacheDirtyKeys) {
                 RocksDBCacheEntry entry = cache.get(key);
 
-                entry.isDirty = false;
+                if (entry != null) {
+                    entry.isDirty = false;
 
-                byte[] rawKey = serdes.rawKey(key);
+                    byte[] rawKey = serdes.rawKey(key);
 
-                if (entry.value != null) {
-                    putBatch.add(new KeyValue<>(rawKey, serdes.rawValue(entry.value)));
-                } else {
-                    deleteBatch.add(rawKey);
+                    if (entry.value != null) {
+                        putBatch.add(new KeyValue<>(rawKey, serdes.rawValue(entry.value)));
+                    } else {
+                        deleteBatch.add(rawKey);
+                    }
                 }
             }
 
@@ -363,7 +377,7 @@
 
             if (loggingEnabled) {
                 for (KeyValue<byte[], byte[]> kv : putBatch)
-                    changeLogger.add(kv.key);
+                    changeLogger.add(Bytes.wrap(kv.key));
             }
 
             // check all removed entries and remove them in rocksDB
@@ -376,7 +390,7 @@
                 }
 
                 if (loggingEnabled) {
-                    changeLogger.delete(removedKey);
+                    changeLogger.delete(Bytes.wrap(removedKey));
                 }
             }
 
@@ -397,6 +411,9 @@
         flushInternal();
     }
 
+    /**
+     * @throws ProcessorStateException if flushing failed because of any internal store exceptions
+     */
     public void flushInternal() {
         try {
             db.flush(fOptions);
@@ -433,6 +450,9 @@
             return iter.isValid();
         }
 
+        /**
+         * @throws NoSuchElementException if no next element exist
+         */
         @Override
         public KeyValue<K, V> next() {
             if (!hasNext())
@@ -443,6 +463,9 @@
             return entry;
         }
 
+        /**
+         * @throws UnsupportedOperationException
+         */
         @Override
         public void remove() {
             throw new UnsupportedOperationException("RocksDB iterator does not support remove");
@@ -455,30 +478,14 @@
 
     }
 
-    private static class LexicographicComparator implements Comparator<byte[]> {
-
-        @Override
-        public int compare(byte[] left, byte[] right) {
-            for (int i = 0, j = 0; i < left.length && j < right.length; i++, j++) {
-                int leftByte = left[i] & 0xff;
-                int rightByte = right[j] & 0xff;
-                if (leftByte != rightByte) {
-                    return leftByte - rightByte;
-                }
-            }
-            return left.length - right.length;
-        }
-    }
-
     private static class RocksDBRangeIterator<K, V> extends RocksDbIterator<K, V> {
         // RocksDB's JNI interface does not expose getters/setters that allow the
         // comparator to be pluggable, and the default is lexicographic, so it's
         // safe to just force lexicographic comparator here for now.
-        private final Comparator<byte[]> comparator = new LexicographicComparator();
-        byte[] rawToKey;
+        private final Comparator<byte[]> comparator = Bytes.BYTES_LEXICO_COMPARATOR;
+        private byte[] rawToKey;
 
-        public RocksDBRangeIterator(RocksIterator iter, StateSerdes<K, V> serdes,
-                                    K from, K to) {
+        public RocksDBRangeIterator(RocksIterator iter, StateSerdes<K, V> serdes, K from, K to) {
             super(iter, serdes);
             iter.seek(serdes.rawKey(from));
             this.rawToKey = serdes.rawKey(to);
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java
index 9851c04..803a089 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java
@@ -20,6 +20,7 @@
 package org.apache.kafka.streams.state.internals;
 
 import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.utils.Bytes;
 import org.apache.kafka.common.utils.Utils;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.processor.ProcessorContext;
@@ -29,8 +30,6 @@
 import org.apache.kafka.streams.state.StateSerdes;
 import org.apache.kafka.streams.state.WindowStore;
 import org.apache.kafka.streams.state.WindowStoreIterator;
-import org.apache.kafka.streams.state.WindowStoreUtils;
-
 
 import java.io.File;
 import java.text.SimpleDateFormat;
@@ -48,11 +47,12 @@
 
     private static final long USE_CURRENT_TIMESTAMP = -1L;
 
-    private static class Segment extends RocksDBStore<byte[], byte[]> {
+    // use the Bytes wrapper for underlying rocksDB keys since they are used for hashing data structures
+    private static class Segment extends RocksDBStore<Bytes, byte[]> {
         public final long id;
 
         Segment(String segmentName, String windowName, long id) {
-            super(segmentName, windowName, WindowStoreUtils.INNER_SERDE, WindowStoreUtils.INNER_SERDE);
+            super(segmentName, windowName, WindowStoreUtils.INNER_KEY_SERDE, WindowStoreUtils.INNER_VALUE_SERDE);
             this.id = id;
         }
 
@@ -63,14 +63,14 @@
 
     private static class RocksDBWindowStoreIterator<V> implements WindowStoreIterator<V> {
         private final StateSerdes<?, V> serdes;
-        private final KeyValueIterator<byte[], byte[]>[] iterators;
+        private final KeyValueIterator<Bytes, byte[]>[] iterators;
         private int index = 0;
 
         RocksDBWindowStoreIterator(StateSerdes<?, V> serdes) {
             this(serdes, WindowStoreUtils.NO_ITERATORS);
         }
 
-        RocksDBWindowStoreIterator(StateSerdes<?, V> serdes, KeyValueIterator<byte[], byte[]>[] iterators) {
+        RocksDBWindowStoreIterator(StateSerdes<?, V> serdes, KeyValueIterator<Bytes, byte[]>[] iterators) {
             this.serdes = serdes;
             this.iterators = iterators;
         }
@@ -86,14 +86,17 @@
             return false;
         }
 
+        /**
+         * @throws NoSuchElementException if no next element exists
+         */
         @Override
         public KeyValue<Long, V> next() {
             if (index >= iterators.length)
                 throw new NoSuchElementException();
 
-            KeyValue<byte[], byte[]> kv = iterators[index].next();
+            KeyValue<Bytes, byte[]> kv = iterators[index].next();
 
-            return new KeyValue<>(WindowStoreUtils.timestampFromBinaryKey(kv.key),
+            return new KeyValue<>(WindowStoreUtils.timestampFromBinaryKey(kv.key.get()),
                                   serdes.valueFrom(kv.value));
         }
 
@@ -105,7 +108,7 @@
 
         @Override
         public void close() {
-            for (KeyValueIterator<byte[], byte[]> iterator : iterators) {
+            for (KeyValueIterator<Bytes, byte[]> iterator : iterators) {
                 iterator.close();
             }
         }
@@ -118,7 +121,7 @@
     private final Serde<K> keySerde;
     private final Serde<V> valueSerde;
     private final SimpleDateFormat formatter;
-    private final StoreChangeLogger.ValueGetter<byte[], byte[]> getter;
+    private final StoreChangeLogger.ValueGetter<Bytes, byte[]> getter;
 
     private ProcessorContext context;
     private int seqnum = 0;
@@ -127,7 +130,7 @@
     private StateSerdes<K, V> serdes;
 
     private boolean loggingEnabled = false;
-    private StoreChangeLogger<byte[], byte[]> changeLogger = null;
+    private StoreChangeLogger<Bytes, byte[]> changeLogger = null;
 
     public RocksDBWindowStore(String name, long retentionPeriod, int numSegments, boolean retainDuplicates, Serde<K> keySerde, Serde<V> valueSerde) {
         this.name = name;
@@ -141,9 +144,9 @@
 
         this.retainDuplicates = retainDuplicates;
 
-        this.getter = new StoreChangeLogger.ValueGetter<byte[], byte[]>() {
-            public byte[] get(byte[] key) {
-                return getInternal(key);
+        this.getter = new StoreChangeLogger.ValueGetter<Bytes, byte[]>() {
+            public byte[] get(Bytes key) {
+                return getInternal(key.get());
             }
         };
 
@@ -175,13 +178,16 @@
 
         openExistingSegments();
 
-        this.changeLogger = this.loggingEnabled ? new RawStoreChangeLogger(name, context) : null;
+        this.changeLogger = this.loggingEnabled ? new StoreChangeLogger(name, context, WindowStoreUtils.INNER_SERDES) : null;
 
         // register and possibly restore the state from the logs
         context.register(root, loggingEnabled, new StateRestoreCallback() {
             @Override
             public void restore(byte[] key, byte[] value) {
-                putInternal(key, value);
+                // if the value is null, it means that this record has already been
+                // deleted while it was captured in the changelog, hence we do not need to put any more.
+                if (value != null)
+                    putInternal(key, value);
             }
         });
 
@@ -246,7 +252,7 @@
         byte[] rawKey = putAndReturnInternalKey(key, value, USE_CURRENT_TIMESTAMP);
 
         if (rawKey != null && loggingEnabled) {
-            changeLogger.add(rawKey);
+            changeLogger.add(Bytes.wrap(rawKey));
             changeLogger.maybeLogChange(this.getter);
         }
     }
@@ -256,7 +262,7 @@
         byte[] rawKey = putAndReturnInternalKey(key, value, timestamp);
 
         if (rawKey != null && loggingEnabled) {
-            changeLogger.add(rawKey);
+            changeLogger.add(Bytes.wrap(rawKey));
             changeLogger.maybeLogChange(this.getter);
         }
     }
@@ -278,7 +284,7 @@
             if (retainDuplicates)
                 seqnum = (seqnum + 1) & 0x7FFFFFFF;
             byte[] binaryKey = WindowStoreUtils.toBinaryKey(key, timestamp, seqnum, serdes);
-            segment.put(binaryKey, serdes.rawValue(value));
+            segment.put(Bytes.wrap(binaryKey), serdes.rawValue(value));
             return binaryKey;
         } else {
             return null;
@@ -297,7 +303,7 @@
         // If the record is within the retention period, put it in the store.
         Segment segment = getSegment(segmentId);
         if (segment != null)
-            segment.put(binaryKey, binaryValue);
+            segment.put(Bytes.wrap(binaryKey), binaryValue);
     }
 
     private byte[] getInternal(byte[] binaryKey) {
@@ -305,7 +311,7 @@
 
         Segment segment = getSegment(segmentId);
         if (segment != null) {
-            return segment.get(binaryKey);
+            return segment.get(Bytes.wrap(binaryKey));
         } else {
             return null;
         }
@@ -320,12 +326,12 @@
         byte[] binaryFrom = WindowStoreUtils.toBinaryKey(key, timeFrom, 0, serdes);
         byte[] binaryTo = WindowStoreUtils.toBinaryKey(key, timeTo, Integer.MAX_VALUE, serdes);
 
-        ArrayList<KeyValueIterator<byte[], byte[]>> iterators = new ArrayList<>();
+        ArrayList<KeyValueIterator<Bytes, byte[]>> iterators = new ArrayList<>();
 
         for (long segmentId = segFrom; segmentId <= segTo; segmentId++) {
             Segment segment = getSegment(segmentId);
             if (segment != null)
-                iterators.add(segment.range(binaryFrom, binaryTo));
+                iterators.add(segment.range(Bytes.wrap(binaryFrom), Bytes.wrap(binaryTo)));
         }
 
         if (iterators.size() > 0) {
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 a439117..3f848fe 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
@@ -27,6 +27,16 @@
 import java.util.HashSet;
 import java.util.Set;
 
+/**
+ * Store change log collector that batches updates before sending to Kafka.
+ *
+ * Note that the use of array-typed keys is discouraged because they result in incorrect caching behavior.
+ * If you intend to work on byte arrays as key, for example, you may want to wrap them with the {@code Bytes} class,
+ * i.e. use {@code RocksDBStore<Bytes, ...>} rather than {@code RocksDBStore<byte[], ...>}.
+ *
+ * @param <K>
+ * @param <V>
+ */
 public class StoreChangeLogger<K, V> {
 
     public interface ValueGetter<K, V> {
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreUtils.java
similarity index 76%
rename from streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java
rename to streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreUtils.java
index fdf3269..30693e7 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreUtils.java
@@ -17,10 +17,13 @@
  * under the License.
  */
 
-package org.apache.kafka.streams.state;
+package org.apache.kafka.streams.state.internals;
 
 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.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.StateSerdes;
 
 import java.nio.ByteBuffer;
 
@@ -30,13 +33,12 @@
     private static final int TIMESTAMP_SIZE = 8;
 
     /** Inner byte array serde used for segments */
-    public static final Serde<byte[]> INNER_SERDE = Serdes.ByteArray();
-
-    /** Inner byte array state serde used for segments */
-    public static final StateSerdes<byte[], byte[]> INNER_SERDES = new StateSerdes<>("", INNER_SERDE, INNER_SERDE);
+    public static final Serde<Bytes> INNER_KEY_SERDE = Serdes.Bytes();
+    public static final Serde<byte[]> INNER_VALUE_SERDE = Serdes.ByteArray();
+    public static final StateSerdes<Bytes, byte[]> INNER_SERDES = new StateSerdes<>("rocksDB-inner", INNER_KEY_SERDE, INNER_VALUE_SERDE);
 
     @SuppressWarnings("unchecked")
-    public static final KeyValueIterator<byte[], byte[]>[] NO_ITERATORS = (KeyValueIterator<byte[], byte[]>[]) new KeyValueIterator[0];
+    public static final KeyValueIterator<Bytes, byte[]>[] NO_ITERATORS = (KeyValueIterator<Bytes, byte[]>[]) new KeyValueIterator[0];
 
     public static <K> byte[] toBinaryKey(K key, final long timestamp, final int seqnum, StateSerdes<K, ?> serdes) {
         byte[] serializedKey = serdes.rawKey(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
new file mode 100644
index 0000000..2a3e767
--- /dev/null
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java
@@ -0,0 +1,169 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.integration;
+
+
+import org.I0Itec.zkclient.ZkClient;
+import org.I0Itec.zkclient.ZkConnection;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.KStreamBuilder;
+import org.apache.kafka.streams.kstream.KeyValueMapper;
+import org.apache.kafka.streams.kstream.ValueMapper;
+import org.apache.kafka.streams.processor.internals.ProcessorStateManager;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.Properties;
+
+import org.apache.kafka.streams.integration.utils.EmbeddedSingleNodeKafkaCluster;
+import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
+import kafka.admin.AdminUtils;
+import kafka.log.LogConfig;
+import kafka.utils.ZKStringSerializer$;
+import kafka.utils.ZkUtils;
+import scala.Tuple2;
+import scala.collection.Iterator;
+import scala.collection.Map;
+
+/**
+ * Tests related to internal topics in streams
+ */
+public class InternalTopicIntegrationTest {
+    @ClassRule
+    public static EmbeddedSingleNodeKafkaCluster cluster = new EmbeddedSingleNodeKafkaCluster();
+    private static final String DEFAULT_INPUT_TOPIC = "inputTopic";
+    private static final String DEFAULT_OUTPUT_TOPIC = "outputTopic";
+    private static final int DEFAULT_ZK_SESSION_TIMEOUT_MS = 10 * 1000;
+    private static final int DEFAULT_ZK_CONNECTION_TIMEOUT_MS = 8 * 1000;
+
+    @BeforeClass
+    public static void startKafkaCluster() throws Exception {
+        cluster.createTopic(DEFAULT_INPUT_TOPIC);
+        cluster.createTopic(DEFAULT_OUTPUT_TOPIC);
+    }
+
+    /**
+     * Validates that any state changelog topics are compacted
+     * @return true if topics have a valid config, false otherwise
+     */
+    private boolean isUsingCompactionForStateChangelogTopics() {
+        boolean valid = true;
+
+        // Note: You must initialize the ZkClient with ZKStringSerializer.  If you don't, then
+        // createTopic() will only seem to work (it will return without error).  The topic will exist in
+        // only ZooKeeper and will be returned when listing topics, but Kafka itself does not create the
+        // topic.
+        ZkClient zkClient = new ZkClient(
+            cluster.zKConnectString(),
+            DEFAULT_ZK_SESSION_TIMEOUT_MS,
+            DEFAULT_ZK_CONNECTION_TIMEOUT_MS,
+            ZKStringSerializer$.MODULE$);
+        boolean isSecure = false;
+        ZkUtils zkUtils = new ZkUtils(zkClient, new ZkConnection(cluster.zKConnectString()), isSecure);
+
+        Map<String, Properties> topicConfigs = AdminUtils.fetchAllTopicConfigs(zkUtils);
+        Iterator it = topicConfigs.iterator();
+        while (it.hasNext()) {
+            Tuple2<String, Properties> topicConfig = (Tuple2<String, Properties>) it.next();
+            String topic = topicConfig._1;
+            Properties prop = topicConfig._2;
+
+            // state changelogs should be compacted
+            if (topic.endsWith(ProcessorStateManager.STATE_CHANGELOG_TOPIC_SUFFIX)) {
+                if (!prop.containsKey(LogConfig.CleanupPolicyProp()) ||
+                    !prop.getProperty(LogConfig.CleanupPolicyProp()).equals(LogConfig.Compact())) {
+                    valid = false;
+                    break;
+                }
+            }
+        }
+        zkClient.close();
+        return valid;
+    }
+
+    @Test
+    public void shouldCompactTopicsForStateChangelogs() throws Exception {
+        List<String> inputValues = Arrays.asList("hello", "world", "world", "hello world");
+
+        //
+        // Step 1: Configure and start a simple word count topology
+        //
+        final Serde<String> stringSerde = Serdes.String();
+        final Serde<Long> longSerde = Serdes.Long();
+
+        Properties streamsConfiguration = new Properties();
+        streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "compact-topics-integration-test");
+        streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers());
+        streamsConfiguration.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, cluster.zKConnectString());
+        streamsConfiguration.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
+        streamsConfiguration.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
+        streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, "/tmp/kafka-streams");
+
+        KStreamBuilder builder = new KStreamBuilder();
+
+        KStream<String, String> textLines = builder.stream(DEFAULT_INPUT_TOPIC);
+
+        KStream<String, Long> wordCounts = textLines
+            .flatMapValues(new ValueMapper<String, Iterable<String>>() {
+                @Override
+                public Iterable<String> apply(String value) {
+                    return Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+"));
+                }
+            }).map(new KeyValueMapper<String, String, KeyValue<String, String>>() {
+                @Override
+                public KeyValue<String, String> apply(String key, String value) {
+                    return new KeyValue<String, String>(value, value);
+                }
+            }).countByKey("Counts").toStream();
+
+        wordCounts.to(stringSerde, longSerde, DEFAULT_OUTPUT_TOPIC);
+
+        // Remove any state from previous test runs
+        IntegrationTestUtils.purgeLocalStreamsState(streamsConfiguration);
+
+        KafkaStreams streams = new KafkaStreams(builder, streamsConfiguration);
+        streams.start();
+
+        //
+        // Step 2: Produce some input data to the input topic.
+        //
+        Properties producerConfig = new Properties();
+        producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers());
+        producerConfig.put(ProducerConfig.ACKS_CONFIG, "all");
+        producerConfig.put(ProducerConfig.RETRIES_CONFIG, 0);
+        producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
+        producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
+        IntegrationTestUtils.produceValuesSynchronously(DEFAULT_INPUT_TOPIC, inputValues, producerConfig);
+
+        //
+        // Step 3: Verify the state changelog topics are compact
+        //
+        streams.close();
+        assertEquals(isUsingCompactionForStateChangelogTopics(), true);
+    }
+}
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedSingleNodeKafkaCluster.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedSingleNodeKafkaCluster.java
new file mode 100644
index 0000000..34753ae
--- /dev/null
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedSingleNodeKafkaCluster.java
@@ -0,0 +1,128 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.streams.integration.utils;
+
+import kafka.server.KafkaConfig$;
+import kafka.zk.EmbeddedZookeeper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Properties;
+import org.junit.rules.ExternalResource;
+
+/**
+ * Runs an in-memory, "embedded" Kafka cluster with 1 ZooKeeper instance and 1 Kafka broker.
+ */
+public class EmbeddedSingleNodeKafkaCluster extends ExternalResource {
+
+    private static final Logger log = LoggerFactory.getLogger(EmbeddedSingleNodeKafkaCluster.class);
+    private static final int DEFAULT_BROKER_PORT = 0; // 0 results in a random port being selected
+    private EmbeddedZookeeper zookeeper = null;
+    private KafkaEmbedded broker = null;
+
+    /**
+     * Creates and starts a Kafka cluster.
+     */
+    public void start() throws IOException, InterruptedException {
+        Properties brokerConfig = new Properties();
+
+        log.debug("Initiating embedded Kafka cluster startup");
+        log.debug("Starting a ZooKeeper instance");
+        zookeeper = new EmbeddedZookeeper();
+        log.debug("ZooKeeper instance is running at {}", zKConnectString());
+        brokerConfig.put(KafkaConfig$.MODULE$.ZkConnectProp(), zKConnectString());
+        brokerConfig.put(KafkaConfig$.MODULE$.PortProp(), DEFAULT_BROKER_PORT);
+
+        log.debug("Starting a Kafka instance on port {} ...", brokerConfig.getProperty(KafkaConfig$.MODULE$.PortProp()));
+        broker = new KafkaEmbedded(brokerConfig);
+
+        log.debug("Kafka instance is running at {}, connected to ZooKeeper at {}",
+            broker.brokerList(), broker.zookeeperConnect());
+    }
+
+    /**
+     * Stop the Kafka cluster.
+     */
+    public void stop() {
+        broker.stop();
+        zookeeper.shutdown();
+    }
+
+    /**
+     * The ZooKeeper connection string aka `zookeeper.connect` in `hostnameOrIp:port` format.
+     * Example: `127.0.0.1:2181`.
+     *
+     * You can use this to e.g. tell Kafka brokers how to connect to this instance.
+     */
+    public String zKConnectString() {
+        return "localhost:" + zookeeper.port();
+    }
+
+    /**
+     * This cluster's `bootstrap.servers` value.  Example: `127.0.0.1:9092`.
+     *
+     * You can use this to tell Kafka producers how to connect to this cluster.
+     */
+    public String bootstrapServers() {
+        return broker.brokerList();
+    }
+
+    protected void before() throws Throwable {
+        start();
+    }
+
+    protected void after() {
+        stop();
+    }
+
+    /**
+     * Create a Kafka topic with 1 partition and a replication factor of 1.
+     *
+     * @param topic The name of the topic.
+     */
+    public void createTopic(String topic) {
+        createTopic(topic, 1, 1, new Properties());
+    }
+
+    /**
+     * Create a Kafka topic with the given parameters.
+     *
+     * @param topic       The name of the topic.
+     * @param partitions  The number of partitions for this topic.
+     * @param replication The replication factor for (the partitions of) this topic.
+     */
+    public void createTopic(String topic, int partitions, int replication) {
+        createTopic(topic, partitions, replication, new Properties());
+    }
+
+    /**
+     * Create a Kafka topic with the given parameters.
+     *
+     * @param topic       The name of the topic.
+     * @param partitions  The number of partitions for this topic.
+     * @param replication The replication factor for (partitions of) this topic.
+     * @param topicConfig Additional topic-level configuration settings.
+     */
+    public void createTopic(String topic,
+                            int partitions,
+                            int replication,
+                            Properties topicConfig) {
+        broker.createTopic(topic, partitions, replication, topicConfig);
+    }
+}
\ No newline at end of file
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java
new file mode 100644
index 0000000..89fe0c4
--- /dev/null
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java
@@ -0,0 +1,157 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.streams.integration.utils;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsConfig;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+/**
+ * Utility functions to make integration testing more convenient.
+ */
+public class IntegrationTestUtils {
+
+    private static final int UNLIMITED_MESSAGES = -1;
+
+    /**
+     * Returns up to `maxMessages` message-values from the topic.
+     *
+     * @param topic          Kafka topic to read messages from
+     * @param consumerConfig Kafka consumer configuration
+     * @param maxMessages    Maximum number of messages to read via the consumer.
+     * @return The values retrieved via the consumer.
+     */
+    public static <K, V> List<V> readValues(String topic, Properties consumerConfig, int maxMessages) {
+        List<V> returnList = new ArrayList<>();
+        List<KeyValue<K, V>> kvs = readKeyValues(topic, consumerConfig, maxMessages);
+        for (KeyValue<K, V> kv : kvs) {
+            returnList.add(kv.value);
+        }
+        return returnList;
+    }
+
+    /**
+     * Returns as many messages as possible from the topic until a (currently hardcoded) timeout is
+     * reached.
+     *
+     * @param topic          Kafka topic to read messages from
+     * @param consumerConfig Kafka consumer configuration
+     * @return The KeyValue elements retrieved via the consumer.
+     */
+    public static <K, V> List<KeyValue<K, V>> readKeyValues(String topic, Properties consumerConfig) {
+        return readKeyValues(topic, consumerConfig, UNLIMITED_MESSAGES);
+    }
+
+    /**
+     * Returns up to `maxMessages` by reading via the provided consumer (the topic(s) to read from
+     * are already configured in the consumer).
+     *
+     * @param topic          Kafka topic to read messages from
+     * @param consumerConfig Kafka consumer configuration
+     * @param maxMessages    Maximum number of messages to read via the consumer
+     * @return The KeyValue elements retrieved via the consumer
+     */
+    public static <K, V> List<KeyValue<K, V>> readKeyValues(String topic, Properties consumerConfig, int maxMessages) {
+        KafkaConsumer<K, V> consumer = new KafkaConsumer<>(consumerConfig);
+        consumer.subscribe(Collections.singletonList(topic));
+        int pollIntervalMs = 100;
+        int maxTotalPollTimeMs = 2000;
+        int totalPollTimeMs = 0;
+        List<KeyValue<K, V>> consumedValues = new ArrayList<>();
+        while (totalPollTimeMs < maxTotalPollTimeMs && continueConsuming(consumedValues.size(), maxMessages)) {
+            totalPollTimeMs += pollIntervalMs;
+            ConsumerRecords<K, V> records = consumer.poll(pollIntervalMs);
+            for (ConsumerRecord<K, V> record : records) {
+                consumedValues.add(new KeyValue<>(record.key(), record.value()));
+            }
+        }
+        consumer.close();
+        return consumedValues;
+    }
+
+    private static boolean continueConsuming(int messagesConsumed, int maxMessages) {
+        return maxMessages <= 0 || messagesConsumed < maxMessages;
+    }
+
+    /**
+     * Removes local state stores.  Useful to reset state in-between integration test runs.
+     *
+     * @param streamsConfiguration Streams configuration settings
+     */
+    public static void purgeLocalStreamsState(Properties streamsConfiguration) throws IOException {
+        String path = streamsConfiguration.getProperty(StreamsConfig.STATE_DIR_CONFIG);
+        if (path != null) {
+            File node = Paths.get(path).normalize().toFile();
+            // Only purge state when it's under /tmp.  This is a safety net to prevent accidentally
+            // deleting important local directory trees.
+            if (node.getAbsolutePath().startsWith("/tmp")) {
+                Utils.delete(new File(node.getAbsolutePath()));
+            }
+        }
+    }
+
+    /**
+     * @param topic          Kafka topic to write the data records to
+     * @param records        Data records to write to Kafka
+     * @param producerConfig Kafka producer configuration
+     * @param <K>            Key type of the data records
+     * @param <V>            Value type of the data records
+     */
+    public static <K, V> void produceKeyValuesSynchronously(
+        String topic, Collection<KeyValue<K, V>> records, Properties producerConfig)
+        throws ExecutionException, InterruptedException {
+        Producer<K, V> producer = new KafkaProducer<>(producerConfig);
+        for (KeyValue<K, V> record : records) {
+            Future<RecordMetadata> f = producer.send(
+                new ProducerRecord<>(topic, record.key, record.value));
+            f.get();
+        }
+        producer.flush();
+        producer.close();
+    }
+
+    public static <V> void produceValuesSynchronously(
+        String topic, Collection<V> records, Properties producerConfig)
+        throws ExecutionException, InterruptedException {
+        Collection<KeyValue<Object, V>> keyedRecords = new ArrayList<>();
+        for (V value : records) {
+            KeyValue<Object, V> kv = new KeyValue<>(null, value);
+            keyedRecords.add(kv);
+        }
+        produceKeyValuesSynchronously(topic, keyedRecords, producerConfig);
+    }
+
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..348b46b
--- /dev/null
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java
@@ -0,0 +1,189 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.streams.integration.utils;
+
+
+import org.apache.kafka.common.protocol.SecurityProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Properties;
+
+import org.I0Itec.zkclient.ZkClient;
+import org.I0Itec.zkclient.ZkConnection;
+
+import java.io.File;
+import java.util.Collections;
+import java.util.List;
+
+import kafka.admin.AdminUtils;
+import kafka.admin.RackAwareMode;
+import kafka.server.KafkaConfig;
+import kafka.server.KafkaConfig$;
+import kafka.server.KafkaServer;
+import kafka.utils.CoreUtils;
+import kafka.utils.SystemTime$;
+import kafka.utils.TestUtils;
+import kafka.utils.ZKStringSerializer$;
+import kafka.utils.ZkUtils;
+import org.junit.rules.TemporaryFolder;
+/**
+ * Runs an in-memory, "embedded" instance of a Kafka broker, which listens at `127.0.0.1:9092` by
+ * default.
+ *
+ * Requires a running ZooKeeper instance to connect to.
+ */
+public class KafkaEmbedded {
+
+    private static final Logger log = LoggerFactory.getLogger(KafkaEmbedded.class);
+
+    private static final String DEFAULT_ZK_CONNECT = "127.0.0.1:2181";
+    private static final int DEFAULT_ZK_SESSION_TIMEOUT_MS = 10 * 1000;
+    private static final int DEFAULT_ZK_CONNECTION_TIMEOUT_MS = 8 * 1000;
+    private final Properties effectiveConfig;
+    private final File logDir;
+    public final TemporaryFolder tmpFolder;
+    private final KafkaServer kafka;
+
+    /**
+     * Creates and starts an embedded Kafka broker.
+     * @param config Broker configuration settings.  Used to modify, for example, on which port the
+     *               broker should listen to.  Note that you cannot change the `log.dirs` setting
+     *               currently.
+     */
+    public KafkaEmbedded(Properties config) throws IOException {
+        tmpFolder = new TemporaryFolder();
+        tmpFolder.create();
+        logDir = tmpFolder.newFolder();
+        effectiveConfig = effectiveConfigFrom(config);
+        boolean loggingEnabled = true;
+        KafkaConfig kafkaConfig = new KafkaConfig(effectiveConfig, loggingEnabled);
+        log.debug("Starting embedded Kafka broker (with log.dirs={} and ZK ensemble at {}) ...",
+            logDir, zookeeperConnect());
+        kafka = TestUtils.createServer(kafkaConfig, SystemTime$.MODULE$);
+        log.debug("Startup of embedded Kafka broker at {} completed (with ZK ensemble at {}) ...",
+            brokerList(), zookeeperConnect());
+    }
+
+
+    /**
+     * Creates the configuration for starting the Kafka broker by merging default values with
+     * overwrites.
+     * @param initialConfig Broker configuration settings that override the default config.
+     * @return
+     * @throws IOException
+     */
+    private Properties effectiveConfigFrom(Properties initialConfig) throws IOException {
+        Properties effectiveConfig = new Properties();
+        effectiveConfig.put(KafkaConfig$.MODULE$.BrokerIdProp(), 0);
+        effectiveConfig.put(KafkaConfig$.MODULE$.HostNameProp(), "127.0.0.1");
+        effectiveConfig.put(KafkaConfig$.MODULE$.PortProp(), "9092");
+        effectiveConfig.put(KafkaConfig$.MODULE$.NumPartitionsProp(), 1);
+        effectiveConfig.put(KafkaConfig$.MODULE$.AutoCreateTopicsEnableProp(), true);
+        effectiveConfig.put(KafkaConfig$.MODULE$.MessageMaxBytesProp(), 1000000);
+        effectiveConfig.put(KafkaConfig$.MODULE$.ControlledShutdownEnableProp(), true);
+
+        effectiveConfig.putAll(initialConfig);
+        effectiveConfig.setProperty(KafkaConfig$.MODULE$.LogDirProp(), logDir.getAbsolutePath());
+        return effectiveConfig;
+    }
+
+    /**
+     * This broker's `metadata.broker.list` value.  Example: `127.0.0.1:9092`.
+     *
+     * You can use this to tell Kafka producers and consumers how to connect to this instance.
+     */
+    public String brokerList() {
+        return kafka.config().hostName() + ":" + kafka.boundPort(SecurityProtocol.PLAINTEXT);
+    }
+
+
+    /**
+     * The ZooKeeper connection string aka `zookeeper.connect`.
+     */
+    public String zookeeperConnect() {
+        return effectiveConfig.getProperty("zookeeper.connect", DEFAULT_ZK_CONNECT);
+    }
+
+    /**
+     * Stop the broker.
+     */
+    public void stop() {
+        log.debug("Shutting down embedded Kafka broker at {} (with ZK ensemble at {}) ...",
+            brokerList(), zookeeperConnect());
+        kafka.shutdown();
+        kafka.awaitShutdown();
+        log.debug("Removing logs.dir at {} ...", logDir);
+        List<String> logDirs = Collections.singletonList(logDir.getAbsolutePath());
+        CoreUtils.delete(scala.collection.JavaConversions.asScalaBuffer(logDirs).seq());
+        tmpFolder.delete();
+        log.debug("Shutdown of embedded Kafka broker at {} completed (with ZK ensemble at {}) ...",
+            brokerList(), zookeeperConnect());
+    }
+
+    /**
+     * Create a Kafka topic with 1 partition and a replication factor of 1.
+     *
+     * @param topic The name of the topic.
+     */
+    public void createTopic(String topic) {
+        createTopic(topic, 1, 1, new Properties());
+    }
+
+    /**
+     * Create a Kafka topic with the given parameters.
+     *
+     * @param topic       The name of the topic.
+     * @param partitions  The number of partitions for this topic.
+     * @param replication The replication factor for (the partitions of) this topic.
+     */
+    public void createTopic(String topic, int partitions, int replication) {
+        createTopic(topic, partitions, replication, new Properties());
+    }
+
+    /**
+     * Create a Kafka topic with the given parameters.
+     *
+     * @param topic       The name of the topic.
+     * @param partitions  The number of partitions for this topic.
+     * @param replication The replication factor for (partitions of) this topic.
+     * @param topicConfig Additional topic-level configuration settings.
+     */
+    public void createTopic(String topic,
+                            int partitions,
+                            int replication,
+                            Properties topicConfig) {
+        log.debug("Creating topic { name: {}, partitions: {}, replication: {}, config: {} }",
+            topic, partitions, replication, topicConfig);
+
+        // Note: You must initialize the ZkClient with ZKStringSerializer.  If you don't, then
+        // createTopic() will only seem to work (it will return without error).  The topic will exist in
+        // only ZooKeeper and will be returned when listing topics, but Kafka itself does not create the
+        // topic.
+        ZkClient zkClient = new ZkClient(
+            zookeeperConnect(),
+            DEFAULT_ZK_SESSION_TIMEOUT_MS,
+            DEFAULT_ZK_CONNECTION_TIMEOUT_MS,
+            ZKStringSerializer$.MODULE$);
+        boolean isSecure = false;
+        ZkUtils zkUtils = new ZkUtils(zkClient, new ZkConnection(zookeeperConnect()), isSecure);
+        AdminUtils.createTopic(zkUtils, topic, partitions, replication, topicConfig, RackAwareMode.Enforced$.MODULE$);
+        zkClient.close();
+    }
+}
\ No newline at end of file
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java
index e75b595..cdf28db 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java
@@ -22,12 +22,23 @@
 import org.apache.kafka.streams.errors.TopologyBuilderException;
 import org.apache.kafka.test.KStreamTestDriver;
 import org.apache.kafka.test.MockProcessorSupplier;
+import org.junit.After;
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
 
 public class KStreamBuilderTest {
 
+    private KStreamTestDriver driver = null;
+
+    @After
+    public void cleanup() {
+        if (driver != null) {
+            driver.close();
+        }
+        driver = null;
+    }
+
     @Test(expected = TopologyBuilderException.class)
     public void testFrom() {
         final KStreamBuilder builder = new KStreamBuilder();
@@ -66,7 +77,7 @@
         MockProcessorSupplier<String, String> processorSupplier = new MockProcessorSupplier<>();
         merged.process(processorSupplier);
 
-        KStreamTestDriver driver = new KStreamTestDriver(builder);
+        driver = new KStreamTestDriver(builder);
         driver.setTime(0L);
 
         driver.process(topic1, "A", "aa");
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
new file mode 100644
index 0000000..e9ff235
--- /dev/null
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java
@@ -0,0 +1,123 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.kafka.streams.kstream;
+
+import org.apache.kafka.streams.kstream.internals.TimeWindow;
+import org.junit.Test;
+
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class TimeWindowsTest {
+
+    private static String anyName = "window";
+    private static long anySize = 123L;
+
+    @Test
+    public void shouldHaveSaneEqualsAndHashCode() {
+        TimeWindows w1 = TimeWindows.of("w1", anySize);
+        TimeWindows w2 = TimeWindows.of("w2", w1.size);
+
+        // Reflexive
+        assertTrue(w1.equals(w1));
+        assertTrue(w1.hashCode() == w1.hashCode());
+
+        // Symmetric
+        assertTrue(w1.equals(w2));
+        assertTrue(w1.hashCode() == w2.hashCode());
+        assertTrue(w2.hashCode() == w1.hashCode());
+
+        // Transitive
+        TimeWindows w3 = TimeWindows.of("w3", w2.size);
+        assertTrue(w2.equals(w3));
+        assertTrue(w2.hashCode() == w3.hashCode());
+        assertTrue(w1.equals(w3));
+        assertTrue(w1.hashCode() == w3.hashCode());
+
+        // Inequality scenarios
+        assertFalse("must be false for null", w1.equals(null));
+        assertFalse("must be false for different window types", w1.equals(UnlimitedWindows.of("irrelevant")));
+        assertFalse("must be false for different types", w1.equals(new Object()));
+
+        TimeWindows differentWindowSize = TimeWindows.of("differentWindowSize", w1.size + 1);
+        assertFalse("must be false when window sizes are different", w1.equals(differentWindowSize));
+
+        TimeWindows differentAdvanceInterval = w1.advanceBy(w1.advance - 1);
+        assertFalse("must be false when advance intervals are different", w1.equals(differentAdvanceInterval));
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void nameMustNotBeEmpty() {
+        TimeWindows.of("", anySize);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void nameMustNotBeNull() {
+        TimeWindows.of(null, anySize);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void windowSizeMustNotBeNegative() {
+        TimeWindows.of(anyName, -1);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void windowSizeMustNotBeZero() {
+        TimeWindows.of(anyName, 0);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void advanceIntervalMustNotBeNegative() {
+        TimeWindows.of(anyName, anySize).advanceBy(-1);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void advanceIntervalMustNotBeZero() {
+        TimeWindows.of(anyName, anySize).advanceBy(0);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void advanceIntervalMustNotBeLargerThanWindowSize() {
+        long size = anySize;
+        TimeWindows.of(anyName, size).advanceBy(size + 1);
+    }
+
+    @Test
+    public void windowsForHoppingWindows() {
+        TimeWindows windows = TimeWindows.of(anyName, 12L).advanceBy(5L);
+        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));
+        assertEquals(new TimeWindow(20L, 32L), matched.get(20L));
+    }
+
+    @Test
+    public void windowsForTumblingWindows() {
+        TimeWindows windows = TimeWindows.of(anyName, 12L);
+        Map<Long, TimeWindow> matched = windows.windowsFor(21L);
+        assertEquals(1, matched.size());
+        assertEquals(new TimeWindow(12L, 24L), matched.get(12L));
+    }
+
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..da5f159
--- /dev/null
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.kafka.streams.kstream;
+
+import org.apache.kafka.streams.kstream.internals.UnlimitedWindow;
+import org.junit.Test;
+
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class UnlimitedWindowsTest {
+
+    private static String anyName = "window";
+    private static long anyStartTime = 10L;
+
+    @Test(expected = IllegalArgumentException.class)
+    public void nameMustNotBeEmpty() {
+        UnlimitedWindows.of("");
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void nameMustNotBeNull() {
+        UnlimitedWindows.of(null);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void startTimeMustNotBeNegative() {
+        UnlimitedWindows.of(anyName).startOn(-1);
+    }
+
+    @Test
+    public void startTimeCanBeZero() {
+        UnlimitedWindows.of(anyName).startOn(0);
+    }
+
+    @Test
+    public void shouldIncludeRecordsThatHappenedOnWindowStart() {
+        UnlimitedWindows w = UnlimitedWindows.of(anyName).startOn(anyStartTime);
+        Map<Long, UnlimitedWindow> matchedWindows = w.windowsFor(w.start);
+        assertEquals(1, matchedWindows.size());
+        assertEquals(new UnlimitedWindow(anyStartTime), matchedWindows.get(anyStartTime));
+    }
+
+    @Test
+    public void shouldIncludeRecordsThatHappenedAfterWindowStart() {
+        UnlimitedWindows w = UnlimitedWindows.of(anyName).startOn(anyStartTime);
+        long timestamp = w.start + 1;
+        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(anyName).startOn(anyStartTime);
+        long timestamp = w.start - 1;
+        Map<Long, UnlimitedWindow> matchedWindows = w.windowsFor(timestamp);
+        assertTrue(matchedWindows.isEmpty());
+    }
+
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..fc0451a
--- /dev/null
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.KStreamBuilder;
+import org.apache.kafka.streams.kstream.KeyValueMapper;
+import org.apache.kafka.test.KStreamTestDriver;
+import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.TestUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+
+public class KGroupedTableImplTest {
+
+    private File stateDir;
+
+    @Before
+    public void setUp() throws IOException {
+        stateDir = TestUtils.tempDirectory("kafka-test");
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testGroupedCountOccurences() throws IOException {
+        final KStreamBuilder builder = new KStreamBuilder();
+        final String input = "count-test-input";
+        final MockProcessorSupplier processorSupplier = new MockProcessorSupplier<>();
+
+        builder.table(Serdes.String(), Serdes.String(), input)
+                .groupBy(new KeyValueMapper<String, String, KeyValue<String, String>>() {
+                    @Override
+                    public KeyValue<String, String> apply(final String key, final String value) {
+                        return new KeyValue<>(value, value);
+                    }
+                }, Serdes.String(), Serdes.String())
+                .count("count")
+                .toStream()
+                .process(processorSupplier);
+
+
+        final KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir);
+
+
+        driver.process(input, "A", "green");
+        driver.process(input, "B", "green");
+        driver.process(input, "A", "blue");
+        driver.process(input, "C", "yellow");
+        driver.process(input, "D", "green");
+
+        final List<String> expected = Arrays.asList("green:1", "green:2", "blue:1", "green:1", "yellow:1", "green:2");
+        final List<String> actual = processorSupplier.processed;
+        assertEquals(expected, actual);
+    }
+}
\ No newline at end of file
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 e04a273..0650b95 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
@@ -23,6 +23,7 @@
 import org.apache.kafka.streams.kstream.Predicate;
 import org.apache.kafka.test.KStreamTestDriver;
 import org.apache.kafka.test.MockProcessorSupplier;
+import org.junit.After;
 import org.junit.Test;
 
 import java.lang.reflect.Array;
@@ -33,6 +34,16 @@
 
     private String topicName = "topic";
 
+    private KStreamTestDriver driver = null;
+
+    @After
+    public void cleanup() {
+        if (driver != null) {
+            driver.close();
+        }
+        driver = null;
+    }
+
     @SuppressWarnings("unchecked")
     @Test
     public void testKStreamBranch() {
@@ -74,7 +85,7 @@
             branches[i].process(processors[i]);
         }
 
-        KStreamTestDriver driver = new KStreamTestDriver(builder);
+        driver = new KStreamTestDriver(builder);
         for (int i = 0; i < expectedKeys.length; i++) {
             driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]);
         }
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 75465c8..4be8513 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
@@ -24,6 +24,7 @@
 import org.apache.kafka.test.KStreamTestDriver;
 import org.apache.kafka.test.MockProcessorSupplier;
 
+import org.junit.After;
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
@@ -32,6 +33,16 @@
 
     private String topicName = "topic";
 
+    private KStreamTestDriver driver = null;
+
+    @After
+    public void cleanup() {
+        if (driver != null) {
+            driver.close();
+        }
+        driver = null;
+    }
+
     private Predicate<Integer, String> isMultipleOfThree = new Predicate<Integer, String>() {
         @Override
         public boolean test(Integer key, String value) {
@@ -51,7 +62,7 @@
         stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName);
         stream.filter(isMultipleOfThree).process(processor);
 
-        KStreamTestDriver driver = new KStreamTestDriver(builder);
+        driver = new KStreamTestDriver(builder);
         for (int i = 0; i < expectedKeys.length; i++) {
             driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]);
         }
@@ -71,7 +82,7 @@
         stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName);
         stream.filterNot(isMultipleOfThree).process(processor);
 
-        KStreamTestDriver driver = new KStreamTestDriver(builder);
+        driver = new KStreamTestDriver(builder);
         for (int i = 0; i < expectedKeys.length; i++) {
             driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]);
         }
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 bc85757..da57d4b 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
@@ -24,6 +24,7 @@
 import org.apache.kafka.streams.kstream.KeyValueMapper;
 import org.apache.kafka.test.KStreamTestDriver;
 import org.apache.kafka.test.MockProcessorSupplier;
+import org.junit.After;
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
@@ -34,6 +35,16 @@
 
     private String topicName = "topic";
 
+    private KStreamTestDriver driver = null;
+
+    @After
+    public void cleanup() {
+        if (driver != null) {
+            driver.close();
+        }
+        driver = null;
+    }
+
     @Test
     public void testFlatMap() {
         KStreamBuilder builder = new KStreamBuilder();
@@ -59,7 +70,7 @@
         stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName);
         stream.flatMap(mapper).process(processor);
 
-        KStreamTestDriver driver = new KStreamTestDriver(builder);
+        driver = new KStreamTestDriver(builder);
         for (int i = 0; i < expectedKeys.length; i++) {
             driver.process(topicName, expectedKeys[i], "V" + expectedKeys[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 a904cb1..9d1141b 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
@@ -23,9 +23,11 @@
 import org.apache.kafka.streams.kstream.ValueMapper;
 import org.apache.kafka.test.KStreamTestDriver;
 import org.apache.kafka.test.MockProcessorSupplier;
+import org.junit.After;
 import org.junit.Test;
 
 import java.util.ArrayList;
+import java.util.Locale;
 
 import static org.junit.Assert.assertEquals;
 
@@ -33,6 +35,16 @@
 
     private String topicName = "topic";
 
+    private KStreamTestDriver driver = null;
+
+    @After
+    public void cleanup() {
+        if (driver != null) {
+            driver.close();
+        }
+        driver = null;
+    }
+
     @Test
     public void testFlatMapValues() {
         KStreamBuilder builder = new KStreamBuilder();
@@ -42,7 +54,7 @@
                 @Override
                 public Iterable<String> apply(String value) {
                     ArrayList<String> result = new ArrayList<String>();
-                    result.add(value.toLowerCase());
+                    result.add(value.toLowerCase(Locale.ROOT));
                     result.add(value);
                     return result;
                 }
@@ -57,7 +69,7 @@
         stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName);
         stream.flatMapValues(mapper).process(processor);
 
-        KStreamTestDriver driver = new KStreamTestDriver(builder);
+        driver = new KStreamTestDriver(builder);
         for (int i = 0; i < expectedKeys.length; i++) {
             driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]);
         }
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 6573779..0bc5e77 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
@@ -24,8 +24,10 @@
 import org.apache.kafka.streams.kstream.KStreamBuilder;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.test.KStreamTestDriver;
+import org.junit.After;
 import org.junit.Test;
 import java.util.List;
+import java.util.Locale;
 import java.util.ArrayList;
 import java.util.Arrays;
 
@@ -38,6 +40,16 @@
     final private Serde<Integer> intSerde = Serdes.Integer();
     final private Serde<String> stringSerde = Serdes.String();
 
+    private KStreamTestDriver driver = null;
+
+    @After
+    public void cleanup() {
+        if (driver != null) {
+            driver.close();
+        }
+        driver = null;
+    }
+
     @Test
     public void testForeach() {
         // Given
@@ -60,7 +72,7 @@
             new ForeachAction<Integer, String>() {
                 @Override
                 public void apply(Integer key, String value) {
-                    actualRecords.add(new KeyValue<>(key * 2, value.toUpperCase()));
+                    actualRecords.add(new KeyValue<>(key * 2, value.toUpperCase(Locale.ROOT)));
                 }
             };
 
@@ -70,7 +82,7 @@
         stream.foreach(action);
 
         // Then
-        KStreamTestDriver driver = new KStreamTestDriver(builder);
+        driver = new KStreamTestDriver(builder);
         for (KeyValue<Integer, String> record: inputRecords) {
             driver.process(topicName, record.key, record.value);
         }
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 b5c3d47..3d45d1d 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
@@ -133,4 +133,11 @@
             1, // process
             builder.build("X", null).processors().size());
     }
+
+    @Test
+    public void testToWithNullValueSerdeDoesntNPE() {
+        final KStreamBuilder builder = new KStreamBuilder();
+        final KStream<String, String> inputStream = builder.stream(stringSerde, stringSerde, "input");
+        inputStream.to(stringSerde, null, "output");
+    }
 }
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 19a9411..6b0828a 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
@@ -19,17 +19,19 @@
 
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.serialization.Serdes;
-import org.apache.kafka.common.utils.Utils;
 import org.apache.kafka.streams.kstream.JoinWindows;
 import org.apache.kafka.streams.kstream.KStream;
 import org.apache.kafka.streams.kstream.KStreamBuilder;
-import org.apache.kafka.streams.kstream.ValueJoiner;
 import org.apache.kafka.test.KStreamTestDriver;
 import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.MockValueJoiner;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 import java.io.File;
-import java.nio.file.Files;
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
@@ -39,460 +41,447 @@
 
 public class KStreamKStreamJoinTest {
 
-    private String topic1 = "topic1";
-    private String topic2 = "topic2";
+    final private String topic1 = "topic1";
+    final private String topic2 = "topic2";
 
     final private Serde<Integer> intSerde = Serdes.Integer();
     final private Serde<String> stringSerde = Serdes.String();
 
-    private ValueJoiner<String, String, String> joiner = new ValueJoiner<String, String, String>() {
-        @Override
-        public String apply(String value1, String value2) {
-            return value1 + "+" + value2;
+    private KStreamTestDriver driver = null;
+    private File stateDir = null;
+
+    @After
+    public void tearDown() {
+        if (driver != null) {
+            driver.close();
         }
-    };
+        driver = null;
+    }
+
+    @Before
+    public void setUp() throws IOException {
+        stateDir = TestUtils.tempDirectory("kafka-test");
+    }
 
     @Test
     public void testJoin() throws Exception {
-        File baseDir = Files.createTempDirectory("test").toFile();
-        try {
+        KStreamBuilder builder = new KStreamBuilder();
 
-            KStreamBuilder builder = new KStreamBuilder();
+        final int[] expectedKeys = new int[]{0, 1, 2, 3};
 
-            final int[] expectedKeys = new int[]{0, 1, 2, 3};
+        KStream<Integer, String> stream1;
+        KStream<Integer, String> stream2;
+        KStream<Integer, String> joined;
+        MockProcessorSupplier<Integer, String> processor;
 
-            KStream<Integer, String> stream1;
-            KStream<Integer, String> stream2;
-            KStream<Integer, String> joined;
-            MockProcessorSupplier<Integer, String> processor;
+        processor = new MockProcessorSupplier<>();
+        stream1 = builder.stream(intSerde, stringSerde, topic1);
+        stream2 = builder.stream(intSerde, stringSerde, topic2);
+        joined = stream1.join(stream2, MockValueJoiner.STRING_JOINER, JoinWindows.of("test").within(100), intSerde, stringSerde, stringSerde);
+        joined.process(processor);
 
-            processor = new MockProcessorSupplier<>();
-            stream1 = builder.stream(intSerde, stringSerde, topic1);
-            stream2 = builder.stream(intSerde, stringSerde, topic2);
-            joined = stream1.join(stream2, joiner, JoinWindows.of("test").within(100), intSerde, stringSerde, stringSerde);
-            joined.process(processor);
+        Collection<Set<String>> copartitionGroups = builder.copartitionGroups();
 
-            Collection<Set<String>> copartitionGroups = builder.copartitionGroups();
+        assertEquals(1, copartitionGroups.size());
+        assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
 
-            assertEquals(1, copartitionGroups.size());
-            assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
+        driver = new KStreamTestDriver(builder, stateDir);
+        driver.setTime(0L);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir);
-            driver.setTime(0L);
+        // push two items to the primary stream. the other window is empty
+        // w1 = {}
+        // w2 = {}
+        // --> w1 = { 0:X1, 1:X1 }
+        //     w2 = {}
 
-            // push two items to the primary stream. the other window is empty
-            // w1 = {}
-            // w2 = {}
-            // --> w1 = { 0:X1, 1:X1 }
-            //     w2 = {}
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult();
-
-            // push two items to the other stream. this should produce two items.
-            // w1 = { 0:X0, 1:X1 }
-            // w2 = {}
-            // --> w1 = { 0:X1, 1:X1 }
-            //     w2 = { 0:Y0, 1:Y1 }
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1");
-
-            // push all four items to the primary stream. this should produce two items.
-            // w1 = { 0:X0, 1:X1 }
-            // w2 = { 0:Y0, 1:Y1 }
-            // --> w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 }
-            //     w2 = { 0:Y0, 1:Y1 }
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1");
-
-            // push all items to the other stream. this should produce six items.
-            // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 }
-            // w2 = { 0:Y0, 1:Y1 }
-            // --> w1 = { 0:X1, 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 i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+YY0", "0:X0+YY0", "1:X1+YY1", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
-
-            // push all four items to the primary stream. this should produce six items.
-            // w1 = { 0:X1, 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
-            // --> w1 = { 0:X1, 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 i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:XX0+Y0", "0:XX0+YY0", "1:XX1+Y1", "1:XX1+YY1", "2:XX2+YY2", "3:XX3+YY3");
-
-            // push two items to the other stream. this should produce six item.
-            // w1 = { 0:X1, 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
-            // --> w1 = { 0:X1, 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, 0:YYY0, 1:YYY1 }
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], "YYY" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+YYY0", "0:X0+YYY0", "0:XX0+YYY0", "1:X1+YYY1", "1:X1+YYY1", "1:XX1+YYY1");
-
-        } finally {
-            Utils.delete(baseDir);
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
         }
+
+        processor.checkAndClearProcessResult();
+
+        // push two items to the other stream. this should produce two items.
+        // w1 = { 0:X0, 1:X1 }
+        // w2 = {}
+        // --> w1 = { 0:X1, 1:X1 }
+        //     w2 = { 0:Y0, 1:Y1 }
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1");
+
+        // push all four items to the primary stream. this should produce two items.
+        // w1 = { 0:X0, 1:X1 }
+        // w2 = { 0:Y0, 1:Y1 }
+        // --> w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 }
+        //     w2 = { 0:Y0, 1:Y1 }
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1");
+
+        // push all items to the other stream. this should produce six items.
+        // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 }
+        // w2 = { 0:Y0, 1:Y1 }
+        // --> w1 = { 0:X1, 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 i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+YY0", "0:X0+YY0", "1:X1+YY1", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
+
+        // push all four items to the primary stream. this should produce six items.
+        // w1 = { 0:X1, 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
+        // --> w1 = { 0:X1, 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 i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:XX0+Y0", "0:XX0+YY0", "1:XX1+Y1", "1:XX1+YY1", "2:XX2+YY2", "3:XX3+YY3");
+
+        // push two items to the other stream. this should produce six item.
+        // w1 = { 0:X1, 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
+        // --> w1 = { 0:X1, 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, 0:YYY0, 1:YYY1 }
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], "YYY" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+YYY0", "0:X0+YYY0", "0:XX0+YYY0", "1:X1+YYY1", "1:X1+YYY1", "1:XX1+YYY1");
     }
 
     @Test
     public void testOuterJoin() throws Exception {
-        File baseDir = Files.createTempDirectory("test").toFile();
-        try {
+        KStreamBuilder builder = new KStreamBuilder();
 
-            KStreamBuilder builder = new KStreamBuilder();
+        final int[] expectedKeys = new int[]{0, 1, 2, 3};
 
-            final int[] expectedKeys = new int[]{0, 1, 2, 3};
+        KStream<Integer, String> stream1;
+        KStream<Integer, String> stream2;
+        KStream<Integer, String> joined;
+        MockProcessorSupplier<Integer, String> processor;
 
-            KStream<Integer, String> stream1;
-            KStream<Integer, String> stream2;
-            KStream<Integer, String> joined;
-            MockProcessorSupplier<Integer, String> processor;
+        processor = new MockProcessorSupplier<>();
+        stream1 = builder.stream(intSerde, stringSerde, topic1);
+        stream2 = builder.stream(intSerde, stringSerde, topic2);
+        joined = stream1.outerJoin(stream2, MockValueJoiner.STRING_JOINER, JoinWindows.of("test").within(100), intSerde, stringSerde, stringSerde);
+        joined.process(processor);
 
-            processor = new MockProcessorSupplier<>();
-            stream1 = builder.stream(intSerde, stringSerde, topic1);
-            stream2 = builder.stream(intSerde, stringSerde, topic2);
-            joined = stream1.outerJoin(stream2, joiner, JoinWindows.of("test").within(100), intSerde, stringSerde, stringSerde);
-            joined.process(processor);
+        Collection<Set<String>> copartitionGroups = builder.copartitionGroups();
 
-            Collection<Set<String>> copartitionGroups = builder.copartitionGroups();
+        assertEquals(1, copartitionGroups.size());
+        assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
 
-            assertEquals(1, copartitionGroups.size());
-            assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
+        driver = new KStreamTestDriver(builder, stateDir);
+        driver.setTime(0L);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir);
-            driver.setTime(0L);
+        // push two items to the primary stream. the other window is empty.this should produce two items
+        // w1 = {}
+        // w2 = {}
+        // --> w1 = { 0:X1, 1:X1 }
+        //     w2 = {}
 
-            // push two items to the primary stream. the other window is empty.this should produce two items
-            // w1 = {}
-            // w2 = {}
-            // --> w1 = { 0:X1, 1:X1 }
-            //     w2 = {}
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+null", "1:X1+null");
-
-            // push two items to the other stream. this should produce two items.
-            // w1 = { 0:X0, 1:X1 }
-            // w2 = {}
-            // --> w1 = { 0:X1, 1:X1 }
-            //     w2 = { 0:Y0, 1:Y1 }
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1");
-
-            // push all four items to the primary stream. this should produce four items.
-            // w1 = { 0:X0, 1:X1 }
-            // w2 = { 0:Y0, 1:Y1 }
-            // --> w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 }
-            //     w2 = { 0:Y0, 1:Y1 }
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null");
-
-            // push all items to the other stream. this should produce six items.
-            // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 }
-            // w2 = { 0:Y0, 1:Y1 }
-            // --> w1 = { 0:X1, 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 i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+YY0", "0:X0+YY0", "1:X1+YY1", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
-
-            // push all four items to the primary stream. this should produce six items.
-            // w1 = { 0:X1, 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
-            // --> w1 = { 0:X1, 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 i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:XX0+Y0", "0:XX0+YY0", "1:XX1+Y1", "1:XX1+YY1", "2:XX2+YY2", "3:XX3+YY3");
-
-            // push two items to the other stream. this should produce six item.
-            // w1 = { 0:X1, 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
-            // --> w1 = { 0:X1, 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, 0:YYY0, 1:YYY1 }
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], "YYY" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+YYY0", "0:X0+YYY0", "0:XX0+YYY0", "1:X1+YYY1", "1:X1+YYY1", "1:XX1+YYY1");
-
-        } finally {
-            Utils.delete(baseDir);
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
         }
+
+        processor.checkAndClearProcessResult("0:X0+null", "1:X1+null");
+
+        // push two items to the other stream. this should produce two items.
+        // w1 = { 0:X0, 1:X1 }
+        // w2 = {}
+        // --> w1 = { 0:X1, 1:X1 }
+        //     w2 = { 0:Y0, 1:Y1 }
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1");
+
+        // push all four items to the primary stream. this should produce four items.
+        // w1 = { 0:X0, 1:X1 }
+        // w2 = { 0:Y0, 1:Y1 }
+        // --> w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 }
+        //     w2 = { 0:Y0, 1:Y1 }
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null");
+
+        // push all items to the other stream. this should produce six items.
+        // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 }
+        // w2 = { 0:Y0, 1:Y1 }
+        // --> w1 = { 0:X1, 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 i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+YY0", "0:X0+YY0", "1:X1+YY1", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
+
+        // push all four items to the primary stream. this should produce six items.
+        // w1 = { 0:X1, 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
+        // --> w1 = { 0:X1, 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 i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:XX0+Y0", "0:XX0+YY0", "1:XX1+Y1", "1:XX1+YY1", "2:XX2+YY2", "3:XX3+YY3");
+
+        // push two items to the other stream. this should produce six item.
+        // w1 = { 0:X1, 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
+        // --> w1 = { 0:X1, 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, 0:YYY0, 1:YYY1 }
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], "YYY" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+YYY0", "0:X0+YYY0", "0:XX0+YYY0", "1:X1+YYY1", "1:X1+YYY1", "1:XX1+YYY1");
     }
 
     @Test
     public void testWindowing() throws Exception {
-        File baseDir = Files.createTempDirectory("test").toFile();
-        try {
+        long time = 0L;
 
-            long time = 0L;
+        KStreamBuilder builder = new KStreamBuilder();
 
-            KStreamBuilder builder = new KStreamBuilder();
+        final int[] expectedKeys = new int[]{0, 1, 2, 3};
 
-            final int[] expectedKeys = new int[]{0, 1, 2, 3};
+        KStream<Integer, String> stream1;
+        KStream<Integer, String> stream2;
+        KStream<Integer, String> joined;
+        MockProcessorSupplier<Integer, String> processor;
 
-            KStream<Integer, String> stream1;
-            KStream<Integer, String> stream2;
-            KStream<Integer, String> joined;
-            MockProcessorSupplier<Integer, String> processor;
+        processor = new MockProcessorSupplier<>();
+        stream1 = builder.stream(intSerde, stringSerde, topic1);
+        stream2 = builder.stream(intSerde, stringSerde, topic2);
+        joined = stream1.join(stream2, MockValueJoiner.STRING_JOINER, JoinWindows.of("test").within(100), intSerde, stringSerde, stringSerde);
+        joined.process(processor);
 
-            processor = new MockProcessorSupplier<>();
-            stream1 = builder.stream(intSerde, stringSerde, topic1);
-            stream2 = builder.stream(intSerde, stringSerde, topic2);
-            joined = stream1.join(stream2, joiner, JoinWindows.of("test").within(100), intSerde, stringSerde, stringSerde);
-            joined.process(processor);
+        Collection<Set<String>> copartitionGroups = builder.copartitionGroups();
 
-            Collection<Set<String>> copartitionGroups = builder.copartitionGroups();
+        assertEquals(1, copartitionGroups.size());
+        assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
 
-            assertEquals(1, copartitionGroups.size());
-            assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
+        driver = new KStreamTestDriver(builder, stateDir);
+        driver.setTime(time);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir);
-            driver.setTime(time);
+        // push two items to the primary stream. the other window is empty. this should produce no items.
+        // w1 = {}
+        // w2 = {}
+        // --> w1 = { 0:X1, 1:X1 }
+        //     w2 = {}
 
-            // push two items to the primary stream. the other window is empty. this should produce no items.
-            // w1 = {}
-            // w2 = {}
-            // --> w1 = { 0:X1, 1:X1 }
-            //     w2 = {}
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult();
-
-            // push two items to the other stream. this should produce two items.
-            // w1 = { 0:X0, 1:X1 }
-            // w2 = {}
-            // --> w1 = { 0:X1, 1:X1 }
-            //     w2 = { 0:Y0, 1:Y1 }
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1");
-
-            // clear logically
-            time = 1000L;
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.setTime(time + i);
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-            processor.checkAndClearProcessResult();
-
-            // gradually expires items in w1
-            // w1 = { 0:X0, 1:X1, 2:X2, 3:X3 }
-
-            time = 1000 + 100L;
-            driver.setTime(time);
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("2:X2+YY2", "3:X3+YY3");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("3:X3+YY3");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult();
-
-            // go back to the time before expiration
-
-            time = 1000L - 100L - 1L;
-            driver.setTime(time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult();
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+YY0");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
-
-            // clear (logically)
-            time = 2000L;
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.setTime(time + i);
-                driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult();
-
-            // gradually expires items in w2
-            // w2 = { 0:Y0, 1:Y1, 2:Y2, 3:Y3 }
-
-            time = 2000L + 100L;
-            driver.setTime(time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("2:XX2+Y2", "3:XX3+Y3");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("3:XX3+Y3");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult();
-
-            // go back to the time before expiration
-
-            time = 2000L - 100L - 1L;
-            driver.setTime(time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult();
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:XX0+Y0");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3");
-
-        } finally {
-            Utils.delete(baseDir);
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
         }
-    }
 
+        processor.checkAndClearProcessResult();
+
+        // push two items to the other stream. this should produce two items.
+        // w1 = { 0:X0, 1:X1 }
+        // w2 = {}
+        // --> w1 = { 0:X1, 1:X1 }
+        //     w2 = { 0:Y0, 1:Y1 }
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1");
+
+        // clear logically
+        time = 1000L;
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.setTime(time + i);
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+        processor.checkAndClearProcessResult();
+
+        // gradually expires items in w1
+        // w1 = { 0:X0, 1:X1, 2:X2, 3:X3 }
+
+        time = 1000 + 100L;
+        driver.setTime(time);
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("2:X2+YY2", "3:X3+YY3");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("3:X3+YY3");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult();
+
+        // go back to the time before expiration
+
+        time = 1000L - 100L - 1L;
+        driver.setTime(time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult();
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+YY0");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
+
+        // clear (logically)
+        time = 2000L;
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.setTime(time + i);
+            driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult();
+
+        // gradually expires items in w2
+        // w2 = { 0:Y0, 1:Y1, 2:Y2, 3:Y3 }
+
+        time = 2000L + 100L;
+        driver.setTime(time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("2:XX2+Y2", "3:XX3+Y3");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("3:XX3+Y3");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult();
+
+        // go back to the time before expiration
+
+        time = 2000L - 100L - 1L;
+        driver.setTime(time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult();
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:XX0+Y0");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        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/KStreamKStreamLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java
index 65226d3..65a4b54 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
@@ -19,17 +19,19 @@
 
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.serialization.Serdes;
-import org.apache.kafka.common.utils.Utils;
 import org.apache.kafka.streams.kstream.JoinWindows;
 import org.apache.kafka.streams.kstream.KStream;
 import org.apache.kafka.streams.kstream.KStreamBuilder;
-import org.apache.kafka.streams.kstream.ValueJoiner;
 import org.apache.kafka.test.KStreamTestDriver;
 import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.MockValueJoiner;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 import java.io.File;
-import java.nio.file.Files;
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
@@ -39,245 +41,240 @@
 
 public class KStreamKStreamLeftJoinTest {
 
-    private String topic1 = "topic1";
-    private String topic2 = "topic2";
+    final private String topic1 = "topic1";
+    final private String topic2 = "topic2";
 
     final private Serde<Integer> intSerde = Serdes.Integer();
     final private Serde<String> stringSerde = Serdes.String();
 
-    private ValueJoiner<String, String, String> joiner = new ValueJoiner<String, String, String>() {
-        @Override
-        public String apply(String value1, String value2) {
-            return value1 + "+" + value2;
+    private KStreamTestDriver driver = null;
+    private File stateDir = null;
+
+    @After
+    public void tearDown() {
+        if (driver != null) {
+            driver.close();
         }
-    };
+        driver = null;
+    }
+
+    @Before
+    public void setUp() throws IOException {
+        stateDir = TestUtils.tempDirectory("kafka-test");
+    }
+
 
     @Test
     public void testLeftJoin() throws Exception {
-        File baseDir = Files.createTempDirectory("test").toFile();
-        try {
+        final KStreamBuilder builder = new KStreamBuilder();
 
-            KStreamBuilder builder = new KStreamBuilder();
+        final int[] expectedKeys = new int[]{0, 1, 2, 3};
 
-            final int[] expectedKeys = new int[]{0, 1, 2, 3};
+        KStream<Integer, String> stream1;
+        KStream<Integer, String> stream2;
+        KStream<Integer, String> joined;
+        MockProcessorSupplier<Integer, String> processor;
 
-            KStream<Integer, String> stream1;
-            KStream<Integer, String> stream2;
-            KStream<Integer, String> joined;
-            MockProcessorSupplier<Integer, String> processor;
+        processor = new MockProcessorSupplier<>();
+        stream1 = builder.stream(intSerde, stringSerde, topic1);
+        stream2 = builder.stream(intSerde, stringSerde, topic2);
+        joined = stream1.leftJoin(stream2, MockValueJoiner.STRING_JOINER, JoinWindows.of("test").within(100), intSerde, stringSerde);
+        joined.process(processor);
 
-            processor = new MockProcessorSupplier<>();
-            stream1 = builder.stream(intSerde, stringSerde, topic1);
-            stream2 = builder.stream(intSerde, stringSerde, topic2);
-            joined = stream1.leftJoin(stream2, joiner, JoinWindows.of("test").within(100), intSerde, stringSerde);
-            joined.process(processor);
+        Collection<Set<String>> copartitionGroups = builder.copartitionGroups();
 
-            Collection<Set<String>> copartitionGroups = builder.copartitionGroups();
+        assertEquals(1, copartitionGroups.size());
+        assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
 
-            assertEquals(1, copartitionGroups.size());
-            assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
+        driver = new KStreamTestDriver(builder, stateDir);
+        driver.setTime(0L);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir);
-            driver.setTime(0L);
+        // push two items to the primary stream. the other window is empty
+        // w {}
+        // --> w = {}
 
-            // push two items to the primary stream. the other window is empty
-            // w {}
-            // --> w = {}
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+null", "1:X1+null");
-
-            // push two items to the other stream. this should produce two items.
-            // w {}
-            // --> w = { 0:Y0, 1:Y1 }
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult();
-
-            // push all four items to the primary stream. this should produce four items.
-            // w = { 0:Y0, 1:Y1 }
-            // --> w = { 0:Y0, 1:Y1 }
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null");
-
-            // push all items to the other stream. this should produce no items.
-            // w = { 0:Y0, 1:Y1 }
-            // --> w = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 }
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult();
-
-            // push all four items to the primary stream. this should produce four items.
-            // w = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3
-            // --> w = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 }
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:XX0+Y0", "0:XX0+YY0", "1:XX1+Y1", "1:XX1+YY1", "2:XX2+YY2", "3:XX3+YY3");
-
-        } finally {
-            Utils.delete(baseDir);
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
         }
+
+        processor.checkAndClearProcessResult("0:X0+null", "1:X1+null");
+
+        // push two items to the other stream. this should produce two items.
+        // w {}
+        // --> w = { 0:Y0, 1:Y1 }
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult();
+
+        // push all four items to the primary stream. this should produce four items.
+        // w = { 0:Y0, 1:Y1 }
+        // --> w = { 0:Y0, 1:Y1 }
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null");
+
+        // push all items to the other stream. this should produce no items.
+        // w = { 0:Y0, 1:Y1 }
+        // --> w = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 }
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult();
+
+        // push all four items to the primary stream. this should produce four items.
+        // w = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3
+        // --> w = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 }
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:XX0+Y0", "0:XX0+YY0", "1:XX1+Y1", "1:XX1+YY1", "2:XX2+YY2", "3:XX3+YY3");
     }
 
     @Test
     public void testWindowing() throws Exception {
-        File baseDir = Files.createTempDirectory("test").toFile();
-        try {
+        final KStreamBuilder builder = new KStreamBuilder();
 
-            long time = 0L;
+        final int[] expectedKeys = new int[]{0, 1, 2, 3};
 
-            KStreamBuilder builder = new KStreamBuilder();
+        long time = 0L;
 
-            final int[] expectedKeys = new int[]{0, 1, 2, 3};
+        KStream<Integer, String> stream1;
+        KStream<Integer, String> stream2;
+        KStream<Integer, String> joined;
+        MockProcessorSupplier<Integer, String> processor;
 
-            KStream<Integer, String> stream1;
-            KStream<Integer, String> stream2;
-            KStream<Integer, String> joined;
-            MockProcessorSupplier<Integer, String> processor;
+        processor = new MockProcessorSupplier<>();
+        stream1 = builder.stream(intSerde, stringSerde, topic1);
+        stream2 = builder.stream(intSerde, stringSerde, topic2);
+        joined = stream1.leftJoin(stream2, MockValueJoiner.STRING_JOINER, JoinWindows.of("test").within(100), intSerde, stringSerde);
+        joined.process(processor);
 
-            processor = new MockProcessorSupplier<>();
-            stream1 = builder.stream(intSerde, stringSerde, topic1);
-            stream2 = builder.stream(intSerde, stringSerde, topic2);
-            joined = stream1.leftJoin(stream2, joiner, JoinWindows.of("test").within(100), intSerde, stringSerde);
-            joined.process(processor);
+        Collection<Set<String>> copartitionGroups = builder.copartitionGroups();
 
-            Collection<Set<String>> copartitionGroups = builder.copartitionGroups();
+        assertEquals(1, copartitionGroups.size());
+        assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
 
-            assertEquals(1, copartitionGroups.size());
-            assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
+        driver = new KStreamTestDriver(builder, stateDir);
+        driver.setTime(time);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir);
-            driver.setTime(time);
+        // push two items to the primary stream. the other window is empty. this should produce two items
+        // w = {}
+        // --> w = {}
 
-            // push two items to the primary stream. the other window is empty. this should produce two items
-            // w = {}
-            // --> w = {}
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+null", "1:X1+null");
-
-            // push two items to the other stream. this should produce no items.
-            // w = {}
-            // --> w = { 0:Y0, 1:Y1 }
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult();
-
-            // clear logically
-            time = 1000L;
-
-            // push all items to the other stream. this should produce no items.
-            // w = {}
-            // --> w = { 0:Y0, 1:Y1, 2:Y2, 3:Y3 }
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.setTime(time + i);
-                driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult();
-
-            // gradually expire items in window.
-            // w = { 0:Y0, 1:Y1, 2:Y2, 3:Y3 }
-
-            time = 1000L + 100L;
-            driver.setTime(time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+Y2", "3:XX3+Y3");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+null", "3:XX3+Y3");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+null", "3:XX3+null");
-
-            // go back to the time before expiration
-
-            time = 1000L - 100L - 1L;
-            driver.setTime(time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+null", "3:XX3+null");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+null", "2:XX2+null", "3:XX3+null");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+null", "3:XX3+null");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+null");
-
-            driver.setTime(++time);
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3");
-
-        } finally {
-            Utils.delete(baseDir);
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
         }
-    }
 
+        processor.checkAndClearProcessResult("0:X0+null", "1:X1+null");
+
+        // push two items to the other stream. this should produce no items.
+        // w = {}
+        // --> w = { 0:Y0, 1:Y1 }
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult();
+
+        // clear logically
+        time = 1000L;
+
+        // push all items to the other stream. this should produce no items.
+        // w = {}
+        // --> w = { 0:Y0, 1:Y1, 2:Y2, 3:Y3 }
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.setTime(time + i);
+            driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult();
+
+        // gradually expire items in window.
+        // w = { 0:Y0, 1:Y1, 2:Y2, 3:Y3 }
+
+        time = 1000L + 100L;
+        driver.setTime(time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+Y2", "3:XX3+Y3");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+null", "3:XX3+Y3");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+null", "3:XX3+null");
+
+        // go back to the time before expiration
+
+        time = 1000L - 100L - 1L;
+        driver.setTime(time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+null", "3:XX3+null");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+null", "2:XX2+null", "3:XX3+null");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+null", "3:XX3+null");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+null");
+
+        driver.setTime(++time);
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        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/KStreamKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java
index 3acb59a..2c6108b 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java
@@ -20,19 +20,20 @@
 import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.serialization.Serdes;
-import org.apache.kafka.common.utils.Utils;
 import org.apache.kafka.streams.kstream.KStream;
 import org.apache.kafka.streams.kstream.KStreamBuilder;
 import org.apache.kafka.streams.kstream.KTable;
-import org.apache.kafka.streams.KeyValue;
-import org.apache.kafka.streams.kstream.KeyValueMapper;
-import org.apache.kafka.streams.kstream.ValueJoiner;
 import org.apache.kafka.test.KStreamTestDriver;
+import org.apache.kafka.test.MockKeyValueMapper;
 import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.MockValueJoiner;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 import java.io.File;
-import java.nio.file.Files;
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
@@ -42,111 +43,105 @@
 
 public class KStreamKTableLeftJoinTest {
 
-    private String topic1 = "topic1";
-    private String topic2 = "topic2";
+    final private String topic1 = "topic1";
+    final private String topic2 = "topic2";
 
-    final private Serde<Integer> intSerde = new Serdes.IntegerSerde();
-    final private Serde<String> stringSerde = new Serdes.StringSerde();
+    final private Serde<Integer> intSerde = Serdes.Integer();
+    final private Serde<String> stringSerde = Serdes.String();
 
-    private ValueJoiner<String, String, String> joiner = new ValueJoiner<String, String, String>() {
-        @Override
-        public String apply(String value1, String value2) {
-            return value1 + "+" + value2;
+    private KStreamTestDriver driver = null;
+    private File stateDir = null;
+
+    @After
+    public void tearDown() {
+        if (driver != null) {
+            driver.close();
         }
-    };
+        driver = null;
+    }
 
-    private KeyValueMapper<Integer, String, KeyValue<Integer, String>> keyValueMapper =
-        new KeyValueMapper<Integer, String, KeyValue<Integer, String>>() {
-            @Override
-            public KeyValue<Integer, String> apply(Integer key, String value) {
-                return KeyValue.pair(key, value);
-            }
-        };
+    @Before
+    public void setUp() throws IOException {
+        stateDir = TestUtils.tempDirectory("kafka-test");
+    }
 
     @Test
     public void testJoin() throws Exception {
-        File baseDir = Files.createTempDirectory("test").toFile();
-        try {
+        KStreamBuilder builder = new KStreamBuilder();
 
-            KStreamBuilder builder = new KStreamBuilder();
+        final int[] expectedKeys = new int[]{0, 1, 2, 3};
 
-            final int[] expectedKeys = new int[]{0, 1, 2, 3};
+        KStream<Integer, String> stream;
+        KTable<Integer, String> table;
+        MockProcessorSupplier<Integer, String> processor;
 
-            KStream<Integer, String> stream;
-            KTable<Integer, String> table;
-            MockProcessorSupplier<Integer, String> processor;
+        processor = new MockProcessorSupplier<>();
+        stream = builder.stream(intSerde, stringSerde, topic1);
+        table = builder.table(intSerde, stringSerde, topic2);
+        stream.leftJoin(table, MockValueJoiner.STRING_JOINER).process(processor);
 
-            processor = new MockProcessorSupplier<>();
-            stream = builder.stream(intSerde, stringSerde, topic1);
-            table = builder.table(intSerde, stringSerde, topic2);
-            stream.leftJoin(table, joiner).process(processor);
+        Collection<Set<String>> copartitionGroups = builder.copartitionGroups();
 
-            Collection<Set<String>> copartitionGroups = builder.copartitionGroups();
+        assertEquals(1, copartitionGroups.size());
+        assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
 
-            assertEquals(1, copartitionGroups.size());
-            assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
+        driver = new KStreamTestDriver(builder, stateDir);
+        driver.setTime(0L);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir);
-            driver.setTime(0L);
+        // push two items to the primary stream. the other table is empty
 
-            // push two items to the primary stream. the other table is empty
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+null", "1:X1+null");
-
-            // push two items to the other stream. this should not produce any item.
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult();
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null");
-
-            // push all items to the other stream. this should not produce any item
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult();
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
-
-            // push two items with null to the other stream as deletes. this should not produce any item.
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], null);
-            }
-
-            processor.checkAndClearProcessResult();
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+YY2", "3:XX3+YY3");
-
-        } finally {
-            Utils.delete(baseDir);
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
         }
+
+        processor.checkAndClearProcessResult("0:X0+null", "1:X1+null");
+
+        // push two items to the other stream. this should not produce any item.
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult();
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null");
+
+        // push all items to the other stream. this should not produce any item
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult();
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
+
+        // push two items with null to the other stream as deletes. this should not produce any item.
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], null);
+        }
+
+        processor.checkAndClearProcessResult();
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+YY2", "3:XX3+YY3");
     }
 
     @Test(expected = KafkaException.class)
@@ -158,10 +153,10 @@
         MockProcessorSupplier<Integer, String> processor;
 
         processor = new MockProcessorSupplier<>();
-        stream = builder.stream(intSerde, stringSerde, topic1).map(keyValueMapper);
+        stream = builder.stream(intSerde, stringSerde, topic1).map(MockKeyValueMapper.<Integer, String>NoOpKeyValueMapper());
         table = builder.table(intSerde, stringSerde, topic2);
 
-        stream.leftJoin(table, joiner).process(processor);
+        stream.leftJoin(table, MockValueJoiner.STRING_JOINER).process(processor);
     }
 
 }
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 68fa656..00e5d70 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
@@ -25,6 +25,7 @@
 import org.apache.kafka.streams.kstream.KeyValueMapper;
 import org.apache.kafka.test.KStreamTestDriver;
 import org.apache.kafka.test.MockProcessorSupplier;
+import org.junit.After;
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
@@ -36,6 +37,16 @@
     final private Serde<Integer> intSerde = Serdes.Integer();
     final private Serde<String> stringSerde = Serdes.String();
 
+    private KStreamTestDriver driver = null;
+
+    @After
+    public void cleanup() {
+        if (driver != null) {
+            driver.close();
+        }
+        driver = null;
+    }
+
     @Test
     public void testMap() {
         KStreamBuilder builder = new KStreamBuilder();
@@ -56,7 +67,7 @@
         processor = new MockProcessorSupplier<>();
         stream.map(mapper).process(processor);
 
-        KStreamTestDriver driver = new KStreamTestDriver(builder);
+        driver = new KStreamTestDriver(builder);
         for (int i = 0; i < expectedKeys.length; i++) {
             driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]);
         }
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 e671aab..e48b677 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
@@ -24,6 +24,7 @@
 import org.apache.kafka.streams.kstream.ValueMapper;
 import org.apache.kafka.test.KStreamTestDriver;
 import org.apache.kafka.test.MockProcessorSupplier;
+import org.junit.After;
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
@@ -35,6 +36,16 @@
     final private Serde<Integer> intSerde = Serdes.Integer();
     final private Serde<String> stringSerde = Serdes.String();
 
+    private KStreamTestDriver driver;
+
+    @After
+    public void cleanup() {
+        if (driver != null) {
+            driver.close();
+        }
+        driver = null;
+    }
+
     @Test
     public void testFlatMapValues() {
         KStreamBuilder builder = new KStreamBuilder();
@@ -54,7 +65,7 @@
         stream = builder.stream(intSerde, stringSerde, topicName);
         stream.mapValues(mapper).process(processor);
 
-        KStreamTestDriver driver = new KStreamTestDriver(builder);
+        driver = new KStreamTestDriver(builder);
         for (int i = 0; i < expectedKeys.length; i++) {
             driver.process(topicName, expectedKeys[i], Integer.toString(expectedKeys[i]));
         }
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
new file mode 100644
index 0000000..1bd870e
--- /dev/null
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java
@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.kafka.streams.kstream.internals;
+
+
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.KStreamBuilder;
+import org.apache.kafka.streams.kstream.KeyValueMapper;
+import org.apache.kafka.test.KStreamTestDriver;
+import org.apache.kafka.test.MockProcessorSupplier;
+import org.junit.After;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class KStreamSelectKeyTest {
+
+    private String topicName = "topic_key_select";
+
+    final private Serde<Integer> integerSerde = Serdes.Integer();
+    final private Serde<String> stringSerde = Serdes.String();
+
+    private KStreamTestDriver driver;
+
+    @After
+    public void cleanup() {
+        if (driver != null) {
+            driver.close();
+        }
+        driver = null;
+    }
+
+    @Test
+    public void testSelectKey() {
+        KStreamBuilder builder = new KStreamBuilder();
+
+        final Map<Integer, String> keyMap = new HashMap<>();
+        keyMap.put(1, "ONE");
+        keyMap.put(2, "TWO");
+        keyMap.put(3, "THREE");
+
+
+        KeyValueMapper<String, Integer, String> selector = new KeyValueMapper<String, Integer, String>() {
+            @Override
+            public String apply(String key, Integer value) {
+                return keyMap.get(value);
+            }
+        };
+
+        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(stringSerde, integerSerde, topicName);
+
+        MockProcessorSupplier<String, Integer> processor = new MockProcessorSupplier<>();
+
+        stream.selectKey(selector).process(processor);
+
+        driver = new KStreamTestDriver(builder);
+
+        for (int expectedValue : expectedValues) {
+            driver.process(topicName, null, expectedValue);
+        }
+
+        assertEquals(3, processor.processed.size());
+
+        for (int i = 0; i < expected.length; i++) {
+            assertEquals(expected[i], processor.processed.get(i));
+        }
+
+    }
+
+}
\ No newline at end of file
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 a0a61f2..e0bdfbc 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
@@ -27,6 +27,7 @@
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.test.KStreamTestDriver;
 import org.apache.kafka.test.MockProcessorSupplier;
+import org.junit.After;
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
@@ -37,6 +38,16 @@
 
     final private Serde<Integer> intSerde = Serdes.Integer();
 
+    private KStreamTestDriver driver;
+
+    @After
+    public void cleanup() {
+        if (driver != null) {
+            driver.close();
+        }
+        driver = null;
+    }
+
     @Test
     public void testTransform() {
         KStreamBuilder builder = new KStreamBuilder();
@@ -76,7 +87,7 @@
         KStream<Integer, Integer> stream = builder.stream(intSerde, intSerde, topicName);
         stream.transform(transformerSupplier).process(processor);
 
-        KStreamTestDriver driver = new KStreamTestDriver(builder);
+        driver = new KStreamTestDriver(builder);
         for (int i = 0; i < expectedKeys.length; i++) {
             driver.process(topicName, expectedKeys[i], expectedKeys[i] * 10);
         }
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 f5f9698..aebcc76 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
@@ -26,6 +26,7 @@
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.test.KStreamTestDriver;
 import org.apache.kafka.test.MockProcessorSupplier;
+import org.junit.After;
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
@@ -36,6 +37,16 @@
 
     final private Serde<Integer> intSerde = Serdes.Integer();
 
+    private KStreamTestDriver driver;
+
+    @After
+    public void cleanup() {
+        if (driver != null) {
+            driver.close();
+        }
+        driver = null;
+    }
+
     @Test
     public void testTransform() {
         KStreamBuilder builder = new KStreamBuilder();
@@ -76,7 +87,7 @@
         stream = builder.stream(intSerde, intSerde, topicName);
         stream.transformValues(valueTransformerSupplier).process(processor);
 
-        KStreamTestDriver driver = new KStreamTestDriver(builder);
+        driver = new KStreamTestDriver(builder);
         for (int i = 0; i < expectedKeys.length; i++) {
             driver.process(topicName, expectedKeys[i], expectedKeys[i] * 10);
         }
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java
index 3c7a1bd..f4fe3a6 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java
@@ -20,41 +20,45 @@
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.utils.Utils;
-import org.apache.kafka.streams.kstream.Aggregator;
-import org.apache.kafka.streams.kstream.HoppingWindows;
-import org.apache.kafka.streams.kstream.Initializer;
 import org.apache.kafka.streams.kstream.KStream;
 import org.apache.kafka.streams.kstream.KStreamBuilder;
 import org.apache.kafka.streams.kstream.KTable;
+import org.apache.kafka.streams.kstream.TimeWindows;
 import org.apache.kafka.streams.kstream.ValueJoiner;
 import org.apache.kafka.streams.kstream.Windowed;
 import org.apache.kafka.test.KStreamTestDriver;
+import org.apache.kafka.test.MockAggregator;
+import org.apache.kafka.test.MockInitializer;
 import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 import java.io.File;
+import java.io.IOException;
 import java.nio.file.Files;
 
 import static org.junit.Assert.assertEquals;
 
 public class KStreamWindowAggregateTest {
 
-    final private Serde<String> strSerde = new Serdes.StringSerde();
+    final private Serde<String> strSerde = Serdes.String();
 
-    private class StringAdd implements Aggregator<String, String, String> {
+    private KStreamTestDriver driver = null;
+    private File stateDir = null;
 
-        @Override
-        public String apply(String aggKey, String value, String aggregate) {
-            return aggregate + "+" + value;
+    @After
+    public void tearDown() {
+        if (driver != null) {
+            driver.close();
         }
+        driver = null;
     }
 
-    private class StringInit implements Initializer<String> {
-
-        @Override
-        public String apply() {
-            return "0";
-        }
+    @Before
+    public void setUp() throws IOException {
+        stateDir = TestUtils.tempDirectory("kafka-test");
     }
 
     @Test
@@ -66,8 +70,9 @@
             String topic1 = "topic1";
 
             KStream<String, String> stream1 = builder.stream(strSerde, strSerde, topic1);
-            KTable<Windowed<String>, String> table2 = stream1.aggregateByKey(new StringInit(), new StringAdd(),
-                    HoppingWindows.of("topic1-Canonized").with(10L).every(5L),
+            KTable<Windowed<String>, String> table2 =
+                stream1.aggregateByKey(MockInitializer.STRING_INIT, MockAggregator.STRING_ADDER,
+                    TimeWindows.of("topic1-Canonized", 10).advanceBy(5),
                     strSerde,
                     strSerde);
 
@@ -143,8 +148,9 @@
             String topic2 = "topic2";
 
             KStream<String, String> stream1 = builder.stream(strSerde, strSerde, topic1);
-            KTable<Windowed<String>, String> table1 = stream1.aggregateByKey(new StringInit(), new StringAdd(),
-                    HoppingWindows.of("topic1-Canonized").with(10L).every(5L),
+            KTable<Windowed<String>, String> table1 =
+                stream1.aggregateByKey(MockInitializer.STRING_INIT, MockAggregator.STRING_ADDER,
+                    TimeWindows.of("topic1-Canonized", 10).advanceBy(5),
                     strSerde,
                     strSerde);
 
@@ -152,8 +158,9 @@
             table1.toStream().process(proc1);
 
             KStream<String, String> stream2 = builder.stream(strSerde, strSerde, topic2);
-            KTable<Windowed<String>, String> table2 = stream2.aggregateByKey(new StringInit(), new StringAdd(),
-                    HoppingWindows.of("topic2-Canonized").with(10L).every(5L),
+            KTable<Windowed<String>, String> table2 =
+                stream2.aggregateByKey(MockInitializer.STRING_INIT, MockAggregator.STRING_ADDER,
+                    TimeWindows.of("topic2-Canonized", 10).advanceBy(5),
                     strSerde,
                     strSerde);
 
@@ -277,7 +284,6 @@
                     "[B@0]:0+2+2+2%0+b+b+b", "[B@5]:0+2+2%0+b+b",
                     "[C@0]:0+3+3%0+c+c", "[C@5]:0+3%0+c"
             );
-
         } finally {
             Utils.delete(baseDir);
         }
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 fc01e5e..a614479 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,91 +20,80 @@
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.utils.Utils;
-import org.apache.kafka.streams.kstream.Aggregator;
-import org.apache.kafka.streams.kstream.Initializer;
 import org.apache.kafka.streams.kstream.KStreamBuilder;
 import org.apache.kafka.streams.kstream.KTable;
 import org.apache.kafka.test.KStreamTestDriver;
+import org.apache.kafka.test.MockAggregator;
+import org.apache.kafka.test.MockInitializer;
+import org.apache.kafka.test.MockKeyValueMapper;
 import org.apache.kafka.test.MockProcessorSupplier;
-import org.apache.kafka.test.NoOpKeyValueMapper;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 import java.io.File;
-import java.nio.file.Files;
+import java.io.IOException;
 
 import static org.junit.Assert.assertEquals;
 
 public class KTableAggregateTest {
 
-    final private Serde<String> stringSerde = new Serdes.StringSerde();
+    final private Serde<String> stringSerde = Serdes.String();
 
-    private class StringAdd implements Aggregator<String, String, String> {
+    private KStreamTestDriver driver = null;
+    private File stateDir = null;
 
-        @Override
-        public String apply(String aggKey, String value, String aggregate) {
-            return aggregate + "+" + value;
+    @After
+    public void tearDown() {
+        if (driver != null) {
+            driver.close();
         }
+        driver = null;
     }
 
-    private class StringRemove implements Aggregator<String, String, String> {
-
-        @Override
-        public String apply(String aggKey, String value, String aggregate) {
-            return aggregate + "-" + value;
-        }
+    @Before
+    public void setUp() throws IOException {
+        stateDir = TestUtils.tempDirectory("kafka-test");
     }
 
-    private class StringInit implements Initializer<String> {
-
-        @Override
-        public String apply() {
-            return "0";
-        }
-    }
-
-
     @Test
     public void testAggBasic() throws Exception {
-        final File baseDir = Files.createTempDirectory("test").toFile();
+        final KStreamBuilder builder = new KStreamBuilder();
+        String topic1 = "topic1";
 
-        try {
-            final KStreamBuilder builder = new KStreamBuilder();
-            String topic1 = "topic1";
+        KTable<String, String> table1 = builder.table(stringSerde, stringSerde, topic1);
+        KTable<String, String> table2 = table1.groupBy(MockKeyValueMapper.<String, String>NoOpKeyValueMapper(),
+                stringSerde,
+                stringSerde
+        ).aggregate(MockInitializer.STRING_INIT,
+                MockAggregator.STRING_ADDER,
+                MockAggregator.STRING_REMOVER,
+                stringSerde,
+                "topic1-Canonized");
 
-            KTable<String, String> table1 = builder.table(stringSerde, stringSerde, topic1);
-            KTable<String, String> table2 = table1.aggregate(new StringInit(), new StringAdd(), new StringRemove(),
-                    new NoOpKeyValueMapper<String, String>(),
-                    stringSerde,
-                    stringSerde,
-                    stringSerde,
-                    "topic1-Canonized");
+        MockProcessorSupplier<String, String> proc2 = new MockProcessorSupplier<>();
+        table2.toStream().process(proc2);
 
-            MockProcessorSupplier<String, String> proc2 = new MockProcessorSupplier<>();
-            table2.toStream().process(proc2);
+        driver = new KStreamTestDriver(builder, stateDir);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir);
+        driver.process(topic1, "A", "1");
+        driver.process(topic1, "B", "2");
+        driver.process(topic1, "A", "3");
+        driver.process(topic1, "B", "4");
+        driver.process(topic1, "C", "5");
+        driver.process(topic1, "D", "6");
+        driver.process(topic1, "B", "7");
+        driver.process(topic1, "C", "8");
 
-            driver.process(topic1, "A", "1");
-            driver.process(topic1, "B", "2");
-            driver.process(topic1, "A", "3");
-            driver.process(topic1, "B", "4");
-            driver.process(topic1, "C", "5");
-            driver.process(topic1, "D", "6");
-            driver.process(topic1, "B", "7");
-            driver.process(topic1, "C", "8");
-
-            assertEquals(Utils.mkList(
-                    "A:0+1",
-                    "B:0+2",
-                    "A:0+1+3", "A:0+1+3-1",
-                    "B:0+2+4", "B:0+2+4-2",
-                    "C:0+5",
-                    "D:0+6",
-                    "B:0+2+4-2+7", "B:0+2+4-2+7-4",
-                    "C:0+5+8", "C:0+5+8-5"), proc2.processed);
-
-        } finally {
-            Utils.delete(baseDir);
-        }
+        assertEquals(Utils.mkList(
+                "A:0+1",
+                "B:0+2",
+                "A:0+1+3", "A:0+1+3-1",
+                "B:0+2+4", "B:0+2+4-2",
+                "C:0+5",
+                "D:0+6",
+                "B:0+2+4-2+7", "B:0+2+4-2+7-4",
+                "C:0+5+8", "C:0+5+8-5"), proc2.processed);
     }
 }
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 ee26058..a3af133 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
@@ -19,25 +19,42 @@
 
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.serialization.Serdes;
-import org.apache.kafka.common.utils.Utils;
 import org.apache.kafka.streams.kstream.KStreamBuilder;
 import org.apache.kafka.streams.kstream.KTable;
 import org.apache.kafka.streams.kstream.Predicate;
 import org.apache.kafka.test.KStreamTestDriver;
 import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 import java.io.File;
 import java.io.IOException;
-import java.nio.file.Files;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 
 public class KTableFilterTest {
 
-    final private Serde<Integer> intSerde = new Serdes.IntegerSerde();
-    final private Serde<String> stringSerde = new Serdes.StringSerde();
+    final private Serde<Integer> intSerde = Serdes.Integer();
+    final private Serde<String> stringSerde = Serdes.String();
+
+    private KStreamTestDriver driver = null;
+    private File stateDir = null;
+
+    @After
+    public void tearDown() {
+        if (driver != null) {
+            driver.close();
+        }
+        driver = null;
+    }
+
+    @Before
+    public void setUp() throws IOException {
+        stateDir = TestUtils.tempDirectory("kafka-test");
+    }
 
     @Test
     public void testKTable() {
@@ -65,7 +82,7 @@
         table2.toStream().process(proc2);
         table3.toStream().process(proc3);
 
-        KStreamTestDriver driver = new KStreamTestDriver(builder);
+        driver = new KStreamTestDriver(builder);
 
         driver.process(topic1, "A", 1);
         driver.process(topic1, "B", 2);
@@ -80,199 +97,181 @@
 
     @Test
     public void testValueGetter() throws IOException {
-        File stateDir = Files.createTempDirectory("test").toFile();
-        try {
-            final KStreamBuilder builder = new KStreamBuilder();
+        KStreamBuilder builder = new KStreamBuilder();
 
-            String topic1 = "topic1";
+        String topic1 = "topic1";
 
-            KTableImpl<String, Integer, Integer> table1 =
-                    (KTableImpl<String, Integer, Integer>) builder.table(stringSerde, intSerde, topic1);
-            KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
-                    new Predicate<String, Integer>() {
-                        @Override
-                        public boolean test(String key, Integer value) {
-                            return (value % 2) == 0;
-                        }
-                    });
-            KTableImpl<String, Integer, Integer> table3 = (KTableImpl<String, Integer, Integer>) table1.filterNot(
-                    new Predicate<String, Integer>() {
-                        @Override
-                        public boolean test(String key, Integer value) {
-                            return (value % 2) == 0;
-                        }
-                    });
+        KTableImpl<String, Integer, Integer> table1 =
+                (KTableImpl<String, Integer, Integer>) builder.table(stringSerde, intSerde, topic1);
+        KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
+                new Predicate<String, Integer>() {
+                    @Override
+                    public boolean test(String key, Integer value) {
+                        return (value % 2) == 0;
+                    }
+                });
+        KTableImpl<String, Integer, Integer> table3 = (KTableImpl<String, Integer, Integer>) table1.filterNot(
+                new Predicate<String, Integer>() {
+                    @Override
+                    public boolean test(String key, Integer value) {
+                        return (value % 2) == 0;
+                    }
+                });
 
-            KTableValueGetterSupplier<String, Integer> getterSupplier2 = table2.valueGetterSupplier();
-            KTableValueGetterSupplier<String, Integer> getterSupplier3 = table3.valueGetterSupplier();
+        KTableValueGetterSupplier<String, Integer> getterSupplier2 = table2.valueGetterSupplier();
+        KTableValueGetterSupplier<String, Integer> getterSupplier3 = table3.valueGetterSupplier();
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null);
+        driver = new KStreamTestDriver(builder, stateDir, null, null);
 
-            KTableValueGetter<String, Integer> getter2 = getterSupplier2.get();
-            KTableValueGetter<String, Integer> getter3 = getterSupplier3.get();
+        KTableValueGetter<String, Integer> getter2 = getterSupplier2.get();
+        KTableValueGetter<String, Integer> getter3 = getterSupplier3.get();
 
-            getter2.init(driver.context());
-            getter3.init(driver.context());
+        getter2.init(driver.context());
+        getter3.init(driver.context());
 
-            driver.process(topic1, "A", 1);
-            driver.process(topic1, "B", 1);
-            driver.process(topic1, "C", 1);
+        driver.process(topic1, "A", 1);
+        driver.process(topic1, "B", 1);
+        driver.process(topic1, "C", 1);
 
-            assertNull(getter2.get("A"));
-            assertNull(getter2.get("B"));
-            assertNull(getter2.get("C"));
+        assertNull(getter2.get("A"));
+        assertNull(getter2.get("B"));
+        assertNull(getter2.get("C"));
 
-            assertEquals(1, (int) getter3.get("A"));
-            assertEquals(1, (int) getter3.get("B"));
-            assertEquals(1, (int) getter3.get("C"));
+        assertEquals(1, (int) getter3.get("A"));
+        assertEquals(1, (int) getter3.get("B"));
+        assertEquals(1, (int) getter3.get("C"));
 
-            driver.process(topic1, "A", 2);
-            driver.process(topic1, "B", 2);
+        driver.process(topic1, "A", 2);
+        driver.process(topic1, "B", 2);
 
-            assertEquals(2, (int) getter2.get("A"));
-            assertEquals(2, (int) getter2.get("B"));
-            assertNull(getter2.get("C"));
+        assertEquals(2, (int) getter2.get("A"));
+        assertEquals(2, (int) getter2.get("B"));
+        assertNull(getter2.get("C"));
 
-            assertNull(getter3.get("A"));
-            assertNull(getter3.get("B"));
-            assertEquals(1, (int) getter3.get("C"));
+        assertNull(getter3.get("A"));
+        assertNull(getter3.get("B"));
+        assertEquals(1, (int) getter3.get("C"));
 
-            driver.process(topic1, "A", 3);
+        driver.process(topic1, "A", 3);
 
-            assertNull(getter2.get("A"));
-            assertEquals(2, (int) getter2.get("B"));
-            assertNull(getter2.get("C"));
+        assertNull(getter2.get("A"));
+        assertEquals(2, (int) getter2.get("B"));
+        assertNull(getter2.get("C"));
 
-            assertEquals(3, (int) getter3.get("A"));
-            assertNull(getter3.get("B"));
-            assertEquals(1, (int) getter3.get("C"));
+        assertEquals(3, (int) getter3.get("A"));
+        assertNull(getter3.get("B"));
+        assertEquals(1, (int) getter3.get("C"));
 
-            driver.process(topic1, "A", null);
-            driver.process(topic1, "B", null);
+        driver.process(topic1, "A", null);
+        driver.process(topic1, "B", null);
 
-            assertNull(getter2.get("A"));
-            assertNull(getter2.get("B"));
-            assertNull(getter2.get("C"));
+        assertNull(getter2.get("A"));
+        assertNull(getter2.get("B"));
+        assertNull(getter2.get("C"));
 
-            assertNull(getter3.get("A"));
-            assertNull(getter3.get("B"));
-            assertEquals(1, (int) getter3.get("C"));
-
-        } finally {
-            Utils.delete(stateDir);
-        }
+        assertNull(getter3.get("A"));
+        assertNull(getter3.get("B"));
+        assertEquals(1, (int) getter3.get("C"));
     }
 
     @Test
     public void testNotSendingOldValue() throws IOException {
-        File stateDir = Files.createTempDirectory("test").toFile();
-        try {
-            final KStreamBuilder builder = new KStreamBuilder();
+        KStreamBuilder builder = new KStreamBuilder();
 
-            String topic1 = "topic1";
+        String topic1 = "topic1";
 
-            KTableImpl<String, Integer, Integer> table1 =
-                    (KTableImpl<String, Integer, Integer>) builder.table(stringSerde, intSerde, topic1);
-            KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
-                    new Predicate<String, Integer>() {
-                        @Override
-                        public boolean test(String key, Integer value) {
-                            return (value % 2) == 0;
-                        }
-                    });
+        KTableImpl<String, Integer, Integer> table1 =
+                (KTableImpl<String, Integer, Integer>) builder.table(stringSerde, intSerde, topic1);
+        KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
+                new Predicate<String, Integer>() {
+                    @Override
+                    public boolean test(String key, Integer value) {
+                        return (value % 2) == 0;
+                    }
+                });
 
-            MockProcessorSupplier<String, Integer> proc1 = new MockProcessorSupplier<>();
-            MockProcessorSupplier<String, Integer> proc2 = new MockProcessorSupplier<>();
+        MockProcessorSupplier<String, Integer> proc1 = new MockProcessorSupplier<>();
+        MockProcessorSupplier<String, Integer> proc2 = new MockProcessorSupplier<>();
 
-            builder.addProcessor("proc1", proc1, table1.name);
-            builder.addProcessor("proc2", proc2, table2.name);
+        builder.addProcessor("proc1", proc1, table1.name);
+        builder.addProcessor("proc2", proc2, table2.name);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null);
+        driver = new KStreamTestDriver(builder, stateDir, null, null);
 
-            driver.process(topic1, "A", 1);
-            driver.process(topic1, "B", 1);
-            driver.process(topic1, "C", 1);
+        driver.process(topic1, "A", 1);
+        driver.process(topic1, "B", 1);
+        driver.process(topic1, "C", 1);
 
-            proc1.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)");
-            proc2.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)", "C:(null<-null)");
+        proc1.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)");
+        proc2.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)", "C:(null<-null)");
 
-            driver.process(topic1, "A", 2);
-            driver.process(topic1, "B", 2);
+        driver.process(topic1, "A", 2);
+        driver.process(topic1, "B", 2);
 
-            proc1.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)");
-            proc2.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)");
+        proc1.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)");
+        proc2.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)");
 
-            driver.process(topic1, "A", 3);
+        driver.process(topic1, "A", 3);
 
-            proc1.checkAndClearProcessResult("A:(3<-null)");
-            proc2.checkAndClearProcessResult("A:(null<-null)");
+        proc1.checkAndClearProcessResult("A:(3<-null)");
+        proc2.checkAndClearProcessResult("A:(null<-null)");
 
-            driver.process(topic1, "A", null);
-            driver.process(topic1, "B", null);
+        driver.process(topic1, "A", null);
+        driver.process(topic1, "B", null);
 
-            proc1.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)");
-            proc2.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)");
-
-        } finally {
-            Utils.delete(stateDir);
-        }
+        proc1.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)");
+        proc2.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)");
     }
 
     @Test
     public void testSendingOldValue() throws IOException {
-        File stateDir = Files.createTempDirectory("test").toFile();
-        try {
-            final KStreamBuilder builder = new KStreamBuilder();
+        KStreamBuilder builder = new KStreamBuilder();
 
-            String topic1 = "topic1";
+        String topic1 = "topic1";
 
-            KTableImpl<String, Integer, Integer> table1 =
-                    (KTableImpl<String, Integer, Integer>) builder.table(stringSerde, intSerde, topic1);
-            KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
-                    new Predicate<String, Integer>() {
-                        @Override
-                        public boolean test(String key, Integer value) {
-                            return (value % 2) == 0;
-                        }
-                    });
+        KTableImpl<String, Integer, Integer> table1 =
+                (KTableImpl<String, Integer, Integer>) builder.table(stringSerde, intSerde, topic1);
+        KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
+                new Predicate<String, Integer>() {
+                    @Override
+                    public boolean test(String key, Integer value) {
+                        return (value % 2) == 0;
+                    }
+                });
 
-            table2.enableSendingOldValues();
+        table2.enableSendingOldValues();
 
-            MockProcessorSupplier<String, Integer> proc1 = new MockProcessorSupplier<>();
-            MockProcessorSupplier<String, Integer> proc2 = new MockProcessorSupplier<>();
+        MockProcessorSupplier<String, Integer> proc1 = new MockProcessorSupplier<>();
+        MockProcessorSupplier<String, Integer> proc2 = new MockProcessorSupplier<>();
 
-            builder.addProcessor("proc1", proc1, table1.name);
-            builder.addProcessor("proc2", proc2, table2.name);
+        builder.addProcessor("proc1", proc1, table1.name);
+        builder.addProcessor("proc2", proc2, table2.name);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null);
+        driver = new KStreamTestDriver(builder, stateDir, null, null);
 
-            driver.process(topic1, "A", 1);
-            driver.process(topic1, "B", 1);
-            driver.process(topic1, "C", 1);
+        driver.process(topic1, "A", 1);
+        driver.process(topic1, "B", 1);
+        driver.process(topic1, "C", 1);
 
-            proc1.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)");
-            proc2.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)", "C:(null<-null)");
+        proc1.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)");
+        proc2.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)", "C:(null<-null)");
 
-            driver.process(topic1, "A", 2);
-            driver.process(topic1, "B", 2);
+        driver.process(topic1, "A", 2);
+        driver.process(topic1, "B", 2);
 
-            proc1.checkAndClearProcessResult("A:(2<-1)", "B:(2<-1)");
-            proc2.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)");
+        proc1.checkAndClearProcessResult("A:(2<-1)", "B:(2<-1)");
+        proc2.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)");
 
-            driver.process(topic1, "A", 3);
+        driver.process(topic1, "A", 3);
 
-            proc1.checkAndClearProcessResult("A:(3<-2)");
-            proc2.checkAndClearProcessResult("A:(null<-2)");
+        proc1.checkAndClearProcessResult("A:(3<-2)");
+        proc2.checkAndClearProcessResult("A:(null<-2)");
 
-            driver.process(topic1, "A", null);
-            driver.process(topic1, "B", null);
+        driver.process(topic1, "A", null);
+        driver.process(topic1, "B", null);
 
-            proc1.checkAndClearProcessResult("A:(null<-3)", "B:(null<-2)");
-            proc2.checkAndClearProcessResult("A:(null<-null)", "B:(null<-2)");
-
-        } finally {
-            Utils.delete(stateDir);
-        }
+        proc1.checkAndClearProcessResult("A:(null<-3)", "B:(null<-2)");
+        proc2.checkAndClearProcessResult("A:(null<-null)", "B:(null<-2)");
     }
 
 }
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableForeachTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableForeachTest.java
index 4b612a5..af131c2 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableForeachTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableForeachTest.java
@@ -24,8 +24,10 @@
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.kstream.KTable;
 import org.apache.kafka.test.KStreamTestDriver;
+import org.junit.After;
 import org.junit.Test;
 import java.util.List;
+import java.util.Locale;
 import java.util.ArrayList;
 import java.util.Arrays;
 
@@ -38,6 +40,16 @@
     final private Serde<Integer> intSerde = Serdes.Integer();
     final private Serde<String> stringSerde = Serdes.String();
 
+    private KStreamTestDriver driver;
+
+    @After
+    public void cleanup() {
+        if (driver != null) {
+            driver.close();
+        }
+        driver = null;
+    }
+
     @Test
     public void testForeach() {
         // Given
@@ -60,7 +72,7 @@
             new ForeachAction<Integer, String>() {
                 @Override
                 public void apply(Integer key, String value) {
-                    actualRecords.add(new KeyValue<>(key * 2, value.toUpperCase()));
+                    actualRecords.add(new KeyValue<>(key * 2, value.toUpperCase(Locale.ROOT)));
                 }
             };
 
@@ -70,7 +82,7 @@
         table.foreach(action);
 
         // Then
-        KStreamTestDriver driver = new KStreamTestDriver(builder);
+        driver = new KStreamTestDriver(builder);
         for (KeyValue<Integer, String> record: inputRecords) {
             driver.process(topicName, record.key, record.value);
         }
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 6f49b6a..ca3bbe1 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
@@ -25,20 +25,46 @@
 import org.apache.kafka.streams.kstream.Predicate;
 import org.apache.kafka.streams.kstream.ValueJoiner;
 import org.apache.kafka.streams.kstream.ValueMapper;
+import org.apache.kafka.streams.processor.internals.SinkNode;
+import org.apache.kafka.streams.processor.internals.SourceNode;
 import org.apache.kafka.test.KStreamTestDriver;
+import org.apache.kafka.test.MockAggregator;
+import org.apache.kafka.test.MockInitializer;
+import org.apache.kafka.test.MockKeyValueMapper;
 import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.MockReducer;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 import java.io.File;
 import java.io.IOException;
-import java.nio.file.Files;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 public class KTableImplTest {
 
-    final private Serde<String> stringSerde = new Serdes.StringSerde();
+    final private Serde<String> stringSerde = Serdes.String();
+
+    private KStreamTestDriver driver = null;
+    private File stateDir = null;
+
+    @After
+    public void tearDown() {
+        if (driver != null) {
+            driver.close();
+        }
+        driver = null;
+    }
+
+    @Before
+    public void setUp() throws IOException {
+        stateDir = TestUtils.tempDirectory("kafka-test");
+    }
 
     @Test
     public void testKTable() {
@@ -77,7 +103,7 @@
         MockProcessorSupplier<String, String> proc4 = new MockProcessorSupplier<>();
         table4.toStream().process(proc4);
 
-        KStreamTestDriver driver = new KStreamTestDriver(builder);
+        driver = new KStreamTestDriver(builder);
 
         driver.process(topic1, "A", "01");
         driver.process(topic1, "B", "02");
@@ -92,129 +118,157 @@
 
     @Test
     public void testValueGetter() throws IOException {
-        File stateDir = Files.createTempDirectory("test").toFile();
-        try {
-            final KStreamBuilder builder = new KStreamBuilder();
+        final KStreamBuilder builder = new KStreamBuilder();
 
-            String topic1 = "topic1";
-            String topic2 = "topic2";
+        String topic1 = "topic1";
+        String topic2 = "topic2";
 
-            KTableImpl<String, String, String> table1 =
-                    (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
-            KTableImpl<String, String, Integer> table2 = (KTableImpl<String, String, Integer>) table1.mapValues(
-                    new ValueMapper<String, Integer>() {
-                        @Override
-                        public Integer apply(String value) {
-                            return new Integer(value);
-                        }
-                    });
-            KTableImpl<String, Integer, Integer> table3 = (KTableImpl<String, Integer, Integer>) table2.filter(
-                    new Predicate<String, Integer>() {
-                        @Override
-                        public boolean test(String key, Integer value) {
-                            return (value % 2) == 0;
-                        }
-                    });
-            KTableImpl<String, String, String> table4 = (KTableImpl<String, String, String>)
-                    table1.through(stringSerde, stringSerde, topic2);
+        KTableImpl<String, String, String> table1 =
+                (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
+        KTableImpl<String, String, Integer> table2 = (KTableImpl<String, String, Integer>) table1.mapValues(
+                new ValueMapper<String, Integer>() {
+                    @Override
+                    public Integer apply(String value) {
+                        return new Integer(value);
+                    }
+                });
+        KTableImpl<String, Integer, Integer> table3 = (KTableImpl<String, Integer, Integer>) table2.filter(
+                new Predicate<String, Integer>() {
+                    @Override
+                    public boolean test(String key, Integer value) {
+                        return (value % 2) == 0;
+                    }
+                });
+        KTableImpl<String, String, String> table4 = (KTableImpl<String, String, String>)
+                table1.through(stringSerde, stringSerde, topic2);
 
-            KTableValueGetterSupplier<String, String> getterSupplier1 = table1.valueGetterSupplier();
-            KTableValueGetterSupplier<String, Integer> getterSupplier2 = table2.valueGetterSupplier();
-            KTableValueGetterSupplier<String, Integer> getterSupplier3 = table3.valueGetterSupplier();
-            KTableValueGetterSupplier<String, String> getterSupplier4 = table4.valueGetterSupplier();
+        KTableValueGetterSupplier<String, String> getterSupplier1 = table1.valueGetterSupplier();
+        KTableValueGetterSupplier<String, Integer> getterSupplier2 = table2.valueGetterSupplier();
+        KTableValueGetterSupplier<String, Integer> getterSupplier3 = table3.valueGetterSupplier();
+        KTableValueGetterSupplier<String, String> getterSupplier4 = table4.valueGetterSupplier();
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null);
+        driver = new KStreamTestDriver(builder, stateDir, null, null);
 
-            // two state store should be created
-            assertEquals(2, driver.allStateStores().size());
+        // two state store should be created
+        assertEquals(2, driver.allStateStores().size());
 
-            KTableValueGetter<String, String> getter1 = getterSupplier1.get();
-            getter1.init(driver.context());
-            KTableValueGetter<String, Integer> getter2 = getterSupplier2.get();
-            getter2.init(driver.context());
-            KTableValueGetter<String, Integer> getter3 = getterSupplier3.get();
-            getter3.init(driver.context());
-            KTableValueGetter<String, String> getter4 = getterSupplier4.get();
-            getter4.init(driver.context());
+        KTableValueGetter<String, String> getter1 = getterSupplier1.get();
+        getter1.init(driver.context());
+        KTableValueGetter<String, Integer> getter2 = getterSupplier2.get();
+        getter2.init(driver.context());
+        KTableValueGetter<String, Integer> getter3 = getterSupplier3.get();
+        getter3.init(driver.context());
+        KTableValueGetter<String, String> getter4 = getterSupplier4.get();
+        getter4.init(driver.context());
 
-            driver.process(topic1, "A", "01");
-            driver.process(topic1, "B", "01");
-            driver.process(topic1, "C", "01");
+        driver.process(topic1, "A", "01");
+        driver.process(topic1, "B", "01");
+        driver.process(topic1, "C", "01");
 
-            assertEquals("01", getter1.get("A"));
-            assertEquals("01", getter1.get("B"));
-            assertEquals("01", getter1.get("C"));
+        assertEquals("01", getter1.get("A"));
+        assertEquals("01", getter1.get("B"));
+        assertEquals("01", getter1.get("C"));
 
-            assertEquals(new Integer(1), getter2.get("A"));
-            assertEquals(new Integer(1), getter2.get("B"));
-            assertEquals(new Integer(1), getter2.get("C"));
+        assertEquals(new Integer(1), getter2.get("A"));
+        assertEquals(new Integer(1), getter2.get("B"));
+        assertEquals(new Integer(1), getter2.get("C"));
 
-            assertNull(getter3.get("A"));
-            assertNull(getter3.get("B"));
-            assertNull(getter3.get("C"));
+        assertNull(getter3.get("A"));
+        assertNull(getter3.get("B"));
+        assertNull(getter3.get("C"));
 
-            assertEquals("01", getter4.get("A"));
-            assertEquals("01", getter4.get("B"));
-            assertEquals("01", getter4.get("C"));
+        assertEquals("01", getter4.get("A"));
+        assertEquals("01", getter4.get("B"));
+        assertEquals("01", getter4.get("C"));
 
-            driver.process(topic1, "A", "02");
-            driver.process(topic1, "B", "02");
+        driver.process(topic1, "A", "02");
+        driver.process(topic1, "B", "02");
 
-            assertEquals("02", getter1.get("A"));
-            assertEquals("02", getter1.get("B"));
-            assertEquals("01", getter1.get("C"));
+        assertEquals("02", getter1.get("A"));
+        assertEquals("02", getter1.get("B"));
+        assertEquals("01", getter1.get("C"));
 
-            assertEquals(new Integer(2), getter2.get("A"));
-            assertEquals(new Integer(2), getter2.get("B"));
-            assertEquals(new Integer(1), getter2.get("C"));
+        assertEquals(new Integer(2), getter2.get("A"));
+        assertEquals(new Integer(2), getter2.get("B"));
+        assertEquals(new Integer(1), getter2.get("C"));
 
-            assertEquals(new Integer(2), getter3.get("A"));
-            assertEquals(new Integer(2), getter3.get("B"));
-            assertNull(getter3.get("C"));
+        assertEquals(new Integer(2), getter3.get("A"));
+        assertEquals(new Integer(2), getter3.get("B"));
+        assertNull(getter3.get("C"));
 
-            assertEquals("02", getter4.get("A"));
-            assertEquals("02", getter4.get("B"));
-            assertEquals("01", getter4.get("C"));
+        assertEquals("02", getter4.get("A"));
+        assertEquals("02", getter4.get("B"));
+        assertEquals("01", getter4.get("C"));
 
-            driver.process(topic1, "A", "03");
+        driver.process(topic1, "A", "03");
 
-            assertEquals("03", getter1.get("A"));
-            assertEquals("02", getter1.get("B"));
-            assertEquals("01", getter1.get("C"));
+        assertEquals("03", getter1.get("A"));
+        assertEquals("02", getter1.get("B"));
+        assertEquals("01", getter1.get("C"));
 
-            assertEquals(new Integer(3), getter2.get("A"));
-            assertEquals(new Integer(2), getter2.get("B"));
-            assertEquals(new Integer(1), getter2.get("C"));
+        assertEquals(new Integer(3), getter2.get("A"));
+        assertEquals(new Integer(2), getter2.get("B"));
+        assertEquals(new Integer(1), getter2.get("C"));
 
-            assertNull(getter3.get("A"));
-            assertEquals(new Integer(2), getter3.get("B"));
-            assertNull(getter3.get("C"));
+        assertNull(getter3.get("A"));
+        assertEquals(new Integer(2), getter3.get("B"));
+        assertNull(getter3.get("C"));
 
-            assertEquals("03", getter4.get("A"));
-            assertEquals("02", getter4.get("B"));
-            assertEquals("01", getter4.get("C"));
+        assertEquals("03", getter4.get("A"));
+        assertEquals("02", getter4.get("B"));
+        assertEquals("01", getter4.get("C"));
 
-            driver.process(topic1, "A", null);
+        driver.process(topic1, "A", null);
 
-            assertNull(getter1.get("A"));
-            assertEquals("02", getter1.get("B"));
-            assertEquals("01", getter1.get("C"));
+        assertNull(getter1.get("A"));
+        assertEquals("02", getter1.get("B"));
+        assertEquals("01", getter1.get("C"));
 
-            assertNull(getter2.get("A"));
-            assertEquals(new Integer(2), getter2.get("B"));
-            assertEquals(new Integer(1), getter2.get("C"));
+        assertNull(getter2.get("A"));
+        assertEquals(new Integer(2), getter2.get("B"));
+        assertEquals(new Integer(1), getter2.get("C"));
 
-            assertNull(getter3.get("A"));
-            assertEquals(new Integer(2), getter3.get("B"));
-            assertNull(getter3.get("C"));
+        assertNull(getter3.get("A"));
+        assertEquals(new Integer(2), getter3.get("B"));
+        assertNull(getter3.get("C"));
 
-            assertNull(getter4.get("A"));
-            assertEquals("02", getter4.get("B"));
-            assertEquals("01", getter4.get("C"));
+        assertNull(getter4.get("A"));
+        assertEquals("02", getter4.get("B"));
+        assertEquals("01", getter4.get("C"));
+    }
 
-        } finally {
-            Utils.delete(stateDir);
-        }
+    @Test
+    public void testStateStoreLazyEval() throws IOException {
+        String topic1 = "topic1";
+        String topic2 = "topic2";
+
+        final KStreamBuilder builder = new KStreamBuilder();
+
+        KTableImpl<String, String, String> table1 =
+                (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
+        KTableImpl<String, String, String> table2 =
+                (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic2);
+
+        KTableImpl<String, String, Integer> table1Mapped = (KTableImpl<String, String, Integer>) table1.mapValues(
+                new ValueMapper<String, Integer>() {
+                    @Override
+                    public Integer apply(String value) {
+                        return new Integer(value);
+                    }
+                });
+        KTableImpl<String, Integer, Integer> table1MappedFiltered = (KTableImpl<String, Integer, Integer>) table1Mapped.filter(
+                new Predicate<String, Integer>() {
+                    @Override
+                    public boolean test(String key, Integer value) {
+                        return (value % 2) == 0;
+                    }
+                });
+
+        driver = new KStreamTestDriver(builder, stateDir, null, null);
+        driver.setTime(0L);
+
+        // no state store should be created
+        assertEquals(0, driver.allStateStores().size());
     }
 
     @Test
@@ -222,79 +276,75 @@
         String topic1 = "topic1";
         String topic2 = "topic2";
 
-        File stateDir = Files.createTempDirectory("test").toFile();
-        try {
-            KStreamBuilder builder = new KStreamBuilder();
+        final KStreamBuilder builder = new KStreamBuilder();
 
-            KTableImpl<String, String, String> table1 =
-                    (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
-            KTableImpl<String, String, String> table2 =
-                    (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic2);
+        KTableImpl<String, String, String> table1 =
+                (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
+        KTableImpl<String, String, String> table2 =
+                (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic2);
 
-            KTableImpl<String, String, Integer> table1Mapped = (KTableImpl<String, String, Integer>) table1.mapValues(
-                    new ValueMapper<String, Integer>() {
-                        @Override
-                        public Integer apply(String value) {
-                            return new Integer(value);
-                        }
-                    });
-            KTableImpl<String, Integer, Integer> table1MappedFiltered = (KTableImpl<String, Integer, Integer>) table1Mapped.filter(
-                    new Predicate<String, Integer>() {
-                        @Override
-                        public boolean test(String key, Integer value) {
-                            return (value % 2) == 0;
-                        }
-                    });
+        KTableImpl<String, String, Integer> table1Mapped = (KTableImpl<String, String, Integer>) table1.mapValues(
+                new ValueMapper<String, Integer>() {
+                    @Override
+                    public Integer apply(String value) {
+                        return new Integer(value);
+                    }
+                });
+        KTableImpl<String, Integer, Integer> table1MappedFiltered = (KTableImpl<String, Integer, Integer>) table1Mapped.filter(
+                new Predicate<String, Integer>() {
+                    @Override
+                    public boolean test(String key, Integer value) {
+                        return (value % 2) == 0;
+                    }
+                });
+        table2.join(table1MappedFiltered,
+                new ValueJoiner<String, Integer, String>() {
+                    @Override
+                    public String apply(String v1, Integer v2) {
+                        return v1 + v2;
+                    }
+                });
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null);
-            driver.setTime(0L);
+        driver = new KStreamTestDriver(builder, stateDir, null, null);
+        driver.setTime(0L);
 
-            // no state store should be created
-            assertEquals(0, driver.allStateStores().size());
+        // two state store should be created
+        assertEquals(2, driver.allStateStores().size());
+    }
 
-        } finally {
-            Utils.delete(stateDir);
-        }
+    @Test
+    public void testRepartition() throws IOException {
+        String topic1 = "topic1";
 
-        try {
-            KStreamBuilder builder = new KStreamBuilder();
+        final KStreamBuilder builder = new KStreamBuilder();
 
-            KTableImpl<String, String, String> table1 =
-                    (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
-            KTableImpl<String, String, String> table2 =
-                    (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic2);
+        KTableImpl<String, String, String> table1 =
+                (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
 
-            KTableImpl<String, String, Integer> table1Mapped = (KTableImpl<String, String, Integer>) table1.mapValues(
-                    new ValueMapper<String, Integer>() {
-                        @Override
-                        public Integer apply(String value) {
-                            return new Integer(value);
-                        }
-                    });
-            KTableImpl<String, Integer, Integer> table1MappedFiltered = (KTableImpl<String, Integer, Integer>) table1Mapped.filter(
-                    new Predicate<String, Integer>() {
-                        @Override
-                        public boolean test(String key, Integer value) {
-                            return (value % 2) == 0;
-                        }
-                    });
-            table2.join(table1MappedFiltered,
-                    new ValueJoiner<String, Integer, String>() {
-                        @Override
-                        public String apply(String v1, Integer v2) {
-                            return v1 + v2;
-                        }
-                    });
+        KTableImpl<String, String, String> table1Aggregated = (KTableImpl<String, String, String>) table1
+                .groupBy(MockKeyValueMapper.<String, String>NoOpKeyValueMapper())
+                .aggregate(MockInitializer.STRING_INIT, MockAggregator.STRING_ADDER, MockAggregator.STRING_REMOVER, "mock-result1");
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null);
-            driver.setTime(0L);
 
-            // two state store should be created
-            assertEquals(2, driver.allStateStores().size());
+        KTableImpl<String, String, String> table1Reduced = (KTableImpl<String, String, String>) table1
+                .groupBy(MockKeyValueMapper.<String, String>NoOpKeyValueMapper())
+                .reduce(MockReducer.STRING_ADDER, MockReducer.STRING_REMOVER, "mock-result2");
 
-        } finally {
-            Utils.delete(stateDir);
-        }
+        driver = new KStreamTestDriver(builder, stateDir, stringSerde, stringSerde);
+        driver.setTime(0L);
 
+        // three state store should be created, one for source, one for aggregate and one for reduce
+        assertEquals(3, driver.allStateStores().size());
+
+        // contains the corresponding repartition source / sink nodes
+        assertTrue(driver.allProcessorNames().contains("KSTREAM-SINK-0000000003"));
+        assertTrue(driver.allProcessorNames().contains("KSTREAM-SOURCE-0000000004"));
+        assertTrue(driver.allProcessorNames().contains("KSTREAM-SINK-0000000007"));
+        assertTrue(driver.allProcessorNames().contains("KSTREAM-SOURCE-0000000008"));
+
+        assertNotNull(((ChangedSerializer) ((SinkNode) driver.processor("KSTREAM-SINK-0000000003")).valueSerializer()).inner());
+        assertNotNull(((ChangedDeserializer) ((SourceNode) driver.processor("KSTREAM-SOURCE-0000000004")).valueDeserializer()).inner());
+        assertNotNull(((ChangedSerializer) ((SinkNode) driver.processor("KSTREAM-SINK-0000000007")).valueSerializer()).inner());
+        assertNotNull(((ChangedDeserializer) ((SourceNode) driver.processor("KSTREAM-SOURCE-0000000008")).valueDeserializer()).inner());
     }
 }
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java
index f6ebbe1..16015fe 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java
@@ -19,17 +19,19 @@
 
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.serialization.Serdes;
-import org.apache.kafka.common.utils.Utils;
 import org.apache.kafka.streams.kstream.KStreamBuilder;
 import org.apache.kafka.streams.kstream.KTable;
 import org.apache.kafka.streams.KeyValue;
-import org.apache.kafka.streams.kstream.ValueJoiner;
 import org.apache.kafka.test.KStreamTestDriver;
 import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.MockValueJoiner;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 import java.io.File;
-import java.nio.file.Files;
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
@@ -42,309 +44,291 @@
 
 public class KTableKTableJoinTest {
 
-    private String topic1 = "topic1";
-    private String topic2 = "topic2";
+    final private String topic1 = "topic1";
+    final private String topic2 = "topic2";
 
-    final private Serde<Integer> intSerde = new Serdes.IntegerSerde();
-    final private Serde<String> stringSerde = new Serdes.StringSerde();
+    final private Serde<Integer> intSerde = Serdes.Integer();
+    final private Serde<String> stringSerde = Serdes.String();
 
-    private ValueJoiner<String, String, String> joiner = new ValueJoiner<String, String, String>() {
-        @Override
-        public String apply(String value1, String value2) {
-            return value1 + "+" + value2;
+    private KStreamTestDriver driver = null;
+    private File stateDir = null;
+
+    @After
+    public void tearDown() {
+        if (driver != null) {
+            driver.close();
         }
-    };
+        driver = null;
+    }
 
-    private static class JoinedKeyValue extends KeyValue<Integer, String> {
-        public JoinedKeyValue(Integer key, String value) {
-            super(key, value);
-        }
+    @Before
+    public void setUp() throws IOException {
+        stateDir = TestUtils.tempDirectory("kafka-test");
     }
 
     @Test
     public void testJoin() throws Exception {
-        File baseDir = Files.createTempDirectory("test").toFile();
-        try {
+        final KStreamBuilder builder = new KStreamBuilder();
 
-            KStreamBuilder builder = new KStreamBuilder();
+        final int[] expectedKeys = new int[]{0, 1, 2, 3};
 
-            final int[] expectedKeys = new int[]{0, 1, 2, 3};
+        KTable<Integer, String> table1;
+        KTable<Integer, String> table2;
+        KTable<Integer, String> joined;
+        MockProcessorSupplier<Integer, String> processor;
 
-            KTable<Integer, String> table1;
-            KTable<Integer, String> table2;
-            KTable<Integer, String> joined;
-            MockProcessorSupplier<Integer, String> processor;
+        processor = new MockProcessorSupplier<>();
+        table1 = builder.table(intSerde, stringSerde, topic1);
+        table2 = builder.table(intSerde, stringSerde, topic2);
+        joined = table1.join(table2, MockValueJoiner.STRING_JOINER);
+        joined.toStream().process(processor);
 
-            processor = new MockProcessorSupplier<>();
-            table1 = builder.table(intSerde, stringSerde, topic1);
-            table2 = builder.table(intSerde, stringSerde, topic2);
-            joined = table1.join(table2, joiner);
-            joined.toStream().process(processor);
+        Collection<Set<String>> copartitionGroups = builder.copartitionGroups();
 
-            Collection<Set<String>> copartitionGroups = builder.copartitionGroups();
+        assertEquals(1, copartitionGroups.size());
+        assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
 
-            assertEquals(1, copartitionGroups.size());
-            assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
+        KTableValueGetterSupplier<Integer, String> getterSupplier = ((KTableImpl<Integer, String, String>) joined).valueGetterSupplier();
 
-            KTableValueGetterSupplier<Integer, String> getterSupplier = ((KTableImpl<Integer, String, String>) joined).valueGetterSupplier();
+        driver = new KStreamTestDriver(builder, stateDir);
+        driver.setTime(0L);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir);
-            driver.setTime(0L);
+        KTableValueGetter<Integer, String> getter = getterSupplier.get();
+        getter.init(driver.context());
 
-            KTableValueGetter<Integer, String> getter = getterSupplier.get();
-            getter.init(driver.context());
+        // push two items to the primary stream. the other table is empty
 
-            // push two items to the primary stream. the other table is empty
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:null", "1:null");
-            checkJoinedValues(getter, kv(0, null), kv(1, null), kv(2, null), kv(3, null));
-
-            // push two items to the other stream. this should produce two items.
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1");
-            checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null));
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:null", "3:null");
-            checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null));
-
-            // push all items to the other stream. this should produce four items.
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
-            checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3"));
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
-            checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3"));
-
-            // push two items with null to the other stream as deletes. this should produce two item.
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], null);
-            }
-
-            processor.checkAndClearProcessResult("0:null", "1:null");
-            checkJoinedValues(getter, kv(0, null), kv(1, null), kv(2, "X2+YY2"), kv(3, "X3+YY3"));
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:null", "1:null", "2:XX2+YY2", "3:XX3+YY3");
-            checkJoinedValues(getter, kv(0, null), kv(1, null), kv(2, "XX2+YY2"), kv(3, "XX3+YY3"));
-
-        } finally {
-            Utils.delete(baseDir);
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
         }
+
+        processor.checkAndClearProcessResult("0:null", "1:null");
+        checkJoinedValues(getter, kv(0, null), kv(1, null), kv(2, null), kv(3, null));
+
+        // push two items to the other stream. this should produce two items.
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1");
+        checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null));
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:null", "3:null");
+        checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null));
+
+        // push all items to the other stream. this should produce four items.
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
+        checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3"));
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
+        checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3"));
+
+        // push two items with null to the other stream as deletes. this should produce two item.
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], null);
+        }
+
+        processor.checkAndClearProcessResult("0:null", "1:null");
+        checkJoinedValues(getter, kv(0, null), kv(1, null), kv(2, "X2+YY2"), kv(3, "X3+YY3"));
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:null", "1:null", "2:XX2+YY2", "3:XX3+YY3");
+        checkJoinedValues(getter, kv(0, null), kv(1, null), kv(2, "XX2+YY2"), kv(3, "XX3+YY3"));
     }
 
     @Test
     public void testNotSendingOldValues() throws Exception {
-        File baseDir = Files.createTempDirectory("test").toFile();
-        try {
+        final KStreamBuilder builder = new KStreamBuilder();
 
-            KStreamBuilder builder = new KStreamBuilder();
+        final int[] expectedKeys = new int[]{0, 1, 2, 3};
 
-            final int[] expectedKeys = new int[]{0, 1, 2, 3};
+        KTable<Integer, String> table1;
+        KTable<Integer, String> table2;
+        KTable<Integer, String> joined;
+        MockProcessorSupplier<Integer, String> proc;
 
-            KTable<Integer, String> table1;
-            KTable<Integer, String> table2;
-            KTable<Integer, String> joined;
-            MockProcessorSupplier<Integer, String> proc;
+        table1 = builder.table(intSerde, stringSerde, topic1);
+        table2 = builder.table(intSerde, stringSerde, topic2);
+        joined = table1.join(table2, MockValueJoiner.STRING_JOINER);
 
-            table1 = builder.table(intSerde, stringSerde, topic1);
-            table2 = builder.table(intSerde, stringSerde, topic2);
-            joined = table1.join(table2, joiner);
+        proc = new MockProcessorSupplier<>();
+        builder.addProcessor("proc", proc, ((KTableImpl<?, ?, ?>) joined).name);
 
-            proc = new MockProcessorSupplier<>();
-            builder.addProcessor("proc", proc, ((KTableImpl<?, ?, ?>) joined).name);
+        driver = new KStreamTestDriver(builder, stateDir);
+        driver.setTime(0L);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir);
-            driver.setTime(0L);
+        assertFalse(((KTableImpl<?, ?, ?>) table1).sendingOldValueEnabled());
+        assertFalse(((KTableImpl<?, ?, ?>) table2).sendingOldValueEnabled());
+        assertFalse(((KTableImpl<?, ?, ?>) joined).sendingOldValueEnabled());
 
-            assertFalse(((KTableImpl<?, ?, ?>) table1).sendingOldValueEnabled());
-            assertFalse(((KTableImpl<?, ?, ?>) table2).sendingOldValueEnabled());
-            assertFalse(((KTableImpl<?, ?, ?>) joined).sendingOldValueEnabled());
+        // push two items to the primary stream. the other table is empty
 
-            // push two items to the primary stream. the other table is empty
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)");
-
-            // push two items to the other stream. this should produce two items.
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)");
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(null<-null)", "3:(null<-null)");
-
-            // push all items to the other stream. this should produce four items.
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)");
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)");
-
-            // push two items with null to the other stream as deletes. this should produce two item.
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], null);
-            }
-
-            proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)");
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)", "2:(XX2+YY2<-null)", "3:(XX3+YY3<-null)");
-
-        } finally {
-            Utils.delete(baseDir);
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
         }
+
+        proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)");
+
+        // push two items to the other stream. this should produce two items.
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)");
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(null<-null)", "3:(null<-null)");
+
+        // push all items to the other stream. this should produce four items.
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)");
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)");
+
+        // push two items with null to the other stream as deletes. this should produce two item.
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], null);
+        }
+
+        proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)");
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)", "2:(XX2+YY2<-null)", "3:(XX3+YY3<-null)");
     }
 
     @Test
     public void testSendingOldValues() throws Exception {
-        File baseDir = Files.createTempDirectory("test").toFile();
-        try {
+        final KStreamBuilder builder = new KStreamBuilder();
 
-            KStreamBuilder builder = new KStreamBuilder();
+        final int[] expectedKeys = new int[]{0, 1, 2, 3};
 
-            final int[] expectedKeys = new int[]{0, 1, 2, 3};
+        KTable<Integer, String> table1;
+        KTable<Integer, String> table2;
+        KTable<Integer, String> joined;
+        MockProcessorSupplier<Integer, String> proc;
 
-            KTable<Integer, String> table1;
-            KTable<Integer, String> table2;
-            KTable<Integer, String> joined;
-            MockProcessorSupplier<Integer, String> proc;
+        table1 = builder.table(intSerde, stringSerde, topic1);
+        table2 = builder.table(intSerde, stringSerde, topic2);
+        joined = table1.join(table2, MockValueJoiner.STRING_JOINER);
 
-            table1 = builder.table(intSerde, stringSerde, topic1);
-            table2 = builder.table(intSerde, stringSerde, topic2);
-            joined = table1.join(table2, joiner);
+        ((KTableImpl<?, ?, ?>) joined).enableSendingOldValues();
 
-            ((KTableImpl<?, ?, ?>) joined).enableSendingOldValues();
+        proc = new MockProcessorSupplier<>();
+        builder.addProcessor("proc", proc, ((KTableImpl<?, ?, ?>) joined).name);
 
-            proc = new MockProcessorSupplier<>();
-            builder.addProcessor("proc", proc, ((KTableImpl<?, ?, ?>) joined).name);
+        driver = new KStreamTestDriver(builder, stateDir);
+        driver.setTime(0L);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir);
-            driver.setTime(0L);
+        assertTrue(((KTableImpl<?, ?, ?>) table1).sendingOldValueEnabled());
+        assertTrue(((KTableImpl<?, ?, ?>) table2).sendingOldValueEnabled());
+        assertTrue(((KTableImpl<?, ?, ?>) joined).sendingOldValueEnabled());
 
-            assertTrue(((KTableImpl<?, ?, ?>) table1).sendingOldValueEnabled());
-            assertTrue(((KTableImpl<?, ?, ?>) table2).sendingOldValueEnabled());
-            assertTrue(((KTableImpl<?, ?, ?>) joined).sendingOldValueEnabled());
+        // push two items to the primary stream. the other table is empty
 
-            // push two items to the primary stream. the other table is empty
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)");
-
-            // push two items to the other stream. this should produce two items.
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)");
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(null<-null)", "3:(null<-null)");
-
-            // push all items to the other stream. this should produce four items.
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+YY0<-X0+Y0)", "1:(X1+YY1<-X1+Y1)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)");
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+YY0<-X0+YY0)", "1:(X1+YY1<-X1+YY1)", "2:(X2+YY2<-X2+YY2)", "3:(X3+YY3<-X3+YY3)");
-
-            // push two items with null to the other stream as deletes. this should produce two item.
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], null);
-            }
-
-            proc.checkAndClearProcessResult("0:(null<-X0+YY0)", "1:(null<-X1+YY1)");
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)", "2:(XX2+YY2<-X2+YY2)", "3:(XX3+YY3<-X3+YY3)");
-
-        } finally {
-            Utils.delete(baseDir);
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
         }
+
+        proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)");
+
+        // push two items to the other stream. this should produce two items.
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)");
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(null<-null)", "3:(null<-null)");
+
+        // push all items to the other stream. this should produce four items.
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+YY0<-X0+Y0)", "1:(X1+YY1<-X1+Y1)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)");
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+YY0<-X0+YY0)", "1:(X1+YY1<-X1+YY1)", "2:(X2+YY2<-X2+YY2)", "3:(X3+YY3<-X3+YY3)");
+
+        // push two items with null to the other stream as deletes. this should produce two item.
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], null);
+        }
+
+        proc.checkAndClearProcessResult("0:(null<-X0+YY0)", "1:(null<-X1+YY1)");
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)", "2:(XX2+YY2<-X2+YY2)", "3:(XX3+YY3<-X3+YY3)");
     }
 
-    private JoinedKeyValue kv(Integer key, String value) {
-        return new JoinedKeyValue(key, value);
+    private KeyValue<Integer, String> kv(Integer key, String value) {
+        return new KeyValue<>(key, value);
     }
 
-    private void checkJoinedValues(KTableValueGetter<Integer, String> getter, JoinedKeyValue... expected) {
-        for (JoinedKeyValue kv : expected) {
+    private void checkJoinedValues(KTableValueGetter<Integer, String> getter, KeyValue<Integer, String>... expected) {
+        for (KeyValue<Integer, String> kv : expected) {
             String value = getter.get(kv.key);
             if (kv.value == null) {
                 assertNull(value);
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java
index 449ea05..5132ce3 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java
@@ -19,18 +19,19 @@
 
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.serialization.Serdes;
-import org.apache.kafka.common.utils.Utils;
 import org.apache.kafka.streams.kstream.KStreamBuilder;
 import org.apache.kafka.streams.kstream.KTable;
 import org.apache.kafka.streams.KeyValue;
-import org.apache.kafka.streams.kstream.KeyValueMapper;
-import org.apache.kafka.streams.kstream.ValueJoiner;
 import org.apache.kafka.test.KStreamTestDriver;
 import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.MockValueJoiner;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 import java.io.File;
-import java.nio.file.Files;
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
@@ -43,313 +44,287 @@
 
 public class KTableKTableLeftJoinTest {
 
-    private String topic1 = "topic1";
-    private String topic2 = "topic2";
+    final private String topic1 = "topic1";
+    final private String topic2 = "topic2";
 
-    final private Serde<Integer> intSerde = new Serdes.IntegerSerde();
-    final private Serde<String> stringSerde = new Serdes.StringSerde();
+    final private Serde<Integer> intSerde = Serdes.Integer();
+    final private Serde<String> stringSerde = Serdes.String();
 
-    private ValueJoiner<String, String, String> joiner = new ValueJoiner<String, String, String>() {
-        @Override
-        public String apply(String value1, String value2) {
-            return value1 + "+" + value2;
+    private KStreamTestDriver driver = null;
+    private File stateDir = null;
+
+    @After
+    public void tearDown() {
+        if (driver != null) {
+            driver.close();
         }
-    };
+        driver = null;
+    }
 
-    private KeyValueMapper<Integer, String, KeyValue<Integer, String>> keyValueMapper =
-        new KeyValueMapper<Integer, String, KeyValue<Integer, String>>() {
-            @Override
-            public KeyValue<Integer, String> apply(Integer key, String value) {
-                return KeyValue.pair(key, value);
-            }
-        };
-
-    private static class JoinedKeyValue extends KeyValue<Integer, String> {
-        public JoinedKeyValue(Integer key, String value) {
-            super(key, value);
-        }
+    @Before
+    public void setUp() throws IOException {
+        stateDir = TestUtils.tempDirectory("kafka-test");
     }
 
     @Test
     public void testJoin() throws Exception {
-        File baseDir = Files.createTempDirectory("test").toFile();
-        try {
+        final KStreamBuilder builder = new KStreamBuilder();
 
-            KStreamBuilder builder = new KStreamBuilder();
+        final int[] expectedKeys = new int[]{0, 1, 2, 3};
 
-            final int[] expectedKeys = new int[]{0, 1, 2, 3};
+        KTable<Integer, String> table1 = builder.table(intSerde, stringSerde, topic1);
+        KTable<Integer, String> table2 = builder.table(intSerde, stringSerde, topic2);
+        KTable<Integer, String> joined = table1.leftJoin(table2, MockValueJoiner.STRING_JOINER);
+        MockProcessorSupplier<Integer, String> processor;
+        processor = new MockProcessorSupplier<>();
+        joined.toStream().process(processor);
 
-            KTable<Integer, String> table1 = builder.table(intSerde, stringSerde, topic1);
-            KTable<Integer, String> table2 = builder.table(intSerde, stringSerde, topic2);
-            KTable<Integer, String> joined = table1.leftJoin(table2, joiner);
-            MockProcessorSupplier<Integer, String> processor;
-            processor = new MockProcessorSupplier<>();
-            joined.toStream().process(processor);
+        Collection<Set<String>> copartitionGroups = builder.copartitionGroups();
 
-            Collection<Set<String>> copartitionGroups = builder.copartitionGroups();
+        assertEquals(1, copartitionGroups.size());
+        assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
 
-            assertEquals(1, copartitionGroups.size());
-            assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
+        KTableValueGetterSupplier<Integer, String> getterSupplier = ((KTableImpl<Integer, String, String>) joined).valueGetterSupplier();
 
-            KTableValueGetterSupplier<Integer, String> getterSupplier = ((KTableImpl<Integer, String, String>) joined).valueGetterSupplier();
+        driver = new KStreamTestDriver(builder, stateDir);
+        driver.setTime(0L);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir);
-            driver.setTime(0L);
+        KTableValueGetter<Integer, String> getter = getterSupplier.get();
+        getter.init(driver.context());
 
-            KTableValueGetter<Integer, String> getter = getterSupplier.get();
-            getter.init(driver.context());
+        // push two items to the primary stream. the other table is empty
 
-            // push two items to the primary stream. the other table is empty
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+null", "1:X1+null");
-            checkJoinedValues(getter, kv(0, "X0+null"), kv(1, "X1+null"), kv(2, null), kv(3, null));
-
-            // push two items to the other stream. this should produce two items.
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1");
-            checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null));
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null");
-            checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, "X2+null"), kv(3, "X3+null"));
-
-            // push all items to the other stream. this should produce four items.
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
-            checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3"));
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
-            checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3"));
-
-            // push two items with null to the other stream as deletes. this should produce two item.
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], null);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+null", "1:X1+null");
-            checkJoinedValues(getter, kv(0, "X0+null"), kv(1, "X1+null"), kv(2, "X2+YY2"), kv(3, "X3+YY3"));
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+YY2", "3:XX3+YY3");
-            checkJoinedValues(getter, kv(0, "XX0+null"), kv(1, "XX1+null"), kv(2, "XX2+YY2"), kv(3, "XX3+YY3"));
-
-        } finally {
-            Utils.delete(baseDir);
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
         }
+
+        processor.checkAndClearProcessResult("0:X0+null", "1:X1+null");
+        checkJoinedValues(getter, kv(0, "X0+null"), kv(1, "X1+null"), kv(2, null), kv(3, null));
+
+        // push two items to the other stream. this should produce two items.
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1");
+        checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null));
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null");
+        checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, "X2+null"), kv(3, "X3+null"));
+
+        // push all items to the other stream. this should produce four items.
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
+        checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3"));
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
+        checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3"));
+
+        // push two items with null to the other stream as deletes. this should produce two item.
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], null);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+null", "1:X1+null");
+        checkJoinedValues(getter, kv(0, "X0+null"), kv(1, "X1+null"), kv(2, "X2+YY2"), kv(3, "X3+YY3"));
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+YY2", "3:XX3+YY3");
+        checkJoinedValues(getter, kv(0, "XX0+null"), kv(1, "XX1+null"), kv(2, "XX2+YY2"), kv(3, "XX3+YY3"));
     }
 
     @Test
     public void testNotSendingOldValue() throws Exception {
-        File baseDir = Files.createTempDirectory("test").toFile();
-        try {
+        final KStreamBuilder builder = new KStreamBuilder();
 
-            KStreamBuilder builder = new KStreamBuilder();
+        final int[] expectedKeys = new int[]{0, 1, 2, 3};
 
-            final int[] expectedKeys = new int[]{0, 1, 2, 3};
+        KTable<Integer, String> table1;
+        KTable<Integer, String> table2;
+        KTable<Integer, String> joined;
+        MockProcessorSupplier<Integer, String> proc;
 
-            KTable<Integer, String> table1;
-            KTable<Integer, String> table2;
-            KTable<Integer, String> joined;
-            MockProcessorSupplier<Integer, String> proc;
+        table1 = builder.table(intSerde, stringSerde, topic1);
+        table2 = builder.table(intSerde, stringSerde, topic2);
+        joined = table1.leftJoin(table2, MockValueJoiner.STRING_JOINER);
 
-            table1 = builder.table(intSerde, stringSerde, topic1);
-            table2 = builder.table(intSerde, stringSerde, topic2);
-            joined = table1.leftJoin(table2, joiner);
+        proc = new MockProcessorSupplier<>();
+        builder.addProcessor("proc", proc, ((KTableImpl<?, ?, ?>) joined).name);
 
-            proc = new MockProcessorSupplier<>();
-            builder.addProcessor("proc", proc, ((KTableImpl<?, ?, ?>) joined).name);
+        driver = new KStreamTestDriver(builder, stateDir);
+        driver.setTime(0L);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir);
-            driver.setTime(0L);
+        assertFalse(((KTableImpl<?, ?, ?>) table1).sendingOldValueEnabled());
+        assertFalse(((KTableImpl<?, ?, ?>) table2).sendingOldValueEnabled());
+        assertFalse(((KTableImpl<?, ?, ?>) joined).sendingOldValueEnabled());
 
-            assertFalse(((KTableImpl<?, ?, ?>) table1).sendingOldValueEnabled());
-            assertFalse(((KTableImpl<?, ?, ?>) table2).sendingOldValueEnabled());
-            assertFalse(((KTableImpl<?, ?, ?>) joined).sendingOldValueEnabled());
+        // push two items to the primary stream. the other table is empty
 
-            // push two items to the primary stream. the other table is empty
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)");
-
-            // push two items to the other stream. this should produce two items.
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)");
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(X2+null<-null)", "3:(X3+null<-null)");
-
-            // push all items to the other stream. this should produce four items.
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)");
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)");
-
-            // push two items with null to the other stream as deletes. this should produce two item.
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], null);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)");
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(XX0+null<-null)", "1:(XX1+null<-null)", "2:(XX2+YY2<-null)", "3:(XX3+YY3<-null)");
-
-        } finally {
-            Utils.delete(baseDir);
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
         }
+
+        proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)");
+
+        // push two items to the other stream. this should produce two items.
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)");
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(X2+null<-null)", "3:(X3+null<-null)");
+
+        // push all items to the other stream. this should produce four items.
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)");
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)");
+
+        // push two items with null to the other stream as deletes. this should produce two item.
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], null);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)");
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(XX0+null<-null)", "1:(XX1+null<-null)", "2:(XX2+YY2<-null)", "3:(XX3+YY3<-null)");
     }
 
     @Test
     public void testSendingOldValue() throws Exception {
-        File baseDir = Files.createTempDirectory("test").toFile();
-        try {
+        final KStreamBuilder builder = new KStreamBuilder();
 
-            KStreamBuilder builder = new KStreamBuilder();
+        final int[] expectedKeys = new int[]{0, 1, 2, 3};
 
-            final int[] expectedKeys = new int[]{0, 1, 2, 3};
+        KTable<Integer, String> table1;
+        KTable<Integer, String> table2;
+        KTable<Integer, String> joined;
+        MockProcessorSupplier<Integer, String> proc;
 
-            KTable<Integer, String> table1;
-            KTable<Integer, String> table2;
-            KTable<Integer, String> joined;
-            MockProcessorSupplier<Integer, String> proc;
+        table1 = builder.table(intSerde, stringSerde, topic1);
+        table2 = builder.table(intSerde, stringSerde, topic2);
+        joined = table1.leftJoin(table2, MockValueJoiner.STRING_JOINER);
 
-            table1 = builder.table(intSerde, stringSerde, topic1);
-            table2 = builder.table(intSerde, stringSerde, topic2);
-            joined = table1.leftJoin(table2, joiner);
+        ((KTableImpl<?, ?, ?>) joined).enableSendingOldValues();
 
-            ((KTableImpl<?, ?, ?>) joined).enableSendingOldValues();
+        proc = new MockProcessorSupplier<>();
+        builder.addProcessor("proc", proc, ((KTableImpl<?, ?, ?>) joined).name);
 
-            proc = new MockProcessorSupplier<>();
-            builder.addProcessor("proc", proc, ((KTableImpl<?, ?, ?>) joined).name);
+        driver = new KStreamTestDriver(builder, stateDir);
+        driver.setTime(0L);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir);
-            driver.setTime(0L);
+        assertTrue(((KTableImpl<?, ?, ?>) table1).sendingOldValueEnabled());
+        assertTrue(((KTableImpl<?, ?, ?>) table2).sendingOldValueEnabled());
+        assertTrue(((KTableImpl<?, ?, ?>) joined).sendingOldValueEnabled());
 
-            assertTrue(((KTableImpl<?, ?, ?>) table1).sendingOldValueEnabled());
-            assertTrue(((KTableImpl<?, ?, ?>) table2).sendingOldValueEnabled());
-            assertTrue(((KTableImpl<?, ?, ?>) joined).sendingOldValueEnabled());
+        // push two items to the primary stream. the other table is empty
 
-            // push two items to the primary stream. the other table is empty
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)");
-
-            // push two items to the other stream. this should produce two items.
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+Y0<-X0+null)", "1:(X1+Y1<-X1+null)");
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(X2+null<-null)", "3:(X3+null<-null)");
-
-            // push all items to the other stream. this should produce four items.
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+YY0<-X0+Y0)", "1:(X1+YY1<-X1+Y1)", "2:(X2+YY2<-X2+null)", "3:(X3+YY3<-X3+null)");
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+YY0<-X0+YY0)", "1:(X1+YY1<-X1+YY1)", "2:(X2+YY2<-X2+YY2)", "3:(X3+YY3<-X3+YY3)");
-
-            // push two items with null to the other stream as deletes. this should produce two item.
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], null);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+null<-X0+YY0)", "1:(X1+null<-X1+YY1)");
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(XX0+null<-X0+null)", "1:(XX1+null<-X1+null)", "2:(XX2+YY2<-X2+YY2)", "3:(XX3+YY3<-X3+YY3)");
-
-        } finally {
-            Utils.delete(baseDir);
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
         }
+
+        proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)");
+
+        // push two items to the other stream. this should produce two items.
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+Y0<-X0+null)", "1:(X1+Y1<-X1+null)");
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(X2+null<-null)", "3:(X3+null<-null)");
+
+        // push all items to the other stream. this should produce four items.
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+YY0<-X0+Y0)", "1:(X1+YY1<-X1+Y1)", "2:(X2+YY2<-X2+null)", "3:(X3+YY3<-X3+null)");
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+YY0<-X0+YY0)", "1:(X1+YY1<-X1+YY1)", "2:(X2+YY2<-X2+YY2)", "3:(X3+YY3<-X3+YY3)");
+
+        // push two items with null to the other stream as deletes. this should produce two item.
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], null);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+null<-X0+YY0)", "1:(X1+null<-X1+YY1)");
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(XX0+null<-X0+null)", "1:(XX1+null<-X1+null)", "2:(XX2+YY2<-X2+YY2)", "3:(XX3+YY3<-X3+YY3)");
     }
 
-    private JoinedKeyValue kv(Integer key, String value) {
-        return new JoinedKeyValue(key, value);
+    private KeyValue<Integer, String> kv(Integer key, String value) {
+        return new KeyValue<>(key, value);
     }
 
-    private void checkJoinedValues(KTableValueGetter<Integer, String> getter, JoinedKeyValue... expected) {
-        for (JoinedKeyValue kv : expected) {
+    private void checkJoinedValues(KTableValueGetter<Integer, String> getter, KeyValue<Integer, String>... expected) {
+        for (KeyValue<Integer, String> kv : expected) {
             String value = getter.get(kv.key);
             if (kv.value == null) {
                 assertNull(value);
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java
index ea7476a..3124556 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java
@@ -19,17 +19,19 @@
 
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.serialization.Serdes;
-import org.apache.kafka.common.utils.Utils;
 import org.apache.kafka.streams.kstream.KStreamBuilder;
 import org.apache.kafka.streams.kstream.KTable;
 import org.apache.kafka.streams.KeyValue;
-import org.apache.kafka.streams.kstream.ValueJoiner;
 import org.apache.kafka.test.KStreamTestDriver;
 import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.MockValueJoiner;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 import java.io.File;
-import java.nio.file.Files;
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
@@ -42,334 +44,316 @@
 
 public class KTableKTableOuterJoinTest {
 
-    private String topic1 = "topic1";
-    private String topic2 = "topic2";
+    final private String topic1 = "topic1";
+    final private String topic2 = "topic2";
 
-    final private Serde<Integer> intSerde = new Serdes.IntegerSerde();
-    final private Serde<String> stringSerde = new Serdes.StringSerde();
+    final private Serde<Integer> intSerde = Serdes.Integer();
+    final private Serde<String> stringSerde = Serdes.String();
 
-    private ValueJoiner<String, String, String> joiner = new ValueJoiner<String, String, String>() {
-        @Override
-        public String apply(String value1, String value2) {
-            return value1 + "+" + value2;
+    private KStreamTestDriver driver = null;
+    private File stateDir = null;
+
+    @After
+    public void tearDown() {
+        if (driver != null) {
+            driver.close();
         }
-    };
+        driver = null;
+    }
 
-    private static class JoinedKeyValue extends KeyValue<Integer, String> {
-        public JoinedKeyValue(Integer key, String value) {
-            super(key, value);
-        }
+    @Before
+    public void setUp() throws IOException {
+        stateDir = TestUtils.tempDirectory("kafka-test");
     }
 
     @Test
     public void testJoin() throws Exception {
-        File baseDir = Files.createTempDirectory("test").toFile();
-        try {
+        KStreamBuilder builder = new KStreamBuilder();
 
-            KStreamBuilder builder = new KStreamBuilder();
+        final int[] expectedKeys = new int[]{0, 1, 2, 3};
 
-            final int[] expectedKeys = new int[]{0, 1, 2, 3};
+        KTable<Integer, String> table1;
+        KTable<Integer, String> table2;
+        KTable<Integer, String> joined;
+        MockProcessorSupplier<Integer, String> processor;
 
-            KTable<Integer, String> table1;
-            KTable<Integer, String> table2;
-            KTable<Integer, String> joined;
-            MockProcessorSupplier<Integer, String> processor;
+        processor = new MockProcessorSupplier<>();
+        table1 = builder.table(intSerde, stringSerde, topic1);
+        table2 = builder.table(intSerde, stringSerde, topic2);
+        joined = table1.outerJoin(table2, MockValueJoiner.STRING_JOINER);
+        joined.toStream().process(processor);
 
-            processor = new MockProcessorSupplier<>();
-            table1 = builder.table(intSerde, stringSerde, topic1);
-            table2 = builder.table(intSerde, stringSerde, topic2);
-            joined = table1.outerJoin(table2, joiner);
-            joined.toStream().process(processor);
+        Collection<Set<String>> copartitionGroups = builder.copartitionGroups();
 
-            Collection<Set<String>> copartitionGroups = builder.copartitionGroups();
+        assertEquals(1, copartitionGroups.size());
+        assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
 
-            assertEquals(1, copartitionGroups.size());
-            assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
+        KTableValueGetterSupplier<Integer, String> getterSupplier = ((KTableImpl<Integer, String, String>) joined).valueGetterSupplier();
 
-            KTableValueGetterSupplier<Integer, String> getterSupplier = ((KTableImpl<Integer, String, String>) joined).valueGetterSupplier();
+        driver = new KStreamTestDriver(builder, stateDir);
+        driver.setTime(0L);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir);
-            driver.setTime(0L);
+        KTableValueGetter<Integer, String> getter = getterSupplier.get();
+        getter.init(driver.context());
 
-            KTableValueGetter<Integer, String> getter = getterSupplier.get();
-            getter.init(driver.context());
+        // push two items to the primary stream. the other table is empty
 
-            // push two items to the primary stream. the other table is empty
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+null", "1:X1+null");
-            checkJoinedValues(getter, kv(0, "X0+null"), kv(1, "X1+null"), kv(2, null), kv(3, null));
-
-            // push two items to the other stream. this should produce two items.
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1");
-            checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null));
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null");
-            checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, "X2+null"), kv(3, "X3+null"));
-
-            // push all items to the other stream. this should produce four items.
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
-            checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3"));
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
-            checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3"));
-
-            // push two items with null to the other stream as deletes. this should produce two item.
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], null);
-            }
-
-            processor.checkAndClearProcessResult("0:X0+null", "1:X1+null");
-            checkJoinedValues(getter, kv(0, "X0+null"), kv(1, "X1+null"), kv(2, "X2+YY2"), kv(3, "X3+YY3"));
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+YY2", "3:XX3+YY3");
-            checkJoinedValues(getter, kv(0, "XX0+null"), kv(1, "XX1+null"), kv(2, "XX2+YY2"), kv(3, "XX3+YY3"));
-
-            // push middle two items to the primary stream with null. this should produce two items.
-
-            for (int i = 1; i < 3; i++) {
-                driver.process(topic1, expectedKeys[i], null);
-            }
-
-            processor.checkAndClearProcessResult("1:null", "2:null+YY2");
-            checkJoinedValues(getter, kv(0, "XX0+null"), kv(1, null), kv(2, "null+YY2"), kv(3, "XX3+YY3"));
-
-        } finally {
-            Utils.delete(baseDir);
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
         }
+
+        processor.checkAndClearProcessResult("0:X0+null", "1:X1+null");
+        checkJoinedValues(getter, kv(0, "X0+null"), kv(1, "X1+null"), kv(2, null), kv(3, null));
+
+        // push two items to the other stream. this should produce two items.
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1");
+        checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null));
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null");
+        checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, "X2+null"), kv(3, "X3+null"));
+
+        // push all items to the other stream. this should produce four items.
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
+        checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3"));
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3");
+        checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3"));
+
+        // push two items with null to the other stream as deletes. this should produce two item.
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], null);
+        }
+
+        processor.checkAndClearProcessResult("0:X0+null", "1:X1+null");
+        checkJoinedValues(getter, kv(0, "X0+null"), kv(1, "X1+null"), kv(2, "X2+YY2"), kv(3, "X3+YY3"));
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+YY2", "3:XX3+YY3");
+        checkJoinedValues(getter, kv(0, "XX0+null"), kv(1, "XX1+null"), kv(2, "XX2+YY2"), kv(3, "XX3+YY3"));
+
+        // push middle two items to the primary stream with null. this should produce two items.
+
+        for (int i = 1; i < 3; i++) {
+            driver.process(topic1, expectedKeys[i], null);
+        }
+
+        processor.checkAndClearProcessResult("1:null", "2:null+YY2");
+        checkJoinedValues(getter, kv(0, "XX0+null"), kv(1, null), kv(2, "null+YY2"), kv(3, "XX3+YY3"));
     }
 
     @Test
     public void testNotSendingOldValue() throws Exception {
-        File baseDir = Files.createTempDirectory("test").toFile();
-        try {
+        final KStreamBuilder builder = new KStreamBuilder();
 
-            KStreamBuilder builder = new KStreamBuilder();
+        final int[] expectedKeys = new int[]{0, 1, 2, 3};
 
-            final int[] expectedKeys = new int[]{0, 1, 2, 3};
+        KTable<Integer, String> table1;
+        KTable<Integer, String> table2;
+        KTable<Integer, String> joined;
+        MockProcessorSupplier<Integer, String> proc;
 
-            KTable<Integer, String> table1;
-            KTable<Integer, String> table2;
-            KTable<Integer, String> joined;
-            MockProcessorSupplier<Integer, String> proc;
+        table1 = builder.table(intSerde, stringSerde, topic1);
+        table2 = builder.table(intSerde, stringSerde, topic2);
+        joined = table1.outerJoin(table2, MockValueJoiner.STRING_JOINER);
 
-            table1 = builder.table(intSerde, stringSerde, topic1);
-            table2 = builder.table(intSerde, stringSerde, topic2);
-            joined = table1.outerJoin(table2, joiner);
+        proc = new MockProcessorSupplier<>();
+        builder.addProcessor("proc", proc, ((KTableImpl<?, ?, ?>) joined).name);
 
-            proc = new MockProcessorSupplier<>();
-            builder.addProcessor("proc", proc, ((KTableImpl<?, ?, ?>) joined).name);
+        driver = new KStreamTestDriver(builder, stateDir);
+        driver.setTime(0L);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir);
-            driver.setTime(0L);
+        assertFalse(((KTableImpl<?, ?, ?>) table1).sendingOldValueEnabled());
+        assertFalse(((KTableImpl<?, ?, ?>) table2).sendingOldValueEnabled());
+        assertFalse(((KTableImpl<?, ?, ?>) joined).sendingOldValueEnabled());
 
-            assertFalse(((KTableImpl<?, ?, ?>) table1).sendingOldValueEnabled());
-            assertFalse(((KTableImpl<?, ?, ?>) table2).sendingOldValueEnabled());
-            assertFalse(((KTableImpl<?, ?, ?>) joined).sendingOldValueEnabled());
+        // push two items to the primary stream. the other table is empty
 
-            // push two items to the primary stream. the other table is empty
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)");
-
-            // push two items to the other stream. this should produce two items.
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)");
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(X2+null<-null)", "3:(X3+null<-null)");
-
-            // push all items to the other stream. this should produce four items.
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)");
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)");
-
-            // push two items with null to the other stream as deletes. this should produce two item.
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], null);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)");
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(XX0+null<-null)", "1:(XX1+null<-null)", "2:(XX2+YY2<-null)", "3:(XX3+YY3<-null)");
-
-            // push middle two items to the primary stream with null. this should produce two items.
-
-            for (int i = 1; i < 3; i++) {
-                driver.process(topic1, expectedKeys[i], null);
-            }
-
-            proc.checkAndClearProcessResult("1:(null<-null)", "2:(null+YY2<-null)");
-
-        } finally {
-            Utils.delete(baseDir);
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
         }
+
+        proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)");
+
+        // push two items to the other stream. this should produce two items.
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)");
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(X2+null<-null)", "3:(X3+null<-null)");
+
+        // push all items to the other stream. this should produce four items.
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)");
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)");
+
+        // push two items with null to the other stream as deletes. this should produce two item.
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], null);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)");
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(XX0+null<-null)", "1:(XX1+null<-null)", "2:(XX2+YY2<-null)", "3:(XX3+YY3<-null)");
+
+        // push middle two items to the primary stream with null. this should produce two items.
+
+        for (int i = 1; i < 3; i++) {
+            driver.process(topic1, expectedKeys[i], null);
+        }
+
+        proc.checkAndClearProcessResult("1:(null<-null)", "2:(null+YY2<-null)");
     }
 
     @Test
     public void testSendingOldValue() throws Exception {
-        File baseDir = Files.createTempDirectory("test").toFile();
-        try {
+        final KStreamBuilder builder = new KStreamBuilder();
 
-            KStreamBuilder builder = new KStreamBuilder();
+        final int[] expectedKeys = new int[]{0, 1, 2, 3};
 
-            final int[] expectedKeys = new int[]{0, 1, 2, 3};
+        KTable<Integer, String> table1;
+        KTable<Integer, String> table2;
+        KTable<Integer, String> joined;
+        MockProcessorSupplier<Integer, String> proc;
 
-            KTable<Integer, String> table1;
-            KTable<Integer, String> table2;
-            KTable<Integer, String> joined;
-            MockProcessorSupplier<Integer, String> proc;
+        table1 = builder.table(intSerde, stringSerde, topic1);
+        table2 = builder.table(intSerde, stringSerde, topic2);
+        joined = table1.outerJoin(table2, MockValueJoiner.STRING_JOINER);
 
-            table1 = builder.table(intSerde, stringSerde, topic1);
-            table2 = builder.table(intSerde, stringSerde, topic2);
-            joined = table1.outerJoin(table2, joiner);
+        ((KTableImpl<?, ?, ?>) joined).enableSendingOldValues();
 
-            ((KTableImpl<?, ?, ?>) joined).enableSendingOldValues();
+        proc = new MockProcessorSupplier<>();
+        builder.addProcessor("proc", proc, ((KTableImpl<?, ?, ?>) joined).name);
 
-            proc = new MockProcessorSupplier<>();
-            builder.addProcessor("proc", proc, ((KTableImpl<?, ?, ?>) joined).name);
+        driver = new KStreamTestDriver(builder, stateDir);
+        driver.setTime(0L);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir);
-            driver.setTime(0L);
+        assertTrue(((KTableImpl<?, ?, ?>) table1).sendingOldValueEnabled());
+        assertTrue(((KTableImpl<?, ?, ?>) table2).sendingOldValueEnabled());
+        assertTrue(((KTableImpl<?, ?, ?>) joined).sendingOldValueEnabled());
 
-            assertTrue(((KTableImpl<?, ?, ?>) table1).sendingOldValueEnabled());
-            assertTrue(((KTableImpl<?, ?, ?>) table2).sendingOldValueEnabled());
-            assertTrue(((KTableImpl<?, ?, ?>) joined).sendingOldValueEnabled());
+        // push two items to the primary stream. the other table is empty
 
-            // push two items to the primary stream. the other table is empty
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)");
-
-            // push two items to the other stream. this should produce two items.
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+Y0<-X0+null)", "1:(X1+Y1<-X1+null)");
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(X2+null<-null)", "3:(X3+null<-null)");
-
-            // push all items to the other stream. this should produce four items.
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+YY0<-X0+Y0)", "1:(X1+YY1<-X1+Y1)", "2:(X2+YY2<-X2+null)", "3:(X3+YY3<-X3+null)");
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+YY0<-X0+YY0)", "1:(X1+YY1<-X1+YY1)", "2:(X2+YY2<-X2+YY2)", "3:(X3+YY3<-X3+YY3)");
-
-            // push two items with null to the other stream as deletes. this should produce two item.
-
-            for (int i = 0; i < 2; i++) {
-                driver.process(topic2, expectedKeys[i], null);
-            }
-
-            proc.checkAndClearProcessResult("0:(X0+null<-X0+YY0)", "1:(X1+null<-X1+YY1)");
-
-            // push all four items to the primary stream. this should produce four items.
-
-            for (int i = 0; i < expectedKeys.length; i++) {
-                driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
-            }
-
-            proc.checkAndClearProcessResult("0:(XX0+null<-X0+null)", "1:(XX1+null<-X1+null)", "2:(XX2+YY2<-X2+YY2)", "3:(XX3+YY3<-X3+YY3)");
-
-            // push middle two items to the primary stream with null. this should produce two items.
-
-            for (int i = 1; i < 3; i++) {
-                driver.process(topic1, expectedKeys[i], null);
-            }
-
-            proc.checkAndClearProcessResult("1:(null<-XX1+null)", "2:(null+YY2<-XX2+YY2)");
-
-        } finally {
-            Utils.delete(baseDir);
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
         }
+
+        proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)");
+
+        // push two items to the other stream. this should produce two items.
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+Y0<-X0+null)", "1:(X1+Y1<-X1+null)");
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(X2+null<-null)", "3:(X3+null<-null)");
+
+        // push all items to the other stream. this should produce four items.
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+YY0<-X0+Y0)", "1:(X1+YY1<-X1+Y1)", "2:(X2+YY2<-X2+null)", "3:(X3+YY3<-X3+null)");
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+YY0<-X0+YY0)", "1:(X1+YY1<-X1+YY1)", "2:(X2+YY2<-X2+YY2)", "3:(X3+YY3<-X3+YY3)");
+
+        // push two items with null to the other stream as deletes. this should produce two item.
+
+        for (int i = 0; i < 2; i++) {
+            driver.process(topic2, expectedKeys[i], null);
+        }
+
+        proc.checkAndClearProcessResult("0:(X0+null<-X0+YY0)", "1:(X1+null<-X1+YY1)");
+
+        // push all four items to the primary stream. this should produce four items.
+
+        for (int i = 0; i < expectedKeys.length; i++) {
+            driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]);
+        }
+
+        proc.checkAndClearProcessResult("0:(XX0+null<-X0+null)", "1:(XX1+null<-X1+null)", "2:(XX2+YY2<-X2+YY2)", "3:(XX3+YY3<-X3+YY3)");
+
+        // push middle two items to the primary stream with null. this should produce two items.
+
+        for (int i = 1; i < 3; i++) {
+            driver.process(topic1, expectedKeys[i], null);
+        }
+
+        proc.checkAndClearProcessResult("1:(null<-XX1+null)", "2:(null+YY2<-XX2+YY2)");
     }
 
-    private JoinedKeyValue kv(Integer key, String value) {
-        return new JoinedKeyValue(key, value);
+    private KeyValue<Integer, String> kv(Integer key, String value) {
+        return new KeyValue<>(key, value);
     }
 
-    private void checkJoinedValues(KTableValueGetter<Integer, String> getter, JoinedKeyValue... expected) {
-        for (JoinedKeyValue kv : expected) {
+    private void checkJoinedValues(KTableValueGetter<Integer, String> getter, KeyValue<Integer, String>... expected) {
+        for (KeyValue<Integer, String> kv : expected) {
             String value = getter.get(kv.key);
             if (kv.value == null) {
                 assertNull(value);
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
new file mode 100644
index 0000000..cf74017
--- /dev/null
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.streams.kstream.internals;
+
+
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.KStreamBuilder;
+import org.apache.kafka.streams.kstream.KTable;
+import org.apache.kafka.streams.kstream.KeyValueMapper;
+import org.apache.kafka.test.KStreamTestDriver;
+import org.apache.kafka.test.MockProcessorSupplier;
+import org.junit.After;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class KTableMapKeysTest {
+
+    final private Serde<String> stringSerde = new Serdes.StringSerde();
+    final private Serde<Integer>  integerSerde = new Serdes.IntegerSerde();
+
+    private KStreamTestDriver driver = null;
+
+    @After
+    public void cleanup() {
+        if (driver != null) {
+            driver.close();
+        }
+        driver = null;
+    }
+
+    @Test
+    public void testMapKeysConvertingToStream() {
+        final KStreamBuilder builder = new KStreamBuilder();
+
+        String topic1 = "topic_map_keys";
+
+        KTable<Integer, String> table1 = builder.table(integerSerde, stringSerde, topic1);
+
+        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>() {
+            @Override
+            public  String apply(Integer key, String value) {
+                return keyMap.get(key);
+            }
+        };
+
+        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> processor = new MockProcessorSupplier<>();
+
+        convertedStream.process(processor);
+
+        driver = new KStreamTestDriver(builder);
+
+        for (int i = 0;  i < originalKeys.length; i++) {
+            driver.process(topic1, originalKeys[i], values[i]);
+        }
+
+        assertEquals(3, processor.processed.size());
+
+        for (int i = 0; i < expected.length; i++) {
+            assertEquals(expected[i], processor.processed.get(i));
+        }
+    }
+
+
+
+}
\ No newline at end of file
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 9cafe8b..efb17fc 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
@@ -26,11 +26,13 @@
 import org.apache.kafka.streams.kstream.ValueMapper;
 import org.apache.kafka.test.KStreamTestDriver;
 import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 import java.io.File;
 import java.io.IOException;
-import java.nio.file.Files;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -39,7 +41,23 @@
 
 public class KTableMapValuesTest {
 
-    final private Serde<String> stringSerde = new Serdes.StringSerde();
+    final private Serde<String> stringSerde = Serdes.String();
+
+    private KStreamTestDriver driver = null;
+    private File stateDir = null;
+
+    @After
+    public void tearDown() {
+        if (driver != null) {
+            driver.close();
+        }
+        driver = null;
+    }
+
+    @Before
+    public void setUp() throws IOException {
+        stateDir = TestUtils.tempDirectory("kafka-test");
+    }
 
     @Test
     public void testKTable() {
@@ -58,7 +76,7 @@
         MockProcessorSupplier<String, Integer> proc2 = new MockProcessorSupplier<>();
         table2.toStream().process(proc2);
 
-        KStreamTestDriver driver = new KStreamTestDriver(builder);
+        driver = new KStreamTestDriver(builder);
 
         driver.process(topic1, "A", "01");
         driver.process(topic1, "B", "02");
@@ -70,230 +88,211 @@
 
     @Test
     public void testValueGetter() throws IOException {
-        File stateDir = Files.createTempDirectory("test").toFile();
-        try {
-            final KStreamBuilder builder = new KStreamBuilder();
+        KStreamBuilder builder = new KStreamBuilder();
 
-            String topic1 = "topic1";
-            String topic2 = "topic2";
+        String topic1 = "topic1";
+        String topic2 = "topic2";
 
-            KTableImpl<String, String, String> table1 =
-                    (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
-            KTableImpl<String, String, Integer> table2 = (KTableImpl<String, String, Integer>) table1.mapValues(
-                    new ValueMapper<String, Integer>() {
-                        @Override
-                        public Integer apply(String value) {
-                            return new Integer(value);
-                        }
-                    });
-            KTableImpl<String, Integer, Integer> table3 = (KTableImpl<String, Integer, Integer>) table2.filter(
-                    new Predicate<String, Integer>() {
-                        @Override
-                        public boolean test(String key, Integer value) {
-                            return (value % 2) == 0;
-                        }
-                    });
-            KTableImpl<String, String, String> table4 = (KTableImpl<String, String, String>)
-                    table1.through(stringSerde, stringSerde, topic2);
+        KTableImpl<String, String, String> table1 =
+                (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
+        KTableImpl<String, String, Integer> table2 = (KTableImpl<String, String, Integer>) table1.mapValues(
+                new ValueMapper<String, Integer>() {
+                    @Override
+                    public Integer apply(String value) {
+                        return new Integer(value);
+                    }
+                });
+        KTableImpl<String, Integer, Integer> table3 = (KTableImpl<String, Integer, Integer>) table2.filter(
+                new Predicate<String, Integer>() {
+                    @Override
+                    public boolean test(String key, Integer value) {
+                        return (value % 2) == 0;
+                    }
+                });
+        KTableImpl<String, String, String> table4 = (KTableImpl<String, String, String>)
+                table1.through(stringSerde, stringSerde, topic2);
 
-            KTableValueGetterSupplier<String, String> getterSupplier1 = table1.valueGetterSupplier();
-            KTableValueGetterSupplier<String, Integer> getterSupplier2 = table2.valueGetterSupplier();
-            KTableValueGetterSupplier<String, Integer> getterSupplier3 = table3.valueGetterSupplier();
-            KTableValueGetterSupplier<String, String> getterSupplier4 = table4.valueGetterSupplier();
+        KTableValueGetterSupplier<String, String> getterSupplier1 = table1.valueGetterSupplier();
+        KTableValueGetterSupplier<String, Integer> getterSupplier2 = table2.valueGetterSupplier();
+        KTableValueGetterSupplier<String, Integer> getterSupplier3 = table3.valueGetterSupplier();
+        KTableValueGetterSupplier<String, String> getterSupplier4 = table4.valueGetterSupplier();
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null);
+        driver = new KStreamTestDriver(builder, stateDir, null, null);
 
-            KTableValueGetter<String, String> getter1 = getterSupplier1.get();
-            getter1.init(driver.context());
-            KTableValueGetter<String, Integer> getter2 = getterSupplier2.get();
-            getter2.init(driver.context());
-            KTableValueGetter<String, Integer> getter3 = getterSupplier3.get();
-            getter3.init(driver.context());
-            KTableValueGetter<String, String> getter4 = getterSupplier4.get();
-            getter4.init(driver.context());
+        KTableValueGetter<String, String> getter1 = getterSupplier1.get();
+        getter1.init(driver.context());
+        KTableValueGetter<String, Integer> getter2 = getterSupplier2.get();
+        getter2.init(driver.context());
+        KTableValueGetter<String, Integer> getter3 = getterSupplier3.get();
+        getter3.init(driver.context());
+        KTableValueGetter<String, String> getter4 = getterSupplier4.get();
+        getter4.init(driver.context());
 
-            driver.process(topic1, "A", "01");
-            driver.process(topic1, "B", "01");
-            driver.process(topic1, "C", "01");
+        driver.process(topic1, "A", "01");
+        driver.process(topic1, "B", "01");
+        driver.process(topic1, "C", "01");
 
-            assertEquals("01", getter1.get("A"));
-            assertEquals("01", getter1.get("B"));
-            assertEquals("01", getter1.get("C"));
+        assertEquals("01", getter1.get("A"));
+        assertEquals("01", getter1.get("B"));
+        assertEquals("01", getter1.get("C"));
 
-            assertEquals(new Integer(1), getter2.get("A"));
-            assertEquals(new Integer(1), getter2.get("B"));
-            assertEquals(new Integer(1), getter2.get("C"));
+        assertEquals(new Integer(1), getter2.get("A"));
+        assertEquals(new Integer(1), getter2.get("B"));
+        assertEquals(new Integer(1), getter2.get("C"));
 
-            assertNull(getter3.get("A"));
-            assertNull(getter3.get("B"));
-            assertNull(getter3.get("C"));
+        assertNull(getter3.get("A"));
+        assertNull(getter3.get("B"));
+        assertNull(getter3.get("C"));
 
-            assertEquals("01", getter4.get("A"));
-            assertEquals("01", getter4.get("B"));
-            assertEquals("01", getter4.get("C"));
+        assertEquals("01", getter4.get("A"));
+        assertEquals("01", getter4.get("B"));
+        assertEquals("01", getter4.get("C"));
 
-            driver.process(topic1, "A", "02");
-            driver.process(topic1, "B", "02");
+        driver.process(topic1, "A", "02");
+        driver.process(topic1, "B", "02");
 
-            assertEquals("02", getter1.get("A"));
-            assertEquals("02", getter1.get("B"));
-            assertEquals("01", getter1.get("C"));
+        assertEquals("02", getter1.get("A"));
+        assertEquals("02", getter1.get("B"));
+        assertEquals("01", getter1.get("C"));
 
-            assertEquals(new Integer(2), getter2.get("A"));
-            assertEquals(new Integer(2), getter2.get("B"));
-            assertEquals(new Integer(1), getter2.get("C"));
+        assertEquals(new Integer(2), getter2.get("A"));
+        assertEquals(new Integer(2), getter2.get("B"));
+        assertEquals(new Integer(1), getter2.get("C"));
 
-            assertEquals(new Integer(2), getter3.get("A"));
-            assertEquals(new Integer(2), getter3.get("B"));
-            assertNull(getter3.get("C"));
+        assertEquals(new Integer(2), getter3.get("A"));
+        assertEquals(new Integer(2), getter3.get("B"));
+        assertNull(getter3.get("C"));
 
-            assertEquals("02", getter4.get("A"));
-            assertEquals("02", getter4.get("B"));
-            assertEquals("01", getter4.get("C"));
+        assertEquals("02", getter4.get("A"));
+        assertEquals("02", getter4.get("B"));
+        assertEquals("01", getter4.get("C"));
 
-            driver.process(topic1, "A", "03");
+        driver.process(topic1, "A", "03");
 
-            assertEquals("03", getter1.get("A"));
-            assertEquals("02", getter1.get("B"));
-            assertEquals("01", getter1.get("C"));
+        assertEquals("03", getter1.get("A"));
+        assertEquals("02", getter1.get("B"));
+        assertEquals("01", getter1.get("C"));
 
-            assertEquals(new Integer(3), getter2.get("A"));
-            assertEquals(new Integer(2), getter2.get("B"));
-            assertEquals(new Integer(1), getter2.get("C"));
+        assertEquals(new Integer(3), getter2.get("A"));
+        assertEquals(new Integer(2), getter2.get("B"));
+        assertEquals(new Integer(1), getter2.get("C"));
 
-            assertNull(getter3.get("A"));
-            assertEquals(new Integer(2), getter3.get("B"));
-            assertNull(getter3.get("C"));
+        assertNull(getter3.get("A"));
+        assertEquals(new Integer(2), getter3.get("B"));
+        assertNull(getter3.get("C"));
 
-            assertEquals("03", getter4.get("A"));
-            assertEquals("02", getter4.get("B"));
-            assertEquals("01", getter4.get("C"));
+        assertEquals("03", getter4.get("A"));
+        assertEquals("02", getter4.get("B"));
+        assertEquals("01", getter4.get("C"));
 
-            driver.process(topic1, "A", null);
+        driver.process(topic1, "A", null);
 
-            assertNull(getter1.get("A"));
-            assertEquals("02", getter1.get("B"));
-            assertEquals("01", getter1.get("C"));
+        assertNull(getter1.get("A"));
+        assertEquals("02", getter1.get("B"));
+        assertEquals("01", getter1.get("C"));
 
-            assertNull(getter2.get("A"));
-            assertEquals(new Integer(2), getter2.get("B"));
-            assertEquals(new Integer(1), getter2.get("C"));
+        assertNull(getter2.get("A"));
+        assertEquals(new Integer(2), getter2.get("B"));
+        assertEquals(new Integer(1), getter2.get("C"));
 
-            assertNull(getter3.get("A"));
-            assertEquals(new Integer(2), getter3.get("B"));
-            assertNull(getter3.get("C"));
+        assertNull(getter3.get("A"));
+        assertEquals(new Integer(2), getter3.get("B"));
+        assertNull(getter3.get("C"));
 
-            assertNull(getter4.get("A"));
-            assertEquals("02", getter4.get("B"));
-            assertEquals("01", getter4.get("C"));
-
-        } finally {
-            Utils.delete(stateDir);
-        }
+        assertNull(getter4.get("A"));
+        assertEquals("02", getter4.get("B"));
+        assertEquals("01", getter4.get("C"));
     }
 
     @Test
     public void testNotSendingOldValue() throws IOException {
-        File stateDir = Files.createTempDirectory("test").toFile();
-        try {
-            final KStreamBuilder builder = new KStreamBuilder();
+        KStreamBuilder builder = new KStreamBuilder();
 
-            String topic1 = "topic1";
+        String topic1 = "topic1";
 
-            KTableImpl<String, String, String> table1 =
-                    (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
-            KTableImpl<String, String, Integer> table2 = (KTableImpl<String, String, Integer>) table1.mapValues(
-                    new ValueMapper<String, Integer>() {
-                        @Override
-                        public Integer apply(String value) {
-                            return new Integer(value);
-                        }
-                    });
+        KTableImpl<String, String, String> table1 =
+                (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
+        KTableImpl<String, String, Integer> table2 = (KTableImpl<String, String, Integer>) table1.mapValues(
+                new ValueMapper<String, Integer>() {
+                    @Override
+                    public Integer apply(String value) {
+                        return new Integer(value);
+                    }
+                });
 
-            MockProcessorSupplier<String, Integer> proc = new MockProcessorSupplier<>();
+        MockProcessorSupplier<String, Integer> proc = new MockProcessorSupplier<>();
 
-            builder.addProcessor("proc", proc, table2.name);
+        builder.addProcessor("proc", proc, table2.name);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null);
+        driver = new KStreamTestDriver(builder, stateDir, null, null);
 
-            assertFalse(table1.sendingOldValueEnabled());
-            assertFalse(table2.sendingOldValueEnabled());
+        assertFalse(table1.sendingOldValueEnabled());
+        assertFalse(table2.sendingOldValueEnabled());
 
-            driver.process(topic1, "A", "01");
-            driver.process(topic1, "B", "01");
-            driver.process(topic1, "C", "01");
+        driver.process(topic1, "A", "01");
+        driver.process(topic1, "B", "01");
+        driver.process(topic1, "C", "01");
 
-            proc.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)");
+        proc.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)");
 
-            driver.process(topic1, "A", "02");
-            driver.process(topic1, "B", "02");
+        driver.process(topic1, "A", "02");
+        driver.process(topic1, "B", "02");
 
-            proc.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)");
+        proc.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)");
 
-            driver.process(topic1, "A", "03");
+        driver.process(topic1, "A", "03");
 
-            proc.checkAndClearProcessResult("A:(3<-null)");
+        proc.checkAndClearProcessResult("A:(3<-null)");
 
-            driver.process(topic1, "A", null);
+        driver.process(topic1, "A", null);
 
-            proc.checkAndClearProcessResult("A:(null<-null)");
-
-        } finally {
-            Utils.delete(stateDir);
-        }
+        proc.checkAndClearProcessResult("A:(null<-null)");
     }
 
     @Test
     public void testSendingOldValue() throws IOException {
-        File stateDir = Files.createTempDirectory("test").toFile();
-        try {
-            final KStreamBuilder builder = new KStreamBuilder();
+        KStreamBuilder builder = new KStreamBuilder();
 
-            String topic1 = "topic1";
+        String topic1 = "topic1";
 
-            KTableImpl<String, String, String> table1 =
-                    (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
-            KTableImpl<String, String, Integer> table2 = (KTableImpl<String, String, Integer>) table1.mapValues(
-                    new ValueMapper<String, Integer>() {
-                        @Override
-                        public Integer apply(String value) {
-                            return new Integer(value);
-                        }
-                    });
+        KTableImpl<String, String, String> table1 =
+                (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
+        KTableImpl<String, String, Integer> table2 = (KTableImpl<String, String, Integer>) table1.mapValues(
+                new ValueMapper<String, Integer>() {
+                    @Override
+                    public Integer apply(String value) {
+                        return new Integer(value);
+                    }
+                });
 
-            table2.enableSendingOldValues();
+        table2.enableSendingOldValues();
 
-            MockProcessorSupplier<String, Integer> proc = new MockProcessorSupplier<>();
+        MockProcessorSupplier<String, Integer> proc = new MockProcessorSupplier<>();
 
-            builder.addProcessor("proc", proc, table2.name);
+        builder.addProcessor("proc", proc, table2.name);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null);
+        driver = new KStreamTestDriver(builder, stateDir, null, null);
 
-            assertTrue(table1.sendingOldValueEnabled());
-            assertTrue(table2.sendingOldValueEnabled());
+        assertTrue(table1.sendingOldValueEnabled());
+        assertTrue(table2.sendingOldValueEnabled());
 
-            driver.process(topic1, "A", "01");
-            driver.process(topic1, "B", "01");
-            driver.process(topic1, "C", "01");
+        driver.process(topic1, "A", "01");
+        driver.process(topic1, "B", "01");
+        driver.process(topic1, "C", "01");
 
-            proc.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)");
+        proc.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)");
 
-            driver.process(topic1, "A", "02");
-            driver.process(topic1, "B", "02");
+        driver.process(topic1, "A", "02");
+        driver.process(topic1, "B", "02");
 
-            proc.checkAndClearProcessResult("A:(2<-1)", "B:(2<-1)");
+        proc.checkAndClearProcessResult("A:(2<-1)", "B:(2<-1)");
 
-            driver.process(topic1, "A", "03");
+        driver.process(topic1, "A", "03");
 
-            proc.checkAndClearProcessResult("A:(3<-2)");
+        proc.checkAndClearProcessResult("A:(3<-2)");
 
-            driver.process(topic1, "A", null);
+        driver.process(topic1, "A", null);
 
-            proc.checkAndClearProcessResult("A:(null<-3)");
-
-        } finally {
-            Utils.delete(stateDir);
-        }
+        proc.checkAndClearProcessResult("A:(null<-3)");
     }
-
 }
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java
index 7c158e2..aaa6cc7 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java
@@ -24,11 +24,13 @@
 import org.apache.kafka.streams.kstream.KTable;
 import org.apache.kafka.test.KStreamTestDriver;
 import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 import java.io.File;
 import java.io.IOException;
-import java.nio.file.Files;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
@@ -36,7 +38,23 @@
 
 public class KTableSourceTest {
 
-    final private Serde<String> stringSerde = new Serdes.StringSerde();
+    final private Serde<String> stringSerde = Serdes.String();
+
+    private KStreamTestDriver driver = null;
+    private File stateDir = null;
+
+    @After
+    public void tearDown() {
+        if (driver != null) {
+            driver.close();
+        }
+        driver = null;
+    }
+
+    @Before
+    public void setUp() throws IOException {
+        stateDir = TestUtils.tempDirectory("kafka-test");
+    }
 
     @Test
     public void testKTable() {
@@ -49,7 +67,7 @@
         MockProcessorSupplier<String, String> proc1 = new MockProcessorSupplier<>();
         table1.toStream().process(proc1);
 
-        KStreamTestDriver driver = new KStreamTestDriver(builder);
+        driver = new KStreamTestDriver(builder);
 
         driver.process(topic1, "A", 1);
         driver.process(topic1, "B", 2);
@@ -63,138 +81,120 @@
 
     @Test
     public void testValueGetter() throws IOException {
-        File stateDir = Files.createTempDirectory("test").toFile();
-        try {
-            final KStreamBuilder builder = new KStreamBuilder();
+        final KStreamBuilder builder = new KStreamBuilder();
 
-            String topic1 = "topic1";
+        String topic1 = "topic1";
 
-            KTableImpl<String, String, String> table1 = (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
+        KTableImpl<String, String, String> table1 = (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
 
-            KTableValueGetterSupplier<String, String> getterSupplier1 = table1.valueGetterSupplier();
+        KTableValueGetterSupplier<String, String> getterSupplier1 = table1.valueGetterSupplier();
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null);
+        driver = new KStreamTestDriver(builder, stateDir, null, null);
 
-            KTableValueGetter<String, String> getter1 = getterSupplier1.get();
-            getter1.init(driver.context());
+        KTableValueGetter<String, String> getter1 = getterSupplier1.get();
+        getter1.init(driver.context());
 
-            driver.process(topic1, "A", "01");
-            driver.process(topic1, "B", "01");
-            driver.process(topic1, "C", "01");
+        driver.process(topic1, "A", "01");
+        driver.process(topic1, "B", "01");
+        driver.process(topic1, "C", "01");
 
-            assertEquals("01", getter1.get("A"));
-            assertEquals("01", getter1.get("B"));
-            assertEquals("01", getter1.get("C"));
+        assertEquals("01", getter1.get("A"));
+        assertEquals("01", getter1.get("B"));
+        assertEquals("01", getter1.get("C"));
 
-            driver.process(topic1, "A", "02");
-            driver.process(topic1, "B", "02");
+        driver.process(topic1, "A", "02");
+        driver.process(topic1, "B", "02");
 
-            assertEquals("02", getter1.get("A"));
-            assertEquals("02", getter1.get("B"));
-            assertEquals("01", getter1.get("C"));
+        assertEquals("02", getter1.get("A"));
+        assertEquals("02", getter1.get("B"));
+        assertEquals("01", getter1.get("C"));
 
-            driver.process(topic1, "A", "03");
+        driver.process(topic1, "A", "03");
 
-            assertEquals("03", getter1.get("A"));
-            assertEquals("02", getter1.get("B"));
-            assertEquals("01", getter1.get("C"));
+        assertEquals("03", getter1.get("A"));
+        assertEquals("02", getter1.get("B"));
+        assertEquals("01", getter1.get("C"));
 
-            driver.process(topic1, "A", null);
-            driver.process(topic1, "B", null);
+        driver.process(topic1, "A", null);
+        driver.process(topic1, "B", null);
 
-            assertNull(getter1.get("A"));
-            assertNull(getter1.get("B"));
-            assertEquals("01", getter1.get("C"));
+        assertNull(getter1.get("A"));
+        assertNull(getter1.get("B"));
+        assertEquals("01", getter1.get("C"));
 
-        } finally {
-            Utils.delete(stateDir);
-        }
     }
 
     @Test
     public void testNotSedingOldValue() throws IOException {
-        File stateDir = Files.createTempDirectory("test").toFile();
-        try {
-            final KStreamBuilder builder = new KStreamBuilder();
+        final KStreamBuilder builder = new KStreamBuilder();
 
-            String topic1 = "topic1";
+        String topic1 = "topic1";
 
-            KTableImpl<String, String, String> table1 = (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
+        KTableImpl<String, String, String> table1 = (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
 
-            MockProcessorSupplier<String, Integer> proc1 = new MockProcessorSupplier<>();
+        MockProcessorSupplier<String, Integer> proc1 = new MockProcessorSupplier<>();
 
-            builder.addProcessor("proc1", proc1, table1.name);
+        builder.addProcessor("proc1", proc1, table1.name);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null);
+        driver = new KStreamTestDriver(builder, stateDir, null, null);
 
-            driver.process(topic1, "A", "01");
-            driver.process(topic1, "B", "01");
-            driver.process(topic1, "C", "01");
+        driver.process(topic1, "A", "01");
+        driver.process(topic1, "B", "01");
+        driver.process(topic1, "C", "01");
 
-            proc1.checkAndClearProcessResult("A:(01<-null)", "B:(01<-null)", "C:(01<-null)");
+        proc1.checkAndClearProcessResult("A:(01<-null)", "B:(01<-null)", "C:(01<-null)");
 
-            driver.process(topic1, "A", "02");
-            driver.process(topic1, "B", "02");
+        driver.process(topic1, "A", "02");
+        driver.process(topic1, "B", "02");
 
-            proc1.checkAndClearProcessResult("A:(02<-null)", "B:(02<-null)");
+        proc1.checkAndClearProcessResult("A:(02<-null)", "B:(02<-null)");
 
-            driver.process(topic1, "A", "03");
+        driver.process(topic1, "A", "03");
 
-            proc1.checkAndClearProcessResult("A:(03<-null)");
+        proc1.checkAndClearProcessResult("A:(03<-null)");
 
-            driver.process(topic1, "A", null);
-            driver.process(topic1, "B", null);
+        driver.process(topic1, "A", null);
+        driver.process(topic1, "B", null);
 
-            proc1.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)");
-
-        } finally {
-            Utils.delete(stateDir);
-        }
+        proc1.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)");
     }
 
     @Test
     public void testSedingOldValue() throws IOException {
-        File stateDir = Files.createTempDirectory("test").toFile();
-        try {
-            final KStreamBuilder builder = new KStreamBuilder();
+        final KStreamBuilder builder = new KStreamBuilder();
 
-            String topic1 = "topic1";
+        String topic1 = "topic1";
 
-            KTableImpl<String, String, String> table1 = (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
+        KTableImpl<String, String, String> table1 = (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
 
-            table1.enableSendingOldValues();
+        table1.enableSendingOldValues();
 
-            assertTrue(table1.sendingOldValueEnabled());
+        assertTrue(table1.sendingOldValueEnabled());
 
-            MockProcessorSupplier<String, Integer> proc1 = new MockProcessorSupplier<>();
+        MockProcessorSupplier<String, Integer> proc1 = new MockProcessorSupplier<>();
 
-            builder.addProcessor("proc1", proc1, table1.name);
+        builder.addProcessor("proc1", proc1, table1.name);
 
-            KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null);
+        driver = new KStreamTestDriver(builder, stateDir, null, null);
 
-            driver.process(topic1, "A", "01");
-            driver.process(topic1, "B", "01");
-            driver.process(topic1, "C", "01");
+        driver.process(topic1, "A", "01");
+        driver.process(topic1, "B", "01");
+        driver.process(topic1, "C", "01");
 
-            proc1.checkAndClearProcessResult("A:(01<-null)", "B:(01<-null)", "C:(01<-null)");
+        proc1.checkAndClearProcessResult("A:(01<-null)", "B:(01<-null)", "C:(01<-null)");
 
-            driver.process(topic1, "A", "02");
-            driver.process(topic1, "B", "02");
+        driver.process(topic1, "A", "02");
+        driver.process(topic1, "B", "02");
 
-            proc1.checkAndClearProcessResult("A:(02<-01)", "B:(02<-01)");
+        proc1.checkAndClearProcessResult("A:(02<-01)", "B:(02<-01)");
 
-            driver.process(topic1, "A", "03");
+        driver.process(topic1, "A", "03");
 
-            proc1.checkAndClearProcessResult("A:(03<-02)");
+        proc1.checkAndClearProcessResult("A:(03<-02)");
 
-            driver.process(topic1, "A", null);
-            driver.process(topic1, "B", null);
+        driver.process(topic1, "A", null);
+        driver.process(topic1, "B", null);
 
-            proc1.checkAndClearProcessResult("A:(null<-03)", "B:(null<-02)");
-
-        } finally {
-            Utils.delete(stateDir);
-        }
+        proc1.checkAndClearProcessResult("A:(null<-03)", "B:(null<-02)");
     }
-
 }
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinterProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinterProcessorTest.java
index 22948ab..c8707af 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinterProcessorTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinterProcessorTest.java
@@ -26,6 +26,7 @@
 import org.apache.kafka.streams.kstream.KStream;
 import org.apache.kafka.streams.kstream.KStreamBuilder;
 import org.apache.kafka.test.KStreamTestDriver;
+import org.junit.After;
 import org.junit.Test;
 
 import java.io.ByteArrayOutputStream;
@@ -39,11 +40,19 @@
 
     private String topicName = "topic";
     private Serde<String> stringSerde = Serdes.String();
-    private Serde<byte[]> bytesSerde = Serdes.ByteArray();
     private ByteArrayOutputStream baos = new ByteArrayOutputStream();
     private KStreamBuilder builder = new KStreamBuilder();
     private PrintStream printStream = new PrintStream(baos);
 
+    private KStreamTestDriver driver = null;
+
+    @After
+    public void cleanup() {
+        if (driver != null) {
+            driver.close();
+        }
+        driver = null;
+    }
 
     @Test
     public void testPrintKeyValueDefaultSerde() throws Exception {
@@ -57,7 +66,7 @@
         KStream<String, String> stream = builder.stream(stringSerde, stringSerde, topicName);
         stream.process(keyValuePrinter);
 
-        KStreamTestDriver driver = new KStreamTestDriver(builder);
+        driver = new KStreamTestDriver(builder);
         for (int i = 0; i < suppliedKeys.length; i++) {
             driver.process(topicName, suppliedKeys[i], suppliedValues[i]);
         }
@@ -79,7 +88,7 @@
 
         stream.process(keyValuePrinter);
 
-        KStreamTestDriver driver = new KStreamTestDriver(builder);
+        driver = new KStreamTestDriver(builder);
 
         String suppliedKey = null;
         byte[] suppliedValue = "{\"name\":\"print\", \"label\":\"test\"}".getBytes(Charset.forName("UTF-8"));
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 7c6d5ec..b31b20d 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
@@ -49,7 +49,7 @@
             new PartitionInfo(topicName, 5, Node.noNode(), new Node[0], new Node[0])
     );
 
-    private Cluster cluster = new Cluster(Arrays.asList(Node.noNode()), infos, Collections.<String>emptySet());
+    private Cluster cluster = new Cluster(Collections.singletonList(Node.noNode()), infos, Collections.<String>emptySet());
 
     @Test
     public void testCopartitioning() {
@@ -71,7 +71,7 @@
             Integer expected = defaultPartitioner.partition("topic", key, keyBytes, value, valueBytes, cluster);
 
             for (int w = 0; w < 10; w++) {
-                HoppingWindow window = new HoppingWindow(10 * w, 20 * w);
+                TimeWindow window = new TimeWindow(10 * w, 20 * w);
 
                 Windowed<Integer> windowedKey = new Windowed<>(key, window);
                 Integer actual = streamPartitioner.partition(windowedKey, value, infos.size());
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowsTest.java
deleted file mode 100644
index f9b6ba5..0000000
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowsTest.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.kafka.streams.kstream.internals;
-
-import org.apache.kafka.streams.kstream.HoppingWindows;
-import org.apache.kafka.streams.kstream.TumblingWindows;
-import org.apache.kafka.streams.kstream.UnlimitedWindows;
-import org.junit.Test;
-
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
-
-public class WindowsTest {
-
-    @Test
-    public void hoppingWindows() {
-
-        HoppingWindows windows = HoppingWindows.of("test").with(12L).every(5L);
-
-        Map<Long, HoppingWindow> matched = windows.windowsFor(21L);
-
-        assertEquals(3, matched.size());
-
-        assertEquals(new HoppingWindow(10L, 22L), matched.get(10L));
-        assertEquals(new HoppingWindow(15L, 27L), matched.get(15L));
-        assertEquals(new HoppingWindow(20L, 32L), matched.get(20L));
-    }
-
-    @Test
-    public void tumblineWindows() {
-
-        TumblingWindows windows = TumblingWindows.of("test").with(12L);
-
-        Map<Long, TumblingWindow> matched = windows.windowsFor(21L);
-
-        assertEquals(1, matched.size());
-
-        assertEquals(new TumblingWindow(12L, 24L), matched.get(12L));
-    }
-
-    @Test
-    public void unlimitedWindows() {
-
-        UnlimitedWindows windows = UnlimitedWindows.of("test").startOn(10L);
-
-        Map<Long, UnlimitedWindow> matched = windows.windowsFor(21L);
-
-        assertEquals(1, matched.size());
-
-        assertEquals(new UnlimitedWindow(10L), matched.get(10L));
-    }
-}
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 c883090..a92fb1b 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
@@ -253,6 +253,7 @@
 
         long endTime = System.currentTimeMillis();
 
+        consumer.close();
         System.out.println("Consumer Performance [MB/sec read]: " + megaBytePerSec(endTime - startTime));
     }
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java
index 84b59e6..e3669e8 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java
@@ -213,7 +213,10 @@
             try {
                 assertNotNull(lock);
             } finally {
-                if (lock != null) lock.release();
+                if (lock != null) {
+                    lock.release();
+                    lock.channel().close();
+                }
             }
         } finally {
             Utils.delete(baseDir);
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java
index be851bf..3e8b110 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java
@@ -556,7 +556,7 @@
         }
 
         @Override
-        public void makeReady(String topic, int numPartitions) {
+        public void makeReady(String topic, int numPartitions, boolean compactTopic) {
             readyTopics.put(topic, numPartitions);
 
             List<PartitionInfo> partitions = new ArrayList<>();
diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java
index 0a02824..fbe7754 100644
--- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java
+++ b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java
@@ -28,7 +28,7 @@
 import org.apache.kafka.streams.kstream.KTable;
 import org.apache.kafka.streams.kstream.KeyValueMapper;
 import org.apache.kafka.streams.kstream.Predicate;
-import org.apache.kafka.streams.kstream.TumblingWindows;
+import org.apache.kafka.streams.kstream.TimeWindows;
 import org.apache.kafka.streams.kstream.UnlimitedWindows;
 import org.apache.kafka.streams.kstream.ValueJoiner;
 import org.apache.kafka.streams.kstream.Windowed;
@@ -207,28 +207,27 @@
 
         // windowed count
         data.countByKey(
-                TumblingWindows.of("tumbling-win-cnt").with(WINDOW_SIZE),
+                TimeWindows.of("tumbling-win-cnt", WINDOW_SIZE),
                 stringSerde
         ).toStream().map(
                 new KeyValueMapper<Windowed<String>, Long, KeyValue<String, Long>>() {
                     @Override
                     public KeyValue<String, Long> apply(Windowed<String> key, Long value) {
-                        return new KeyValue<>(key.value() + "@" + key.window().start(), value);
+                        return new KeyValue<>(key.key() + "@" + key.window().start(), value);
                     }
                 }
         ).to(stringSerde, longSerde, "wcnt");
 
         // test repartition
         Agg agg = new Agg();
-        cntTable.aggregate(
-                agg.init(),
-                agg.adder(),
-                agg.remover(),
-                agg.selector(),
-                stringSerde,
-                longSerde,
-                longSerde,
-                "cntByCnt"
+        cntTable.groupBy(agg.selector(),
+                         stringSerde,
+                         longSerde
+        ).aggregate(agg.init(),
+                    agg.adder(),
+                    agg.remover(),
+                    longSerde,
+                    "cntByCnt"
         ).to(stringSerde, longSerde, "tagg");
 
         return new KafkaStreams(builder, props);
diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java
index d7b0139..205ba4b 100644
--- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java
+++ b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java
@@ -278,7 +278,7 @@
                 }
             }
         }
-
+        consumer.close();
 
         System.out.println("-------------------");
         System.out.println("Result Verification");
diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestUtil.java b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestUtil.java
index c5ded5e..b0d7a0b 100644
--- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestUtil.java
+++ b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestUtil.java
@@ -77,7 +77,7 @@
 
     public static final class Unwindow<K, V> implements KeyValueMapper<Windowed<K>, V, KeyValue<K, V>> {
         public KeyValue<K, V> apply(Windowed<K> winKey, V value) {
-            return new KeyValue<K, V>(winKey.value(), value);
+            return new KeyValue<K, V>(winKey.key(), value);
         }
     }
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java
index 0468f49..3a35d75 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java
@@ -198,19 +198,8 @@
             public <K1, V1> void send(ProducerRecord<K1, V1> record, Serializer<K1> keySerializer, Serializer<V1> valueSerializer) {
                 // for byte arrays we need to wrap it for comparison
 
-                K key;
-                if (record.key() instanceof byte[]) {
-                    key = serdes.keyFrom((byte[]) record.key());
-                } else {
-                    key = (K) record.key();
-                }
-
-                V value;
-                if (record.key() instanceof byte[]) {
-                    value = serdes.valueFrom((byte[]) record.value());
-                } else {
-                    value = (V) record.value();
-                }
+                K key = serdes.keyFrom(keySerializer.serialize(record.topic(), record.key()));
+                V value = serdes.valueFrom(valueSerializer.serialize(record.topic(), record.value()));
 
                 recordFlushed(key, value);
             }
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 fb0efc9..2bfe644 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
@@ -192,8 +192,6 @@
         }
     }
 
-
-
     @Test
     public void testPutIfAbsent() {
         // Create the test driver ...
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java
index 502870b..e9888ad 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java
@@ -31,7 +31,6 @@
 import org.apache.kafka.streams.state.StateSerdes;
 import org.apache.kafka.streams.state.WindowStore;
 import org.apache.kafka.streams.state.WindowStoreIterator;
-import org.apache.kafka.streams.state.WindowStoreUtils;
 import org.apache.kafka.test.MockProcessorContext;
 import org.junit.Test;
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java
index 9a477df..09f12fb 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java
@@ -24,10 +24,7 @@
 import java.util.Map;
 
 import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.common.serialization.IntegerDeserializer;
-import org.apache.kafka.common.serialization.IntegerSerializer;
 import org.apache.kafka.common.serialization.Serializer;
-import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.streams.processor.StreamPartitioner;
 import org.apache.kafka.streams.processor.internals.RecordCollector;
@@ -63,8 +60,6 @@
 
     private final StoreChangeLogger<Integer, String> changeLogger = new StoreChangeLogger<>(topic, context, StateSerdes.withBuiltinTypes(topic, Integer.class, String.class), 3, 3);
 
-    private final StoreChangeLogger<byte[], byte[]> rawChangeLogger = new RawStoreChangeLogger(topic, context, 3, 3);
-
     private final StoreChangeLogger.ValueGetter<Integer, String> getter = new StoreChangeLogger.ValueGetter<Integer, String>() {
         @Override
         public String get(Integer key) {
@@ -72,16 +67,6 @@
         }
     };
 
-    private final StoreChangeLogger.ValueGetter<byte[], byte[]> rawGetter = new StoreChangeLogger.ValueGetter<byte[], byte[]>() {
-        private IntegerDeserializer deserializer = new IntegerDeserializer();
-        private StringSerializer serializer = new StringSerializer();
-
-        @Override
-        public byte[] get(byte[] key) {
-            return serializer.serialize(topic, written.get(deserializer.deserialize(topic, key)));
-        }
-    };
-
     @Test
     public void testAddRemove() {
         written.put(0, "zero");
@@ -117,30 +102,4 @@
         assertEquals("three", logged.get(3));
         assertEquals("four", logged.get(4));
     }
-
-    @Test
-    public void testRaw() {
-        IntegerSerializer serializer = new IntegerSerializer();
-
-        written.put(0, "zero");
-        rawChangeLogger.add(serializer.serialize(topic, 0));
-        written.put(1, "one");
-        rawChangeLogger.add(serializer.serialize(topic, 1));
-        written.put(2, "two");
-        rawChangeLogger.add(serializer.serialize(topic, 2));
-        assertEquals(3, rawChangeLogger.numDirty());
-        assertEquals(0, rawChangeLogger.numRemoved());
-
-        rawChangeLogger.delete(serializer.serialize(topic, 0));
-        rawChangeLogger.delete(serializer.serialize(topic, 1));
-        written.put(3, "three");
-        rawChangeLogger.add(serializer.serialize(topic, 3));
-        assertEquals(2, rawChangeLogger.numDirty());
-        assertEquals(2, rawChangeLogger.numRemoved());
-
-        written.put(0, "zero-again");
-        rawChangeLogger.add(serializer.serialize(topic, 0));
-        assertEquals(3, rawChangeLogger.numDirty());
-        assertEquals(1, rawChangeLogger.numRemoved());
-    }
 }
diff --git a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java
index 2ee8730..7316804 100644
--- a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java
+++ b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java
@@ -27,12 +27,15 @@
 import org.apache.kafka.streams.processor.StateStoreSupplier;
 import org.apache.kafka.streams.processor.StreamPartitioner;
 import org.apache.kafka.streams.processor.internals.ProcessorNode;
+import org.apache.kafka.streams.processor.internals.ProcessorStateManager;
 import org.apache.kafka.streams.processor.internals.ProcessorTopology;
 import org.apache.kafka.streams.processor.internals.RecordCollector;
 
 import java.io.File;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 public class KStreamTestDriver {
 
@@ -80,6 +83,12 @@
 
     public void process(String topicName, Object key, Object value) {
         currNode = topology.source(topicName);
+
+        // if currNode is null, check if this topic is a changelog topic;
+        // if yes, skip
+        if (topicName.endsWith(ProcessorStateManager.STATE_CHANGELOG_TOPIC_SUFFIX))
+            return;
+
         try {
             forward(key, value);
         } finally {
@@ -106,10 +115,6 @@
         context.setTime(timestamp);
     }
 
-    public StateStore getStateStore(String name) {
-        return context.getStateStore(name);
-    }
-
     @SuppressWarnings("unchecked")
     public <K, V> void forward(K key, V value) {
         ProcessorNode thisNode = currNode;
@@ -151,11 +156,50 @@
         }
     }
 
+    public void close() {
+        // close all processors
+        for (ProcessorNode node : topology.processors()) {
+            currNode = node;
+            try {
+                node.close();
+            } finally {
+                currNode = null;
+            }
+        }
+
+        // close all state stores
+        for (StateStore store : context.allStateStores().values()) {
+            store.close();
+        }
+    }
+
+    public Set<String> allProcessorNames() {
+        Set<String> names = new HashSet<>();
+
+        List<ProcessorNode> nodes = topology.processors();
+
+        for (ProcessorNode node: nodes) {
+            names.add(node.name());
+        }
+
+        return names;
+    }
+
+    public ProcessorNode processor(String name) {
+        List<ProcessorNode> nodes = topology.processors();
+
+        for (ProcessorNode node: nodes) {
+            if (node.name().equals(name))
+                return node;
+        }
+
+        return null;
+    }
+
     public Map<String, StateStore> allStateStores() {
         return context.allStateStores();
     }
 
-
     private class MockRecordCollector extends RecordCollector {
         public MockRecordCollector() {
             super(null);
diff --git a/streams/src/test/java/org/apache/kafka/test/MockAggregator.java b/streams/src/test/java/org/apache/kafka/test/MockAggregator.java
new file mode 100644
index 0000000..e8bb10b
--- /dev/null
+++ b/streams/src/test/java/org/apache/kafka/test/MockAggregator.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.test;
+
+import org.apache.kafka.streams.kstream.Aggregator;
+
+public class MockAggregator {
+
+    private static class StringAdd implements Aggregator<String, String, String> {
+
+        @Override
+        public String apply(String aggKey, String value, String aggregate) {
+            return aggregate + "+" + value;
+        }
+    }
+
+    private static class StringRemove implements Aggregator<String, String, String> {
+
+        @Override
+        public String apply(String aggKey, String value, String aggregate) {
+            return aggregate + "-" + value;
+        }
+    }
+
+    public final static Aggregator<String, String, String> STRING_ADDER = new StringAdd();
+
+    public final static Aggregator<String, String, String> STRING_REMOVER = new StringRemove();
+}
diff --git a/streams/src/test/java/org/apache/kafka/test/NoOpKeyValueMapper.java b/streams/src/test/java/org/apache/kafka/test/MockInitializer.java
similarity index 72%
rename from streams/src/test/java/org/apache/kafka/test/NoOpKeyValueMapper.java
rename to streams/src/test/java/org/apache/kafka/test/MockInitializer.java
index 828b5ae..9bfe7f8 100644
--- a/streams/src/test/java/org/apache/kafka/test/NoOpKeyValueMapper.java
+++ b/streams/src/test/java/org/apache/kafka/test/MockInitializer.java
@@ -17,13 +17,17 @@
 
 package org.apache.kafka.test;
 
-import org.apache.kafka.streams.KeyValue;
-import org.apache.kafka.streams.kstream.KeyValueMapper;
+import org.apache.kafka.streams.kstream.Initializer;
 
-public class NoOpKeyValueMapper<K, V> implements KeyValueMapper<K, V, KeyValue<K, V>> {
+public class MockInitializer {
 
-    @Override
-    public KeyValue<K, V> apply(K key, V value) {
-        return new KeyValue<>(key, value);
+    private static class StringInit implements Initializer<String> {
+
+        @Override
+        public String apply() {
+            return "0";
+        }
     }
+
+    public final static Initializer<String> STRING_INIT = new StringInit();
 }
diff --git a/streams/src/test/java/org/apache/kafka/test/NoOpKeyValueMapper.java b/streams/src/test/java/org/apache/kafka/test/MockKeyValueMapper.java
similarity index 70%
copy from streams/src/test/java/org/apache/kafka/test/NoOpKeyValueMapper.java
copy to streams/src/test/java/org/apache/kafka/test/MockKeyValueMapper.java
index 828b5ae..769ee71 100644
--- a/streams/src/test/java/org/apache/kafka/test/NoOpKeyValueMapper.java
+++ b/streams/src/test/java/org/apache/kafka/test/MockKeyValueMapper.java
@@ -20,10 +20,17 @@
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.kstream.KeyValueMapper;
 
-public class NoOpKeyValueMapper<K, V> implements KeyValueMapper<K, V, KeyValue<K, V>> {
+public class MockKeyValueMapper {
 
-    @Override
-    public KeyValue<K, V> apply(K key, V value) {
-        return new KeyValue<>(key, value);
+    private static class NoOpKeyValueMapper<K, V> implements KeyValueMapper<K, V, KeyValue<K, V>> {
+
+        @Override
+        public KeyValue<K, V> apply(K key, V value) {
+            return KeyValue.pair(key, value);
+        }
     }
-}
+
+    public static <K, V> KeyValueMapper<K, V, KeyValue<K, V>> NoOpKeyValueMapper() {
+        return new NoOpKeyValueMapper<>();
+    }
+}
\ No newline at end of file
diff --git a/streams/src/test/java/org/apache/kafka/test/MockReducer.java b/streams/src/test/java/org/apache/kafka/test/MockReducer.java
new file mode 100644
index 0000000..24a8fea
--- /dev/null
+++ b/streams/src/test/java/org/apache/kafka/test/MockReducer.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.test;
+
+import org.apache.kafka.streams.kstream.Reducer;
+
+public class MockReducer {
+
+    private static class StringAdd implements Reducer<String> {
+
+        @Override
+        public String apply(String value1, String value2) {
+            return value1 + "+" + value2;
+        }
+    }
+
+    private static class StringRemove implements Reducer<String> {
+
+        @Override
+        public String apply(String value1, String value2) {
+            return value1 + "-" + value2;
+        }
+    }
+
+    public final static Reducer<String> STRING_ADDER = new StringAdd();
+
+    public final static Reducer<String> STRING_REMOVER = new StringRemove();
+}
\ No newline at end of file
diff --git a/streams/src/test/java/org/apache/kafka/test/NoOpKeyValueMapper.java b/streams/src/test/java/org/apache/kafka/test/MockValueJoiner.java
similarity index 67%
copy from streams/src/test/java/org/apache/kafka/test/NoOpKeyValueMapper.java
copy to streams/src/test/java/org/apache/kafka/test/MockValueJoiner.java
index 828b5ae..4d44166 100644
--- a/streams/src/test/java/org/apache/kafka/test/NoOpKeyValueMapper.java
+++ b/streams/src/test/java/org/apache/kafka/test/MockValueJoiner.java
@@ -17,13 +17,17 @@
 
 package org.apache.kafka.test;
 
-import org.apache.kafka.streams.KeyValue;
-import org.apache.kafka.streams.kstream.KeyValueMapper;
+import org.apache.kafka.streams.kstream.ValueJoiner;
 
-public class NoOpKeyValueMapper<K, V> implements KeyValueMapper<K, V, KeyValue<K, V>> {
+public class MockValueJoiner {
 
-    @Override
-    public KeyValue<K, V> apply(K key, V value) {
-        return new KeyValue<>(key, value);
-    }
-}
+    private static class StringJoin implements ValueJoiner<String, String, String> {
+
+        @Override
+        public String apply(String value1, String value2) {
+            return value1 + "+" + value2;
+        }
+    };
+
+    public final static ValueJoiner<String, String, String> STRING_JOINER = new StringJoin();
+}
\ No newline at end of file
diff --git a/tests/kafkatest/benchmarks/core/benchmark_test.py b/tests/kafkatest/benchmarks/core/benchmark_test.py
index 9c2e32d..83f4b2a 100644
--- a/tests/kafkatest/benchmarks/core/benchmark_test.py
+++ b/tests/kafkatest/benchmarks/core/benchmark_test.py
@@ -20,7 +20,8 @@
 
 from kafkatest.services.zookeeper import ZookeeperService
 from kafkatest.services.kafka import KafkaService
-from kafkatest.services.performance import ProducerPerformanceService, EndToEndLatencyService, ConsumerPerformanceService
+from kafkatest.services.kafka.version import TRUNK, KafkaVersion
+from kafkatest.services.performance import ProducerPerformanceService, EndToEndLatencyService, ConsumerPerformanceService, throughput, latency, compute_aggregate_throughput
 
 
 TOPIC_REP_ONE = "topic-replication-factor-one"
@@ -54,11 +55,12 @@
     def setUp(self):
         self.zk.start()
 
-    def start_kafka(self, security_protocol, interbroker_security_protocol):
+    def start_kafka(self, security_protocol, interbroker_security_protocol, version):
         self.kafka = KafkaService(
             self.test_context, self.num_brokers,
             self.zk, security_protocol=security_protocol,
-            interbroker_security_protocol=interbroker_security_protocol, topics=self.topics)
+            interbroker_security_protocol=interbroker_security_protocol, topics=self.topics,
+            version=version)
         self.kafka.log_level = "INFO"  # We don't DEBUG logging here
         self.kafka.start()
 
@@ -66,8 +68,10 @@
     @parametrize(acks=1, topic=TOPIC_REP_THREE)
     @parametrize(acks=-1, topic=TOPIC_REP_THREE)
     @parametrize(acks=1, topic=TOPIC_REP_THREE, num_producers=3)
-    @matrix(acks=[1], topic=[TOPIC_REP_THREE], message_size=[10, 100, 1000, 10000, 100000], security_protocol=['PLAINTEXT', 'SSL'])
-    def test_producer_throughput(self, acks, topic, num_producers=1, message_size=DEFAULT_RECORD_SIZE, security_protocol='PLAINTEXT'):
+    @matrix(acks=[1], topic=[TOPIC_REP_THREE], message_size=[10, 100, 1000, 10000, 100000], compression_type=["none", "snappy"], security_protocol=['PLAINTEXT', 'SSL'])
+    def test_producer_throughput(self, acks, topic, num_producers=1, message_size=DEFAULT_RECORD_SIZE,
+                                 compression_type="none", security_protocol='PLAINTEXT', client_version=str(TRUNK),
+                                 broker_version=str(TRUNK)):
         """
         Setup: 1 node zk + 3 node kafka cluster
         Produce ~128MB worth of messages to a topic with 6 partitions. Required acks, topic replication factor,
@@ -76,23 +80,29 @@
         Collect and return aggregate throughput statistics after all messages have been acknowledged.
         (This runs ProducerPerformance.java under the hood)
         """
-        self.start_kafka(security_protocol, security_protocol)
+        client_version = KafkaVersion(client_version)
+        broker_version = KafkaVersion(broker_version)
+        self.validate_versions(client_version, broker_version)
+        self.start_kafka(security_protocol, security_protocol, broker_version)
         # Always generate the same total amount of data
         nrecords = int(self.target_data_size / message_size)
 
         self.producer = ProducerPerformanceService(
             self.test_context, num_producers, self.kafka, topic=topic,
-            num_records=nrecords, record_size=message_size,  throughput=-1,
+            num_records=nrecords, record_size=message_size,  throughput=-1, version=client_version,
             settings={
                 'acks': acks,
+                'compression.type': compression_type,
                 'batch.size': self.batch_size,
                 'buffer.memory': self.buffer_memory})
         self.producer.run()
         return compute_aggregate_throughput(self.producer)
 
     @parametrize(security_protocol='SSL', interbroker_security_protocol='PLAINTEXT')
-    @matrix(security_protocol=['PLAINTEXT', 'SSL'])
-    def test_long_term_producer_throughput(self, security_protocol, interbroker_security_protocol=None):
+    @matrix(security_protocol=['PLAINTEXT', 'SSL'], compression_type=["none", "snappy"])
+    def test_long_term_producer_throughput(self, compression_type="none", security_protocol='PLAINTEXT',
+                                           interbroker_security_protocol=None, client_version=str(TRUNK),
+                                           broker_version=str(TRUNK)):
         """
         Setup: 1 node zk + 3 node kafka cluster
         Produce 10e6 100 byte messages to a topic with 6 partitions, replication-factor 3, and acks=1.
@@ -101,13 +111,21 @@
 
         (This runs ProducerPerformance.java under the hood)
         """
+        client_version = KafkaVersion(client_version)
+        broker_version = KafkaVersion(broker_version)
+        self.validate_versions(client_version, broker_version)
         if interbroker_security_protocol is None:
             interbroker_security_protocol = security_protocol
-        self.start_kafka(security_protocol, interbroker_security_protocol)
+        self.start_kafka(security_protocol, interbroker_security_protocol, broker_version)
         self.producer = ProducerPerformanceService(
             self.test_context, 1, self.kafka,
             topic=TOPIC_REP_THREE, num_records=self.msgs_large, record_size=DEFAULT_RECORD_SIZE,
-            throughput=-1, settings={'acks': 1, 'batch.size': self.batch_size, 'buffer.memory': self.buffer_memory},
+            throughput=-1, version=client_version, settings={
+                'acks': 1,
+                'compression.type': compression_type,
+                'batch.size': self.batch_size,
+                'buffer.memory': self.buffer_memory
+            },
             intermediate_stats=True
         )
         self.producer.run()
@@ -135,10 +153,11 @@
         self.logger.info("\n".join(summary))
         return data
 
-
     @parametrize(security_protocol='SSL', interbroker_security_protocol='PLAINTEXT')
-    @matrix(security_protocol=['PLAINTEXT', 'SSL', 'SASL_PLAINTEXT', 'SASL_SSL'])
-    def test_end_to_end_latency(self, security_protocol, interbroker_security_protocol=None):
+    @matrix(security_protocol=['PLAINTEXT', 'SSL', 'SASL_PLAINTEXT', 'SASL_SSL'], compression_type=["none", "snappy"])
+    def test_end_to_end_latency(self, compression_type="none", security_protocol="PLAINTEXT",
+                                interbroker_security_protocol=None, client_version=str(TRUNK),
+                                broker_version=str(TRUNK)):
         """
         Setup: 1 node zk + 3 node kafka cluster
         Produce (acks = 1) and consume 10e3 messages to a topic with 6 partitions and replication-factor 3,
@@ -148,21 +167,27 @@
 
         (Under the hood, this simply runs EndToEndLatency.scala)
         """
+        client_version = KafkaVersion(client_version)
+        broker_version = KafkaVersion(broker_version)
+        self.validate_versions(client_version, broker_version)
         if interbroker_security_protocol is None:
             interbroker_security_protocol = security_protocol
-        self.start_kafka(security_protocol, interbroker_security_protocol)
+        self.start_kafka(security_protocol, interbroker_security_protocol, broker_version)
         self.logger.info("BENCHMARK: End to end latency")
         self.perf = EndToEndLatencyService(
             self.test_context, 1, self.kafka,
-            topic=TOPIC_REP_THREE, num_records=10000
+            topic=TOPIC_REP_THREE, num_records=10000,
+            compression_type=compression_type, version=client_version
         )
         self.perf.run()
         return latency(self.perf.results[0]['latency_50th_ms'],  self.perf.results[0]['latency_99th_ms'], self.perf.results[0]['latency_999th_ms'])
 
     @parametrize(security_protocol='PLAINTEXT', new_consumer=False)
     @parametrize(security_protocol='SSL', interbroker_security_protocol='PLAINTEXT')
-    @matrix(security_protocol=['PLAINTEXT', 'SSL'])
-    def test_producer_and_consumer(self, security_protocol, interbroker_security_protocol=None, new_consumer=True):
+    @matrix(security_protocol=['PLAINTEXT', 'SSL'], compression_type=["none", "snappy"])
+    def test_producer_and_consumer(self, compression_type="none", security_protocol="PLAINTEXT",
+                                   interbroker_security_protocol=None, new_consumer=True,
+                                   client_version=str(TRUNK), broker_version=str(TRUNK)):
         """
         Setup: 1 node zk + 3 node kafka cluster
         Concurrently produce and consume 10e6 messages with a single producer and a single consumer,
@@ -172,16 +197,24 @@
 
         (Under the hood, this runs ProducerPerformance.java, and ConsumerPerformance.scala)
         """
+        client_version = KafkaVersion(client_version)
+        broker_version = KafkaVersion(broker_version)
+        self.validate_versions(client_version, broker_version)
         if interbroker_security_protocol is None:
             interbroker_security_protocol = security_protocol
-        self.start_kafka(security_protocol, interbroker_security_protocol)
+        self.start_kafka(security_protocol, interbroker_security_protocol, broker_version)
         num_records = 10 * 1000 * 1000  # 10e6
 
         self.producer = ProducerPerformanceService(
             self.test_context, 1, self.kafka,
             topic=TOPIC_REP_THREE,
-            num_records=num_records, record_size=DEFAULT_RECORD_SIZE, throughput=-1,
-            settings={'acks': 1, 'batch.size': self.batch_size, 'buffer.memory': self.buffer_memory}
+            num_records=num_records, record_size=DEFAULT_RECORD_SIZE, throughput=-1, version=client_version,
+            settings={
+                'acks': 1,
+                'compression.type': compression_type,
+                'batch.size': self.batch_size,
+                'buffer.memory': self.buffer_memory
+            }
         )
         self.consumer = ConsumerPerformanceService(
             self.test_context, 1, self.kafka, topic=TOPIC_REP_THREE, new_consumer=new_consumer, messages=num_records)
@@ -199,23 +232,33 @@
 
     @parametrize(security_protocol='PLAINTEXT', new_consumer=False)
     @parametrize(security_protocol='SSL', interbroker_security_protocol='PLAINTEXT')
-    @matrix(security_protocol=['PLAINTEXT', 'SSL'])
-    def test_consumer_throughput(self, security_protocol, interbroker_security_protocol=None, new_consumer=True, num_consumers=1):
+    @matrix(security_protocol=['PLAINTEXT', 'SSL'], compression_type=["none", "snappy"])
+    def test_consumer_throughput(self, compression_type="none", security_protocol="PLAINTEXT",
+                                 interbroker_security_protocol=None, new_consumer=True, num_consumers=1,
+                                 client_version=str(TRUNK), broker_version=str(TRUNK)):
         """
         Consume 10e6 100-byte messages with 1 or more consumers from a topic with 6 partitions
         (using new consumer iff new_consumer == True), and report throughput.
         """
+        client_version = KafkaVersion(client_version)
+        broker_version = KafkaVersion(broker_version)
+        self.validate_versions(client_version, broker_version)
         if interbroker_security_protocol is None:
             interbroker_security_protocol = security_protocol
-        self.start_kafka(security_protocol, interbroker_security_protocol)
+        self.start_kafka(security_protocol, interbroker_security_protocol, broker_version)
         num_records = 10 * 1000 * 1000  # 10e6
 
         # seed kafka w/messages
         self.producer = ProducerPerformanceService(
             self.test_context, 1, self.kafka,
             topic=TOPIC_REP_THREE,
-            num_records=num_records, record_size=DEFAULT_RECORD_SIZE, throughput=-1,
-            settings={'acks': 1, 'batch.size': self.batch_size, 'buffer.memory': self.buffer_memory}
+            num_records=num_records, record_size=DEFAULT_RECORD_SIZE, throughput=-1, version=client_version,
+            settings={
+                'acks': 1,
+                'compression.type': compression_type,
+                'batch.size': self.batch_size,
+                'buffer.memory': self.buffer_memory
+            }
         )
         self.producer.run()
 
@@ -227,27 +270,5 @@
         self.consumer.run()
         return compute_aggregate_throughput(self.consumer)
 
-
-def throughput(records_per_sec, mb_per_sec):
-    """Helper method to ensure uniform representation of throughput data"""
-    return {
-        "records_per_sec": records_per_sec,
-        "mb_per_sec": mb_per_sec
-    }
-
-
-def latency(latency_50th_ms, latency_99th_ms, latency_999th_ms):
-    """Helper method to ensure uniform representation of latency data"""
-    return {
-        "latency_50th_ms": latency_50th_ms,
-        "latency_99th_ms": latency_99th_ms,
-        "latency_999th_ms": latency_999th_ms
-    }
-
-
-def compute_aggregate_throughput(perf):
-    """Helper method for computing throughput after running a performance service."""
-    aggregate_rate = sum([r['records_per_sec'] for r in perf.results])
-    aggregate_mbps = sum([r['mbps'] for r in perf.results])
-
-    return throughput(aggregate_rate, aggregate_mbps)
+    def validate_versions(self, client_version, broker_version):
+        assert client_version <= broker_version, "Client version %s should be <= than broker version %s" (client_version, broker_version)
diff --git a/tests/kafkatest/sanity_checks/test_console_consumer.py b/tests/kafkatest/sanity_checks/test_console_consumer.py
index 139c74a..d6a152a 100644
--- a/tests/kafkatest/sanity_checks/test_console_consumer.py
+++ b/tests/kafkatest/sanity_checks/test_console_consumer.py
@@ -45,11 +45,14 @@
         self.zk.start()
 
     @parametrize(security_protocol='PLAINTEXT', new_consumer=False)
+    @parametrize(security_protocol='SASL_SSL', sasl_mechanism='PLAIN')
     @matrix(security_protocol=['PLAINTEXT', 'SSL', 'SASL_PLAINTEXT', 'SASL_SSL'])
-    def test_lifecycle(self, security_protocol, new_consumer=True):
+    def test_lifecycle(self, security_protocol, new_consumer=True, sasl_mechanism='GSSAPI'):
         """Check that console consumer starts/stops properly, and that we are capturing log output."""
 
         self.kafka.security_protocol = security_protocol
+        self.kafka.client_sasl_mechanism = sasl_mechanism
+        self.kafka.interbroker_sasl_mechanism = sasl_mechanism
         self.kafka.start()
 
         self.consumer.security_protocol = security_protocol
diff --git a/tests/kafkatest/sanity_checks/test_performance_services.py b/tests/kafkatest/sanity_checks/test_performance_services.py
new file mode 100644
index 0000000..16d5d32
--- /dev/null
+++ b/tests/kafkatest/sanity_checks/test_performance_services.py
@@ -0,0 +1,88 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from ducktape.tests.test import Test
+from ducktape.mark import parametrize
+
+from kafkatest.services.zookeeper import ZookeeperService
+from kafkatest.services.kafka import KafkaService
+from kafkatest.services.kafka.version import TRUNK, LATEST_0_8_2, LATEST_0_9, KafkaVersion
+from kafkatest.services.performance import ProducerPerformanceService, ConsumerPerformanceService, EndToEndLatencyService
+from kafkatest.services.performance import latency, compute_aggregate_throughput
+
+
+class PerformanceServiceTest(Test):
+    def __init__(self, test_context):
+        super(PerformanceServiceTest, self).__init__(test_context)
+        self.record_size = 100
+        self.num_records = 10000
+        self.topic = "topic"
+
+        self.zk = ZookeeperService(test_context, 1)
+
+    def setUp(self):
+        self.zk.start()
+
+    # We are keeping 0.8.2 here so that we don't inadvertently break support for it. Since this is just a sanity check,
+    # the overhead should be manageable.
+    @parametrize(version=str(LATEST_0_8_2))
+    @parametrize(version=str(LATEST_0_9), new_consumer=False)
+    @parametrize(version=str(LATEST_0_9), new_consumer=True)
+    @parametrize(version=str(TRUNK), new_consumer=False)
+    @parametrize(version=str(TRUNK), new_consumer=True)
+    def test_version(self, version=str(LATEST_0_9), new_consumer=False):
+        """
+        Sanity check out producer performance service - verify that we can run the service with a small
+        number of messages. The actual stats here are pretty meaningless since the number of messages is quite small.
+        """
+        version = KafkaVersion(version)
+        self.kafka = KafkaService(
+            self.test_context, 1,
+            self.zk, topics={self.topic: {'partitions': 1, 'replication-factor': 1}}, version=version)
+        self.kafka.start()
+
+        # check basic run of producer performance
+        self.producer_perf = ProducerPerformanceService(
+            self.test_context, 1, self.kafka, topic=self.topic,
+            num_records=self.num_records, record_size=self.record_size,
+            throughput=1000000000,  # Set impossibly for no throttling for equivalent behavior between 0.8.X and 0.9.X
+            version=version,
+            settings={
+                'acks': 1,
+                'batch.size': 8*1024,
+                'buffer.memory': 64*1024*1024})
+        self.producer_perf.run()
+        producer_perf_data = compute_aggregate_throughput(self.producer_perf)
+
+        # check basic run of end to end latency
+        self.end_to_end = EndToEndLatencyService(
+            self.test_context, 1, self.kafka,
+            topic=self.topic, num_records=self.num_records, version=version)
+        self.end_to_end.run()
+        end_to_end_data = latency(self.end_to_end.results[0]['latency_50th_ms'],  self.end_to_end.results[0]['latency_99th_ms'], self.end_to_end.results[0]['latency_999th_ms'])
+
+        # check basic run of consumer performance service
+        self.consumer_perf = ConsumerPerformanceService(
+            self.test_context, 1, self.kafka, new_consumer=new_consumer,
+            topic=self.topic, version=version, messages=self.num_records)
+        self.consumer_perf.group = "test-consumer-group"
+        self.consumer_perf.run()
+        consumer_perf_data = compute_aggregate_throughput(self.consumer_perf)
+
+        return {
+            "producer_performance": producer_perf_data,
+            "end_to_end_latency": end_to_end_data,
+            "consumer_performance": consumer_perf_data
+        }
diff --git a/tests/kafkatest/services/console_consumer.py b/tests/kafkatest/services/console_consumer.py
index e5f2196..37638e2 100644
--- a/tests/kafkatest/services/console_consumer.py
+++ b/tests/kafkatest/services/console_consumer.py
@@ -123,6 +123,7 @@
         self.from_beginning = from_beginning
         self.message_validator = message_validator
         self.messages_consumed = {idx: [] for idx in range(1, num_nodes + 1)}
+        self.clean_shutdown_nodes = set()
         self.client_id = client_id
         self.print_key = print_key
         self.log_level = "TRACE"
@@ -185,6 +186,7 @@
         if node.version > LATEST_0_9:
             cmd+=" --formatter kafka.tools.LoggingMessageFormatter"
 
+        cmd += " --enable-systest-events"
         cmd += " 2>> %(stderr)s | tee -a %(stdout)s &" % args
         return cmd
 
@@ -226,10 +228,15 @@
 
             for line in itertools.chain([first_line], consumer_output):
                 msg = line.strip()
-                if self.message_validator is not None:
-                    msg = self.message_validator(msg)
-                if msg is not None:
-                    self.messages_consumed[idx].append(msg)
+                if msg == "shutdown_complete":
+                    if node in self.clean_shutdown_nodes:
+                        raise Exception("Unexpected shutdown event from consumer, already shutdown. Consumer index: %d" % idx)
+                    self.clean_shutdown_nodes.add(node)
+                else:
+                    if self.message_validator is not None:
+                        msg = self.message_validator(msg)
+                    if msg is not None:
+                        self.messages_consumed[idx].append(msg)
 
             self.read_jmx_output(idx, node)
 
diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py
index 33ece35..a74bb00 100644
--- a/tests/kafkatest/services/kafka/kafka.py
+++ b/tests/kafkatest/services/kafka/kafka.py
@@ -63,7 +63,8 @@
     }
 
     def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAINTEXT, interbroker_security_protocol=SecurityConfig.PLAINTEXT,
-                 sasl_mechanism=SecurityConfig.SASL_MECHANISM_GSSAPI, authorizer_class_name=None, topics=None, version=TRUNK, quota_config=None, jmx_object_names=None,
+                 client_sasl_mechanism=SecurityConfig.SASL_MECHANISM_GSSAPI, interbroker_sasl_mechanism=SecurityConfig.SASL_MECHANISM_GSSAPI,
+                 authorizer_class_name=None, topics=None, version=TRUNK, quota_config=None, jmx_object_names=None,
                  jmx_attributes=[], zk_connect_timeout=5000):
         """
         :type context
@@ -78,7 +79,8 @@
 
         self.security_protocol = security_protocol
         self.interbroker_security_protocol = interbroker_security_protocol
-        self.sasl_mechanism = sasl_mechanism
+        self.client_sasl_mechanism = client_sasl_mechanism
+        self.interbroker_sasl_mechanism = interbroker_sasl_mechanism
         self.topics = topics
         self.minikdc = None
         self.authorizer_class_name = authorizer_class_name
@@ -108,7 +110,9 @@
 
     @property
     def security_config(self):
-        return SecurityConfig(self.security_protocol, self.interbroker_security_protocol, zk_sasl = self.zk.zk_sasl , sasl_mechanism=self.sasl_mechanism)
+        return SecurityConfig(self.security_protocol, self.interbroker_security_protocol,
+                              zk_sasl = self.zk.zk_sasl,
+                              client_sasl_mechanism=self.client_sasl_mechanism, interbroker_sasl_mechanism=self.interbroker_sasl_mechanism)
 
     def open_port(self, protocol):
         self.port_mappings[protocol] = self.port_mappings[protocol]._replace(open=True)
@@ -163,9 +167,7 @@
         # TODO - clean up duplicate configuration logic
         prop_file = cfg.render()
         prop_file += self.render('kafka.properties', node=node, broker_id=self.idx(node),
-                                 security_config=self.security_config,
-                                 interbroker_security_protocol=self.interbroker_security_protocol,
-                                 sasl_mechanism=self.sasl_mechanism)
+                                 security_config=self.security_config)
         return prop_file
 
     def start_cmd(self, node):
diff --git a/tests/kafkatest/services/kafka/templates/kafka.properties b/tests/kafkatest/services/kafka/templates/kafka.properties
index a718ee2..1e4f17c 100644
--- a/tests/kafkatest/services/kafka/templates/kafka.properties
+++ b/tests/kafkatest/services/kafka/templates/kafka.properties
@@ -50,7 +50,7 @@
 quota.consumer.bytes.per.second.overrides={{ quota_config.quota_consumer_bytes_per_second_overrides }}
 {% endif %}
 
-security.inter.broker.protocol={{ interbroker_security_protocol }}
+security.inter.broker.protocol={{ security_config.interbroker_security_protocol }}
 
 ssl.keystore.location=/mnt/security/test.keystore.jks
 ssl.keystore.password=test-ks-passwd
@@ -59,7 +59,8 @@
 ssl.truststore.location=/mnt/security/test.truststore.jks
 ssl.truststore.password=test-ts-passwd
 ssl.truststore.type=JKS
-sasl.mechanism={{ sasl_mechanism }}
+sasl.mechanism.inter.broker.protocol={{ security_config.interbroker_sasl_mechanism }}
+sasl.enabled.mechanisms={{ ",".join(security_config.enabled_sasl_mechanisms) }}
 sasl.kerberos.service.name=kafka
 {% if authorizer_class_name is not none %}
 ssl.client.auth=required
diff --git a/tests/kafkatest/services/performance/__init__.py b/tests/kafkatest/services/performance/__init__.py
index a72e3b7..9eddcaa 100644
--- a/tests/kafkatest/services/performance/__init__.py
+++ b/tests/kafkatest/services/performance/__init__.py
@@ -13,7 +13,7 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-from performance import PerformanceService
+from performance import PerformanceService, throughput, latency, compute_aggregate_throughput
 from end_to_end_latency import EndToEndLatencyService
 from producer_performance import ProducerPerformanceService
-from consumer_performance import ConsumerPerformanceService
\ No newline at end of file
+from consumer_performance import ConsumerPerformanceService
diff --git a/tests/kafkatest/services/performance/consumer_performance.py b/tests/kafkatest/services/performance/consumer_performance.py
index f8289bc..def27b1 100644
--- a/tests/kafkatest/services/performance/consumer_performance.py
+++ b/tests/kafkatest/services/performance/consumer_performance.py
@@ -14,8 +14,9 @@
 # limitations under the License.
 
 from kafkatest.services.performance import PerformanceService
-from kafkatest.services.kafka.directory import kafka_dir
 from kafkatest.services.security.security_config import SecurityConfig
+from kafkatest.services.kafka.directory import kafka_dir
+from kafkatest.services.kafka.version import TRUNK, V_0_9_0_0
 
 import os
 
@@ -69,7 +70,7 @@
             "collect_default": True}
     }
 
-    def __init__(self, context, num_nodes, kafka, topic, messages, new_consumer=False, settings={}):
+    def __init__(self, context, num_nodes, kafka, topic, messages, version=TRUNK, new_consumer=False, settings={}):
         super(ConsumerPerformanceService, self).__init__(context, num_nodes)
         self.kafka = kafka
         self.security_config = kafka.security_config.client_config()
@@ -78,6 +79,13 @@
         self.new_consumer = new_consumer
         self.settings = settings
 
+        assert version >= V_0_9_0_0 or (not new_consumer), \
+            "new_consumer is only supported if version >= 0.9.0.0, version %s" % str(version)
+
+        security_protocol = self.security_config.security_protocol
+        assert version >= V_0_9_0_0 or security_protocol == SecurityConfig.PLAINTEXT, \
+            "Security protocol %s is only supported if version >= 0.9.0.0, version %s" % (self.security_config, str(version))
+
         # These less-frequently used settings can be updated manually after instantiation
         self.fetch_size = None
         self.socket_buffer_size = None
@@ -86,6 +94,9 @@
         self.group = None
         self.from_latest = None
 
+        for node in self.nodes:
+            node.version = version
+
     @property
     def args(self):
         """Dictionary of arguments used to start the Consumer Performance script."""
@@ -127,7 +138,10 @@
         cmd += " /opt/%s/bin/kafka-consumer-perf-test.sh" % kafka_dir(node)
         for key, value in self.args.items():
             cmd += " --%s %s" % (key, value)
-        cmd += " --consumer.config %s" % ConsumerPerformanceService.CONFIG_FILE
+
+        if node.version >= V_0_9_0_0:
+            # This is only used for security settings
+            cmd += " --consumer.config %s" % ConsumerPerformanceService.CONFIG_FILE
 
         for key, value in self.settings.items():
             cmd += " %s=%s" % (str(key), str(value))
@@ -136,6 +150,22 @@
                                                         'stderr': ConsumerPerformanceService.STDERR_CAPTURE}
         return cmd
 
+    def parse_results(self, line, version):
+        parts = line.split(',')
+        if version >= V_0_9_0_0:
+            result = {
+                'total_mb': float(parts[2]),
+                'mbps': float(parts[3]),
+                'records_per_sec': float(parts[5]),
+            }
+        else:
+            result = {
+                'total_mb': float(parts[3]),
+                'mbps': float(parts[4]),
+                'records_per_sec': float(parts[6]),
+            }
+        return result
+
     def _worker(self, idx, node):
         node.account.ssh("mkdir -p %s" % ConsumerPerformanceService.PERSISTENT_ROOT, allow_fail=False)
 
@@ -149,11 +179,6 @@
         last = None
         for line in node.account.ssh_capture(cmd):
             last = line
-        # Parse and save the last line's information
-        parts = last.split(',')
 
-        self.results[idx-1] = {
-            'total_mb': float(parts[2]),
-            'mbps': float(parts[3]),
-            'records_per_sec': float(parts[5]),
-        }
+        # Parse and save the last line's information
+        self.results[idx-1] = self.parse_results(last, node.version)
diff --git a/tests/kafkatest/services/performance/end_to_end_latency.py b/tests/kafkatest/services/performance/end_to_end_latency.py
index 049eebc..6d21151 100644
--- a/tests/kafkatest/services/performance/end_to_end_latency.py
+++ b/tests/kafkatest/services/performance/end_to_end_latency.py
@@ -17,47 +17,97 @@
 from kafkatest.services.security.security_config import SecurityConfig
 
 from kafkatest.services.kafka.directory import kafka_dir
+from kafkatest.services.kafka.version import TRUNK, V_0_9_0_0, V_0_10_0_0
 
+import os
 
 class EndToEndLatencyService(PerformanceService):
+    MESSAGE_BYTES = 21  # 0.8.X messages are fixed at 21 bytes, so we'll match that for other versions
+
+    # Root directory for persistent output
+    PERSISTENT_ROOT = "/mnt/end_to_end_latency"
+    LOG_DIR = os.path.join(PERSISTENT_ROOT, "logs")
+    STDOUT_CAPTURE = os.path.join(PERSISTENT_ROOT, "end_to_end_latency.stdout")
+    STDERR_CAPTURE = os.path.join(PERSISTENT_ROOT, "end_to_end_latency.stderr")
+    LOG_FILE = os.path.join(LOG_DIR, "end_to_end_latency.log")
+    LOG4J_CONFIG = os.path.join(PERSISTENT_ROOT, "tools-log4j.properties")
+    CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "client.properties")
 
     logs = {
-        "end_to_end_latency_log": {
-            "path": "/mnt/end-to-end-latency.log",
+        "end_to_end_latency_output": {
+            "path": STDOUT_CAPTURE,
             "collect_default": True},
+        "end_to_end_latency_stderr": {
+            "path": STDERR_CAPTURE,
+            "collect_default": True},
+        "end_to_end_latency_log": {
+            "path": LOG_FILE,
+            "collect_default": True}
     }
 
-    def __init__(self, context, num_nodes, kafka, topic, num_records, consumer_fetch_max_wait=100, acks=1):
+
+    def __init__(self, context, num_nodes, kafka, topic, num_records, compression_type="none", version=TRUNK, acks=1):
         super(EndToEndLatencyService, self).__init__(context, num_nodes)
         self.kafka = kafka
         self.security_config = kafka.security_config.client_config()
+
+        security_protocol = self.security_config.security_protocol
+
+        if version < V_0_9_0_0:
+            assert security_protocol == SecurityConfig.PLAINTEXT, \
+                "Security protocol %s is only supported if version >= 0.9.0.0, version %s" % (self.security_config, str(version))
+            assert compression_type == "none", \
+                "Compression type %s is only supported if version >= 0.9.0.0, version %s" % (compression_type, str(version))
+
         self.args = {
             'topic': topic,
             'num_records': num_records,
-            'consumer_fetch_max_wait': consumer_fetch_max_wait,
             'acks': acks,
-            'kafka_opts': self.security_config.kafka_opts
+            'compression_type': compression_type,
+            'kafka_opts': self.security_config.kafka_opts,
+            'message_bytes': EndToEndLatencyService.MESSAGE_BYTES
         }
 
-    def _worker(self, idx, node):
+        for node in self.nodes:
+            node.version = version
+
+    def start_cmd(self, node):
         args = self.args.copy()
-        self.security_config.setup_node(node)
-        if self.security_config.security_protocol != SecurityConfig.PLAINTEXT:
-            security_config_file = SecurityConfig.CONFIG_DIR + "/security.properties"
-            node.account.create_file(security_config_file, str(self.security_config))
-        else:
-            security_config_file = ""
         args.update({
             'zk_connect': self.kafka.zk.connect_setting(),
             'bootstrap_servers': self.kafka.bootstrap_servers(self.security_config.security_protocol),
-            'security_config_file': security_config_file,
+            'config_file': EndToEndLatencyService.CONFIG_FILE,
             'kafka_dir': kafka_dir(node)
         })
 
-        cmd = "KAFKA_OPTS=%(kafka_opts)s /opt/%(kafka_dir)s/bin/kafka-run-class.sh kafka.tools.EndToEndLatency " % args
-        cmd += "%(bootstrap_servers)s %(topic)s %(num_records)d %(acks)d 20 %(security_config_file)s" % args
-        cmd += " | tee /mnt/end-to-end-latency.log"
+        cmd = "export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\"; " % EndToEndLatencyService.LOG4J_CONFIG
+        if node.version >= V_0_9_0_0:
+            cmd += "KAFKA_OPTS=%(kafka_opts)s /opt/%(kafka_dir)s/bin/kafka-run-class.sh kafka.tools.EndToEndLatency " % args
+            cmd += "%(bootstrap_servers)s %(topic)s %(num_records)d %(acks)d %(message_bytes)d %(config_file)s" % args
+        else:
+            # Set fetch max wait to 0 to match behavior in later versions
+            cmd += "KAFKA_OPTS=%(kafka_opts)s /opt/%(kafka_dir)s/bin/kafka-run-class.sh kafka.tools.TestEndToEndLatency " % args
+            cmd += "%(bootstrap_servers)s %(zk_connect)s %(topic)s %(num_records)d 0 %(acks)d" % args
 
+        cmd += " 2>> %(stderr)s | tee -a %(stdout)s" % {'stdout': EndToEndLatencyService.STDOUT_CAPTURE,
+                                                        'stderr': EndToEndLatencyService.STDERR_CAPTURE}
+
+        return cmd
+
+    def _worker(self, idx, node):
+        node.account.ssh("mkdir -p %s" % EndToEndLatencyService.PERSISTENT_ROOT, allow_fail=False)
+
+        log_config = self.render('tools_log4j.properties', log_file=EndToEndLatencyService.LOG_FILE)
+
+        node.account.create_file(EndToEndLatencyService.LOG4J_CONFIG, log_config)
+        client_config = str(self.security_config)
+        if node.version >= V_0_9_0_0:
+            client_config += "compression_type=%(compression_type)s" % self.args
+        node.account.create_file(EndToEndLatencyService.CONFIG_FILE, client_config)
+        
+        self.security_config.setup_node(node)
+
+        cmd = self.start_cmd(node)
         self.logger.debug("End-to-end latency %d command: %s", idx, cmd)
         results = {}
         for line in node.account.ssh_capture(cmd):
diff --git a/tests/kafkatest/services/performance/performance.py b/tests/kafkatest/services/performance/performance.py
index 6d286f6..1eab197 100644
--- a/tests/kafkatest/services/performance/performance.py
+++ b/tests/kafkatest/services/performance/performance.py
@@ -27,3 +27,26 @@
         node.account.kill_process("java", clean_shutdown=False, allow_fail=True)
         node.account.ssh("rm -rf /mnt/*", allow_fail=False)
 
+def throughput(records_per_sec, mb_per_sec):
+    """Helper method to ensure uniform representation of throughput data"""
+    return {
+        "records_per_sec": records_per_sec,
+        "mb_per_sec": mb_per_sec
+    }
+
+
+def latency(latency_50th_ms, latency_99th_ms, latency_999th_ms):
+    """Helper method to ensure uniform representation of latency data"""
+    return {
+        "latency_50th_ms": latency_50th_ms,
+        "latency_99th_ms": latency_99th_ms,
+        "latency_999th_ms": latency_999th_ms
+    }
+
+
+def compute_aggregate_throughput(perf):
+    """Helper method for computing throughput after running a performance service."""
+    aggregate_rate = sum([r['records_per_sec'] for r in perf.results])
+    aggregate_mbps = sum([r['mbps'] for r in perf.results])
+
+    return throughput(aggregate_rate, aggregate_mbps)
diff --git a/tests/kafkatest/services/performance/producer_performance.py b/tests/kafkatest/services/performance/producer_performance.py
index 7cbc7bb..f4887ed 100644
--- a/tests/kafkatest/services/performance/producer_performance.py
+++ b/tests/kafkatest/services/performance/producer_performance.py
@@ -13,26 +13,56 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+from ducktape.utils.util import wait_until
+
 from kafkatest.services.monitor.jmx import JmxMixin
 from kafkatest.services.performance import PerformanceService
-import itertools
 from kafkatest.services.security.security_config import SecurityConfig
-from kafkatest.services.kafka.directory import kafka_dir
+from kafkatest.services.kafka.directory import kafka_dir, KAFKA_TRUNK
+from kafkatest.services.kafka.version import TRUNK, V_0_9_0_0
+
+import os
+import subprocess
+
 
 class ProducerPerformanceService(JmxMixin, PerformanceService):
 
-    logs = {
-        "producer_performance_log": {
-            "path": "/mnt/producer-performance.log",
-            "collect_default": True},
-    }
+    PERSISTENT_ROOT = "/mnt/producer_performance"
+    STDOUT_CAPTURE = os.path.join(PERSISTENT_ROOT, "producer_performance.stdout")
+    STDERR_CAPTURE = os.path.join(PERSISTENT_ROOT, "producer_performance.stderr")
+    LOG_DIR = os.path.join(PERSISTENT_ROOT, "logs")
+    LOG_FILE = os.path.join(LOG_DIR, "producer_performance.log")
+    LOG4J_CONFIG = os.path.join(PERSISTENT_ROOT, "tools-log4j.properties")
 
-    def __init__(self, context, num_nodes, kafka, topic, num_records, record_size, throughput, settings={},
+    def __init__(self, context, num_nodes, kafka, topic, num_records, record_size, throughput, version=TRUNK, settings={},
                  intermediate_stats=False, client_id="producer-performance", jmx_object_names=None, jmx_attributes=[]):
         JmxMixin.__init__(self, num_nodes, jmx_object_names, jmx_attributes)
         PerformanceService.__init__(self, context, num_nodes)
+
+        self.logs = {
+            "producer_performance_stdout": {
+                "path": ProducerPerformanceService.STDOUT_CAPTURE,
+                "collect_default": True},
+            "producer_performance_stderr": {
+                "path": ProducerPerformanceService.STDERR_CAPTURE,
+                "collect_default": True},
+            "producer_performance_log": {
+                "path": ProducerPerformanceService.LOG_FILE,
+                "collect_default": True},
+            "jmx_log": {
+                "path": "/mnt/jmx_tool.log",
+                "collect_default": jmx_object_names is not None
+            }
+
+        }
+
         self.kafka = kafka
         self.security_config = kafka.security_config.client_config()
+
+        security_protocol = self.security_config.security_protocol
+        assert version >= V_0_9_0_0 or security_protocol == SecurityConfig.PLAINTEXT, \
+            "Security protocol %s is only supported if version >= 0.9.0.0, version %s" % (self.security_config, str(version))
+
         self.args = {
             'topic': topic,
             'kafka_opts': self.security_config.kafka_opts,
@@ -44,7 +74,10 @@
         self.intermediate_stats = intermediate_stats
         self.client_id = client_id
 
-    def _worker(self, idx, node):
+        for node in self.nodes:
+            node.version = version
+
+    def start_cmd(self, node):
         args = self.args.copy()
         args.update({
             'bootstrap_servers': self.kafka.bootstrap_servers(self.security_config.security_protocol),
@@ -52,48 +85,92 @@
             'client_id': self.client_id,
             'kafka_directory': kafka_dir(node)
             })
-        cmd = "JMX_PORT=%(jmx_port)d KAFKA_OPTS=%(kafka_opts)s /opt/%(kafka_directory)s/bin/kafka-run-class.sh org.apache.kafka.tools.ProducerPerformance " \
+
+        cmd = ""
+
+        if node.version < TRUNK:
+            # In order to ensure more consistent configuration between versions, always use the ProducerPerformance
+            # tool from trunk
+            cmd += "for file in /opt/%s/tools/build/libs/kafka-tools*.jar; do CLASSPATH=$CLASSPATH:$file; done; " % KAFKA_TRUNK
+            cmd += "for file in /opt/%s/tools/build/dependant-libs-${SCALA_VERSION}*/*.jar; do CLASSPATH=$CLASSPATH:$file; done; " % KAFKA_TRUNK
+            cmd += "export CLASSPATH; "
+
+        cmd += " export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\"; " % ProducerPerformanceService.LOG4J_CONFIG
+        cmd += "JMX_PORT=%(jmx_port)d KAFKA_OPTS=%(kafka_opts)s /opt/%(kafka_directory)s/bin/kafka-run-class.sh org.apache.kafka.tools.ProducerPerformance " \
               "--topic %(topic)s --num-records %(num_records)d --record-size %(record_size)d --throughput %(throughput)d --producer-props bootstrap.servers=%(bootstrap_servers)s client.id=%(client_id)s" % args
 
         self.security_config.setup_node(node)
         if self.security_config.security_protocol != SecurityConfig.PLAINTEXT:
             self.settings.update(self.security_config.properties)
+
         for key, value in self.settings.items():
             cmd += " %s=%s" % (str(key), str(value))
-        cmd += " | tee /mnt/producer-performance.log"
 
+        cmd += " 2>>%s | tee %s" % (ProducerPerformanceService.STDERR_CAPTURE, ProducerPerformanceService.STDOUT_CAPTURE)
+        return cmd
+
+    def pids(self, node):
+        try:
+            cmd = "jps | grep -i ProducerPerformance | awk '{print $1}'"
+            pid_arr = [pid for pid in node.account.ssh_capture(cmd, allow_fail=True, callback=int)]
+            return pid_arr
+        except (subprocess.CalledProcessError, ValueError) as e:
+            return []
+
+    def alive(self, node):
+        return len(self.pids(node)) > 0
+
+    def _worker(self, idx, node):
+
+        node.account.ssh("mkdir -p %s" % ProducerPerformanceService.PERSISTENT_ROOT, allow_fail=False)
+
+        # Create and upload log properties
+        log_config = self.render('tools_log4j.properties', log_file=ProducerPerformanceService.LOG_FILE)
+        node.account.create_file(ProducerPerformanceService.LOG4J_CONFIG, log_config)
+
+        cmd = self.start_cmd(node)
         self.logger.debug("Producer performance %d command: %s", idx, cmd)
 
-        def parse_stats(line):
-            parts = line.split(',')
-            return {
-                'records': int(parts[0].split()[0]),
-                'records_per_sec': float(parts[1].split()[0]),
-                'mbps': float(parts[1].split('(')[1].split()[0]),
-                'latency_avg_ms': float(parts[2].split()[0]),
-                'latency_max_ms': float(parts[3].split()[0]),
-                'latency_50th_ms': float(parts[4].split()[0]),
-                'latency_95th_ms': float(parts[5].split()[0]),
-                'latency_99th_ms': float(parts[6].split()[0]),
-                'latency_999th_ms': float(parts[7].split()[0]),
-            }
-        last = None
+        # start ProducerPerformance process
         producer_output = node.account.ssh_capture(cmd)
+        wait_until(lambda: self.alive(node), timeout_sec=20, err_msg="ProducerPerformance failed to start")
+        # block until there is at least one line of output
         first_line = next(producer_output, None)
+        if first_line is None:
+            raise Exception("No output from ProducerPerformance")
 
-        if first_line is not None:
-            self.start_jmx_tool(idx, node)
-            for line in itertools.chain([first_line], producer_output):
-                if self.intermediate_stats:
-                    try:
-                        self.stats[idx-1].append(parse_stats(line))
-                    except:
-                        # Sometimes there are extraneous log messages
-                        pass
+        self.start_jmx_tool(idx, node)
+        wait_until(lambda: not self.alive(node), timeout_sec=1200, err_msg="ProducerPerformance failed to finish")
+        self.read_jmx_output(idx, node)
 
-                last = line
-            try:
-                self.results[idx-1] = parse_stats(last)
-            except:
-                raise Exception("Unable to parse aggregate performance statistics on node %d: %s" % (idx, last))
-            self.read_jmx_output(idx, node)
+        # parse producer output from file
+        last = None
+        producer_output = node.account.ssh_capture("cat %s" % ProducerPerformanceService.STDOUT_CAPTURE)
+        for line in producer_output:
+            if self.intermediate_stats:
+                try:
+                    self.stats[idx-1].append(self.parse_stats(line))
+                except:
+                    # Sometimes there are extraneous log messages
+                    pass
+
+            last = line
+        try:
+            self.results[idx-1] = self.parse_stats(last)
+        except:
+            raise Exception("Unable to parse aggregate performance statistics on node %d: %s" % (idx, last))
+
+    def parse_stats(self, line):
+
+        parts = line.split(',')
+        return {
+            'records': int(parts[0].split()[0]),
+            'records_per_sec': float(parts[1].split()[0]),
+            'mbps': float(parts[1].split('(')[1].split()[0]),
+            'latency_avg_ms': float(parts[2].split()[0]),
+            'latency_max_ms': float(parts[3].split()[0]),
+            'latency_50th_ms': float(parts[4].split()[0]),
+            'latency_95th_ms': float(parts[5].split()[0]),
+            'latency_99th_ms': float(parts[6].split()[0]),
+            'latency_999th_ms': float(parts[7].split()[0]),
+        }
diff --git a/tests/kafkatest/services/replica_verification_tool.py b/tests/kafkatest/services/replica_verification_tool.py
new file mode 100644
index 0000000..f6374fb
--- /dev/null
+++ b/tests/kafkatest/services/replica_verification_tool.py
@@ -0,0 +1,81 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from ducktape.services.background_thread import BackgroundThreadService
+
+from kafkatest.services.kafka.directory import kafka_dir
+from kafkatest.services.security.security_config import SecurityConfig
+
+import re
+
+class ReplicaVerificationTool(BackgroundThreadService):
+
+    logs = {
+        "producer_log": {
+            "path": "/mnt/replica_verification_tool.log",
+            "collect_default": False}
+    }
+
+    def __init__(self, context, num_nodes, kafka, topic, report_interval_ms, security_protocol="PLAINTEXT"):
+        super(ReplicaVerificationTool, self).__init__(context, num_nodes)
+
+        self.kafka = kafka
+        self.topic = topic
+        self.report_interval_ms = report_interval_ms
+        self.security_protocol = security_protocol
+        self.security_config = SecurityConfig(security_protocol)
+        self.partition_lag = {}
+
+    def _worker(self, idx, node):
+        cmd = self.start_cmd(node)
+        self.logger.debug("ReplicaVerificationTool %d command: %s" % (idx, cmd))
+        self.security_config.setup_node(node)
+        for line in node.account.ssh_capture(cmd):
+            self.logger.debug("Parsing line:{}".format(line))
+
+            parsed = re.search('.*max lag is (.+?) for partition \[(.+?)\] at', line)
+            if parsed:
+                lag = int(parsed.group(1))
+                topic_partition = parsed.group(2)
+                self.logger.debug("Setting max lag for {} as {}".format(topic_partition, lag))
+                self.partition_lag[topic_partition] = lag
+
+    def get_lag_for_partition(self, topic, partition):
+        """
+        Get latest lag for given topic-partition
+
+        Args:
+            topic:          a topic
+            partition:      a partition of the topic
+        """
+        topic_partition = topic + ',' + str(partition)
+        lag = self.partition_lag[topic_partition]
+        self.logger.debug("Retuning lag for {} as {}".format(topic_partition, lag))
+        return lag
+
+    def start_cmd(self, node):
+        cmd = "/opt/%s/bin/" % kafka_dir(node)
+        cmd += "kafka-run-class.sh kafka.tools.ReplicaVerificationTool"
+        cmd += " --broker-list %s --topic-white-list %s --time -2 --report-interval-ms %s" % (self.kafka.bootstrap_servers(self.security_protocol), self.topic, self.report_interval_ms)
+
+        cmd += " 2>> /mnt/replica_verification_tool.log | tee -a /mnt/replica_verification_tool.log &"
+        return cmd
+
+    def stop_node(self, node):
+        node.account.kill_process("java", clean_shutdown=True, allow_fail=True)
+
+    def clean_node(self, node):
+        node.account.kill_process("java", clean_shutdown=False, allow_fail=True)
+        node.account.ssh("rm -rf /mnt/replica_verification_tool.log", allow_fail=False)
\ No newline at end of file
diff --git a/tests/kafkatest/services/security/minikdc.py b/tests/kafkatest/services/security/minikdc.py
index b376e26..d83aede 100644
--- a/tests/kafkatest/services/security/minikdc.py
+++ b/tests/kafkatest/services/security/minikdc.py
@@ -23,6 +23,7 @@
 from io import open
 import uuid
 
+
 class MiniKdc(Service):
 
     logs = {
@@ -54,9 +55,7 @@
         remove(file_path)
         move(abs_path, file_path)
 
-
     def start_node(self, node):
-
         node.account.ssh("mkdir -p %s" % MiniKdc.WORK_DIR, allow_fail=False)
         props_file = self.render('minikdc.properties',  node=node)
         node.account.create_file(MiniKdc.PROPS_FILE, props_file)
@@ -69,7 +68,7 @@
 
         jar_paths = self.core_jar_paths(node, "dependant-testlibs") + self.core_jar_paths(node, "libs")
         classpath = ":".join(jar_paths)
-        cmd = "CLASSPATH=%s /opt/%s/bin/kafka-run-class.sh kafka.security.minikdc.MiniKdc %s %s %s %s 1>> %s 2>> %s &" % (classpath, kafka_dir(node), MiniKdc.WORK_DIR, MiniKdc.PROPS_FILE, MiniKdc.KEYTAB_FILE, principals, MiniKdc.LOG_FILE, MiniKdc.LOG_FILE)
+        cmd = "INCLUDE_TEST_JARS=true CLASSPATH=%s /opt/%s/bin/kafka-run-class.sh kafka.security.minikdc.MiniKdc %s %s %s %s 1>> %s 2>> %s &" % (classpath, kafka_dir(node), MiniKdc.WORK_DIR, MiniKdc.PROPS_FILE, MiniKdc.KEYTAB_FILE, principals, MiniKdc.LOG_FILE, MiniKdc.LOG_FILE)
         self.logger.debug("Attempting to start MiniKdc on %s with command: %s" % (str(node.account), cmd))
         with node.account.monitor_log(MiniKdc.LOG_FILE) as monitor:
             node.account.ssh(cmd)
diff --git a/tests/kafkatest/services/security/security_config.py b/tests/kafkatest/services/security/security_config.py
index 1bbabd2..d7cc3c0 100644
--- a/tests/kafkatest/services/security/security_config.py
+++ b/tests/kafkatest/services/security/security_config.py
@@ -75,7 +75,9 @@
 
     ssl_stores = Keytool.generate_keystore_truststore('.')
 
-    def __init__(self, security_protocol=None, interbroker_security_protocol=None, sasl_mechanism=SASL_MECHANISM_GSSAPI, zk_sasl=False, template_props=""):
+    def __init__(self, security_protocol=None, interbroker_security_protocol=None,
+                 client_sasl_mechanism=SASL_MECHANISM_GSSAPI, interbroker_sasl_mechanism=SASL_MECHANISM_GSSAPI,
+                 zk_sasl=False, template_props=""):
         """
         Initialize the security properties for the node and copy
         keystore and truststore to the remote node if the transport protocol 
@@ -104,13 +106,14 @@
             'ssl.key.password' : SecurityConfig.ssl_stores['ssl.key.password'],
             'ssl.truststore.location' : SecurityConfig.TRUSTSTORE_PATH,
             'ssl.truststore.password' : SecurityConfig.ssl_stores['ssl.truststore.password'],
-            'sasl.mechanism' : sasl_mechanism,
+            'sasl.mechanism' : client_sasl_mechanism,
+            'sasl.mechanism.inter.broker.protocol' : interbroker_sasl_mechanism,
             'sasl.kerberos.service.name' : 'kafka'
         }
 
 
     def client_config(self, template_props=""):
-        return SecurityConfig(self.security_protocol, sasl_mechanism=self.sasl_mechanism, template_props=template_props)
+        return SecurityConfig(self.security_protocol, client_sasl_mechanism=self.client_sasl_mechanism, template_props=template_props)
 
     def setup_node(self, node):
         if self.has_ssl:
@@ -120,13 +123,15 @@
 
         if self.has_sasl:
             node.account.ssh("mkdir -p %s" % SecurityConfig.CONFIG_DIR, allow_fail=False)
-            jaas_conf_file = self.sasl_mechanism.lower() + "_jaas.conf"
+            jaas_conf_file = "jaas.conf"
             java_version = node.account.ssh_capture("java -version")
             if any('IBM' in line for line in java_version):
                 is_ibm_jdk = True
             else:
                 is_ibm_jdk = False
-            jaas_conf = self.render(jaas_conf_file,  node=node, is_ibm_jdk=is_ibm_jdk)
+            jaas_conf = self.render(jaas_conf_file,  node=node, is_ibm_jdk=is_ibm_jdk,
+                                    client_sasl_mechanism=self.client_sasl_mechanism,
+                                    enabled_sasl_mechanisms=self.enabled_sasl_mechanisms)
             node.account.create_file(SecurityConfig.JAAS_CONF_PATH, jaas_conf)
             if self.has_sasl_kerberos:
                 node.account.scp_to(MiniKdc.LOCAL_KEYTAB_FILE, SecurityConfig.KEYTAB_PATH)
@@ -159,12 +164,20 @@
         return self.properties['security.protocol']
 
     @property
-    def sasl_mechanism(self):
+    def client_sasl_mechanism(self):
         return self.properties['sasl.mechanism']
 
     @property
+    def interbroker_sasl_mechanism(self):
+        return self.properties['sasl.mechanism.inter.broker.protocol']
+
+    @property
+    def enabled_sasl_mechanisms(self):
+        return set([self.client_sasl_mechanism, self.interbroker_sasl_mechanism])
+
+    @property
     def has_sasl_kerberos(self):
-        return self.has_sasl and self.sasl_mechanism == SecurityConfig.SASL_MECHANISM_GSSAPI
+        return self.has_sasl and (SecurityConfig.SASL_MECHANISM_GSSAPI in self.enabled_sasl_mechanisms)
 
     @property
     def kafka_opts(self):
diff --git a/tests/kafkatest/services/security/templates/gssapi_jaas.conf b/tests/kafkatest/services/security/templates/jaas.conf
similarity index 61%
rename from tests/kafkatest/services/security/templates/gssapi_jaas.conf
rename to tests/kafkatest/services/security/templates/jaas.conf
index 6a629d9..fbfa8af 100644
--- a/tests/kafkatest/services/security/templates/gssapi_jaas.conf
+++ b/tests/kafkatest/services/security/templates/jaas.conf
@@ -11,76 +11,85 @@
   * specific language governing permissions and limitations under the License.
   */
 
-{% if is_ibm_jdk %}
 
 KafkaClient {
+{% if client_sasl_mechanism == "GSSAPI" %}
+{% if is_ibm_jdk %}
     com.ibm.security.auth.module.Krb5LoginModule required debug=false
     credsType=both
     useKeytab="file:/mnt/security/keytab"
     principal="client@EXAMPLE.COM";
+{% else %}
+    com.sun.security.auth.module.Krb5LoginModule required debug=false
+    doNotPrompt=true
+    useKeyTab=true
+    storeKey=true
+    keyTab="/mnt/security/keytab"
+    principal="client@EXAMPLE.COM";
+{% endif %}
+{% elif client_sasl_mechanism == "PLAIN" %}
+	org.apache.kafka.common.security.plain.PlainLoginModule required
+	username="client"
+	password="client-secret";
+{% endif %}
+
 };
 
 KafkaServer {
+{% if "GSSAPI" in enabled_sasl_mechanisms %}
+{% if is_ibm_jdk %}
     com.ibm.security.auth.module.Krb5LoginModule required debug=false
     credsType=both
     useKeytab="file:/mnt/security/keytab"
     principal="kafka/{{ node.account.hostname }}@EXAMPLE.COM";
+{% else %}
+    com.sun.security.auth.module.Krb5LoginModule required debug=false
+    doNotPrompt=true
+    useKeyTab=true
+    storeKey=true
+    keyTab="/mnt/security/keytab"
+    principal="kafka/{{ node.account.hostname }}@EXAMPLE.COM";
+{% endif %}
+{% endif %}
+{% if "PLAIN" in enabled_sasl_mechanisms %}
+	org.apache.kafka.common.security.plain.PlainLoginModule required
+	username="kafka"
+	password="kafka-secret"
+	user_client="client-secret"
+	user_kafka="kafka-secret";
+{% endif %}
 };
+
 {% if zk_sasl %}
 Client {
+{% if is_ibm_jdk %}
     com.ibm.security.auth.module.Krb5LoginModule required debug=false
     credsType=both
     useKeytab="file:/mnt/security/keytab"
     principal="zkclient@EXAMPLE.COM";
-};
-
-Server {
-       com.ibm.security.auth.module.Krb5LoginModule required debug=false
-       credsType=both
-       useKeyTab="file:/mnt/security/keytab"
-       principal="zookeeper/{{ node.account.hostname }}@EXAMPLE.COM";
-};
-{% endif %}
 {% else %}
-
-KafkaClient {
-    com.sun.security.auth.module.Krb5LoginModule required debug=false
-    doNotPrompt=true
-    useKeyTab=true
-    storeKey=true
-    keyTab="/mnt/security/keytab"
-    principal="client@EXAMPLE.COM";
-};
-
-KafkaServer {
-    com.sun.security.auth.module.Krb5LoginModule required debug=false
-    doNotPrompt=true
-    useKeyTab=true
-    storeKey=true
-    keyTab="/mnt/security/keytab"
-    principal="kafka/{{ node.account.hostname }}@EXAMPLE.COM";
-};
-
-{% if zk_sasl %}
-Client {
-       com.sun.security.auth.module.Krb5LoginModule required
-       useKeyTab=true
-       keyTab="/mnt/security/keytab"
-       storeKey=true
-       useTicketCache=false
-       principal="zkclient@EXAMPLE.COM";
+   com.sun.security.auth.module.Krb5LoginModule required
+   useKeyTab=true
+   keyTab="/mnt/security/keytab"
+   storeKey=true
+   useTicketCache=false
+   principal="zkclient@EXAMPLE.COM";
+{% endif %}
 };
 
 Server {
-       com.sun.security.auth.module.Krb5LoginModule required
-       useKeyTab=true
-       keyTab="/mnt/security/keytab"
-       storeKey=true
-       useTicketCache=false
-       principal="zookeeper/{{ node.account.hostname }}@EXAMPLE.COM";
+{% if is_ibm_jdk %}
+   com.ibm.security.auth.module.Krb5LoginModule required debug=false
+   credsType=both
+   useKeyTab="file:/mnt/security/keytab"
+   principal="zookeeper/{{ node.account.hostname }}@EXAMPLE.COM";
+{% else %}
+   com.sun.security.auth.module.Krb5LoginModule required
+   useKeyTab=true
+   keyTab="/mnt/security/keytab"
+   storeKey=true
+   useTicketCache=false
+   principal="zookeeper/{{ node.account.hostname }}@EXAMPLE.COM";
+{% endif %}
 };
 {% endif %}
-{% endif %}
-
-
-
diff --git a/tests/kafkatest/services/streams.py b/tests/kafkatest/services/streams.py
index dcbcc69..53d967e 100644
--- a/tests/kafkatest/services/streams.py
+++ b/tests/kafkatest/services/streams.py
@@ -15,10 +15,11 @@
 
 from ducktape.services.service import Service
 from ducktape.utils.util import wait_until
-from ducktape.errors import DucktapeError
 
 from kafkatest.services.kafka.directory import kafka_dir
-import signal, random, requests, os.path, json
+import signal
+import os.path
+
 
 class StreamsSmokeTestBaseService(Service):
     """Base class for Streams Smoke Test services providing some common settings and functionality"""
@@ -46,7 +47,7 @@
     def __init__(self, context, kafka, command):
         super(StreamsSmokeTestBaseService, self).__init__(context, 1)
         self.kafka = kafka
-        self.args = { 'command': command }
+        self.args = {'command': command}
 
     @property
     def node(self):
@@ -107,7 +108,7 @@
         args['kafka_dir'] = kafka_dir(node)
 
         cmd = "( export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%(log4j)s\"; " \
-              "/opt/%(kafka_dir)s/bin/kafka-run-class.sh org.apache.kafka.streams.smoketest.StreamsSmokeTest " \
+              "INCLUDE_TEST_JARS=true /opt/%(kafka_dir)s/bin/kafka-run-class.sh org.apache.kafka.streams.smoketest.StreamsSmokeTest " \
               " %(command)s %(kafka)s %(zk)s %(state_dir)s " \
               " & echo $! >&3 ) 1>> %(stdout)s 2>> %(stderr)s 3> %(pidfile)s" % args
 
@@ -131,6 +132,7 @@
     def __init__(self, context, kafka):
         super(StreamsSmokeTestDriverService, self).__init__(context, kafka, "run")
 
+
 class StreamsSmokeTestJobRunnerService(StreamsSmokeTestBaseService):
     def __init__(self, context, kafka):
         super(StreamsSmokeTestJobRunnerService, self).__init__(context, kafka, "process")
diff --git a/tests/kafkatest/services/templates/tools_log4j.properties b/tests/kafkatest/services/templates/tools_log4j.properties
index 6fec1d6..55ae4e0 100644
--- a/tests/kafkatest/services/templates/tools_log4j.properties
+++ b/tests/kafkatest/services/templates/tools_log4j.properties
@@ -22,4 +22,4 @@
 # Set the append to true
 log4j.appender.FILE.Append=true
 log4j.appender.FILE.layout=org.apache.log4j.PatternLayout
-log4j.appender.FILE.layout.conversionPattern=[%d] %p %m (%c)%n
\ No newline at end of file
+log4j.appender.FILE.layout.conversionPattern=[%d] %p %m (%c)%n
diff --git a/tests/kafkatest/services/verifiable_producer.py b/tests/kafkatest/services/verifiable_producer.py
index 414da84..4fec776 100644
--- a/tests/kafkatest/services/verifiable_producer.py
+++ b/tests/kafkatest/services/verifiable_producer.py
@@ -43,7 +43,7 @@
         }
 
     def __init__(self, context, num_nodes, kafka, topic, max_messages=-1, throughput=100000,
-                 message_validator=is_int, compression_types=None, version=TRUNK):
+                 message_validator=is_int, compression_types=None, version=TRUNK, acks=None):
         """
         :param max_messages is a number of messages to be produced per producer
         :param message_validator checks for an expected format of messages produced. There are
@@ -71,6 +71,8 @@
         self.acked_values = []
         self.not_acked_values = []
         self.produced_count = {}
+        self.clean_shutdown_nodes = set()
+        self.acks = acks
 
 
     @property
@@ -96,6 +98,9 @@
 
         # Create and upload config file
         producer_prop_file = self.prop_file(node)
+        if self.acks is not None:
+            self.logger.info("VerifiableProducer (index = %d) will use acks = %s", idx, self.acks)
+            producer_prop_file += "\nacks=%s\n" % self.acks
         self.logger.info("verifiable_producer.properties:")
         self.logger.info(producer_prop_file)
         node.account.create_file(VerifiableProducer.CONFIG_FILE, producer_prop_file)
@@ -135,6 +140,11 @@
                         last_produced_time = t
                         prev_msg = data
 
+                    elif data["name"] == "shutdown_complete":
+                        if node in self.clean_shutdown_nodes:
+                            raise Exception("Unexpected shutdown event from producer, already shutdown. Producer index: %d" % idx)
+                        self.clean_shutdown_nodes.add(node)
+
     def start_cmd(self, node, idx):
 
         cmd = ""
@@ -156,6 +166,8 @@
             cmd += " --throughput %s" % str(self.throughput)
         if self.message_validator == is_int_with_prefix:
             cmd += " --value-prefix %s" % str(idx)
+        if self.acks is not None:
+            cmd += " --acks %s\n" % str(self.acks)
 
         cmd += " --producer.config %s" % VerifiableProducer.CONFIG_FILE
         cmd += " 2>> %s | tee -a %s &" % (VerifiableProducer.STDOUT_CAPTURE, VerifiableProducer.STDOUT_CAPTURE)
diff --git a/tests/kafkatest/tests/core/replication_test.py b/tests/kafkatest/tests/core/replication_test.py
index 7b360ab..8e9474a 100644
--- a/tests/kafkatest/tests/core/replication_test.py
+++ b/tests/kafkatest/tests/core/replication_test.py
@@ -128,7 +128,10 @@
     @matrix(failure_mode=["clean_shutdown", "hard_shutdown", "clean_bounce", "hard_bounce"],
             broker_type=["controller"],
             security_protocol=["PLAINTEXT", "SASL_SSL"])
-    def test_replication_with_broker_failure(self, failure_mode, security_protocol, broker_type):
+    @matrix(failure_mode=["hard_bounce"],
+            broker_type=["leader"],
+            security_protocol=["SASL_SSL"], client_sasl_mechanism=["PLAIN"], interbroker_sasl_mechanism=["PLAIN", "GSSAPI"])
+    def test_replication_with_broker_failure(self, failure_mode, security_protocol, broker_type, client_sasl_mechanism="GSSAPI", interbroker_sasl_mechanism="GSSAPI"):
         """Replication tests.
         These tests verify that replication provides simple durability guarantees by checking that data acked by
         brokers is still available for consumption in the face of various failure scenarios.
@@ -144,6 +147,8 @@
 
         self.kafka.security_protocol = security_protocol
         self.kafka.interbroker_security_protocol = security_protocol
+        self.kafka.client_sasl_mechanism = client_sasl_mechanism
+        self.kafka.interbroker_sasl_mechanism = interbroker_sasl_mechanism
         new_consumer = False if  self.kafka.security_protocol == "PLAINTEXT" else True
         self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic, throughput=self.producer_throughput)
         self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic, new_consumer=new_consumer, consumer_timeout_ms=60000, message_validator=is_int)
diff --git a/tests/kafkatest/tests/tools/replica_verification_test.py b/tests/kafkatest/tests/tools/replica_verification_test.py
new file mode 100644
index 0000000..1b625e9
--- /dev/null
+++ b/tests/kafkatest/tests/tools/replica_verification_test.py
@@ -0,0 +1,88 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+
+from ducktape.utils.util import wait_until
+from ducktape.tests.test import Test
+from kafkatest.services.verifiable_producer import VerifiableProducer
+
+from kafkatest.services.zookeeper import ZookeeperService
+from kafkatest.services.kafka import KafkaService
+from kafkatest.services.replica_verification_tool import ReplicaVerificationTool
+
+TOPIC = "topic-replica-verification"
+REPORT_INTERVAL_MS = 1000
+
+class ReplicaVerificationToolTest(Test):
+    """
+    Tests ReplicaVerificationTool
+    """
+    def __init__(self, test_context):
+        super(ReplicaVerificationToolTest, self).__init__(test_context)
+        self.num_zk = 1
+        self.num_brokers = 2
+        self.messages_received_count = 0
+        self.topics = {
+            TOPIC: {'partitions': 1, 'replication-factor': 2}
+        }
+
+        self.zk = ZookeeperService(test_context, self.num_zk)
+        self.kafka = None
+        self.producer = None
+        self.replica_verifier = None
+
+    def setUp(self):
+        self.zk.start()
+
+    def start_kafka(self, security_protocol, interbroker_security_protocol):
+        self.kafka = KafkaService(
+            self.test_context, self.num_brokers,
+            self.zk, security_protocol=security_protocol,
+            interbroker_security_protocol=interbroker_security_protocol, topics=self.topics)
+        self.kafka.start()
+
+    def start_replica_verification_tool(self, security_protocol):
+        self.replica_verifier = ReplicaVerificationTool(self.test_context, 1, self.kafka, TOPIC, report_interval_ms=REPORT_INTERVAL_MS, security_protocol=security_protocol)
+        self.replica_verifier.start()
+
+    def start_producer(self, max_messages, acks, timeout):
+        # This will produce to kafka cluster
+        self.producer = VerifiableProducer(self.test_context, num_nodes=1, kafka=self.kafka, topic=TOPIC, throughput=1000, acks=acks, max_messages=max_messages)
+        current_acked = self.producer.num_acked
+        self.logger.info("current_acked = %s" % current_acked)
+        self.producer.start()
+        wait_until(lambda: acks == 0 or self.producer.num_acked >= current_acked + max_messages, timeout_sec=timeout,
+                   err_msg="Timeout awaiting messages to be produced and acked")
+
+    def stop_producer(self):
+        self.producer.stop()
+
+    def test_replica_lags(self, security_protocol='PLAINTEXT'):
+        """
+        Tests ReplicaVerificationTool
+        :return: None
+        """
+        self.start_kafka(security_protocol, security_protocol)
+        self.start_replica_verification_tool(security_protocol)
+        self.start_producer(max_messages=10, acks=-1, timeout=15)
+        # Verify that there is no lag in replicas and is correctly reported by ReplicaVerificationTool
+        wait_until(lambda: self.replica_verifier.get_lag_for_partition(TOPIC, 0) == 0, timeout_sec=10,
+                   err_msg="Timed out waiting to reach zero replica lags.")
+        self.stop_producer()
+
+        self.start_producer(max_messages=1000, acks=0, timeout=5)
+        # Verify that there is lag in replicas and is correctly reported by ReplicaVerificationTool
+        wait_until(lambda: self.replica_verifier.get_lag_for_partition(TOPIC, 0) > 0, timeout_sec=10,
+                   err_msg="Timed out waiting to reach non-zero number of replica lags.")
\ No newline at end of file
diff --git a/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java b/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java
index 18daf09..b83227f 100644
--- a/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java
+++ b/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java
@@ -17,6 +17,7 @@
 import java.util.Arrays;
 import java.util.List;
 import java.util.Properties;
+import java.util.Random;
 
 import org.apache.kafka.clients.producer.Callback;
 import org.apache.kafka.clients.producer.KafkaProducer;
@@ -59,8 +60,10 @@
 
             /* setup perf test */
             byte[] payload = new byte[recordSize];
-            Arrays.fill(payload, (byte) 1);
-            ProducerRecord<byte[], byte[]> record = new ProducerRecord<byte[], byte[]>(topicName, payload);
+            Random random = new Random(0);
+            for (int i = 0; i < payload.length; ++i)
+                payload[i] = (byte) (random.nextInt(26) + 65);
+            ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(topicName, payload);
             Stats stats = new Stats(numRecords, 5000);
             long startMs = System.currentTimeMillis();
 
diff --git a/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java b/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java
index 9b10a9f..b511fb9 100644
--- a/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java
+++ b/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java
@@ -247,6 +247,14 @@
     /** Close the producer to flush any remaining messages. */
     public void close() {
         producer.close();
+        System.out.println(shutdownString());
+    }
+
+    String shutdownString() {
+        Map<String, Object> data = new HashMap<>();
+        data.put("class", this.getClass().toString());
+        data.put("name", "shutdown_complete");
+        return toJsonString(data);
     }
 
     /**
diff --git a/vagrant/base.sh b/vagrant/base.sh
index d271f87..da7737c 100644
--- a/vagrant/base.sh
+++ b/vagrant/base.sh
@@ -63,7 +63,9 @@
 }
 
 get_kafka 0.8.2.2
+chmod a+rw /opt/kafka-0.8.2.2
 get_kafka 0.9.0.1
+chmod a+rw /opt/kafka-0.9.0.1
 
 # For EC2 nodes, we want to use /mnt, which should have the local disk. On local
 # VMs, we can just create it if it doesn't exist and use it like we'd use