You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by ij...@apache.org on 2017/09/19 04:13:22 UTC

[1/8] kafka git commit: MINOR: Move request/response schemas to the corresponding object representation

Repository: kafka
Updated Branches:
  refs/heads/trunk a64fe2ed8 -> 0cf770800


http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
----------------------------------------------------------------------
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 067fc27..cc65003 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
@@ -484,7 +484,7 @@ public class RequestResponseTest {
     @Test
     public void testControlledShutdownResponse() {
         ControlledShutdownResponse response = createControlledShutdownResponse();
-        short version = ApiKeys.CONTROLLED_SHUTDOWN_KEY.latestVersion();
+        short version = ApiKeys.CONTROLLED_SHUTDOWN.latestVersion();
         Struct struct = response.toStruct(version);
         ByteBuffer buffer = toBuffer(struct);
         ControlledShutdownResponse deserialized = ControlledShutdownResponse.parse(buffer, version);

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java
----------------------------------------------------------------------
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
index d41d61a..c59f2c9 100644
--- 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
@@ -60,6 +60,8 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
+import javax.security.auth.Subject;
+import javax.security.auth.login.Configuration;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
@@ -74,9 +76,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 
-import javax.security.auth.Subject;
-import javax.security.auth.login.Configuration;
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -953,7 +952,7 @@ public class SaslAuthenticatorTest {
 
                     @Override
                     protected ApiVersionsResponse apiVersionsResponse() {
-                        List<ApiVersion> apiVersions = new ArrayList<>(ApiVersionsResponse.API_VERSIONS_RESPONSE.apiVersions());
+                        List<ApiVersion> apiVersions = new ArrayList<>(ApiVersionsResponse.defaultApiVersionsResponse().apiVersions());
                         for (Iterator<ApiVersion> it = apiVersions.iterator(); it.hasNext(); ) {
                             ApiVersion apiVersion = it.next();
                             if (apiVersion.apiKey == ApiKeys.SASL_AUTHENTICATE.id) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
index 6976f7c..58e5543 100755
--- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
+++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
@@ -242,7 +242,7 @@ class RequestSendThread(val controllerId: Int,
       if (clientResponse != null) {
         val requestHeader = clientResponse.requestHeader
         val api = requestHeader.apiKey
-        if (api != ApiKeys.LEADER_AND_ISR && api != ApiKeys.STOP_REPLICA && api != ApiKeys.UPDATE_METADATA_KEY)
+        if (api != ApiKeys.LEADER_AND_ISR && api != ApiKeys.STOP_REPLICA && api != ApiKeys.UPDATE_METADATA)
           throw new KafkaException(s"Unexpected apiKey received: $apiKey")
 
         val response = clientResponse.responseBody
@@ -455,7 +455,7 @@ class ControllerBrokerRequestBatch(controller: KafkaController, stateChangeLogge
       }
 
       updateMetadataRequestBrokerSet.foreach { broker =>
-        controller.sendRequest(broker, ApiKeys.UPDATE_METADATA_KEY, updateMetadataRequest, null)
+        controller.sendRequest(broker, ApiKeys.UPDATE_METADATA, updateMetadataRequest, null)
       }
       updateMetadataRequestBrokerSet.clear()
       updateMetadataRequestPartitionInfoMap.clear()

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
index c2ebbad..aba0249 100644
--- a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
+++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
@@ -36,7 +36,7 @@ import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.internals.Topic
 import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.protocol.types.Type._
-import org.apache.kafka.common.protocol.types.{ArrayOf, Field, Schema, Struct}
+import org.apache.kafka.common.protocol.types._
 import org.apache.kafka.common.record._
 import org.apache.kafka.common.requests.{IsolationLevel, OffsetFetchResponse}
 import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala
index efd315b..2c7178e 100644
--- a/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala
+++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala
@@ -20,7 +20,7 @@ import kafka.common.{KafkaException, MessageFormatter}
 import org.apache.kafka.clients.consumer.ConsumerRecord
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.protocol.types.Type._
-import org.apache.kafka.common.protocol.types.{ArrayOf, Field, Schema, Struct}
+import org.apache.kafka.common.protocol.types._
 import java.io.PrintStream
 import java.nio.ByteBuffer
 import java.nio.charset.StandardCharsets

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/core/src/main/scala/kafka/log/ProducerStateManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/ProducerStateManager.scala b/core/src/main/scala/kafka/log/ProducerStateManager.scala
index 4c3d1a1..69d4e36 100644
--- a/core/src/main/scala/kafka/log/ProducerStateManager.scala
+++ b/core/src/main/scala/kafka/log/ProducerStateManager.scala
@@ -17,7 +17,7 @@
 package kafka.log
 
 import java.io._
-import java.nio.{BufferUnderflowException, ByteBuffer}
+import java.nio.ByteBuffer
 import java.nio.file.Files
 
 import kafka.common.KafkaException

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/core/src/main/scala/kafka/network/RequestChannel.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala
index 2d6370a..1128fd3 100644
--- a/core/src/main/scala/kafka/network/RequestChannel.scala
+++ b/core/src/main/scala/kafka/network/RequestChannel.scala
@@ -28,7 +28,7 @@ import kafka.server.QuotaId
 import kafka.utils.{Logging, NotNothing}
 import org.apache.kafka.common.memory.MemoryPool
 import org.apache.kafka.common.network.Send
-import org.apache.kafka.common.protocol.{ApiKeys, Protocol}
+import org.apache.kafka.common.protocol.ApiKeys
 import org.apache.kafka.common.requests._
 import org.apache.kafka.common.security.auth.KafkaPrincipal
 import org.apache.kafka.common.utils.Time
@@ -71,7 +71,7 @@ object RequestChannel extends Logging {
     //most request types are parsed entirely into objects at this point. for those we can release the underlying buffer.
     //some (like produce, or any time the schema contains fields of types BYTES or NULLABLE_BYTES) retain a reference
     //to the buffer. for those requests we cannot release the buffer early, but only when request processing is done.
-    if (!Protocol.requiresDelayedDeallocation(header.apiKey.id)) {
+    if (!header.apiKey.requiresDelayedAllocation) {
       releaseBuffer()
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/core/src/main/scala/kafka/server/KafkaApis.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala
index 62e8abf..e07e689 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -102,8 +102,8 @@ class KafkaApis(val requestChannel: RequestChannel,
         case ApiKeys.METADATA => handleTopicMetadataRequest(request)
         case ApiKeys.LEADER_AND_ISR => handleLeaderAndIsrRequest(request)
         case ApiKeys.STOP_REPLICA => handleStopReplicaRequest(request)
-        case ApiKeys.UPDATE_METADATA_KEY => handleUpdateMetadataRequest(request)
-        case ApiKeys.CONTROLLED_SHUTDOWN_KEY => handleControlledShutdownRequest(request)
+        case ApiKeys.UPDATE_METADATA => handleUpdateMetadataRequest(request)
+        case ApiKeys.CONTROLLED_SHUTDOWN => handleControlledShutdownRequest(request)
         case ApiKeys.OFFSET_COMMIT => handleOffsetCommitRequest(request)
         case ApiKeys.OFFSET_FETCH => handleOffsetFetchRequest(request)
         case ApiKeys.FIND_COORDINATOR => handleFindCoordinatorRequest(request)
@@ -1126,7 +1126,7 @@ class KafkaApis(val requestChannel: RequestChannel,
   def handleListGroupsRequest(request: RequestChannel.Request) {
     if (!authorize(request.session, Describe, Resource.ClusterResource)) {
       sendResponseMaybeThrottle(request, requestThrottleMs =>
-        ListGroupsResponse.fromError(requestThrottleMs, Errors.CLUSTER_AUTHORIZATION_FAILED))
+        request.body[ListGroupsRequest].getErrorResponse(requestThrottleMs, Errors.CLUSTER_AUTHORIZATION_FAILED.exception))
     } else {
       val (error, groups) = groupCoordinator.handleListGroups()
       val allGroups = groups.map { group => new ListGroupsResponse.Group(group.groupId, group.protocolType) }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
index dcd2038..7aeffb5 100644
--- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
@@ -117,7 +117,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
       ApiKeys.OFFSET_COMMIT -> classOf[requests.OffsetCommitResponse],
       ApiKeys.OFFSET_FETCH -> classOf[requests.OffsetFetchResponse],
       ApiKeys.FIND_COORDINATOR -> classOf[FindCoordinatorResponse],
-      ApiKeys.UPDATE_METADATA_KEY -> classOf[requests.UpdateMetadataResponse],
+      ApiKeys.UPDATE_METADATA -> classOf[requests.UpdateMetadataResponse],
       ApiKeys.JOIN_GROUP -> classOf[JoinGroupResponse],
       ApiKeys.SYNC_GROUP -> classOf[SyncGroupResponse],
       ApiKeys.DESCRIBE_GROUPS -> classOf[DescribeGroupsResponse],
@@ -125,7 +125,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
       ApiKeys.LEAVE_GROUP -> classOf[LeaveGroupResponse],
       ApiKeys.LEADER_AND_ISR -> classOf[requests.LeaderAndIsrResponse],
       ApiKeys.STOP_REPLICA -> classOf[requests.StopReplicaResponse],
-      ApiKeys.CONTROLLED_SHUTDOWN_KEY -> classOf[requests.ControlledShutdownResponse],
+      ApiKeys.CONTROLLED_SHUTDOWN -> classOf[requests.ControlledShutdownResponse],
       ApiKeys.CREATE_TOPICS -> classOf[CreateTopicsResponse],
       ApiKeys.DELETE_TOPICS -> classOf[requests.DeleteTopicsResponse],
       ApiKeys.OFFSET_FOR_LEADER_EPOCH -> classOf[OffsetsForLeaderEpochResponse],
@@ -152,7 +152,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
     ApiKeys.OFFSET_COMMIT -> ((resp: requests.OffsetCommitResponse) => resp.responseData.asScala.find(_._1 == tp).get._2),
     ApiKeys.OFFSET_FETCH -> ((resp: requests.OffsetFetchResponse) => resp.error),
     ApiKeys.FIND_COORDINATOR -> ((resp: FindCoordinatorResponse) => resp.error),
-    ApiKeys.UPDATE_METADATA_KEY -> ((resp: requests.UpdateMetadataResponse) => resp.error),
+    ApiKeys.UPDATE_METADATA -> ((resp: requests.UpdateMetadataResponse) => resp.error),
     ApiKeys.JOIN_GROUP -> ((resp: JoinGroupResponse) => resp.error),
     ApiKeys.SYNC_GROUP -> ((resp: SyncGroupResponse) => resp.error),
     ApiKeys.DESCRIBE_GROUPS -> ((resp: DescribeGroupsResponse) => resp.groups.get(group).error),
@@ -160,7 +160,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
     ApiKeys.LEAVE_GROUP -> ((resp: LeaveGroupResponse) => resp.error),
     ApiKeys.LEADER_AND_ISR -> ((resp: requests.LeaderAndIsrResponse) => resp.responses.asScala.find(_._1 == tp).get._2),
     ApiKeys.STOP_REPLICA -> ((resp: requests.StopReplicaResponse) => resp.responses.asScala.find(_._1 == tp).get._2),
-    ApiKeys.CONTROLLED_SHUTDOWN_KEY -> ((resp: requests.ControlledShutdownResponse) => resp.error),
+    ApiKeys.CONTROLLED_SHUTDOWN -> ((resp: requests.ControlledShutdownResponse) => resp.error),
     ApiKeys.CREATE_TOPICS -> ((resp: CreateTopicsResponse) => resp.errors.asScala.find(_._1 == createTopic).get._2.error),
     ApiKeys.DELETE_TOPICS -> ((resp: requests.DeleteTopicsResponse) => resp.errors.asScala.find(_._1 == deleteTopic).get._2),
     ApiKeys.OFFSET_FOR_LEADER_EPOCH -> ((resp: OffsetsForLeaderEpochResponse) => resp.responses.get(tp).error),
@@ -190,7 +190,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
     ApiKeys.OFFSET_COMMIT -> (topicReadAcl ++ groupReadAcl),
     ApiKeys.OFFSET_FETCH -> (topicReadAcl ++ groupDescribeAcl),
     ApiKeys.FIND_COORDINATOR -> (topicReadAcl ++ groupDescribeAcl ++ transactionalIdDescribeAcl),
-    ApiKeys.UPDATE_METADATA_KEY -> clusterAcl,
+    ApiKeys.UPDATE_METADATA -> clusterAcl,
     ApiKeys.JOIN_GROUP -> groupReadAcl,
     ApiKeys.SYNC_GROUP -> groupReadAcl,
     ApiKeys.DESCRIBE_GROUPS -> groupDescribeAcl,
@@ -198,7 +198,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
     ApiKeys.LEAVE_GROUP -> groupReadAcl,
     ApiKeys.LEADER_AND_ISR -> clusterAcl,
     ApiKeys.STOP_REPLICA -> clusterAcl,
-    ApiKeys.CONTROLLED_SHUTDOWN_KEY -> clusterAcl,
+    ApiKeys.CONTROLLED_SHUTDOWN -> clusterAcl,
     ApiKeys.CREATE_TOPICS -> clusterCreateAcl,
     ApiKeys.DELETE_TOPICS -> topicDeleteAcl,
     ApiKeys.OFFSET_FOR_LEADER_EPOCH -> clusterAcl,
@@ -293,7 +293,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
     val brokers = Set(new requests.UpdateMetadataRequest.Broker(brokerId,
       Seq(new requests.UpdateMetadataRequest.EndPoint("localhost", 0, securityProtocol,
         ListenerName.forSecurityProtocol(securityProtocol))).asJava, null)).asJava
-    val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion
+    val version = ApiKeys.UPDATE_METADATA.latestVersion
     new requests.UpdateMetadataRequest.Builder(version, brokerId, Int.MaxValue, partitionState, brokers).build()
   }
 
@@ -373,7 +373,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
       ApiKeys.LIST_OFFSETS -> createListOffsetsRequest,
       ApiKeys.OFFSET_FETCH -> createOffsetFetchRequest,
       ApiKeys.FIND_COORDINATOR -> createFindCoordinatorRequest,
-      ApiKeys.UPDATE_METADATA_KEY -> createUpdateMetadataRequest,
+      ApiKeys.UPDATE_METADATA -> createUpdateMetadataRequest,
       ApiKeys.JOIN_GROUP -> createJoinGroupRequest,
       ApiKeys.SYNC_GROUP -> createSyncGroupRequest,
       ApiKeys.DESCRIBE_GROUPS -> createDescribeGroupsRequest,
@@ -382,7 +382,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
       ApiKeys.LEAVE_GROUP -> leaveGroupRequest,
       ApiKeys.LEADER_AND_ISR -> leaderAndIsrRequest,
       ApiKeys.STOP_REPLICA -> stopReplicaRequest,
-      ApiKeys.CONTROLLED_SHUTDOWN_KEY -> controlledShutdownRequest,
+      ApiKeys.CONTROLLED_SHUTDOWN -> controlledShutdownRequest,
       ApiKeys.CREATE_TOPICS -> createTopicsRequest,
       ApiKeys.DELETE_TOPICS -> deleteTopicsRequest,
       ApiKeys.OFFSET_FOR_LEADER_EPOCH -> offsetsForLeaderEpochRequest,

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala
index c52020c..83f7111 100644
--- a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala
@@ -28,7 +28,7 @@ import scala.collection.JavaConverters._
 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.API_VERSIONS_RESPONSE.apiVersions.asScala) {
+    for (expectedApiVersion: ApiVersion <- ApiVersionsResponse.defaultApiVersionsResponse().apiVersions.asScala) {
       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)

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/core/src/test/scala/unit/kafka/server/ApiVersionsTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsTest.scala
index 262686a..3ba7cd5 100644
--- a/core/src/test/scala/unit/kafka/server/ApiVersionsTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ApiVersionsTest.scala
@@ -18,7 +18,7 @@
 package kafka.server
 
 import org.apache.kafka.common.requests.ApiVersionsResponse
-import org.apache.kafka.common.protocol.{ApiKeys, Protocol}
+import org.apache.kafka.common.protocol.ApiKeys
 import org.junit.Assert._
 import org.junit.Test
 
@@ -26,25 +26,25 @@ class ApiVersionsTest {
 
   @Test
   def testApiVersions(): Unit = {
-    val apiVersions = ApiVersionsResponse.API_VERSIONS_RESPONSE.apiVersions
+    val apiVersions = ApiVersionsResponse.defaultApiVersionsResponse().apiVersions
     assertEquals("API versions for all API keys must be maintained.", apiVersions.size, ApiKeys.values().length)
 
     for (key <- ApiKeys.values) {
-      val version = ApiVersionsResponse.API_VERSIONS_RESPONSE.apiVersion(key.id)
+      val version = ApiVersionsResponse.defaultApiVersionsResponse().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, key.oldestVersion)
       assertEquals(s"Incorrect max version for Api ${key.name}.", version.maxVersion, key.latestVersion)
 
       // 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))
+        assertNull(s"Request version $i for API ${version.apiKey} must be null.", key.requestSchemas(i))
+        assertNull(s"Response version $i for API ${version.apiKey} must be null.", key.responseSchemas(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))
+        assertNotNull(s"Request version $i for API ${version.apiKey} must not be null.", key.requestSchemas(i))
+        assertNotNull(s"Response version $i for API ${version.apiKey} must not be null.", key.responseSchemas(i))
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala
index de3ccdb..6b3c6c0 100644
--- a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala
+++ b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala
@@ -69,7 +69,7 @@ class MetadataCacheTest {
       new TopicPartition(topic0, 1) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, 1, 1, asList(1, 0), zkVersion, asList(1, 2, 0, 4), asList()),
       new TopicPartition(topic1, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, 2, 2, asList(2, 1), zkVersion, asList(2, 1, 3), asList()))
 
-    val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion
+    val version = ApiKeys.UPDATE_METADATA.latestVersion
     val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch,
       partitionStates.asJava, brokers.asJava).build()
     cache.updateCache(15, updateMetadataRequest)
@@ -127,7 +127,7 @@ class MetadataCacheTest {
     val partitionStates = Map(
       new TopicPartition(topic, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, leader, leaderEpoch, asList(0), zkVersion, asList(0), asList()))
 
-    val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion
+    val version = ApiKeys.UPDATE_METADATA.latestVersion
     val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch,
       partitionStates.asJava, brokers.asJava).build()
     cache.updateCache(15, updateMetadataRequest)
@@ -171,7 +171,7 @@ class MetadataCacheTest {
     val partitionStates = Map(
       new TopicPartition(topic, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas, asList()))
 
-    val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion
+    val version = ApiKeys.UPDATE_METADATA.latestVersion
     val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch,
       partitionStates.asJava, brokers.asJava).build()
     cache.updateCache(15, updateMetadataRequest)
@@ -231,7 +231,7 @@ class MetadataCacheTest {
     val partitionStates = Map(
       new TopicPartition(topic, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas, asList()))
 
-    val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion
+    val version = ApiKeys.UPDATE_METADATA.latestVersion
     val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch,
       partitionStates.asJava, brokers.asJava).build()
     cache.updateCache(15, updateMetadataRequest)
@@ -283,7 +283,7 @@ class MetadataCacheTest {
     val isr = asList[Integer](0, 1)
     val partitionStates = Map(
       new TopicPartition(topic, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas, asList()))
-    val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion
+    val version = ApiKeys.UPDATE_METADATA.latestVersion
     val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, 2, controllerEpoch, partitionStates.asJava,
       brokers.asJava).build()
     cache.updateCache(15, updateMetadataRequest)
@@ -316,7 +316,7 @@ class MetadataCacheTest {
       val isr = asList[Integer](0, 1)
       val partitionStates = Map(
         new TopicPartition(topic, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas, asList()))
-      val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion
+      val version = ApiKeys.UPDATE_METADATA.latestVersion
       val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, 2, controllerEpoch, partitionStates.asJava,
         brokers.asJava).build()
       cache.updateCache(15, updateMetadataRequest)

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
index bb9f82e..1a3b5c2 100644
--- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
+++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
@@ -187,16 +187,16 @@ class RequestQuotaTest extends BaseRequestTest {
         case ApiKeys.STOP_REPLICA =>
           new StopReplicaRequest.Builder(brokerId, Int.MaxValue, true, Set(tp).asJava)
 
-        case ApiKeys.UPDATE_METADATA_KEY =>
+        case ApiKeys.UPDATE_METADATA =>
           val partitionState = Map(tp -> new UpdateMetadataRequest.PartitionState(
             Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava, Seq.empty[Integer].asJava)).asJava
           val securityProtocol = SecurityProtocol.PLAINTEXT
           val brokers = Set(new UpdateMetadataRequest.Broker(brokerId,
             Seq(new UpdateMetadataRequest.EndPoint("localhost", 0, securityProtocol,
             ListenerName.forSecurityProtocol(securityProtocol))).asJava, null)).asJava
-          new UpdateMetadataRequest.Builder(ApiKeys.UPDATE_METADATA_KEY.latestVersion, brokerId, Int.MaxValue, partitionState, brokers)
+          new UpdateMetadataRequest.Builder(ApiKeys.UPDATE_METADATA.latestVersion, brokerId, Int.MaxValue, partitionState, brokers)
 
-        case ApiKeys.CONTROLLED_SHUTDOWN_KEY =>
+        case ApiKeys.CONTROLLED_SHUTDOWN =>
           new ControlledShutdownRequest.Builder(brokerId)
 
         case ApiKeys.OFFSET_COMMIT =>


[4/8] kafka git commit: MINOR: Move request/response schemas to the corresponding object representation

Posted by ij...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
index 39c027b..4a60c94 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
@@ -19,6 +19,9 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.record.MemoryRecords;
 
@@ -28,6 +31,12 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.INT64;
+import static org.apache.kafka.common.protocol.types.Type.INT8;
+
 public class FetchRequest extends AbstractRequest {
     public static final int CONSUMER_REPLICA_ID = -1;
     private static final String REPLICA_ID_KEY_NAME = "replica_id";
@@ -40,14 +49,101 @@ public class FetchRequest extends AbstractRequest {
     private static final String MAX_BYTES_KEY_NAME = "max_bytes";
 
     // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
     private static final String PARTITIONS_KEY_NAME = "partitions";
 
     // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
     private static final String FETCH_OFFSET_KEY_NAME = "fetch_offset";
     private static final String LOG_START_OFFSET_KEY_NAME = "log_start_offset";
 
+    private static final Schema FETCH_REQUEST_PARTITION_V0 = new Schema(
+            PARTITION_ID,
+            new Field(FETCH_OFFSET_KEY_NAME, INT64, "Message offset."),
+            new Field(MAX_BYTES_KEY_NAME, INT32, "Maximum bytes to fetch."));
+
+    // FETCH_REQUEST_PARTITION_V1 added log_start_offset field - the earliest available offset of partition data that can be consumed.
+    private static final Schema FETCH_REQUEST_PARTITION_V5 = new Schema(
+            PARTITION_ID,
+            new Field(FETCH_OFFSET_KEY_NAME, INT64, "Message offset."),
+            new Field(LOG_START_OFFSET_KEY_NAME, INT64, "Earliest available offset of the follower replica. " +
+                            "The field is only used when request is sent by follower. "),
+            new Field(MAX_BYTES_KEY_NAME, INT32, "Maximum bytes to fetch."));
+
+    private static final Schema FETCH_REQUEST_TOPIC_V0 = new Schema(
+            TOPIC_NAME,
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(FETCH_REQUEST_PARTITION_V0), "Partitions to fetch."));
+
+    private static final Schema FETCH_REQUEST_TOPIC_V5 = new Schema(
+            TOPIC_NAME,
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(FETCH_REQUEST_PARTITION_V5), "Partitions to fetch."));
+
+    private static final Schema FETCH_REQUEST_V0 = new Schema(
+            new Field(REPLICA_ID_KEY_NAME, INT32, "Broker id of the follower. For normal consumers, use -1."),
+            new Field(MAX_WAIT_KEY_NAME, INT32, "Maximum time in ms to wait for the response."),
+            new Field(MIN_BYTES_KEY_NAME, INT32, "Minimum bytes to accumulate in the response."),
+            new Field(TOPICS_KEY_NAME, new ArrayOf(FETCH_REQUEST_TOPIC_V0), "Topics to fetch."));
+
+    // The V1 Fetch Request body is the same as V0.
+    // Only the version number is incremented to indicate a newer client
+    private static final Schema FETCH_REQUEST_V1 = FETCH_REQUEST_V0;
+    // The V2 Fetch Request body is the same as V1.
+    // Only the version number is incremented to indicate the client support message format V1 which uses
+    // relative offset and has timestamp.
+    private static final Schema FETCH_REQUEST_V2 = FETCH_REQUEST_V1;
+    // Fetch Request V3 added top level max_bytes field - the total size of partition data to accumulate in response.
+    // The partition ordering is now relevant - partitions will be processed in order they appear in request.
+    private static final Schema FETCH_REQUEST_V3 = new Schema(
+            new Field(REPLICA_ID_KEY_NAME, INT32, "Broker id of the follower. For normal consumers, use -1."),
+            new Field(MAX_WAIT_KEY_NAME, INT32, "Maximum time in ms to wait for the response."),
+            new Field(MIN_BYTES_KEY_NAME, INT32, "Minimum bytes to accumulate in the response."),
+            new Field(MAX_BYTES_KEY_NAME, INT32, "Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, " +
+                    "if the first message in the first non-empty partition of the fetch is larger than this " +
+                    "value, the message will still be returned to ensure that progress can be made."),
+            new Field(TOPICS_KEY_NAME, new ArrayOf(FETCH_REQUEST_TOPIC_V0), "Topics to fetch in the order provided."));
+
+    // The V4 Fetch Request adds the fetch isolation level and exposes magic v2 (via the response).
+    private static final Schema FETCH_REQUEST_V4 = new Schema(
+            new Field(REPLICA_ID_KEY_NAME, INT32, "Broker id of the follower. For normal consumers, use -1."),
+            new Field(MAX_WAIT_KEY_NAME, INT32, "Maximum time in ms to wait for the response."),
+            new Field(MIN_BYTES_KEY_NAME, INT32, "Minimum bytes to accumulate in the response."),
+            new Field(MAX_BYTES_KEY_NAME, INT32, "Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, " +
+                    "if the first message in the first non-empty partition of the fetch is larger than this " +
+                    "value, the message will still be returned to ensure that progress can be made."),
+            new Field(ISOLATION_LEVEL_KEY_NAME, INT8, "This setting controls the visibility of transactional records. Using READ_UNCOMMITTED " +
+                    "(isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), " +
+                    "non-transactional and COMMITTED transactional records are visible. To be more concrete, " +
+                    "READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), " +
+                    "and enables the inclusion of the list of aborted transactions in the result, which allows " +
+                    "consumers to discard ABORTED transactional records"),
+            new Field(TOPICS_KEY_NAME, new ArrayOf(FETCH_REQUEST_TOPIC_V0), "Topics to fetch in the order provided."));
+
+    // FETCH_REQUEST_V5 added a per-partition log_start_offset field - the earliest available offset of partition data that can be consumed.
+    private static final Schema FETCH_REQUEST_V5 = new Schema(
+            new Field(REPLICA_ID_KEY_NAME, INT32, "Broker id of the follower. For normal consumers, use -1."),
+            new Field(MAX_WAIT_KEY_NAME, INT32, "Maximum time in ms to wait for the response."),
+            new Field(MIN_BYTES_KEY_NAME, INT32, "Minimum bytes to accumulate in the response."),
+            new Field(MAX_BYTES_KEY_NAME, INT32, "Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, " +
+                    "if the first message in the first non-empty partition of the fetch is larger than this " +
+                    "value, the message will still be returned to ensure that progress can be made."),
+            new Field(ISOLATION_LEVEL_KEY_NAME, INT8, "This setting controls the visibility of transactional records. Using READ_UNCOMMITTED " +
+                    "(isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), " +
+                    "non-transactional and COMMITTED transactional records are visible. To be more concrete, " +
+                    "READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), " +
+                    "and enables the inclusion of the list of aborted transactions in the result, which allows " +
+                    "consumers to discard ABORTED transactional records"),
+            new Field(TOPICS_KEY_NAME, new ArrayOf(FETCH_REQUEST_TOPIC_V5), "Topics to fetch in the order provided."));
+
+    /**
+     * The body of FETCH_REQUEST_V6 is the same as FETCH_REQUEST_V5.
+     * The version number is bumped up to indicate that the client supports KafkaStorageException.
+     * The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 5
+     */
+    private static final Schema FETCH_REQUEST_V6 = FETCH_REQUEST_V5;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{FETCH_REQUEST_V0, FETCH_REQUEST_V1, FETCH_REQUEST_V2, FETCH_REQUEST_V3, FETCH_REQUEST_V4,
+            FETCH_REQUEST_V5, FETCH_REQUEST_V6};
+    };
+
     // default values for older versions where a request level limit did not exist
     public static final int DEFAULT_RESPONSE_MAX_BYTES = Integer.MAX_VALUE;
     public static final long INVALID_LOG_START_OFFSET = -1L;
@@ -191,10 +287,10 @@ public class FetchRequest extends AbstractRequest {
         fetchData = new LinkedHashMap<>();
         for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
             Struct topicResponse = (Struct) topicResponseObj;
-            String topic = topicResponse.getString(TOPIC_KEY_NAME);
+            String topic = topicResponse.get(TOPIC_NAME);
             for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
                 Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                int partition = partitionResponse.get(PARTITION_ID);
                 long offset = partitionResponse.getLong(FETCH_OFFSET_KEY_NAME);
                 int maxBytes = partitionResponse.getInt(MAX_BYTES_KEY_NAME);
                 long logStartOffset = partitionResponse.hasField(LOG_START_OFFSET_KEY_NAME) ?
@@ -266,12 +362,12 @@ public class FetchRequest extends AbstractRequest {
         List<Struct> topicArray = new ArrayList<>();
         for (TopicAndPartitionData<PartitionData> topicEntry : topicsData) {
             Struct topicData = struct.instance(TOPICS_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, topicEntry.topic);
+            topicData.set(TOPIC_NAME, topicEntry.topic);
             List<Struct> partitionArray = new ArrayList<>();
             for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.partitions.entrySet()) {
                 PartitionData fetchPartitionData = partitionEntry.getValue();
                 Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
+                partitionData.set(PARTITION_ID, partitionEntry.getKey());
                 partitionData.set(FETCH_OFFSET_KEY_NAME, fetchPartitionData.fetchOffset);
                 if (partitionData.hasField(LOG_START_OFFSET_KEY_NAME))
                     partitionData.set(LOG_START_OFFSET_KEY_NAME, fetchPartitionData.logStartOffset);

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java
index 281ad44..417e845 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java
@@ -22,9 +22,10 @@ import org.apache.kafka.common.network.MultiSend;
 import org.apache.kafka.common.network.Send;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
-
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
-import org.apache.kafka.common.protocol.types.Type;
 import org.apache.kafka.common.record.Records;
 
 import java.nio.ByteBuffer;
@@ -33,6 +34,14 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.INT64;
+import static org.apache.kafka.common.protocol.types.Type.RECORDS;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 /**
  * This wrapper supports all versions of the Fetch API
  */
@@ -41,13 +50,10 @@ public class FetchResponse extends AbstractResponse {
     private static final String RESPONSES_KEY_NAME = "responses";
 
     // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
     private static final String PARTITIONS_KEY_NAME = "partition_responses";
 
     // partition level field names
     private static final String PARTITION_HEADER_KEY_NAME = "partition_header";
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
     private static final String HIGH_WATERMARK_KEY_NAME = "high_watermark";
     private static final String LAST_STABLE_OFFSET_KEY_NAME = "last_stable_offset";
     private static final String LOG_START_OFFSET_KEY_NAME = "log_start_offset";
@@ -58,7 +64,93 @@ public class FetchResponse extends AbstractResponse {
     private static final String PRODUCER_ID_KEY_NAME = "producer_id";
     private static final String FIRST_OFFSET_KEY_NAME = "first_offset";
 
-    private static final int DEFAULT_THROTTLE_TIME = 0;
+    private static final Schema FETCH_RESPONSE_PARTITION_HEADER_V0 = new Schema(
+            PARTITION_ID,
+            ERROR_CODE,
+            new Field(HIGH_WATERMARK_KEY_NAME, INT64, "Last committed offset."));
+    private static final Schema FETCH_RESPONSE_PARTITION_V0 = new Schema(
+            new Field(PARTITION_HEADER_KEY_NAME, FETCH_RESPONSE_PARTITION_HEADER_V0),
+            new Field(RECORD_SET_KEY_NAME, RECORDS));
+
+    private static final Schema FETCH_RESPONSE_TOPIC_V0 = new Schema(
+            TOPIC_NAME,
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(FETCH_RESPONSE_PARTITION_V0)));
+
+    private static final Schema FETCH_RESPONSE_V0 = new Schema(
+            new Field(RESPONSES_KEY_NAME, new ArrayOf(FETCH_RESPONSE_TOPIC_V0)));
+
+    private static final Schema FETCH_RESPONSE_V1 = new Schema(
+            THROTTLE_TIME_MS,
+            new Field(RESPONSES_KEY_NAME, new ArrayOf(FETCH_RESPONSE_TOPIC_V0)));
+    // Even though fetch response v2 has the same protocol as v1, the record set in the response is different. In v1,
+    // record set only includes messages of v0 (magic byte 0). In v2, record set can include messages of v0 and v1
+    // (magic byte 0 and 1). For details, see ByteBufferMessageSet.
+    private static final Schema FETCH_RESPONSE_V2 = FETCH_RESPONSE_V1;
+    private static final Schema FETCH_RESPONSE_V3 = FETCH_RESPONSE_V2;
+
+    // The v4 Fetch Response adds features for transactional consumption (the aborted transaction list and the
+    // last stable offset). It also exposes messages with magic v2 (along with older formats).
+    private static final Schema FETCH_RESPONSE_ABORTED_TRANSACTION_V4 = new Schema(
+            new Field(PRODUCER_ID_KEY_NAME, INT64, "The producer id associated with the aborted transactions"),
+            new Field(FIRST_OFFSET_KEY_NAME, INT64, "The first offset in the aborted transaction"));
+
+    private static final Schema FETCH_RESPONSE_ABORTED_TRANSACTION_V5 = FETCH_RESPONSE_ABORTED_TRANSACTION_V4;
+
+    private static final Schema FETCH_RESPONSE_PARTITION_HEADER_V4 = new Schema(
+            PARTITION_ID,
+            ERROR_CODE,
+            new Field(HIGH_WATERMARK_KEY_NAME, INT64, "Last committed offset."),
+            new Field(LAST_STABLE_OFFSET_KEY_NAME, INT64, "The last stable offset (or LSO) of the partition. This is the last offset such that the state " +
+                    "of all transactional records prior to this offset have been decided (ABORTED or COMMITTED)"),
+            new Field(ABORTED_TRANSACTIONS_KEY_NAME, ArrayOf.nullable(FETCH_RESPONSE_ABORTED_TRANSACTION_V4)));
+
+    // FETCH_RESPONSE_PARTITION_HEADER_V5 added log_start_offset field - the earliest available offset of partition data that can be consumed.
+    private static final Schema FETCH_RESPONSE_PARTITION_HEADER_V5 = new Schema(
+            PARTITION_ID,
+            ERROR_CODE,
+            new Field(HIGH_WATERMARK_KEY_NAME, INT64, "Last committed offset."),
+            new Field(LAST_STABLE_OFFSET_KEY_NAME, INT64, "The last stable offset (or LSO) of the partition. This is the last offset such that the state " +
+                    "of all transactional records prior to this offset have been decided (ABORTED or COMMITTED)"),
+            new Field(LOG_START_OFFSET_KEY_NAME, INT64, "Earliest available offset."),
+            new Field(ABORTED_TRANSACTIONS_KEY_NAME, ArrayOf.nullable(FETCH_RESPONSE_ABORTED_TRANSACTION_V5)));
+
+    private static final Schema FETCH_RESPONSE_PARTITION_V4 = new Schema(
+            new Field(PARTITION_HEADER_KEY_NAME, FETCH_RESPONSE_PARTITION_HEADER_V4),
+            new Field(RECORD_SET_KEY_NAME, RECORDS));
+
+    private static final Schema FETCH_RESPONSE_PARTITION_V5 = new Schema(
+            new Field(PARTITION_HEADER_KEY_NAME, FETCH_RESPONSE_PARTITION_HEADER_V5),
+            new Field(RECORD_SET_KEY_NAME, RECORDS));
+
+    private static final Schema FETCH_RESPONSE_TOPIC_V4 = new Schema(
+            TOPIC_NAME,
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(FETCH_RESPONSE_PARTITION_V4)));
+
+    private static final Schema FETCH_RESPONSE_TOPIC_V5 = new Schema(
+            TOPIC_NAME,
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(FETCH_RESPONSE_PARTITION_V5)));
+
+    private static final Schema FETCH_RESPONSE_V4 = new Schema(
+            THROTTLE_TIME_MS,
+            new Field(RESPONSES_KEY_NAME, new ArrayOf(FETCH_RESPONSE_TOPIC_V4)));
+
+    private static final Schema FETCH_RESPONSE_V5 = new Schema(
+            THROTTLE_TIME_MS,
+            new Field(RESPONSES_KEY_NAME, new ArrayOf(FETCH_RESPONSE_TOPIC_V5)));
+
+    /**
+     * The body of FETCH_RESPONSE_V6 is the same as FETCH_RESPONSE_V5.
+     * The version number is bumped up to indicate that the client supports KafkaStorageException.
+     * The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 5
+     */
+    private static final Schema FETCH_RESPONSE_V6 = FETCH_RESPONSE_V5;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {FETCH_RESPONSE_V0, FETCH_RESPONSE_V1, FETCH_RESPONSE_V2,
+            FETCH_RESPONSE_V3, FETCH_RESPONSE_V4, FETCH_RESPONSE_V5, FETCH_RESPONSE_V6};
+    }
+
+
     public static final long INVALID_HIGHWATERMARK = -1L;
     public static final long INVALID_LAST_STABLE_OFFSET = -1L;
     public static final long INVALID_LOG_START_OFFSET = -1L;
@@ -190,12 +282,12 @@ public class FetchResponse extends AbstractResponse {
         LinkedHashMap<TopicPartition, PartitionData> responseData = new LinkedHashMap<>();
         for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
             Struct topicResponse = (Struct) topicResponseObj;
-            String topic = topicResponse.getString(TOPIC_KEY_NAME);
+            String topic = topicResponse.get(TOPIC_NAME);
             for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
                 Struct partitionResponse = (Struct) partitionResponseObj;
                 Struct partitionResponseHeader = partitionResponse.getStruct(PARTITION_HEADER_KEY_NAME);
-                int partition = partitionResponseHeader.getInt(PARTITION_KEY_NAME);
-                Errors error = Errors.forCode(partitionResponseHeader.getShort(ERROR_CODE_KEY_NAME));
+                int partition = partitionResponseHeader.get(PARTITION_ID);
+                Errors error = Errors.forCode(partitionResponseHeader.get(ERROR_CODE));
                 long highWatermark = partitionResponseHeader.getLong(HIGH_WATERMARK_KEY_NAME);
                 long lastStableOffset = INVALID_LAST_STABLE_OFFSET;
                 if (partitionResponseHeader.hasField(LAST_STABLE_OFFSET_KEY_NAME))
@@ -226,7 +318,7 @@ public class FetchResponse extends AbstractResponse {
             }
         }
         this.responseData = responseData;
-        this.throttleTimeMs = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME;
+        this.throttleTimeMs = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME);
     }
 
     @Override
@@ -266,7 +358,7 @@ public class FetchResponse extends AbstractResponse {
     private static void addResponseData(Struct struct, int throttleTimeMs, String dest, List<Send> sends) {
         Object[] allTopicData = struct.getArray(RESPONSES_KEY_NAME);
 
-        if (struct.hasField(THROTTLE_TIME_KEY_NAME)) {
+        if (struct.hasField(THROTTLE_TIME_MS)) {
             ByteBuffer buffer = ByteBuffer.allocate(8);
             buffer.putInt(throttleTimeMs);
             buffer.putInt(allTopicData.length);
@@ -284,12 +376,12 @@ public class FetchResponse extends AbstractResponse {
     }
 
     private static void addTopicData(String dest, List<Send> sends, Struct topicData) {
-        String topic = topicData.getString(TOPIC_KEY_NAME);
+        String topic = topicData.get(TOPIC_NAME);
         Object[] allPartitionData = topicData.getArray(PARTITIONS_KEY_NAME);
 
         // include the topic header and the count for the number of partitions
-        ByteBuffer buffer = ByteBuffer.allocate(Type.STRING.sizeOf(topic) + 4);
-        Type.STRING.write(buffer, topic);
+        ByteBuffer buffer = ByteBuffer.allocate(STRING.sizeOf(topic) + 4);
+        STRING.write(buffer, topic);
         buffer.putInt(allPartitionData.length);
         buffer.rewind();
         sends.add(new ByteBufferSend(dest, buffer));
@@ -313,13 +405,13 @@ public class FetchResponse extends AbstractResponse {
         sends.add(new RecordsSend(dest, records));
     }
 
-    private static Struct toStruct(short version, LinkedHashMap<TopicPartition, PartitionData> responseData, int throttleTime) {
+    private static Struct toStruct(short version, LinkedHashMap<TopicPartition, PartitionData> responseData, int throttleTimeMs) {
         Struct struct = new Struct(ApiKeys.FETCH.responseSchema(version));
         List<FetchRequest.TopicAndPartitionData<PartitionData>> topicsData = FetchRequest.TopicAndPartitionData.batchByTopic(responseData);
         List<Struct> topicArray = new ArrayList<>();
         for (FetchRequest.TopicAndPartitionData<PartitionData> topicEntry: topicsData) {
             Struct topicData = struct.instance(RESPONSES_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, topicEntry.topic);
+            topicData.set(TOPIC_NAME, topicEntry.topic);
             List<Struct> partitionArray = new ArrayList<>();
             for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.partitions.entrySet()) {
                 PartitionData fetchPartitionData = partitionEntry.getValue();
@@ -332,8 +424,8 @@ public class FetchResponse extends AbstractResponse {
                     errorCode = Errors.NOT_LEADER_FOR_PARTITION.code();
                 Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
                 Struct partitionDataHeader = partitionData.instance(PARTITION_HEADER_KEY_NAME);
-                partitionDataHeader.set(PARTITION_KEY_NAME, partitionEntry.getKey());
-                partitionDataHeader.set(ERROR_CODE_KEY_NAME, errorCode);
+                partitionDataHeader.set(PARTITION_ID, partitionEntry.getKey());
+                partitionDataHeader.set(ERROR_CODE, errorCode);
                 partitionDataHeader.set(HIGH_WATERMARK_KEY_NAME, fetchPartitionData.highWatermark);
 
                 if (partitionDataHeader.hasField(LAST_STABLE_OFFSET_KEY_NAME)) {
@@ -363,9 +455,7 @@ public class FetchResponse extends AbstractResponse {
             topicArray.add(topicData);
         }
         struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
-
-        if (struct.hasField(THROTTLE_TIME_KEY_NAME))
-            struct.set(THROTTLE_TIME_KEY_NAME, throttleTime);
+        struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs);
 
         return struct;
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorRequest.java
index fbc7fa2..c94bcde 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorRequest.java
@@ -20,15 +20,32 @@ import org.apache.kafka.common.Node;
 import org.apache.kafka.common.errors.UnsupportedVersionException;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
 
+import static org.apache.kafka.common.protocol.types.Type.INT8;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class FindCoordinatorRequest extends AbstractRequest {
     private static final String GROUP_ID_KEY_NAME = "group_id";
     private static final String COORDINATOR_KEY_KEY_NAME = "coordinator_key";
     private static final String COORDINATOR_TYPE_KEY_NAME = "coordinator_type";
 
+    private static final Schema FIND_COORDINATOR_REQUEST_V0 = new Schema(
+            new Field("group_id", STRING, "The unique group id."));
+
+    private static final Schema FIND_COORDINATOR_REQUEST_V1 = new Schema(
+            new Field("coordinator_key", STRING, "Id to use for finding the coordinator (for groups, this is the groupId, " +
+                            "for transactional producers, this is the transactional id)"),
+            new Field("coordinator_type", INT8, "The type of coordinator to find (0 = group, 1 = transaction)"));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {FIND_COORDINATOR_REQUEST_V0, FIND_COORDINATOR_REQUEST_V1};
+    }
+
     public static class Builder extends AbstractRequest.Builder<FindCoordinatorRequest> {
         private final String coordinatorKey;
         private final CoordinatorType coordinatorType;

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java
index ae6986a..f5d9805 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java
@@ -19,16 +19,46 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.Node;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
 
-public class FindCoordinatorResponse extends AbstractResponse {
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_MESSAGE;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
 
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
-    private static final String ERROR_MESSAGE_KEY_NAME = "error_message";
+public class FindCoordinatorResponse extends AbstractResponse {
     private static final String COORDINATOR_KEY_NAME = "coordinator";
 
+    // coordinator level field names
+    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 Schema FIND_COORDINATOR_BROKER_V0 = new Schema(
+            new Field(NODE_ID_KEY_NAME, INT32, "The broker id."),
+            new Field(HOST_KEY_NAME, STRING, "The hostname of the broker."),
+            new Field(PORT_KEY_NAME, INT32, "The port on which the broker accepts requests."));
+
+    private static final Schema FIND_COORDINATOR_RESPONSE_V0 = new Schema(
+            ERROR_CODE,
+            new Field(COORDINATOR_KEY_NAME, FIND_COORDINATOR_BROKER_V0, "Host and port information for the coordinator " +
+                    "for a consumer group."));
+
+    private static final Schema FIND_COORDINATOR_RESPONSE_V1 = new Schema(
+            THROTTLE_TIME_MS,
+            ERROR_CODE,
+            ERROR_MESSAGE,
+            new Field(COORDINATOR_KEY_NAME, FIND_COORDINATOR_BROKER_V0, "Host and port information for the coordinator"));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {FIND_COORDINATOR_RESPONSE_V0, FIND_COORDINATOR_RESPONSE_V1};
+    }
+
     /**
      * Possible error codes:
      *
@@ -37,10 +67,6 @@ public class FindCoordinatorResponse extends AbstractResponse {
      * GROUP_AUTHORIZATION_FAILED (30)
      */
 
-    // coordinator level field names
-    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 final int throttleTimeMs;
     private final String errorMessage;
@@ -59,12 +85,9 @@ public class FindCoordinatorResponse extends AbstractResponse {
     }
 
     public FindCoordinatorResponse(Struct struct) {
-        this.throttleTimeMs = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME;
-        error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME));
-        if (struct.hasField(ERROR_MESSAGE_KEY_NAME))
-            errorMessage = struct.getString(ERROR_MESSAGE_KEY_NAME);
-        else
-            errorMessage = null;
+        this.throttleTimeMs = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME);
+        error = Errors.forCode(struct.get(ERROR_CODE));
+        errorMessage = struct.getOrElse(ERROR_MESSAGE, null);
 
         Struct broker = (Struct) struct.get(COORDINATOR_KEY_NAME);
         int nodeId = broker.getInt(NODE_ID_KEY_NAME);
@@ -88,11 +111,9 @@ public class FindCoordinatorResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.FIND_COORDINATOR.responseSchema(version));
-        if (struct.hasField(THROTTLE_TIME_KEY_NAME))
-            struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
-        struct.set(ERROR_CODE_KEY_NAME, error.code());
-        if (struct.hasField(ERROR_MESSAGE_KEY_NAME))
-            struct.set(ERROR_MESSAGE_KEY_NAME, errorMessage);
+        struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs);
+        struct.set(ERROR_CODE, error.code());
+        struct.setIfExists(ERROR_MESSAGE, errorMessage);
 
         Struct coordinator = struct.instance(COORDINATOR_KEY_NAME);
         coordinator.set(NODE_ID_KEY_NAME, node.id());

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java
index 7e08a55..00a806f 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java
@@ -18,15 +18,32 @@ 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.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
 
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class HeartbeatRequest extends AbstractRequest {
     private static final String GROUP_ID_KEY_NAME = "group_id";
     private static final String GROUP_GENERATION_ID_KEY_NAME = "group_generation_id";
     private static final String MEMBER_ID_KEY_NAME = "member_id";
 
+    private static final Schema HEARTBEAT_REQUEST_V0 = new Schema(
+            new Field(GROUP_ID_KEY_NAME, STRING, "The group id."),
+            new Field(GROUP_GENERATION_ID_KEY_NAME, INT32, "The generation of the group."),
+            new Field(MEMBER_ID_KEY_NAME, STRING, "The member id assigned by the group coordinator."));
+
+    /* v1 request is the same as v0. Throttle time has been added to response */
+    private static final Schema HEARTBEAT_REQUEST_V1 = HEARTBEAT_REQUEST_V0;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {HEARTBEAT_REQUEST_V0, HEARTBEAT_REQUEST_V1};
+    }
+
     public static class Builder extends AbstractRequest.Builder<HeartbeatRequest> {
         private final String groupId;
         private final int groupGenerationId;

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java
index cec39f0..e41d937 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java
@@ -18,13 +18,25 @@ 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.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+
 public class HeartbeatResponse extends AbstractResponse {
 
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
+    private static final Schema HEARTBEAT_RESPONSE_V0 = new Schema(
+            ERROR_CODE);
+    private static final Schema HEARTBEAT_RESPONSE_V1 = new Schema(
+            THROTTLE_TIME_MS,
+            ERROR_CODE);
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {HEARTBEAT_RESPONSE_V0, HEARTBEAT_RESPONSE_V1};
+    }
 
     /**
      * Possible error codes:
@@ -49,8 +61,8 @@ public class HeartbeatResponse extends AbstractResponse {
     }
 
     public HeartbeatResponse(Struct struct) {
-        this.throttleTimeMs = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME;
-        error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME));
+        this.throttleTimeMs = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME);
+        error = Errors.forCode(struct.get(ERROR_CODE));
     }
 
     public int throttleTimeMs() {
@@ -64,9 +76,8 @@ public class HeartbeatResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.HEARTBEAT.responseSchema(version));
-        if (struct.hasField(THROTTLE_TIME_KEY_NAME))
-            struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
-        struct.set(ERROR_CODE_KEY_NAME, error.code());
+        struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs);
+        struct.set(ERROR_CODE, error.code());
         return struct;
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdRequest.java
index 45f88a2..fa14a97 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdRequest.java
@@ -18,16 +18,29 @@ 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.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
 
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING;
+
 public class InitProducerIdRequest extends AbstractRequest {
     public static final int NO_TRANSACTION_TIMEOUT_MS = Integer.MAX_VALUE;
 
     private static final String TRANSACTIONAL_ID_KEY_NAME = "transactional_id";
     private static final String TRANSACTION_TIMEOUT_KEY_NAME = "transaction_timeout_ms";
 
+    private static final Schema INIT_PRODUCER_ID_REQUEST_V0 = new Schema(
+            new Field(TRANSACTIONAL_ID_KEY_NAME, NULLABLE_STRING, "The transactional id whose producer id we want to retrieve or generate."),
+            new Field(TRANSACTION_TIMEOUT_KEY_NAME, INT32, "The time in ms to wait for before aborting idle transactions sent by this producer."));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{INIT_PRODUCER_ID_REQUEST_V0};
+    }
+
     private final String transactionalId;
     private final int transactionTimeoutMs;
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java
index 88fb09c..8799ad7 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java
@@ -18,11 +18,18 @@ 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.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.record.RecordBatch;
 
 import java.nio.ByteBuffer;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.types.Type.INT16;
+import static org.apache.kafka.common.protocol.types.Type.INT64;
+
 public class InitProducerIdResponse extends AbstractResponse {
     // Possible error codes:
     //   NotCoordinator
@@ -33,7 +40,19 @@ public class InitProducerIdResponse extends AbstractResponse {
 
     private static final String PRODUCER_ID_KEY_NAME = "producer_id";
     private static final String EPOCH_KEY_NAME = "producer_epoch";
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
+
+    private static final Schema INIT_PRODUCER_ID_RESPONSE_V0 = new Schema(
+            THROTTLE_TIME_MS,
+            ERROR_CODE,
+            new Field(PRODUCER_ID_KEY_NAME, INT64, "The producer id for the input transactional id. If the input " +
+                    "id was empty, then this is used only for ensuring idempotence of messages."),
+            new Field(EPOCH_KEY_NAME, INT16, "The epoch for the producer id. Will always be 0 if no transactional " +
+                    "id was specified in the request."));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{INIT_PRODUCER_ID_RESPONSE_V0};
+    }
+
     private final int throttleTimeMs;
     private final Errors error;
     private final long producerId;
@@ -47,8 +66,8 @@ public class InitProducerIdResponse extends AbstractResponse {
     }
 
     public InitProducerIdResponse(Struct struct) {
-        this.throttleTimeMs = struct.getInt(THROTTLE_TIME_KEY_NAME);
-        this.error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME));
+        this.throttleTimeMs = struct.get(THROTTLE_TIME_MS);
+        this.error = Errors.forCode(struct.get(ERROR_CODE));
         this.producerId = struct.getLong(PRODUCER_ID_KEY_NAME);
         this.epoch = struct.getShort(EPOCH_KEY_NAME);
     }
@@ -76,10 +95,10 @@ public class InitProducerIdResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.INIT_PRODUCER_ID.responseSchema(version));
-        struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
+        struct.set(THROTTLE_TIME_MS, throttleTimeMs);
         struct.set(PRODUCER_ID_KEY_NAME, producerId);
         struct.set(EPOCH_KEY_NAME, epoch);
-        struct.set(ERROR_CODE_KEY_NAME, error.code());
+        struct.set(ERROR_CODE, error.code());
         return struct;
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java
index ff07d13..b2ff133 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java
@@ -18,6 +18,9 @@ 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.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.Utils;
 
@@ -26,6 +29,10 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
+import static org.apache.kafka.common.protocol.types.Type.BYTES;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class JoinGroupRequest extends AbstractRequest {
     private static final String GROUP_ID_KEY_NAME = "group_id";
     private static final String SESSION_TIMEOUT_KEY_NAME = "session_timeout";
@@ -36,6 +43,38 @@ public class JoinGroupRequest extends AbstractRequest {
     private static final String PROTOCOL_NAME_KEY_NAME = "protocol_name";
     private static final String PROTOCOL_METADATA_KEY_NAME = "protocol_metadata";
 
+    /* Join group api */
+    private static final Schema JOIN_GROUP_REQUEST_PROTOCOL_V0 = new Schema(
+            new Field(PROTOCOL_NAME_KEY_NAME, STRING),
+            new Field(PROTOCOL_METADATA_KEY_NAME, BYTES));
+
+    private static final Schema JOIN_GROUP_REQUEST_V0 = new Schema(
+            new Field(GROUP_ID_KEY_NAME, STRING, "The group id."),
+            new Field(SESSION_TIMEOUT_KEY_NAME, INT32, "The coordinator considers the consumer dead if it receives " +
+                    "no heartbeat after this timeout in ms."),
+            new Field(MEMBER_ID_KEY_NAME, STRING, "The assigned consumer id or an empty string for a new consumer."),
+            new Field(PROTOCOL_TYPE_KEY_NAME, STRING, "Unique name for class of protocols implemented by group"),
+            new Field(GROUP_PROTOCOLS_KEY_NAME, new ArrayOf(JOIN_GROUP_REQUEST_PROTOCOL_V0), "List of protocols " +
+                    "that the member supports"));
+
+    private static final Schema JOIN_GROUP_REQUEST_V1 = new Schema(
+            new Field(GROUP_ID_KEY_NAME, STRING, "The group id."),
+            new Field(SESSION_TIMEOUT_KEY_NAME, INT32, "The coordinator considers the consumer dead if it receives no " +
+                    "heartbeat after this timeout in ms."),
+            new Field(REBALANCE_TIMEOUT_KEY_NAME, INT32, "The maximum time that the coordinator will wait for each " +
+                    "member to rejoin when rebalancing the group"),
+            new Field(MEMBER_ID_KEY_NAME, STRING, "The assigned consumer id or an empty string for a new consumer."),
+            new Field(PROTOCOL_TYPE_KEY_NAME, STRING, "Unique name for class of protocols implemented by group"),
+            new Field(GROUP_PROTOCOLS_KEY_NAME, new ArrayOf(JOIN_GROUP_REQUEST_PROTOCOL_V0), "List of protocols " +
+                    "that the member supports"));
+
+    /* v2 request is the same as v1. Throttle time has been added to response */
+    private static final Schema JOIN_GROUP_REQUEST_V2 = JOIN_GROUP_REQUEST_V1;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {JOIN_GROUP_REQUEST_V0, JOIN_GROUP_REQUEST_V1, JOIN_GROUP_REQUEST_V2};
+    }
+
     public static final String UNKNOWN_MEMBER_ID = "";
 
     private final String groupId;

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java
index eb86ce7..a4431b9 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java
@@ -18,6 +18,9 @@ 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.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
@@ -26,9 +29,53 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.types.Type.BYTES;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class JoinGroupResponse extends AbstractResponse {
 
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
+    private static final String GENERATION_ID_KEY_NAME = "generation_id";
+    private static final String GROUP_PROTOCOL_KEY_NAME = "group_protocol";
+    private static final String LEADER_ID_KEY_NAME = "leader_id";
+    private static final String MEMBER_ID_KEY_NAME = "member_id";
+    private static final String MEMBERS_KEY_NAME = "members";
+
+    private static final String MEMBER_METADATA_KEY_NAME = "member_metadata";
+
+    private static final Schema JOIN_GROUP_RESPONSE_MEMBER_V0 = new Schema(
+            new Field(MEMBER_ID_KEY_NAME, STRING),
+            new Field(MEMBER_METADATA_KEY_NAME, BYTES));
+
+    private static final Schema JOIN_GROUP_RESPONSE_V0 = new Schema(
+            ERROR_CODE,
+            new Field(GENERATION_ID_KEY_NAME, INT32, "The generation of the consumer group."),
+            new Field(GROUP_PROTOCOL_KEY_NAME, STRING, "The group protocol selected by the coordinator"),
+            new Field(LEADER_ID_KEY_NAME, STRING, "The leader of the group"),
+            new Field(MEMBER_ID_KEY_NAME, STRING, "The consumer id assigned by the group coordinator."),
+            new Field(MEMBERS_KEY_NAME, new ArrayOf(JOIN_GROUP_RESPONSE_MEMBER_V0)));
+
+    private static final Schema JOIN_GROUP_RESPONSE_V1 = JOIN_GROUP_RESPONSE_V0;
+
+    private static final Schema JOIN_GROUP_RESPONSE_V2 = new Schema(
+            THROTTLE_TIME_MS,
+            ERROR_CODE,
+            new Field(GENERATION_ID_KEY_NAME, INT32, "The generation of the consumer group."),
+            new Field(GROUP_PROTOCOL_KEY_NAME, STRING, "The group protocol selected by the coordinator"),
+            new Field(LEADER_ID_KEY_NAME, STRING, "The leader of the group"),
+            new Field(MEMBER_ID_KEY_NAME, STRING, "The consumer id assigned by the group coordinator."),
+            new Field(MEMBERS_KEY_NAME, new ArrayOf(JOIN_GROUP_RESPONSE_MEMBER_V0)));
+
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {JOIN_GROUP_RESPONSE_V0, JOIN_GROUP_RESPONSE_V1, JOIN_GROUP_RESPONSE_V2};
+    }
+
+    public static final String UNKNOWN_PROTOCOL = "";
+    public static final int UNKNOWN_GENERATION_ID = -1;
+    public static final String UNKNOWN_MEMBER_ID = "";
 
     /**
      * Possible error codes:
@@ -42,18 +89,6 @@ public class JoinGroupResponse extends AbstractResponse {
      * GROUP_AUTHORIZATION_FAILED (30)
      */
 
-    private static final String GENERATION_ID_KEY_NAME = "generation_id";
-    private static final String GROUP_PROTOCOL_KEY_NAME = "group_protocol";
-    private static final String LEADER_ID_KEY_NAME = "leader_id";
-    private static final String MEMBER_ID_KEY_NAME = "member_id";
-    private static final String MEMBERS_KEY_NAME = "members";
-
-    private static final String MEMBER_METADATA_KEY_NAME = "member_metadata";
-
-    public static final String UNKNOWN_PROTOCOL = "";
-    public static final int UNKNOWN_GENERATION_ID = -1;
-    public static final String UNKNOWN_MEMBER_ID = "";
-
     private final int throttleTimeMs;
     private final Errors error;
     private final int generationId;
@@ -88,7 +123,7 @@ public class JoinGroupResponse extends AbstractResponse {
     }
 
     public JoinGroupResponse(Struct struct) {
-        this.throttleTimeMs = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME;
+        this.throttleTimeMs = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME);
         members = new HashMap<>();
 
         for (Object memberDataObj : struct.getArray(MEMBERS_KEY_NAME)) {
@@ -97,7 +132,7 @@ public class JoinGroupResponse extends AbstractResponse {
             ByteBuffer memberMetadata = memberData.getBytes(MEMBER_METADATA_KEY_NAME);
             members.put(memberId, memberMetadata);
         }
-        error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME));
+        error = Errors.forCode(struct.get(ERROR_CODE));
         generationId = struct.getInt(GENERATION_ID_KEY_NAME);
         groupProtocol = struct.getString(GROUP_PROTOCOL_KEY_NAME);
         memberId = struct.getString(MEMBER_ID_KEY_NAME);
@@ -143,10 +178,9 @@ public class JoinGroupResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.JOIN_GROUP.responseSchema(version));
-        if (struct.hasField(THROTTLE_TIME_KEY_NAME))
-            struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
+        struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs);
 
-        struct.set(ERROR_CODE_KEY_NAME, error.code());
+        struct.set(ERROR_CODE, error.code());
         struct.set(GENERATION_ID_KEY_NAME, generationId);
         struct.set(GROUP_PROTOCOL_KEY_NAME, groupProtocol);
         struct.set(MEMBER_ID_KEY_NAME, memberId);

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java
index bd379b8..73f037f 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java
@@ -20,6 +20,9 @@ import org.apache.kafka.common.Node;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.Utils;
 
@@ -31,6 +34,12 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.BOOLEAN;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class LeaderAndIsrRequest extends AbstractRequest {
     private static final String CONTROLLER_ID_KEY_NAME = "controller_id";
     private static final String CONTROLLER_EPOCH_KEY_NAME = "controller_epoch";
@@ -38,8 +47,6 @@ public class LeaderAndIsrRequest extends AbstractRequest {
     private static final String LIVE_LEADERS_KEY_NAME = "live_leaders";
 
     // partition_states key names
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITION_KEY_NAME = "partition";
     private static final String LEADER_KEY_NAME = "leader";
     private static final String LEADER_EPOCH_KEY_NAME = "leader_epoch";
     private static final String ISR_KEY_NAME = "isr";
@@ -52,6 +59,51 @@ public class LeaderAndIsrRequest extends AbstractRequest {
     private static final String HOST_KEY_NAME = "host";
     private static final String PORT_KEY_NAME = "port";
 
+    private static final Schema LEADER_AND_ISR_REQUEST_PARTITION_STATE_V0 = new Schema(
+            TOPIC_NAME,
+            PARTITION_ID,
+            new Field(CONTROLLER_EPOCH_KEY_NAME, INT32, "The controller epoch."),
+            new Field(LEADER_KEY_NAME, INT32, "The broker id for the leader."),
+            new Field(LEADER_EPOCH_KEY_NAME, INT32, "The leader epoch."),
+            new Field(ISR_KEY_NAME, new ArrayOf(INT32), "The in sync replica ids."),
+            new Field(ZK_VERSION_KEY_NAME, INT32, "The ZK version."),
+            new Field(REPLICAS_KEY_NAME, new ArrayOf(INT32), "The replica ids."));
+
+    // LEADER_AND_ISR_REQUEST_PARTITION_STATE_V1 added a per-partition is_new Field.
+    // This field specifies whether the replica should have existed on the broker or not.
+    private static final Schema LEADER_AND_ISR_REQUEST_PARTITION_STATE_V1 = new Schema(
+            TOPIC_NAME,
+            PARTITION_ID,
+            new Field(CONTROLLER_EPOCH_KEY_NAME, INT32, "The controller epoch."),
+            new Field(LEADER_KEY_NAME, INT32, "The broker id for the leader."),
+            new Field(LEADER_EPOCH_KEY_NAME, INT32, "The leader epoch."),
+            new Field(ISR_KEY_NAME, new ArrayOf(INT32), "The in sync replica ids."),
+            new Field(ZK_VERSION_KEY_NAME, INT32, "The ZK version."),
+            new Field(REPLICAS_KEY_NAME, new ArrayOf(INT32), "The replica ids."),
+            new Field(IS_NEW_KEY_NAME, BOOLEAN, "Whether the replica should have existed on the broker or not"));
+
+    private static final Schema LEADER_AND_ISR_REQUEST_LIVE_LEADER_V0 = new Schema(
+            new Field(END_POINT_ID_KEY_NAME, INT32, "The broker id."),
+            new Field(HOST_KEY_NAME, STRING, "The hostname of the broker."),
+            new Field(PORT_KEY_NAME, INT32, "The port on which the broker accepts requests."));
+
+    private static final Schema LEADER_AND_ISR_REQUEST_V0 = new Schema(
+            new Field(CONTROLLER_ID_KEY_NAME, INT32, "The controller id."),
+            new Field(CONTROLLER_EPOCH_KEY_NAME, INT32, "The controller epoch."),
+            new Field(PARTITION_STATES_KEY_NAME, new ArrayOf(LEADER_AND_ISR_REQUEST_PARTITION_STATE_V0)),
+            new Field(LIVE_LEADERS_KEY_NAME, new ArrayOf(LEADER_AND_ISR_REQUEST_LIVE_LEADER_V0)));
+
+    // LEADER_AND_ISR_REQUEST_V1 added a per-partition is_new Field. This field specifies whether the replica should have existed on the broker or not.
+    private static final Schema LEADER_AND_ISR_REQUEST_V1 = new Schema(
+            new Field(CONTROLLER_ID_KEY_NAME, INT32, "The controller id."),
+            new Field(CONTROLLER_EPOCH_KEY_NAME, INT32, "The controller epoch."),
+            new Field(PARTITION_STATES_KEY_NAME, new ArrayOf(LEADER_AND_ISR_REQUEST_PARTITION_STATE_V1)),
+            new Field(LIVE_LEADERS_KEY_NAME, new ArrayOf(LEADER_AND_ISR_REQUEST_LIVE_LEADER_V0)));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{LEADER_AND_ISR_REQUEST_V0, LEADER_AND_ISR_REQUEST_V1};
+    }
+
     public static class Builder extends AbstractRequest.Builder<LeaderAndIsrRequest> {
         private final int controllerId;
         private final int controllerEpoch;
@@ -105,8 +157,8 @@ public class LeaderAndIsrRequest extends AbstractRequest {
         Map<TopicPartition, PartitionState> partitionStates = new HashMap<>();
         for (Object partitionStateDataObj : struct.getArray(PARTITION_STATES_KEY_NAME)) {
             Struct partitionStateData = (Struct) partitionStateDataObj;
-            String topic = partitionStateData.getString(TOPIC_KEY_NAME);
-            int partition = partitionStateData.getInt(PARTITION_KEY_NAME);
+            String topic = partitionStateData.get(TOPIC_NAME);
+            int partition = partitionStateData.get(PARTITION_ID);
             int controllerEpoch = partitionStateData.getInt(CONTROLLER_EPOCH_KEY_NAME);
             int leader = partitionStateData.getInt(LEADER_KEY_NAME);
             int leaderEpoch = partitionStateData.getInt(LEADER_EPOCH_KEY_NAME);
@@ -154,8 +206,8 @@ public class LeaderAndIsrRequest extends AbstractRequest {
         for (Map.Entry<TopicPartition, PartitionState> entry : partitionStates.entrySet()) {
             Struct partitionStateData = struct.instance(PARTITION_STATES_KEY_NAME);
             TopicPartition topicPartition = entry.getKey();
-            partitionStateData.set(TOPIC_KEY_NAME, topicPartition.topic());
-            partitionStateData.set(PARTITION_KEY_NAME, topicPartition.partition());
+            partitionStateData.set(TOPIC_NAME, topicPartition.topic());
+            partitionStateData.set(PARTITION_ID, topicPartition.partition());
             PartitionState partitionState = entry.getValue();
             partitionStateData.set(CONTROLLER_EPOCH_KEY_NAME, partitionState.basePartitionState.controllerEpoch);
             partitionStateData.set(LEADER_KEY_NAME, partitionState.basePartitionState.leader);

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java
index bc4400e..39b8c37 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java
@@ -19,6 +19,9 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
@@ -27,14 +30,27 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-public class LeaderAndIsrResponse extends AbstractResponse {
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
 
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
+public class LeaderAndIsrResponse extends AbstractResponse {
     private static final String PARTITIONS_KEY_NAME = "partitions";
 
-    private static final String PARTITIONS_TOPIC_KEY_NAME = "topic";
-    private static final String PARTITIONS_PARTITION_KEY_NAME = "partition";
-    private static final String PARTITIONS_ERROR_CODE_KEY_NAME = "error_code";
+    private static final Schema LEADER_AND_ISR_RESPONSE_PARTITION_V0 = new Schema(
+            TOPIC_NAME,
+            PARTITION_ID,
+            ERROR_CODE);
+    private static final Schema LEADER_AND_ISR_RESPONSE_V0 = new Schema(
+            ERROR_CODE,
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(LEADER_AND_ISR_RESPONSE_PARTITION_V0)));
+
+    // LeaderAndIsrResponse V1 may receive KAFKA_STORAGE_ERROR in the response
+    private static final Schema LEADER_AND_ISR_RESPONSE_V1 = LEADER_AND_ISR_RESPONSE_V0;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{LEADER_AND_ISR_RESPONSE_V0, LEADER_AND_ISR_RESPONSE_V1};
+    }
 
     /**
      * Possible error code:
@@ -54,13 +70,13 @@ public class LeaderAndIsrResponse extends AbstractResponse {
         responses = new HashMap<>();
         for (Object responseDataObj : struct.getArray(PARTITIONS_KEY_NAME)) {
             Struct responseData = (Struct) responseDataObj;
-            String topic = responseData.getString(PARTITIONS_TOPIC_KEY_NAME);
-            int partition = responseData.getInt(PARTITIONS_PARTITION_KEY_NAME);
-            Errors error = Errors.forCode(responseData.getShort(PARTITIONS_ERROR_CODE_KEY_NAME));
+            String topic = responseData.get(TOPIC_NAME);
+            int partition = responseData.get(PARTITION_ID);
+            Errors error = Errors.forCode(responseData.get(ERROR_CODE));
             responses.put(new TopicPartition(topic, partition), error);
         }
 
-        error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME));
+        error = Errors.forCode(struct.get(ERROR_CODE));
     }
 
     public Map<TopicPartition, Errors> responses() {
@@ -83,14 +99,14 @@ public class LeaderAndIsrResponse extends AbstractResponse {
         for (Map.Entry<TopicPartition, Errors> response : responses.entrySet()) {
             Struct partitionData = struct.instance(PARTITIONS_KEY_NAME);
             TopicPartition partition = response.getKey();
-            partitionData.set(PARTITIONS_TOPIC_KEY_NAME, partition.topic());
-            partitionData.set(PARTITIONS_PARTITION_KEY_NAME, partition.partition());
-            partitionData.set(PARTITIONS_ERROR_CODE_KEY_NAME, response.getValue().code());
+            partitionData.set(TOPIC_NAME, partition.topic());
+            partitionData.set(PARTITION_ID, partition.partition());
+            partitionData.set(ERROR_CODE, response.getValue().code());
             responseDatas.add(partitionData);
         }
 
         struct.set(PARTITIONS_KEY_NAME, responseDatas.toArray());
-        struct.set(ERROR_CODE_KEY_NAME, error.code());
+        struct.set(ERROR_CODE, error.code());
 
         return struct;
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java
index 76e076e..661eb7f 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java
@@ -16,15 +16,31 @@
  */
 package org.apache.kafka.common.requests;
 
-import java.nio.ByteBuffer;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
+import java.nio.ByteBuffer;
+
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class LeaveGroupRequest extends AbstractRequest {
     private static final String GROUP_ID_KEY_NAME = "group_id";
     private static final String MEMBER_ID_KEY_NAME = "member_id";
 
+    private static final Schema LEAVE_GROUP_REQUEST_V0 = new Schema(
+            new Field(GROUP_ID_KEY_NAME, STRING, "The group id."),
+            new Field(MEMBER_ID_KEY_NAME, STRING, "The member id assigned by the group coordinator."));
+
+    /* v1 request is the same as v0. Throttle time has been added to response */
+    private static final Schema LEAVE_GROUP_REQUEST_V1 = LEAVE_GROUP_REQUEST_V0;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {LEAVE_GROUP_REQUEST_V0, LEAVE_GROUP_REQUEST_V1};
+    }
+
     public static class Builder extends AbstractRequest.Builder<LeaveGroupRequest> {
         private final String groupId;
         private final String memberId;

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java
index 1c85850..bef21e9 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java
@@ -18,13 +18,25 @@ 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.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+
 public class LeaveGroupResponse extends AbstractResponse {
 
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
+    private static final Schema LEAVE_GROUP_RESPONSE_V0 = new Schema(
+            ERROR_CODE);
+    private static final Schema LEAVE_GROUP_RESPONSE_V1 = new Schema(
+            THROTTLE_TIME_MS,
+            ERROR_CODE);
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {LEAVE_GROUP_RESPONSE_V0, LEAVE_GROUP_RESPONSE_V1};
+    }
 
     /**
      * Possible error code:
@@ -48,8 +60,8 @@ public class LeaveGroupResponse extends AbstractResponse {
     }
 
     public LeaveGroupResponse(Struct struct) {
-        this.throttleTimeMs = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME;
-        error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME));
+        this.throttleTimeMs = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME);
+        this.error = Errors.forCode(struct.get(ERROR_CODE));
     }
 
     public int throttleTimeMs() {
@@ -63,9 +75,8 @@ public class LeaveGroupResponse extends AbstractResponse {
     @Override
     public Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.LEAVE_GROUP.responseSchema(version));
-        if (struct.hasField(THROTTLE_TIME_KEY_NAME))
-            struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
-        struct.set(ERROR_CODE_KEY_NAME, error.code());
+        struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs);
+        struct.set(ERROR_CODE, error.code());
         return struct;
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java
index 3d4f2b8..f279b4c 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java
@@ -18,12 +18,24 @@ 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.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
 import java.util.Collections;
 
 public class ListGroupsRequest extends AbstractRequest {
+
+    /* List groups api */
+    private static final Schema LIST_GROUPS_REQUEST_V0 = new Schema();
+
+    /* v1 request is the same as v0. Throttle time has been added to response */
+    private static final Schema LIST_GROUPS_REQUEST_V1 = LIST_GROUPS_REQUEST_V0;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {LIST_GROUPS_REQUEST_V0, LIST_GROUPS_REQUEST_V1};
+    }
+
     public static class Builder extends AbstractRequest.Builder<ListGroupsRequest> {
         public Builder() {
             super(ApiKeys.LIST_GROUPS);
@@ -49,7 +61,7 @@ public class ListGroupsRequest extends AbstractRequest {
     }
 
     @Override
-    public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
+    public ListGroupsResponse getErrorResponse(int throttleTimeMs, Throwable e) {
         short versionId = version();
         switch (versionId) {
             case 0:

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java
index 8ae3792..cdf4c59 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java
@@ -18,19 +18,39 @@ 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.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+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.Collections;
 import java.util.List;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class ListGroupsResponse extends AbstractResponse {
 
-    public static final String ERROR_CODE_KEY_NAME = "error_code";
-    public static final String GROUPS_KEY_NAME = "groups";
-    public static final String GROUP_ID_KEY_NAME = "group_id";
-    public static final String PROTOCOL_TYPE_KEY_NAME = "protocol_type";
+    private static final String GROUPS_KEY_NAME = "groups";
+    private static final String GROUP_ID_KEY_NAME = "group_id";
+    private static final String PROTOCOL_TYPE_KEY_NAME = "protocol_type";
+
+    private static final Schema LIST_GROUPS_RESPONSE_GROUP_V0 = new Schema(
+            new Field("group_id", STRING),
+            new Field("protocol_type", STRING));
+    private static final Schema LIST_GROUPS_RESPONSE_V0 = new Schema(
+            ERROR_CODE,
+            new Field("groups", new ArrayOf(LIST_GROUPS_RESPONSE_GROUP_V0)));
+    private static final Schema LIST_GROUPS_RESPONSE_V1 = new Schema(
+            THROTTLE_TIME_MS,
+            ERROR_CODE,
+            new Field("groups", new ArrayOf(LIST_GROUPS_RESPONSE_GROUP_V0)));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {LIST_GROUPS_RESPONSE_V0, LIST_GROUPS_RESPONSE_V1};
+    }
 
     /**
      * Possible error codes:
@@ -54,8 +74,8 @@ public class ListGroupsResponse extends AbstractResponse {
     }
 
     public ListGroupsResponse(Struct struct) {
-        this.throttleTimeMs = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME;
-        this.error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME));
+        this.throttleTimeMs = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME);
+        this.error = Errors.forCode(struct.get(ERROR_CODE));
         this.groups = new ArrayList<>();
         for (Object groupObj : struct.getArray(GROUPS_KEY_NAME)) {
             Struct groupStruct = (Struct) groupObj;
@@ -99,9 +119,8 @@ public class ListGroupsResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.LIST_GROUPS.responseSchema(version));
-        if (struct.hasField(THROTTLE_TIME_KEY_NAME))
-            struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
-        struct.set(ERROR_CODE_KEY_NAME, error.code());
+        struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs);
+        struct.set(ERROR_CODE, error.code());
         List<Struct> groupList = new ArrayList<>();
         for (Group group : groups) {
             Struct groupStruct = struct.instance(GROUPS_KEY_NAME);
@@ -113,14 +132,6 @@ public class ListGroupsResponse extends AbstractResponse {
         return struct;
     }
 
-    public static ListGroupsResponse fromError(Errors error) {
-        return fromError(DEFAULT_THROTTLE_TIME, error);
-    }
-
-    public static ListGroupsResponse fromError(int throttleTimeMs, Errors error) {
-        return new ListGroupsResponse(throttleTimeMs, error, Collections.<Group>emptyList());
-    }
-
     public static ListGroupsResponse parse(ByteBuffer buffer, short version) {
         return new ListGroupsResponse(ApiKeys.LIST_GROUPS.parseResponse(version, buffer));
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java
index 03f6ee5..ace582d 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java
@@ -20,6 +20,9 @@ import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.errors.UnsupportedVersionException;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.CollectionUtils;
 
@@ -32,6 +35,12 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.INT64;
+import static org.apache.kafka.common.protocol.types.Type.INT8;
+
 public class ListOffsetRequest extends AbstractRequest {
     public static final long EARLIEST_TIMESTAMP = -2L;
     public static final long LATEST_TIMESTAMP = -1L;
@@ -44,14 +53,49 @@ public class ListOffsetRequest extends AbstractRequest {
     private static final String TOPICS_KEY_NAME = "topics";
 
     // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
     private static final String PARTITIONS_KEY_NAME = "partitions";
 
     // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
     private static final String TIMESTAMP_KEY_NAME = "timestamp";
     private static final String MAX_NUM_OFFSETS_KEY_NAME = "max_num_offsets";
 
+    private static final Schema LIST_OFFSET_REQUEST_PARTITION_V0 = new Schema(
+            PARTITION_ID,
+            new Field(TIMESTAMP_KEY_NAME, INT64, "Timestamp."),
+            new Field(MAX_NUM_OFFSETS_KEY_NAME, INT32, "Maximum offsets to return."));
+    private static final Schema LIST_OFFSET_REQUEST_PARTITION_V1 = new Schema(
+            PARTITION_ID,
+            new Field(TIMESTAMP_KEY_NAME, INT64, "The target timestamp for the partition."));
+
+    private static final Schema LIST_OFFSET_REQUEST_TOPIC_V0 = new Schema(
+            TOPIC_NAME,
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(LIST_OFFSET_REQUEST_PARTITION_V0), "Partitions to list offset."));
+    private static final Schema LIST_OFFSET_REQUEST_TOPIC_V1 = new Schema(
+            TOPIC_NAME,
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(LIST_OFFSET_REQUEST_PARTITION_V1), "Partitions to list offset."));
+
+    private static final Schema LIST_OFFSET_REQUEST_V0 = new Schema(
+            new Field(REPLICA_ID_KEY_NAME, INT32, "Broker id of the follower. For normal consumers, use -1."),
+            new Field(TOPICS_KEY_NAME, new ArrayOf(LIST_OFFSET_REQUEST_TOPIC_V0), "Topics to list offsets."));
+    private static final Schema LIST_OFFSET_REQUEST_V1 = new Schema(
+            new Field(REPLICA_ID_KEY_NAME, INT32, "Broker id of the follower. For normal consumers, use -1."),
+            new Field(TOPICS_KEY_NAME, new ArrayOf(LIST_OFFSET_REQUEST_TOPIC_V1), "Topics to list offsets."));
+
+    private static final Schema LIST_OFFSET_REQUEST_V2 = new Schema(
+            new Field(REPLICA_ID_KEY_NAME, INT32, "Broker id of the follower. For normal consumers, use -1."),
+            new Field(ISOLATION_LEVEL_KEY_NAME, INT8, "This setting controls the visibility of transactional records. " +
+                    "Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED " +
+                    "(isolation_level = 1), non-transactional and COMMITTED transactional records are visible. " +
+                    "To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current " +
+                    "LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the " +
+                    "result, which allows consumers to discard ABORTED transactional records"),
+            new Field(TOPICS_KEY_NAME, new ArrayOf(LIST_OFFSET_REQUEST_TOPIC_V1), "Topics to list offsets."));;
+
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {LIST_OFFSET_REQUEST_V0, LIST_OFFSET_REQUEST_V1, LIST_OFFSET_REQUEST_V2};
+    }
+
     private final int replicaId;
     private final IsolationLevel isolationLevel;
     private final Map<TopicPartition, PartitionData> offsetData;
@@ -193,10 +237,10 @@ public class ListOffsetRequest extends AbstractRequest {
         partitionTimestamps = new HashMap<>();
         for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
             Struct topicResponse = (Struct) topicResponseObj;
-            String topic = topicResponse.getString(TOPIC_KEY_NAME);
+            String topic = topicResponse.get(TOPIC_NAME);
             for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
                 Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                int partition = partitionResponse.get(PARTITION_ID);
                 long timestamp = partitionResponse.getLong(TIMESTAMP_KEY_NAME);
                 TopicPartition tp = new TopicPartition(topic, partition);
                 if (partitionResponse.hasField(MAX_NUM_OFFSETS_KEY_NAME)) {
@@ -283,20 +327,20 @@ public class ListOffsetRequest extends AbstractRequest {
         List<Struct> topicArray = new ArrayList<>();
         for (Map.Entry<String, Map<Integer, Object>> topicEntry: topicsData.entrySet()) {
             Struct topicData = struct.instance(TOPICS_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
+            topicData.set(TOPIC_NAME, topicEntry.getKey());
             List<Struct> partitionArray = new ArrayList<>();
             for (Map.Entry<Integer, Object> partitionEntry : topicEntry.getValue().entrySet()) {
                 if (version == 0) {
                     PartitionData offsetPartitionData = (PartitionData) partitionEntry.getValue();
                     Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                    partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
+                    partitionData.set(PARTITION_ID, partitionEntry.getKey());
                     partitionData.set(TIMESTAMP_KEY_NAME, offsetPartitionData.timestamp);
                     partitionData.set(MAX_NUM_OFFSETS_KEY_NAME, offsetPartitionData.maxNumOffsets);
                     partitionArray.add(partitionData);
                 } else {
                     Long timestamp = (Long) partitionEntry.getValue();
                     Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                    partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
+                    partitionData.set(PARTITION_ID, partitionEntry.getKey());
                     partitionData.set(TIMESTAMP_KEY_NAME, timestamp);
                     partitionArray.add(partitionData);
                 }


[2/8] kafka git commit: MINOR: Move request/response schemas to the corresponding object representation

Posted by ij...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java
index fa23559..103246c 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java
@@ -25,58 +25,69 @@ import org.apache.kafka.common.resource.Resource;
 import org.apache.kafka.common.resource.ResourceFilter;
 import org.apache.kafka.common.resource.ResourceType;
 
+import static org.apache.kafka.common.protocol.CommonFields.HOST;
+import static org.apache.kafka.common.protocol.CommonFields.HOST_FILTER;
+import static org.apache.kafka.common.protocol.CommonFields.OPERATION;
+import static org.apache.kafka.common.protocol.CommonFields.PERMISSION_TYPE;
+import static org.apache.kafka.common.protocol.CommonFields.PRINCIPAL;
+import static org.apache.kafka.common.protocol.CommonFields.PRINCIPAL_FILTER;
+import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_NAME;
+import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_NAME_FILTER;
+import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_TYPE;
+
 class RequestUtils {
+
     static Resource resourceFromStructFields(Struct struct) {
-        byte resourceType = struct.getByte("resource_type");
-        String name = struct.getString("resource_name");
+        byte resourceType = struct.get(RESOURCE_TYPE);
+        String name = struct.get(RESOURCE_NAME);
         return new Resource(ResourceType.fromCode(resourceType), name);
     }
 
     static void resourceSetStructFields(Resource resource, Struct struct) {
-        struct.set("resource_type", resource.resourceType().code());
-        struct.set("resource_name", resource.name());
+        struct.set(RESOURCE_TYPE, resource.resourceType().code());
+        struct.set(RESOURCE_NAME, resource.name());
     }
 
     static ResourceFilter resourceFilterFromStructFields(Struct struct) {
-        byte resourceType = struct.getByte("resource_type");
-        String name = struct.getString("resource_name");
+        byte resourceType = struct.get(RESOURCE_TYPE);
+        String name = struct.get(RESOURCE_NAME_FILTER);
         return new ResourceFilter(ResourceType.fromCode(resourceType), name);
     }
 
     static void resourceFilterSetStructFields(ResourceFilter resourceFilter, Struct struct) {
-        struct.set("resource_type", resourceFilter.resourceType().code());
-        struct.set("resource_name", resourceFilter.name());
+        struct.set(RESOURCE_TYPE, resourceFilter.resourceType().code());
+        struct.set(RESOURCE_NAME_FILTER, resourceFilter.name());
     }
 
     static AccessControlEntry aceFromStructFields(Struct struct) {
-        String principal = struct.getString("principal");
-        String host = struct.getString("host");
-        byte operation = struct.getByte("operation");
-        byte permissionType = struct.getByte("permission_type");
+        String principal = struct.get(PRINCIPAL);
+        String host = struct.get(HOST);
+        byte operation = struct.get(OPERATION);
+        byte permissionType = struct.get(PERMISSION_TYPE);
         return new AccessControlEntry(principal, host, AclOperation.fromCode(operation),
             AclPermissionType.fromCode(permissionType));
     }
 
     static void aceSetStructFields(AccessControlEntry data, Struct struct) {
-        struct.set("principal", data.principal());
-        struct.set("host", data.host());
-        struct.set("operation", data.operation().code());
-        struct.set("permission_type", data.permissionType().code());
+        struct.set(PRINCIPAL, data.principal());
+        struct.set(HOST, data.host());
+        struct.set(OPERATION, data.operation().code());
+        struct.set(PERMISSION_TYPE, data.permissionType().code());
     }
 
     static AccessControlEntryFilter aceFilterFromStructFields(Struct struct) {
-        String principal = struct.getString("principal");
-        String host = struct.getString("host");
-        byte operation = struct.getByte("operation");
-        byte permissionType = struct.getByte("permission_type");
+        String principal = struct.get(PRINCIPAL_FILTER);
+        String host = struct.get(HOST_FILTER);
+        byte operation = struct.get(OPERATION);
+        byte permissionType = struct.get(PERMISSION_TYPE);
         return new AccessControlEntryFilter(principal, host, AclOperation.fromCode(operation),
             AclPermissionType.fromCode(permissionType));
     }
 
     static void aceFilterSetStructFields(AccessControlEntryFilter filter, Struct struct) {
-        struct.set("principal", filter.principal());
-        struct.set("host", filter.host());
-        struct.set("operation", filter.operation().code());
-        struct.set("permission_type", filter.permissionType().code());
+        struct.set(PRINCIPAL_FILTER, filter.principal());
+        struct.set(HOST_FILTER, filter.host());
+        struct.set(OPERATION, filter.operation().code());
+        struct.set(PERMISSION_TYPE, filter.permissionType().code());
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java b/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java
index 895716b..fe452a2 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java
@@ -16,21 +16,22 @@
  */
 package org.apache.kafka.common.requests;
 
-import static org.apache.kafka.common.protocol.Protocol.RESPONSE_HEADER;
-
-import java.nio.ByteBuffer;
-
-import org.apache.kafka.common.protocol.Protocol;
+import org.apache.kafka.common.protocol.types.BoundField;
 import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
+import java.nio.ByteBuffer;
+
+import static org.apache.kafka.common.protocol.types.Type.INT32;
 
 /**
  * A response header in the kafka protocol.
  */
 public class ResponseHeader extends AbstractRequestResponse {
-
-    private static final Field CORRELATION_KEY_FIELD = RESPONSE_HEADER.get("correlation_id");
+    public static final Schema SCHEMA = new Schema(
+            new Field("correlation_id", INT32, "The user-supplied value passed in with the request"));
+    private static final BoundField CORRELATION_KEY_FIELD = SCHEMA.get("correlation_id");
 
     private final int correlationId;
 
@@ -47,7 +48,7 @@ public class ResponseHeader extends AbstractRequestResponse {
     }
 
     public Struct toStruct() {
-        Struct struct = new Struct(Protocol.RESPONSE_HEADER);
+        Struct struct = new Struct(SCHEMA);
         struct.set(CORRELATION_KEY_FIELD, correlationId);
         return struct;
     }
@@ -57,7 +58,7 @@ public class ResponseHeader extends AbstractRequestResponse {
     }
 
     public static ResponseHeader parse(ByteBuffer buffer) {
-        return new ResponseHeader(Protocol.RESPONSE_HEADER.read(buffer));
+        return new ResponseHeader(SCHEMA.read(buffer));
     }
 
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateRequest.java
index f21c896..74d31a6 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateRequest.java
@@ -16,12 +16,15 @@
  */
 package org.apache.kafka.common.requests;
 
-import java.nio.ByteBuffer;
-
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
+import java.nio.ByteBuffer;
+
+import static org.apache.kafka.common.protocol.types.Type.BYTES;
 
 /**
  * Request from SASL client containing client SASL authentication token as defined by the
@@ -32,9 +35,15 @@ import org.apache.kafka.common.protocol.types.Struct;
  * brokers will send SaslHandshake request v0 followed by SASL tokens without the Kafka request headers.
  */
 public class SaslAuthenticateRequest extends AbstractRequest {
-
     private static final String SASL_AUTH_BYTES_KEY_NAME = "sasl_auth_bytes";
 
+    private static final Schema SASL_AUTHENTICATE_REQUEST_V0 = new Schema(
+            new Field(SASL_AUTH_BYTES_KEY_NAME, BYTES, "SASL authentication bytes from client as defined by the SASL mechanism."));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{SASL_AUTHENTICATE_REQUEST_V0};
+    }
+
     private final ByteBuffer saslAuthBytes;
 
     public static class Builder extends AbstractRequest.Builder<SaslAuthenticateRequest> {

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateResponse.java
index 2119f21..1dd0e76 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateResponse.java
@@ -16,23 +16,35 @@
  */
 package org.apache.kafka.common.requests;
 
-import java.nio.ByteBuffer;
-
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
+import java.nio.ByteBuffer;
+
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_MESSAGE;
+import static org.apache.kafka.common.protocol.types.Type.BYTES;
+
 
 /**
  * Response from SASL server which for a SASL challenge as defined by the SASL protocol
  * for the mechanism configured for the client.
  */
 public class SaslAuthenticateResponse extends AbstractResponse {
-
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
-    private static final String ERROR_MESSAGE_KEY_NAME = "error_message";
     private static final String SASL_AUTH_BYTES_KEY_NAME = "sasl_auth_bytes";
 
+    private static final Schema SASL_AUTHENTICATE_RESPONSE_V0 = new Schema(
+            ERROR_CODE,
+            ERROR_MESSAGE,
+            new Field(SASL_AUTH_BYTES_KEY_NAME, BYTES, "SASL authentication bytes from server as defined by the SASL mechanism."));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{SASL_AUTHENTICATE_RESPONSE_V0};
+    }
+
     private static final ByteBuffer EMPTY_BUFFER = ByteBuffer.allocate(0);
 
     private final ByteBuffer saslAuthBytes;
@@ -55,8 +67,8 @@ public class SaslAuthenticateResponse extends AbstractResponse {
     }
 
     public SaslAuthenticateResponse(Struct struct) {
-        error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME));
-        errorMessage = struct.getString(ERROR_MESSAGE_KEY_NAME);
+        error = Errors.forCode(struct.get(ERROR_CODE));
+        errorMessage = struct.get(ERROR_MESSAGE);
         saslAuthBytes = struct.getBytes(SASL_AUTH_BYTES_KEY_NAME);
     }
 
@@ -75,8 +87,8 @@ public class SaslAuthenticateResponse extends AbstractResponse {
     @Override
     public Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.SASL_AUTHENTICATE.responseSchema(version));
-        struct.set(ERROR_CODE_KEY_NAME, error.code());
-        struct.set(ERROR_MESSAGE_KEY_NAME, errorMessage);
+        struct.set(ERROR_CODE, error.code());
+        struct.set(ERROR_MESSAGE, errorMessage);
         struct.set(SASL_AUTH_BYTES_KEY_NAME, saslAuthBytes);
         return struct;
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java
----------------------------------------------------------------------
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
index 9906d13..a06a4db 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java
@@ -16,14 +16,17 @@
  */
 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.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.kafka.common.protocol.types.Type.STRING;
 
 /**
  * Request from SASL client containing client SASL mechanism.
@@ -35,8 +38,17 @@ import org.apache.kafka.common.protocol.types.Struct;
  * making it easy to distinguish from a GSSAPI packet.
  */
 public class SaslHandshakeRequest extends AbstractRequest {
+    private static final String MECHANISM_KEY_NAME = "mechanism";
+
+    private static final Schema SASL_HANDSHAKE_REQUEST_V0 = new Schema(
+            new Field("mechanism", STRING, "SASL Mechanism chosen by the client."));
 
-    public static final String MECHANISM_KEY_NAME = "mechanism";
+    // SASL_HANDSHAKE_REQUEST_V1 added to support SASL_AUTHENTICATE request to improve diagnostics
+    private static final Schema SASL_HANDSHAKE_REQUEST_V1 = SASL_HANDSHAKE_REQUEST_V0;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{SASL_HANDSHAKE_REQUEST_V0, SASL_HANDSHAKE_REQUEST_V1};
+    }
 
     private final String mechanism;
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java
----------------------------------------------------------------------
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
index e70bad3..c9f6369 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java
@@ -16,14 +16,20 @@
  */
 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.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+
 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.Errors;
-import org.apache.kafka.common.protocol.types.Struct;
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
 
 
 /**
@@ -31,10 +37,18 @@ import org.apache.kafka.common.protocol.types.Struct;
  * For error responses, the list of enabled mechanisms is included in the response.
  */
 public class SaslHandshakeResponse extends AbstractResponse {
-
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
     private static final String ENABLED_MECHANISMS_KEY_NAME = "enabled_mechanisms";
 
+    private static final Schema SASL_HANDSHAKE_RESPONSE_V0 = new Schema(
+            ERROR_CODE,
+            new Field(ENABLED_MECHANISMS_KEY_NAME, new ArrayOf(Type.STRING), "Array of mechanisms enabled in the server."));
+
+    private static final Schema SASL_HANDSHAKE_RESPONSE_V1 = SASL_HANDSHAKE_RESPONSE_V0;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{SASL_HANDSHAKE_RESPONSE_V0, SASL_HANDSHAKE_RESPONSE_V1};
+    }
+
     /**
      * Possible error codes:
      *   UNSUPPORTED_SASL_MECHANISM(33): Client mechanism not enabled in server
@@ -49,7 +63,7 @@ public class SaslHandshakeResponse extends AbstractResponse {
     }
 
     public SaslHandshakeResponse(Struct struct) {
-        error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME));
+        error = Errors.forCode(struct.get(ERROR_CODE));
         Object[] mechanisms = struct.getArray(ENABLED_MECHANISMS_KEY_NAME);
         ArrayList<String> enabledMechanisms = new ArrayList<>();
         for (Object mechanism : mechanisms)
@@ -64,7 +78,7 @@ public class SaslHandshakeResponse extends AbstractResponse {
     @Override
     public Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.SASL_HANDSHAKE.responseSchema(version));
-        struct.set(ERROR_CODE_KEY_NAME, error.code());
+        struct.set(ERROR_CODE, error.code());
         struct.set(ENABLED_MECHANISMS_KEY_NAME, enabledMechanisms.toArray());
         return struct;
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java
index 48aa16e..722a604 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java
@@ -19,6 +19,9 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.Utils;
 
@@ -30,13 +33,29 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.BOOLEAN;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+
 public class StopReplicaRequest extends AbstractRequest {
     private static final String CONTROLLER_ID_KEY_NAME = "controller_id";
     private static final String CONTROLLER_EPOCH_KEY_NAME = "controller_epoch";
     private static final String DELETE_PARTITIONS_KEY_NAME = "delete_partitions";
     private static final String PARTITIONS_KEY_NAME = "partitions";
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITION_KEY_NAME = "partition";
+
+    private static final Schema STOP_REPLICA_REQUEST_PARTITION_V0 = new Schema(
+            TOPIC_NAME,
+            PARTITION_ID);
+    private static final Schema STOP_REPLICA_REQUEST_V0 = new Schema(
+            new Field(CONTROLLER_ID_KEY_NAME, INT32, "The controller id."),
+            new Field(CONTROLLER_EPOCH_KEY_NAME, INT32, "The controller epoch."),
+            new Field(DELETE_PARTITIONS_KEY_NAME, BOOLEAN, "Boolean which indicates if replica's partitions must be deleted."),
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(STOP_REPLICA_REQUEST_PARTITION_V0)));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {STOP_REPLICA_REQUEST_V0};
+    }
 
     public static class Builder extends AbstractRequest.Builder<StopReplicaRequest> {
         private final int controllerId;
@@ -92,8 +111,8 @@ public class StopReplicaRequest extends AbstractRequest {
         partitions = new HashSet<>();
         for (Object partitionDataObj : struct.getArray(PARTITIONS_KEY_NAME)) {
             Struct partitionData = (Struct) partitionDataObj;
-            String topic = partitionData.getString(TOPIC_KEY_NAME);
-            int partition = partitionData.getInt(PARTITION_KEY_NAME);
+            String topic = partitionData.get(TOPIC_NAME);
+            int partition = partitionData.get(PARTITION_ID);
             partitions.add(new TopicPartition(topic, partition));
         }
 
@@ -150,8 +169,8 @@ public class StopReplicaRequest extends AbstractRequest {
         List<Struct> partitionDatas = new ArrayList<>(partitions.size());
         for (TopicPartition partition : partitions) {
             Struct partitionData = struct.instance(PARTITIONS_KEY_NAME);
-            partitionData.set(TOPIC_KEY_NAME, partition.topic());
-            partitionData.set(PARTITION_KEY_NAME, partition.partition());
+            partitionData.set(TOPIC_NAME, partition.topic());
+            partitionData.set(PARTITION_ID, partition.partition());
             partitionDatas.add(partitionData);
         }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java
index c859f7f..4196b83 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java
@@ -19,6 +19,9 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
@@ -27,14 +30,24 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-public class StopReplicaResponse extends AbstractResponse {
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
 
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
+public class StopReplicaResponse extends AbstractResponse {
     private static final String PARTITIONS_KEY_NAME = "partitions";
 
-    private static final String PARTITIONS_TOPIC_KEY_NAME = "topic";
-    private static final String PARTITIONS_PARTITION_KEY_NAME = "partition";
-    private static final String PARTITIONS_ERROR_CODE_KEY_NAME = "error_code";
+    private static final Schema STOP_REPLICA_RESPONSE_PARTITION_V0 = new Schema(
+            TOPIC_NAME,
+            PARTITION_ID,
+            ERROR_CODE);
+    private static final Schema STOP_REPLICA_RESPONSE_V0 = new Schema(
+            ERROR_CODE,
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(STOP_REPLICA_RESPONSE_PARTITION_V0)));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {STOP_REPLICA_RESPONSE_V0};
+    }
 
     private final Map<TopicPartition, Errors> responses;
 
@@ -54,13 +67,13 @@ public class StopReplicaResponse extends AbstractResponse {
         responses = new HashMap<>();
         for (Object responseDataObj : struct.getArray(PARTITIONS_KEY_NAME)) {
             Struct responseData = (Struct) responseDataObj;
-            String topic = responseData.getString(PARTITIONS_TOPIC_KEY_NAME);
-            int partition = responseData.getInt(PARTITIONS_PARTITION_KEY_NAME);
-            Errors error = Errors.forCode(responseData.getShort(PARTITIONS_ERROR_CODE_KEY_NAME));
+            String topic = responseData.get(TOPIC_NAME);
+            int partition = responseData.get(PARTITION_ID);
+            Errors error = Errors.forCode(responseData.get(ERROR_CODE));
             responses.put(new TopicPartition(topic, partition), error);
         }
 
-        error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME));
+        error = Errors.forCode(struct.get(ERROR_CODE));
     }
 
     public Map<TopicPartition, Errors> responses() {
@@ -83,15 +96,14 @@ public class StopReplicaResponse extends AbstractResponse {
         for (Map.Entry<TopicPartition, Errors> response : responses.entrySet()) {
             Struct partitionData = struct.instance(PARTITIONS_KEY_NAME);
             TopicPartition partition = response.getKey();
-            partitionData.set(PARTITIONS_TOPIC_KEY_NAME, partition.topic());
-            partitionData.set(PARTITIONS_PARTITION_KEY_NAME, partition.partition());
-            partitionData.set(PARTITIONS_ERROR_CODE_KEY_NAME, response.getValue().code());
+            partitionData.set(TOPIC_NAME, partition.topic());
+            partitionData.set(PARTITION_ID, partition.partition());
+            partitionData.set(ERROR_CODE, response.getValue().code());
             responseDatas.add(partitionData);
         }
 
         struct.set(PARTITIONS_KEY_NAME, responseDatas.toArray());
-        struct.set(ERROR_CODE_KEY_NAME, error.code());
-
+        struct.set(ERROR_CODE, error.code());
         return struct;
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java
index 82df84a..4ff9fcd 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java
@@ -18,6 +18,9 @@ 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.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.Utils;
 
@@ -27,12 +30,32 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.types.Type.BYTES;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class SyncGroupRequest extends AbstractRequest {
-    public static final String GROUP_ID_KEY_NAME = "group_id";
-    public static final String GENERATION_ID_KEY_NAME = "generation_id";
-    public static final String MEMBER_ID_KEY_NAME = "member_id";
-    public static final String MEMBER_ASSIGNMENT_KEY_NAME = "member_assignment";
-    public static final String GROUP_ASSIGNMENT_KEY_NAME = "group_assignment";
+    private static final String GROUP_ID_KEY_NAME = "group_id";
+    private static final String GENERATION_ID_KEY_NAME = "generation_id";
+    private static final String MEMBER_ID_KEY_NAME = "member_id";
+    private static final String MEMBER_ASSIGNMENT_KEY_NAME = "member_assignment";
+    private static final String GROUP_ASSIGNMENT_KEY_NAME = "group_assignment";
+
+    private static final Schema SYNC_GROUP_REQUEST_MEMBER_V0 = new Schema(
+            new Field(MEMBER_ID_KEY_NAME, STRING),
+            new Field(MEMBER_ASSIGNMENT_KEY_NAME, BYTES));
+    private static final Schema SYNC_GROUP_REQUEST_V0 = new Schema(
+            new Field(GROUP_ID_KEY_NAME, STRING),
+            new Field(GENERATION_ID_KEY_NAME, INT32),
+            new Field(MEMBER_ID_KEY_NAME, STRING),
+            new Field(GROUP_ASSIGNMENT_KEY_NAME, new ArrayOf(SYNC_GROUP_REQUEST_MEMBER_V0)));
+
+    /* v1 request is the same as v0. Throttle time has been added to response */
+    private static final Schema SYNC_GROUP_REQUEST_V1 = SYNC_GROUP_REQUEST_V0;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {SYNC_GROUP_REQUEST_V0, SYNC_GROUP_REQUEST_V1};
+    }
 
     public static class Builder extends AbstractRequest.Builder<SyncGroupRequest> {
         private final String groupId;

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java
index c96e21f..d68b2cd 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java
@@ -18,14 +18,30 @@ 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.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.types.Type.BYTES;
+
 public class SyncGroupResponse extends AbstractResponse {
+    private static final String MEMBER_ASSIGNMENT_KEY_NAME = "member_assignment";
+
+    private static final Schema SYNC_GROUP_RESPONSE_V0 = new Schema(
+            ERROR_CODE,
+            new Field(MEMBER_ASSIGNMENT_KEY_NAME, BYTES));
+    private static final Schema SYNC_GROUP_RESPONSE_V1 = new Schema(
+            THROTTLE_TIME_MS,
+            ERROR_CODE,
+            new Field(MEMBER_ASSIGNMENT_KEY_NAME, BYTES));
 
-    public static final String ERROR_CODE_KEY_NAME = "error_code";
-    public static final String MEMBER_ASSIGNMENT_KEY_NAME = "member_assignment";
+    public static Schema[] schemaVersions() {
+        return new Schema[] {SYNC_GROUP_RESPONSE_V0, SYNC_GROUP_RESPONSE_V1};
+    }
 
     /**
      * Possible error codes:
@@ -53,8 +69,8 @@ public class SyncGroupResponse extends AbstractResponse {
     }
 
     public SyncGroupResponse(Struct struct) {
-        this.throttleTimeMs = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME;
-        this.error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME));
+        this.throttleTimeMs = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME);
+        this.error = Errors.forCode(struct.get(ERROR_CODE));
         this.memberState = struct.getBytes(MEMBER_ASSIGNMENT_KEY_NAME);
     }
 
@@ -73,9 +89,8 @@ public class SyncGroupResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.SYNC_GROUP.responseSchema(version));
-        if (struct.hasField(THROTTLE_TIME_KEY_NAME))
-            struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
-        struct.set(ERROR_CODE_KEY_NAME, error.code());
+        struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs);
+        struct.set(ERROR_CODE, error.code());
         struct.set(MEMBER_ASSIGNMENT_KEY_NAME, memberState);
         return struct;
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java
index 20522af..d384192 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java
@@ -19,6 +19,9 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.CollectionUtils;
 
@@ -26,18 +29,42 @@ import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.INT16;
+import static org.apache.kafka.common.protocol.types.Type.INT64;
+import static org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class TxnOffsetCommitRequest extends AbstractRequest {
     private static final String TRANSACTIONAL_ID_KEY_NAME = "transactional_id";
     private static final String CONSUMER_GROUP_ID_KEY_NAME = "consumer_group_id";
     private static final String PRODUCER_ID_KEY_NAME = "producer_id";
     private static final String PRODUCER_EPOCH_KEY_NAME = "producer_epoch";
-    private static final String TOPIC_PARTITIONS_KEY_NAME = "topics";
-    private static final String TOPIC_KEY_NAME = "topic";
+    private static final String TOPICS_KEY_NAME = "topics";
     private static final String PARTITIONS_KEY_NAME = "partitions";
-    private static final String PARTITION_KEY_NAME = "partition";
     private static final String OFFSET_KEY_NAME = "offset";
     private static final String METADATA_KEY_NAME = "metadata";
 
+    private static final Schema TXN_OFFSET_COMMIT_PARTITION_OFFSET_METADATA_REQUEST_V0 = new Schema(
+            PARTITION_ID,
+            new Field(OFFSET_KEY_NAME, INT64),
+            new Field(METADATA_KEY_NAME, NULLABLE_STRING));
+
+    private static final Schema TXN_OFFSET_COMMIT_REQUEST_V0 = new Schema(
+            new Field(TRANSACTIONAL_ID_KEY_NAME, STRING, "The transactional id corresponding to the transaction."),
+            new Field(CONSUMER_GROUP_ID_KEY_NAME, STRING, "Id of the associated consumer group to commit offsets for."),
+            new Field(PRODUCER_ID_KEY_NAME, INT64, "Current producer id in use by the transactional id."),
+            new Field(PRODUCER_EPOCH_KEY_NAME, INT16, "Current epoch associated with the producer id."),
+            new Field(TOPICS_KEY_NAME, new ArrayOf(new Schema(
+                    TOPIC_NAME,
+                    new Field(PARTITIONS_KEY_NAME, new ArrayOf(TXN_OFFSET_COMMIT_PARTITION_OFFSET_METADATA_REQUEST_V0)))),
+                    "The partitions to write markers for."));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{TXN_OFFSET_COMMIT_REQUEST_V0};
+    }
+
     public static class Builder extends AbstractRequest.Builder<TxnOffsetCommitRequest> {
         private final String transactionalId;
         private final String consumerGroupId;
@@ -106,13 +133,13 @@ public class TxnOffsetCommitRequest extends AbstractRequest {
         this.producerEpoch = struct.getShort(PRODUCER_EPOCH_KEY_NAME);
 
         Map<TopicPartition, CommittedOffset> offsets = new HashMap<>();
-        Object[] topicPartitionsArray = struct.getArray(TOPIC_PARTITIONS_KEY_NAME);
+        Object[] topicPartitionsArray = struct.getArray(TOPICS_KEY_NAME);
         for (Object topicPartitionObj : topicPartitionsArray) {
             Struct topicPartitionStruct = (Struct) topicPartitionObj;
-            String topic = topicPartitionStruct.getString(TOPIC_KEY_NAME);
+            String topic = topicPartitionStruct.get(TOPIC_NAME);
             for (Object partitionObj : topicPartitionStruct.getArray(PARTITIONS_KEY_NAME)) {
                 Struct partitionStruct = (Struct) partitionObj;
-                TopicPartition partition = new TopicPartition(topic, partitionStruct.getInt(PARTITION_KEY_NAME));
+                TopicPartition partition = new TopicPartition(topic, partitionStruct.get(PARTITION_ID));
                 long offset = partitionStruct.getLong(OFFSET_KEY_NAME);
                 String metadata = partitionStruct.getString(METADATA_KEY_NAME);
                 offsets.put(partition, new CommittedOffset(offset, metadata));
@@ -153,15 +180,15 @@ public class TxnOffsetCommitRequest extends AbstractRequest {
         Object[] partitionsArray = new Object[mappedPartitionOffsets.size()];
         int i = 0;
         for (Map.Entry<String, Map<Integer, CommittedOffset>> topicAndPartitions : mappedPartitionOffsets.entrySet()) {
-            Struct topicPartitionsStruct = struct.instance(TOPIC_PARTITIONS_KEY_NAME);
-            topicPartitionsStruct.set(TOPIC_KEY_NAME, topicAndPartitions.getKey());
+            Struct topicPartitionsStruct = struct.instance(TOPICS_KEY_NAME);
+            topicPartitionsStruct.set(TOPIC_NAME, topicAndPartitions.getKey());
 
             Map<Integer, CommittedOffset> partitionOffsets = topicAndPartitions.getValue();
             Object[] partitionOffsetsArray = new Object[partitionOffsets.size()];
             int j = 0;
             for (Map.Entry<Integer, CommittedOffset> partitionOffset : partitionOffsets.entrySet()) {
                 Struct partitionOffsetStruct = topicPartitionsStruct.instance(PARTITIONS_KEY_NAME);
-                partitionOffsetStruct.set(PARTITION_KEY_NAME, partitionOffset.getKey());
+                partitionOffsetStruct.set(PARTITION_ID, partitionOffset.getKey());
                 CommittedOffset committedOffset = partitionOffset.getValue();
                 partitionOffsetStruct.set(OFFSET_KEY_NAME, committedOffset.offset);
                 partitionOffsetStruct.set(METADATA_KEY_NAME, committedOffset.metadata);
@@ -171,7 +198,7 @@ public class TxnOffsetCommitRequest extends AbstractRequest {
             partitionsArray[i++] = topicPartitionsStruct;
         }
 
-        struct.set(TOPIC_PARTITIONS_KEY_NAME, partitionsArray);
+        struct.set(TOPICS_KEY_NAME, partitionsArray);
         return struct;
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java
index 9a1cefa..53804d9 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java
@@ -19,6 +19,9 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.CollectionUtils;
 
@@ -26,12 +29,29 @@ import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+
 public class TxnOffsetCommitResponse extends AbstractResponse {
-    private static final String TOPIC_PARTITIONS_KEY_NAME = "topics";
+    private static final String TOPICS_KEY_NAME = "topics";
     private static final String PARTITIONS_KEY_NAME = "partitions";
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
+
+    private static final Schema TXN_OFFSET_COMMIT_PARTITION_ERROR_RESPONSE_V0 = new Schema(
+            PARTITION_ID,
+            ERROR_CODE);
+
+    private static final Schema TXN_OFFSET_COMMIT_RESPONSE_V0 = new Schema(
+            THROTTLE_TIME_MS,
+            new Field(TOPICS_KEY_NAME, new ArrayOf(new Schema(
+                    TOPIC_NAME,
+                    new Field(PARTITIONS_KEY_NAME, new ArrayOf(TXN_OFFSET_COMMIT_PARTITION_ERROR_RESPONSE_V0)))),
+                    "Errors per partition from writing markers."));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{TXN_OFFSET_COMMIT_RESPONSE_V0};
+    }
 
     // Possible error codes:
     //   InvalidProducerEpoch
@@ -53,16 +73,16 @@ public class TxnOffsetCommitResponse extends AbstractResponse {
     }
 
     public TxnOffsetCommitResponse(Struct struct) {
-        this.throttleTimeMs = struct.getInt(THROTTLE_TIME_KEY_NAME);
+        this.throttleTimeMs = struct.get(THROTTLE_TIME_MS);
         Map<TopicPartition, Errors> errors = new HashMap<>();
-        Object[] topicPartitionsArray = struct.getArray(TOPIC_PARTITIONS_KEY_NAME);
+        Object[] topicPartitionsArray = struct.getArray(TOPICS_KEY_NAME);
         for (Object topicPartitionObj : topicPartitionsArray) {
             Struct topicPartitionStruct = (Struct) topicPartitionObj;
-            String topic = topicPartitionStruct.getString(TOPIC_KEY_NAME);
+            String topic = topicPartitionStruct.get(TOPIC_NAME);
             for (Object partitionObj : topicPartitionStruct.getArray(PARTITIONS_KEY_NAME)) {
                 Struct partitionStruct = (Struct) partitionObj;
-                Integer partition = partitionStruct.getInt(PARTITION_KEY_NAME);
-                Errors error = Errors.forCode(partitionStruct.getShort(ERROR_CODE_KEY_NAME));
+                Integer partition = partitionStruct.get(PARTITION_ID);
+                Errors error = Errors.forCode(partitionStruct.get(ERROR_CODE));
                 errors.put(new TopicPartition(topic, partition), error);
             }
         }
@@ -72,28 +92,28 @@ public class TxnOffsetCommitResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.TXN_OFFSET_COMMIT.responseSchema(version));
-        struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
+        struct.set(THROTTLE_TIME_MS, throttleTimeMs);
         Map<String, Map<Integer, Errors>> mappedPartitions = CollectionUtils.groupDataByTopic(errors);
         Object[] partitionsArray = new Object[mappedPartitions.size()];
         int i = 0;
         for (Map.Entry<String, Map<Integer, Errors>> topicAndPartitions : mappedPartitions.entrySet()) {
-            Struct topicPartitionsStruct = struct.instance(TOPIC_PARTITIONS_KEY_NAME);
-            topicPartitionsStruct.set(TOPIC_KEY_NAME, topicAndPartitions.getKey());
+            Struct topicPartitionsStruct = struct.instance(TOPICS_KEY_NAME);
+            topicPartitionsStruct.set(TOPIC_NAME, topicAndPartitions.getKey());
             Map<Integer, Errors> partitionAndErrors = topicAndPartitions.getValue();
 
             Object[] partitionAndErrorsArray = new Object[partitionAndErrors.size()];
             int j = 0;
             for (Map.Entry<Integer, Errors> partitionAndError : partitionAndErrors.entrySet()) {
                 Struct partitionAndErrorStruct = topicPartitionsStruct.instance(PARTITIONS_KEY_NAME);
-                partitionAndErrorStruct.set(PARTITION_KEY_NAME, partitionAndError.getKey());
-                partitionAndErrorStruct.set(ERROR_CODE_KEY_NAME, partitionAndError.getValue().code());
+                partitionAndErrorStruct.set(PARTITION_ID, partitionAndError.getKey());
+                partitionAndErrorStruct.set(ERROR_CODE, partitionAndError.getValue().code());
                 partitionAndErrorsArray[j++] = partitionAndErrorStruct;
             }
             topicPartitionsStruct.set(PARTITIONS_KEY_NAME, partitionAndErrorsArray);
             partitionsArray[i++] = topicPartitionsStruct;
         }
 
-        struct.set(TOPIC_PARTITIONS_KEY_NAME, partitionsArray);
+        struct.set(TOPICS_KEY_NAME, partitionsArray);
         return struct;
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java
index 5f17d0b..67ae8e1 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java
@@ -22,6 +22,9 @@ import org.apache.kafka.common.network.ListenerName;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
 import org.apache.kafka.common.protocol.SecurityProtocol;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.Utils;
 
@@ -34,7 +37,136 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.INT16;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class UpdateMetadataRequest extends AbstractRequest {
+
+    private static final String CONTROLLER_ID_KEY_NAME = "controller_id";
+    private static final String CONTROLLER_EPOCH_KEY_NAME = "controller_epoch";
+    private static final String PARTITION_STATES_KEY_NAME = "partition_states";
+    private static final String LIVE_BROKERS_KEY_NAME = "live_brokers";
+
+    // PartitionState key names
+    private static final String LEADER_KEY_NAME = "leader";
+    private static final String LEADER_EPOCH_KEY_NAME = "leader_epoch";
+    private static final String ISR_KEY_NAME = "isr";
+    private static final String ZK_VERSION_KEY_NAME = "zk_version";
+    private static final String REPLICAS_KEY_NAME = "replicas";
+    private static final String OFFLINE_REPLICAS_KEY_NAME = "offline_replicas";
+
+    // Broker key names
+    private static final String BROKER_ID_KEY_NAME = "id";
+    private static final String ENDPOINTS_KEY_NAME = "end_points";
+    private static final String RACK_KEY_NAME = "rack";
+
+    // EndPoint key names
+    private static final String HOST_KEY_NAME = "host";
+    private static final String PORT_KEY_NAME = "port";
+    private static final String LISTENER_NAME_KEY_NAME = "listener_name";
+    private static final String SECURITY_PROTOCOL_TYPE_KEY_NAME = "security_protocol_type";
+
+    private static final Schema UPDATE_METADATA_REQUEST_PARTITION_STATE_V0 = new Schema(
+                    TOPIC_NAME,
+                    PARTITION_ID,
+                    new Field(CONTROLLER_EPOCH_KEY_NAME, INT32, "The controller epoch."),
+                    new Field(LEADER_KEY_NAME, INT32, "The broker id for the leader."),
+                    new Field(LEADER_EPOCH_KEY_NAME, INT32, "The leader epoch."),
+                    new Field(ISR_KEY_NAME, new ArrayOf(INT32), "The in sync replica ids."),
+                    new Field(ZK_VERSION_KEY_NAME, INT32, "The ZK version."),
+                    new Field(REPLICAS_KEY_NAME, new ArrayOf(INT32), "The replica ids."));
+
+    private static final Schema UPDATE_METADATA_REQUEST_BROKER_V0 = new Schema(
+            new Field(BROKER_ID_KEY_NAME, INT32, "The broker id."),
+            new Field(HOST_KEY_NAME, STRING, "The hostname of the broker."),
+            new Field(PORT_KEY_NAME, INT32, "The port on which the broker accepts requests."));
+
+    private static final Schema UPDATE_METADATA_REQUEST_V0 = new Schema(
+            new Field(CONTROLLER_ID_KEY_NAME, INT32, "The controller id."),
+            new Field(CONTROLLER_EPOCH_KEY_NAME, INT32, "The controller epoch."),
+            new Field(PARTITION_STATES_KEY_NAME, new ArrayOf(UPDATE_METADATA_REQUEST_PARTITION_STATE_V0)),
+            new Field(LIVE_BROKERS_KEY_NAME, new ArrayOf(UPDATE_METADATA_REQUEST_BROKER_V0)));
+
+    private static final Schema UPDATE_METADATA_REQUEST_PARTITION_STATE_V1 = UPDATE_METADATA_REQUEST_PARTITION_STATE_V0;
+
+    // for some reason, V1 sends `port` before `host` while V0 sends `host` before `port
+    private static final Schema UPDATE_METADATA_REQUEST_END_POINT_V1 = new Schema(
+            new Field(PORT_KEY_NAME, INT32, "The port on which the broker accepts requests."),
+            new Field(HOST_KEY_NAME, STRING, "The hostname of the broker."),
+            new Field(SECURITY_PROTOCOL_TYPE_KEY_NAME, INT16, "The security protocol type."));
+
+    private static final Schema UPDATE_METADATA_REQUEST_BROKER_V1 = new Schema(
+            new Field(BROKER_ID_KEY_NAME, INT32, "The broker id."),
+            new Field(ENDPOINTS_KEY_NAME, new ArrayOf(UPDATE_METADATA_REQUEST_END_POINT_V1)));
+
+    private static final Schema UPDATE_METADATA_REQUEST_V1 = new Schema(
+            new Field(CONTROLLER_ID_KEY_NAME, INT32, "The controller id."),
+            new Field(CONTROLLER_EPOCH_KEY_NAME, INT32, "The controller epoch."),
+            new Field(PARTITION_STATES_KEY_NAME, new ArrayOf(UPDATE_METADATA_REQUEST_PARTITION_STATE_V1)),
+            new Field(LIVE_BROKERS_KEY_NAME, new ArrayOf(UPDATE_METADATA_REQUEST_BROKER_V1)));
+
+    private static final Schema UPDATE_METADATA_REQUEST_PARTITION_STATE_V2 = UPDATE_METADATA_REQUEST_PARTITION_STATE_V1;
+
+    private static final Schema UPDATE_METADATA_REQUEST_END_POINT_V2 = UPDATE_METADATA_REQUEST_END_POINT_V1;
+
+    private static final Schema UPDATE_METADATA_REQUEST_BROKER_V2 = new Schema(
+            new Field(BROKER_ID_KEY_NAME, INT32, "The broker id."),
+            new Field(ENDPOINTS_KEY_NAME, new ArrayOf(UPDATE_METADATA_REQUEST_END_POINT_V2)),
+            new Field(RACK_KEY_NAME, NULLABLE_STRING, "The rack"));
+
+    private static final Schema UPDATE_METADATA_REQUEST_V2 = new Schema(
+            new Field(CONTROLLER_ID_KEY_NAME, INT32, "The controller id."),
+            new Field(CONTROLLER_EPOCH_KEY_NAME, INT32, "The controller epoch."),
+            new Field(PARTITION_STATES_KEY_NAME, new ArrayOf(UPDATE_METADATA_REQUEST_PARTITION_STATE_V2)),
+            new Field(LIVE_BROKERS_KEY_NAME, new ArrayOf(UPDATE_METADATA_REQUEST_BROKER_V2)));
+
+    private static final Schema UPDATE_METADATA_REQUEST_PARTITION_STATE_V3 = UPDATE_METADATA_REQUEST_PARTITION_STATE_V2;
+
+    // UPDATE_METADATA_REQUEST_PARTITION_STATE_V4 added a per-partition offline_replicas field. This field specifies the list of replicas that are offline.
+    private static final Schema UPDATE_METADATA_REQUEST_PARTITION_STATE_V4 = new Schema(
+            TOPIC_NAME,
+            PARTITION_ID,
+            new Field(CONTROLLER_EPOCH_KEY_NAME, INT32, "The controller epoch."),
+            new Field(LEADER_KEY_NAME, INT32, "The broker id for the leader."),
+            new Field(LEADER_EPOCH_KEY_NAME, INT32, "The leader epoch."),
+            new Field(ISR_KEY_NAME, new ArrayOf(INT32), "The in sync replica ids."),
+            new Field(ZK_VERSION_KEY_NAME, INT32, "The ZK version."),
+            new Field(REPLICAS_KEY_NAME, new ArrayOf(INT32), "The replica ids."),
+            new Field(OFFLINE_REPLICAS_KEY_NAME, new ArrayOf(INT32), "The offline replica ids"));
+
+    private static final Schema UPDATE_METADATA_REQUEST_END_POINT_V3 = new Schema(
+            new Field(PORT_KEY_NAME, INT32, "The port on which the broker accepts requests."),
+            new Field(HOST_KEY_NAME, STRING, "The hostname of the broker."),
+            new Field(LISTENER_NAME_KEY_NAME, STRING, "The listener name."),
+            new Field(SECURITY_PROTOCOL_TYPE_KEY_NAME, INT16, "The security protocol type."));
+
+    private static final Schema UPDATE_METADATA_REQUEST_BROKER_V3 = new Schema(
+            new Field(BROKER_ID_KEY_NAME, INT32, "The broker id."),
+            new Field(ENDPOINTS_KEY_NAME, new ArrayOf(UPDATE_METADATA_REQUEST_END_POINT_V3)),
+            new Field(RACK_KEY_NAME, NULLABLE_STRING, "The rack"));
+
+    private static final Schema UPDATE_METADATA_REQUEST_V3 = new Schema(
+            new Field(CONTROLLER_ID_KEY_NAME, INT32, "The controller id."),
+            new Field(CONTROLLER_EPOCH_KEY_NAME, INT32, "The controller epoch."),
+            new Field(PARTITION_STATES_KEY_NAME, new ArrayOf(UPDATE_METADATA_REQUEST_PARTITION_STATE_V3)),
+            new Field(LIVE_BROKERS_KEY_NAME, new ArrayOf(UPDATE_METADATA_REQUEST_BROKER_V3)));
+
+    // UPDATE_METADATA_REQUEST_V4 added a per-partition offline_replicas field. This field specifies the list of replicas that are offline.
+    private static final Schema UPDATE_METADATA_REQUEST_V4 = new Schema(
+            new Field(CONTROLLER_ID_KEY_NAME, INT32, "The controller id."),
+            new Field(CONTROLLER_EPOCH_KEY_NAME, INT32, "The controller epoch."),
+            new Field(PARTITION_STATES_KEY_NAME, new ArrayOf(UPDATE_METADATA_REQUEST_PARTITION_STATE_V4)),
+            new Field(LIVE_BROKERS_KEY_NAME, new ArrayOf(UPDATE_METADATA_REQUEST_BROKER_V3)));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {UPDATE_METADATA_REQUEST_V0, UPDATE_METADATA_REQUEST_V1, UPDATE_METADATA_REQUEST_V2,
+            UPDATE_METADATA_REQUEST_V3, UPDATE_METADATA_REQUEST_V4};
+    }
+
     public static class Builder extends AbstractRequest.Builder<UpdateMetadataRequest> {
         private final int controllerId;
         private final int controllerEpoch;
@@ -43,7 +175,7 @@ public class UpdateMetadataRequest extends AbstractRequest {
 
         public Builder(short version, int controllerId, int controllerEpoch,
                        Map<TopicPartition, PartitionState> partitionStates, Set<Broker> liveBrokers) {
-            super(ApiKeys.UPDATE_METADATA_KEY, version);
+            super(ApiKeys.UPDATE_METADATA, version);
             this.controllerId = controllerId;
             this.controllerEpoch = controllerEpoch;
             this.partitionStates = partitionStates;
@@ -144,32 +276,6 @@ public class UpdateMetadataRequest extends AbstractRequest {
         }
     }
 
-    private static final String CONTROLLER_ID_KEY_NAME = "controller_id";
-    private static final String CONTROLLER_EPOCH_KEY_NAME = "controller_epoch";
-    private static final String PARTITION_STATES_KEY_NAME = "partition_states";
-    private static final String LIVE_BROKERS_KEY_NAME = "live_brokers";
-
-    // PartitionState key names
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String LEADER_KEY_NAME = "leader";
-    private static final String LEADER_EPOCH_KEY_NAME = "leader_epoch";
-    private static final String ISR_KEY_NAME = "isr";
-    private static final String ZK_VERSION_KEY_NAME = "zk_version";
-    private static final String REPLICAS_KEY_NAME = "replicas";
-    private static final String OFFLINE_REPLICAS_KEY_NAME = "offline_replicas";
-
-    // Broker key names
-    private static final String BROKER_ID_KEY_NAME = "id";
-    private static final String ENDPOINTS_KEY_NAME = "end_points";
-    private static final String RACK_KEY_NAME = "rack";
-
-    // EndPoint key names
-    private static final String HOST_KEY_NAME = "host";
-    private static final String PORT_KEY_NAME = "port";
-    private static final String LISTENER_NAME_KEY_NAME = "listener_name";
-    private static final String SECURITY_PROTOCOL_TYPE_KEY_NAME = "security_protocol_type";
-
     private final int controllerId;
     private final int controllerEpoch;
     private final Map<TopicPartition, PartitionState> partitionStates;
@@ -189,8 +295,8 @@ public class UpdateMetadataRequest extends AbstractRequest {
         Map<TopicPartition, PartitionState> partitionStates = new HashMap<>();
         for (Object partitionStateDataObj : struct.getArray(PARTITION_STATES_KEY_NAME)) {
             Struct partitionStateData = (Struct) partitionStateDataObj;
-            String topic = partitionStateData.getString(TOPIC_KEY_NAME);
-            int partition = partitionStateData.getInt(PARTITION_KEY_NAME);
+            String topic = partitionStateData.get(TOPIC_NAME);
+            int partition = partitionStateData.get(PARTITION_ID);
             int controllerEpoch = partitionStateData.getInt(CONTROLLER_EPOCH_KEY_NAME);
             int leader = partitionStateData.getInt(LEADER_KEY_NAME);
             int leaderEpoch = partitionStateData.getInt(LEADER_EPOCH_KEY_NAME);
@@ -264,7 +370,7 @@ public class UpdateMetadataRequest extends AbstractRequest {
     @Override
     protected Struct toStruct() {
         short version = version();
-        Struct struct = new Struct(ApiKeys.UPDATE_METADATA_KEY.requestSchema(version));
+        Struct struct = new Struct(ApiKeys.UPDATE_METADATA.requestSchema(version));
         struct.set(CONTROLLER_ID_KEY_NAME, controllerId);
         struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch);
 
@@ -272,8 +378,8 @@ public class UpdateMetadataRequest extends AbstractRequest {
         for (Map.Entry<TopicPartition, PartitionState> entry : partitionStates.entrySet()) {
             Struct partitionStateData = struct.instance(PARTITION_STATES_KEY_NAME);
             TopicPartition topicPartition = entry.getKey();
-            partitionStateData.set(TOPIC_KEY_NAME, topicPartition.topic());
-            partitionStateData.set(PARTITION_KEY_NAME, topicPartition.partition());
+            partitionStateData.set(TOPIC_NAME, topicPartition.topic());
+            partitionStateData.set(PARTITION_ID, topicPartition.partition());
             PartitionState partitionState = entry.getValue();
             partitionStateData.set(CONTROLLER_EPOCH_KEY_NAME, partitionState.basePartitionState.controllerEpoch);
             partitionStateData.set(LEADER_KEY_NAME, partitionState.basePartitionState.leader);
@@ -328,7 +434,7 @@ public class UpdateMetadataRequest extends AbstractRequest {
             return new UpdateMetadataResponse(Errors.forException(e));
         else
             throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
-                    versionId, this.getClass().getSimpleName(), ApiKeys.UPDATE_METADATA_KEY.latestVersion()));
+                    versionId, this.getClass().getSimpleName(), ApiKeys.UPDATE_METADATA.latestVersion()));
     }
 
     public int controllerId() {
@@ -348,7 +454,7 @@ public class UpdateMetadataRequest extends AbstractRequest {
     }
 
     public static UpdateMetadataRequest parse(ByteBuffer buffer, short version) {
-        return new UpdateMetadataRequest(ApiKeys.UPDATE_METADATA_KEY.parseRequest(version, buffer), version);
+        return new UpdateMetadataRequest(ApiKeys.UPDATE_METADATA.parseRequest(version, buffer), version);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java
index 4eae39e..9ff8e27 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java
@@ -18,13 +18,24 @@ 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.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+
 public class UpdateMetadataResponse extends AbstractResponse {
+    private static final Schema UPDATE_METADATA_RESPONSE_V0 = new Schema(ERROR_CODE);
+    private static final Schema UPDATE_METADATA_RESPONSE_V1 = UPDATE_METADATA_RESPONSE_V0;
+    private static final Schema UPDATE_METADATA_RESPONSE_V2 = UPDATE_METADATA_RESPONSE_V1;
+    private static final Schema UPDATE_METADATA_RESPONSE_V3 = UPDATE_METADATA_RESPONSE_V2;
+    private static final Schema UPDATE_METADATA_RESPONSE_V4 = UPDATE_METADATA_RESPONSE_V3;
 
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
+    public static Schema[] schemaVersions() {
+        return new Schema[]{UPDATE_METADATA_RESPONSE_V0, UPDATE_METADATA_RESPONSE_V1, UPDATE_METADATA_RESPONSE_V2,
+            UPDATE_METADATA_RESPONSE_V3, UPDATE_METADATA_RESPONSE_V4};
+    }
 
     /**
      * Possible error code:
@@ -38,7 +49,7 @@ public class UpdateMetadataResponse extends AbstractResponse {
     }
 
     public UpdateMetadataResponse(Struct struct) {
-        error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME));
+        error = Errors.forCode(struct.get(ERROR_CODE));
     }
 
     public Errors error() {
@@ -46,13 +57,13 @@ public class UpdateMetadataResponse extends AbstractResponse {
     }
 
     public static UpdateMetadataResponse parse(ByteBuffer buffer, short version) {
-        return new UpdateMetadataResponse(ApiKeys.UPDATE_METADATA_KEY.parseResponse(version, buffer));
+        return new UpdateMetadataResponse(ApiKeys.UPDATE_METADATA.parseResponse(version, buffer));
     }
 
     @Override
     protected Struct toStruct(short version) {
-        Struct struct = new Struct(ApiKeys.UPDATE_METADATA_KEY.responseSchema(version));
-        struct.set(ERROR_CODE_KEY_NAME, error.code());
+        Struct struct = new Struct(ApiKeys.UPDATE_METADATA.responseSchema(version));
+        struct.set(ERROR_CODE, error.code());
         return struct;
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersRequest.java
index cf2c9fc..96dfb2f 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersRequest.java
@@ -19,6 +19,9 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.CollectionUtils;
 
@@ -29,6 +32,12 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.BOOLEAN;
+import static org.apache.kafka.common.protocol.types.Type.INT16;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.INT64;
+
 public class WriteTxnMarkersRequest extends AbstractRequest {
     private static final String COORDINATOR_EPOCH_KEY_NAME = "coordinator_epoch";
     private static final String TXN_MARKER_ENTRY_KEY_NAME = "transaction_markers";
@@ -36,10 +45,27 @@ public class WriteTxnMarkersRequest extends AbstractRequest {
     private static final String PRODUCER_ID_KEY_NAME = "producer_id";
     private static final String PRODUCER_EPOCH_KEY_NAME = "producer_epoch";
     private static final String TRANSACTION_RESULT_KEY_NAME = "transaction_result";
-    private static final String TOPIC_PARTITIONS_KEY_NAME = "topics";
-    private static final String TOPIC_KEY_NAME = "topic";
+    private static final String TOPICS_KEY_NAME = "topics";
     private static final String PARTITIONS_KEY_NAME = "partitions";
 
+    private static final Schema WRITE_TXN_MARKERS_ENTRY_V0 = new Schema(
+            new Field(PRODUCER_ID_KEY_NAME, INT64, "Current producer id in use by the transactional id."),
+            new Field(PRODUCER_EPOCH_KEY_NAME, INT16, "Current epoch associated with the producer id."),
+            new Field(TRANSACTION_RESULT_KEY_NAME, BOOLEAN, "The result of the transaction to write to the " +
+                    "partitions (false = ABORT, true = COMMIT)."),
+            new Field(TOPICS_KEY_NAME, new ArrayOf(new Schema(
+                    TOPIC_NAME,
+                    new Field(PARTITIONS_KEY_NAME, new ArrayOf(INT32)))), "The partitions to write markers for."),
+            new Field(COORDINATOR_EPOCH_KEY_NAME, INT32, "Epoch associated with the transaction state partition " +
+                    "hosted by this transaction coordinator"));
+
+    private static final Schema WRITE_TXN_MARKERS_REQUEST_V0 = new Schema(
+            new Field(TXN_MARKER_ENTRY_KEY_NAME, new ArrayOf(WRITE_TXN_MARKERS_ENTRY_V0), "The transaction markers to be written."));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{WRITE_TXN_MARKERS_REQUEST_V0};
+    }
+
     public static class TxnMarkerEntry {
         private final long producerId;
         private final short producerEpoch;
@@ -144,10 +170,10 @@ public class WriteTxnMarkersRequest extends AbstractRequest {
             TransactionResult result = TransactionResult.forId(markerStruct.getBoolean(TRANSACTION_RESULT_KEY_NAME));
 
             List<TopicPartition> partitions = new ArrayList<>();
-            Object[] topicPartitionsArray = markerStruct.getArray(TOPIC_PARTITIONS_KEY_NAME);
+            Object[] topicPartitionsArray = markerStruct.getArray(TOPICS_KEY_NAME);
             for (Object topicPartitionObj : topicPartitionsArray) {
                 Struct topicPartitionStruct = (Struct) topicPartitionObj;
-                String topic = topicPartitionStruct.getString(TOPIC_KEY_NAME);
+                String topic = topicPartitionStruct.get(TOPIC_NAME);
                 for (Object partitionObj : topicPartitionStruct.getArray(PARTITIONS_KEY_NAME)) {
                     partitions.add(new TopicPartition(topic, (Integer) partitionObj));
                 }
@@ -181,12 +207,12 @@ public class WriteTxnMarkersRequest extends AbstractRequest {
             Object[] partitionsArray = new Object[mappedPartitions.size()];
             int j = 0;
             for (Map.Entry<String, List<Integer>> topicAndPartitions : mappedPartitions.entrySet()) {
-                Struct topicPartitionsStruct = markerStruct.instance(TOPIC_PARTITIONS_KEY_NAME);
-                topicPartitionsStruct.set(TOPIC_KEY_NAME, topicAndPartitions.getKey());
+                Struct topicPartitionsStruct = markerStruct.instance(TOPICS_KEY_NAME);
+                topicPartitionsStruct.set(TOPIC_NAME, topicAndPartitions.getKey());
                 topicPartitionsStruct.set(PARTITIONS_KEY_NAME, topicAndPartitions.getValue().toArray());
                 partitionsArray[j++] = topicPartitionsStruct;
             }
-            markerStruct.set(TOPIC_PARTITIONS_KEY_NAME, partitionsArray);
+            markerStruct.set(TOPICS_KEY_NAME, partitionsArray);
             markersArray[i++] = markerStruct;
         }
         struct.set(TXN_MARKER_ENTRY_KEY_NAME, markersArray);

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java
index ddddc42..3372670 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java
@@ -19,6 +19,9 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.CollectionUtils;
 
@@ -26,15 +29,35 @@ import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.INT64;
+
 public class WriteTxnMarkersResponse extends AbstractResponse {
     private static final String TXN_MARKER_ENTRY_KEY_NAME = "transaction_markers";
 
     private static final String PRODUCER_ID_KEY_NAME = "producer_id";
-    private static final String TOPIC_PARTITIONS_KEY_NAME = "topics";
+    private static final String TOPICS_KEY_NAME = "topics";
     private static final String PARTITIONS_KEY_NAME = "partitions";
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
+
+    private static final Schema WRITE_TXN_MARKERS_PARTITION_ERROR_RESPONSE_V0 = new Schema(
+            PARTITION_ID,
+            ERROR_CODE);
+
+    private static final Schema WRITE_TXN_MARKERS_ENTRY_RESPONSE_V0 = new Schema(
+            new Field(PRODUCER_ID_KEY_NAME, INT64, "Current producer id in use by the transactional id."),
+            new Field(TOPICS_KEY_NAME, new ArrayOf(new Schema(
+                    TOPIC_NAME,
+                    new Field(PARTITIONS_KEY_NAME, new ArrayOf(WRITE_TXN_MARKERS_PARTITION_ERROR_RESPONSE_V0)))),
+                    "Errors per partition from writing markers."));
+
+    private static final Schema WRITE_TXN_MARKERS_RESPONSE_V0 = new Schema(
+            new Field("transaction_markers", new ArrayOf(WRITE_TXN_MARKERS_ENTRY_RESPONSE_V0), "Errors per partition from writing markers."));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{WRITE_TXN_MARKERS_RESPONSE_V0};
+    }
 
     // Possible error codes:
     //   CorruptRecord
@@ -66,14 +89,14 @@ public class WriteTxnMarkersResponse extends AbstractResponse {
             long producerId = responseStruct.getLong(PRODUCER_ID_KEY_NAME);
 
             Map<TopicPartition, Errors> errorPerPartition = new HashMap<>();
-            Object[] topicPartitionsArray = responseStruct.getArray(TOPIC_PARTITIONS_KEY_NAME);
+            Object[] topicPartitionsArray = responseStruct.getArray(TOPICS_KEY_NAME);
             for (Object topicPartitionObj : topicPartitionsArray) {
                 Struct topicPartitionStruct = (Struct) topicPartitionObj;
-                String topic = topicPartitionStruct.getString(TOPIC_KEY_NAME);
+                String topic = topicPartitionStruct.get(TOPIC_NAME);
                 for (Object partitionObj : topicPartitionStruct.getArray(PARTITIONS_KEY_NAME)) {
                     Struct partitionStruct = (Struct) partitionObj;
-                    Integer partition = partitionStruct.getInt(PARTITION_KEY_NAME);
-                    Errors error = Errors.forCode(partitionStruct.getShort(ERROR_CODE_KEY_NAME));
+                    Integer partition = partitionStruct.get(PARTITION_ID);
+                    Errors error = Errors.forCode(partitionStruct.get(ERROR_CODE));
                     errorPerPartition.put(new TopicPartition(topic, partition), error);
                 }
             }
@@ -98,22 +121,22 @@ public class WriteTxnMarkersResponse extends AbstractResponse {
             Object[] partitionsArray = new Object[mappedPartitions.size()];
             int i = 0;
             for (Map.Entry<String, Map<Integer, Errors>> topicAndPartitions : mappedPartitions.entrySet()) {
-                Struct topicPartitionsStruct = responseStruct.instance(TOPIC_PARTITIONS_KEY_NAME);
-                topicPartitionsStruct.set(TOPIC_KEY_NAME, topicAndPartitions.getKey());
+                Struct topicPartitionsStruct = responseStruct.instance(TOPICS_KEY_NAME);
+                topicPartitionsStruct.set(TOPIC_NAME, topicAndPartitions.getKey());
                 Map<Integer, Errors> partitionIdAndErrors = topicAndPartitions.getValue();
 
                 Object[] partitionAndErrorsArray = new Object[partitionIdAndErrors.size()];
                 int j = 0;
                 for (Map.Entry<Integer, Errors> partitionAndError : partitionIdAndErrors.entrySet()) {
                     Struct partitionAndErrorStruct = topicPartitionsStruct.instance(PARTITIONS_KEY_NAME);
-                    partitionAndErrorStruct.set(PARTITION_KEY_NAME, partitionAndError.getKey());
-                    partitionAndErrorStruct.set(ERROR_CODE_KEY_NAME, partitionAndError.getValue().code());
+                    partitionAndErrorStruct.set(PARTITION_ID, partitionAndError.getKey());
+                    partitionAndErrorStruct.set(ERROR_CODE, partitionAndError.getValue().code());
                     partitionAndErrorsArray[j++] = partitionAndErrorStruct;
                 }
                 topicPartitionsStruct.set(PARTITIONS_KEY_NAME, partitionAndErrorsArray);
                 partitionsArray[i++] = topicPartitionsStruct;
             }
-            responseStruct.set(TOPIC_PARTITIONS_KEY_NAME, partitionsArray);
+            responseStruct.set(TOPICS_KEY_NAME, partitionsArray);
 
             responsesArray[k++] = responseStruct;
         }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java
----------------------------------------------------------------------
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 ce6a6b0..46386cf 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
@@ -35,7 +35,6 @@ 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.SecurityProtocol;
-import org.apache.kafka.common.protocol.Protocol;
 import org.apache.kafka.common.requests.AbstractResponse;
 import org.apache.kafka.common.requests.ApiVersionsRequest;
 import org.apache.kafka.common.requests.ApiVersionsResponse;
@@ -372,7 +371,7 @@ public class SaslServerAuthenticator implements Authenticator {
                 sendKafkaResponse(requestContext, requestAndSize.request.getErrorResponse(e));
                 throw e;
             }
-            if (!Protocol.apiVersionSupported(apiKey.id, version)) {
+            if (!apiKey.isVersionSupported(version)) {
                 this.error = Errors.UNSUPPORTED_VERSION;
                 // We cannot create an error response if the request version of SaslAuthenticate is not supported
                 // This should not normally occur since clients typically check supported versions using ApiVersionsRequest
@@ -471,7 +470,7 @@ public class SaslServerAuthenticator implements Authenticator {
 
     // Visible to override for testing
     protected ApiVersionsResponse apiVersionsResponse() {
-        return ApiVersionsResponse.API_VERSIONS_RESPONSE;
+        return ApiVersionsResponse.defaultApiVersionsResponse();
     }
 
     // Visible to override for testing

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
index 4c3019c..86ad2ff 100644
--- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
@@ -161,8 +161,9 @@ public class NetworkClientTest {
     }
 
     private void maybeSetExpectedApiVersionsResponse() {
-        short apiVersionsResponseVersion = ApiVersionsResponse.API_VERSIONS_RESPONSE.apiVersion(ApiKeys.API_VERSIONS.id).maxVersion;
-        ByteBuffer buffer = ApiVersionsResponse.API_VERSIONS_RESPONSE.serialize(apiVersionsResponseVersion, new ResponseHeader(0));
+        ApiVersionsResponse response = ApiVersionsResponse.defaultApiVersionsResponse();
+        short apiVersionsResponseVersion = response.apiVersion(ApiKeys.API_VERSIONS.id).maxVersion;
+        ByteBuffer buffer = response.serialize(apiVersionsResponseVersion, new ResponseHeader(0));
         selector.delayedReceive(new DelayedReceive(node.idString(), new NetworkReceive(node.idString(), buffer)));
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java b/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java
index 41ac42a..6b1d92a 100644
--- a/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java
@@ -138,7 +138,7 @@ public class NodeApiVersionsTest {
     @Test
     public void testUsableVersionLatestVersions() {
         List<ApiVersion> versionList = new LinkedList<>();
-        for (ApiVersion apiVersion: ApiVersionsResponse.API_VERSIONS_RESPONSE.apiVersions()) {
+        for (ApiVersion apiVersion: ApiVersionsResponse.defaultApiVersionsResponse().apiVersions()) {
             versionList.add(apiVersion);
         }
         // Add an API key that we don't know about.

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java
index d68e676..a1dd775 100644
--- a/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java
@@ -16,16 +16,16 @@
  */
 package org.apache.kafka.common.protocol;
 
+import org.apache.kafka.common.protocol.types.BoundField;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.junit.Test;
+
 import java.util.Arrays;
 import java.util.List;
 
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 
-import org.apache.kafka.common.protocol.types.Field;
-import org.apache.kafka.common.protocol.types.Schema;
-import org.junit.Test;
-
 public class ApiKeysTest {
 
     @Test(expected = IllegalArgumentException.class)
@@ -40,7 +40,7 @@ public class ApiKeysTest {
 
     @Test(expected = IllegalArgumentException.class)
     public void schemaVersionOutOfRange() {
-        ApiKeys.PRODUCE.requestSchema((short) Protocol.REQUESTS[ApiKeys.PRODUCE.id].length);
+        ApiKeys.PRODUCE.requestSchema((short) ApiKeys.PRODUCE.requestSchemas.length);
     }
 
     /**
@@ -56,10 +56,9 @@ public class ApiKeysTest {
     @Test
     public void testResponseThrottleTime() {
         List<ApiKeys> authenticationKeys = Arrays.asList(ApiKeys.SASL_HANDSHAKE, ApiKeys.SASL_AUTHENTICATE);
-
         for (ApiKeys apiKey: ApiKeys.values()) {
             Schema responseSchema = apiKey.responseSchema(apiKey.latestVersion());
-            Field throttleTimeField = responseSchema.get("throttle_time_ms");
+            BoundField throttleTimeField = responseSchema.get(CommonFields.THROTTLE_TIME_MS.name);
             if (apiKey.clusterAction || authenticationKeys.contains(apiKey))
                 assertNull("Unexpected throttle time field: " + apiKey, throttleTimeField);
             else

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/test/java/org/apache/kafka/common/protocol/ProtoUtilsTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/ProtoUtilsTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/ProtoUtilsTest.java
index 8cb6b80..c4b3fc4 100644
--- a/clients/src/test/java/org/apache/kafka/common/protocol/ProtoUtilsTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/protocol/ProtoUtilsTest.java
@@ -16,20 +16,21 @@
  */
 package org.apache.kafka.common.protocol;
 
-import org.junit.Assert;
 import org.junit.Test;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 public class ProtoUtilsTest {
     @Test
     public void testDelayedAllocationSchemaDetection() throws Exception {
         //verifies that schemas known to retain a reference to the underlying byte buffer are correctly detected.
         for (ApiKeys key : ApiKeys.values()) {
             if (key == ApiKeys.PRODUCE || key == ApiKeys.JOIN_GROUP || key == ApiKeys.SYNC_GROUP || key == ApiKeys.SASL_AUTHENTICATE) {
-                Assert.assertTrue(Protocol.requiresDelayedDeallocation(key.id));
+                assertTrue(key + " should require delayed allocation", key.requiresDelayedAllocation);
             } else {
-                Assert.assertFalse(Protocol.requiresDelayedDeallocation(key.id));
+                assertFalse(key + " should not require delayed allocation", key.requiresDelayedAllocation);
             }
         }
-
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java
----------------------------------------------------------------------
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 136b55a..6e9341a 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
@@ -16,17 +16,17 @@
  */
 package org.apache.kafka.common.protocol.types;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
+import org.junit.Before;
+import org.junit.Test;
 
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 
-import org.junit.Before;
-import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
 
 public class ProtocolSerializationTest {
 
@@ -96,15 +96,15 @@ public class ProtocolSerializationTest {
 
     @Test
     public void testNulls() {
-        for (Field f : this.schema.fields()) {
+        for (BoundField f : this.schema.fields()) {
             Object o = this.struct.get(f);
             try {
                 this.struct.set(f, null);
                 this.struct.validate();
-                if (!f.type.isNullable())
+                if (!f.def.type.isNullable())
                     fail("Should not allow serialization of null value.");
             } catch (SchemaException e) {
-                assertFalse(f.type.isNullable());
+                assertFalse(f.def.type.isNullable());
             } finally {
                 this.struct.set(f, o);
             }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java
index 1e8e3b4..6d488cd 100644
--- a/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java
@@ -40,7 +40,7 @@ public class ApiVersionsResponseTest {
 
     @Test
     public void shouldCreateApiResponseThatHasAllApiKeysSupportedByBroker() throws Exception {
-        assertEquals(apiKeysInResponse(ApiVersionsResponse.API_VERSIONS_RESPONSE), Utils.mkSet(ApiKeys.values()));
+        assertEquals(apiKeysInResponse(ApiVersionsResponse.defaultApiVersionsResponse()), Utils.mkSet(ApiKeys.values()));
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/test/java/org/apache/kafka/common/requests/RequestHeaderTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestHeaderTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestHeaderTest.java
index bc3bd37..f73ee2f 100644
--- a/clients/src/test/java/org/apache/kafka/common/requests/RequestHeaderTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestHeaderTest.java
@@ -33,13 +33,13 @@ public class RequestHeaderTest {
 
         int correlationId = 2342;
         ByteBuffer rawBuffer = ByteBuffer.allocate(32);
-        rawBuffer.putShort(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id);
+        rawBuffer.putShort(ApiKeys.CONTROLLED_SHUTDOWN.id);
         rawBuffer.putShort((short) 0);
         rawBuffer.putInt(correlationId);
         rawBuffer.flip();
 
         RequestHeader deserialized = RequestHeader.parse(rawBuffer);
-        assertEquals(ApiKeys.CONTROLLED_SHUTDOWN_KEY, deserialized.apiKey());
+        assertEquals(ApiKeys.CONTROLLED_SHUTDOWN, deserialized.apiKey());
         assertEquals(0, deserialized.apiVersion());
         assertEquals(correlationId, deserialized.correlationId());
         assertEquals("", deserialized.clientId());
@@ -47,7 +47,7 @@ public class RequestHeaderTest {
         Struct serialized = deserialized.toStruct();
         ByteBuffer serializedBuffer = toBuffer(serialized);
 
-        assertEquals(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id, serializedBuffer.getShort(0));
+        assertEquals(ApiKeys.CONTROLLED_SHUTDOWN.id, serializedBuffer.getShort(0));
         assertEquals(0, serializedBuffer.getShort(2));
         assertEquals(correlationId, serializedBuffer.getInt(4));
         assertEquals(8, serializedBuffer.limit());


[3/8] kafka git commit: MINOR: Move request/response schemas to the corresponding object representation

Posted by ij...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java
index 8fe8ba7..732fb49 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java
@@ -19,6 +19,9 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.CollectionUtils;
 import org.apache.kafka.common.utils.Utils;
@@ -29,6 +32,12 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.INT64;
+
 public class ListOffsetResponse extends AbstractResponse {
     public static final long UNKNOWN_TIMESTAMP = -1L;
     public static final long UNKNOWN_OFFSET = -1L;
@@ -36,13 +45,8 @@ public class ListOffsetResponse extends AbstractResponse {
     private static final String RESPONSES_KEY_NAME = "responses";
 
     // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
     private static final String PARTITIONS_KEY_NAME = "partition_responses";
 
-    // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
-
     /**
      * Possible error code:
      *
@@ -58,6 +62,38 @@ public class ListOffsetResponse extends AbstractResponse {
     private static final String TIMESTAMP_KEY_NAME = "timestamp";
     private static final String OFFSET_KEY_NAME = "offset";
 
+    private static final Schema LIST_OFFSET_RESPONSE_PARTITION_V0 = new Schema(
+            PARTITION_ID,
+            ERROR_CODE,
+            new Field(OFFSETS_KEY_NAME, new ArrayOf(INT64), "A list of offsets."));
+
+    private static final Schema LIST_OFFSET_RESPONSE_PARTITION_V1 = new Schema(
+            PARTITION_ID,
+            ERROR_CODE,
+            new Field(TIMESTAMP_KEY_NAME, INT64, "The timestamp associated with the returned offset"),
+            new Field(OFFSET_KEY_NAME, INT64, "offset found"));
+
+    private static final Schema LIST_OFFSET_RESPONSE_TOPIC_V0 = new Schema(
+            TOPIC_NAME,
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(LIST_OFFSET_RESPONSE_PARTITION_V0)));
+
+    private static final Schema LIST_OFFSET_RESPONSE_TOPIC_V1 = new Schema(
+            TOPIC_NAME,
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(LIST_OFFSET_RESPONSE_PARTITION_V1)));
+
+    private static final Schema LIST_OFFSET_RESPONSE_V0 = new Schema(
+            new Field(RESPONSES_KEY_NAME, new ArrayOf(LIST_OFFSET_RESPONSE_TOPIC_V0)));
+
+    private static final Schema LIST_OFFSET_RESPONSE_V1 = new Schema(
+            new Field(RESPONSES_KEY_NAME, new ArrayOf(LIST_OFFSET_RESPONSE_TOPIC_V1)));
+    private static final Schema LIST_OFFSET_RESPONSE_V2 = new Schema(
+            THROTTLE_TIME_MS,
+            new Field(RESPONSES_KEY_NAME, new ArrayOf(LIST_OFFSET_RESPONSE_TOPIC_V1)));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {LIST_OFFSET_RESPONSE_V0, LIST_OFFSET_RESPONSE_V1, LIST_OFFSET_RESPONSE_V2};
+    }
+
     public static final class PartitionData {
         public final Errors error;
         // The offsets list is only used in ListOffsetResponse v0.
@@ -121,19 +157,19 @@ public class ListOffsetResponse extends AbstractResponse {
     }
 
     public ListOffsetResponse(Struct struct) {
-        this.throttleTimeMs = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME;
+        this.throttleTimeMs = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME);
         responseData = new HashMap<>();
         for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
             Struct topicResponse = (Struct) topicResponseObj;
-            String topic = topicResponse.getString(TOPIC_KEY_NAME);
+            String topic = topicResponse.get(TOPIC_NAME);
             for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
                 Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
-                Errors error = Errors.forCode(partitionResponse.getShort(ERROR_CODE_KEY_NAME));
+                int partition = partitionResponse.get(PARTITION_ID);
+                Errors error = Errors.forCode(partitionResponse.get(ERROR_CODE));
                 PartitionData partitionData;
                 if (partitionResponse.hasField(OFFSETS_KEY_NAME)) {
                     Object[] offsets = partitionResponse.getArray(OFFSETS_KEY_NAME);
-                    List<Long> offsetsList = new ArrayList<Long>();
+                    List<Long> offsetsList = new ArrayList<>();
                     for (Object offset : offsets)
                         offsetsList.add((Long) offset);
                     partitionData = new PartitionData(error, offsetsList);
@@ -162,20 +198,19 @@ public class ListOffsetResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.LIST_OFFSETS.responseSchema(version));
-        if (struct.hasField(THROTTLE_TIME_KEY_NAME))
-            struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
+        struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs);
         Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(responseData);
 
         List<Struct> topicArray = new ArrayList<>();
         for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
             Struct topicData = struct.instance(RESPONSES_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
+            topicData.set(TOPIC_NAME, topicEntry.getKey());
             List<Struct> partitionArray = new ArrayList<>();
             for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
                 PartitionData offsetPartitionData = partitionEntry.getValue();
                 Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
-                partitionData.set(ERROR_CODE_KEY_NAME, offsetPartitionData.error.code());
+                partitionData.set(PARTITION_ID, partitionEntry.getKey());
+                partitionData.set(ERROR_CODE, offsetPartitionData.error.code());
                 if (version == 0)
                     partitionData.set(OFFSETS_KEY_NAME, offsetPartitionData.offsets.toArray());
                 else {

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java
----------------------------------------------------------------------
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 8aa2fc3..934b0ed 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
@@ -20,6 +20,9 @@ import org.apache.kafka.common.Node;
 import org.apache.kafka.common.errors.UnsupportedVersionException;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.Utils;
 
@@ -28,11 +31,42 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
+import static org.apache.kafka.common.protocol.types.Type.BOOLEAN;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class MetadataRequest extends AbstractRequest {
 
     private static final String TOPICS_KEY_NAME = "topics";
     private static final String ALLOW_AUTO_TOPIC_CREATION_KEY_NAME = "allow_auto_topic_creation";
 
+    private static final Schema METADATA_REQUEST_V0 = new Schema(
+            new Field(TOPICS_KEY_NAME, new ArrayOf(STRING), "An array of topics to fetch metadata for. If no topics are specified fetch metadata for all topics."));
+
+    private static final Schema METADATA_REQUEST_V1 = new Schema(
+            new Field(TOPICS_KEY_NAME, ArrayOf.nullable(STRING), "An array of topics to fetch metadata for. If the topics array is null fetch metadata for all topics."));
+
+    /* The v2 metadata request is the same as v1. An additional field for cluster id has been added to the v2 metadata response */
+    private static final Schema METADATA_REQUEST_V2 = METADATA_REQUEST_V1;
+
+    /* The v3 metadata request is the same as v1 and v2. An additional field for throttle time has been added to the v3 metadata response */
+    private static final Schema METADATA_REQUEST_V3 = METADATA_REQUEST_V2;
+
+    /* The v4 metadata request has an additional field for allowing auto topic creation. The response is the same as v3. */
+    private static final Schema METADATA_REQUEST_V4 = new Schema(
+            new Field(TOPICS_KEY_NAME, ArrayOf.nullable(STRING), "An array of topics to fetch metadata for. " +
+                    "If the topics array is null fetch metadata for all topics."),
+            new Field(ALLOW_AUTO_TOPIC_CREATION_KEY_NAME, BOOLEAN, "If this and the broker config " +
+                    "'auto.create.topics.enable' are true, topics that don't exist will be created by the broker. " +
+                    "Otherwise, no topics will be created by the broker."));
+
+    /* The v5 metadata request is the same as v4. An additional field for offline_replicas has been added to the v5 metadata response */
+    private static final Schema METADATA_REQUEST_V5 = METADATA_REQUEST_V4;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {METADATA_REQUEST_V0, METADATA_REQUEST_V1, METADATA_REQUEST_V2, METADATA_REQUEST_V3,
+            METADATA_REQUEST_V4, METADATA_REQUEST_V5};
+    }
+
     public static class Builder extends AbstractRequest.Builder<MetadataRequest> {
         private static final List<String> ALL_TOPICS = null;
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java
----------------------------------------------------------------------
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 10f5c13..fb69cef 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
@@ -22,6 +22,9 @@ import org.apache.kafka.common.PartitionInfo;
 import org.apache.kafka.common.errors.InvalidMetadataException;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.Utils;
 
@@ -34,8 +37,16 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-public class MetadataResponse extends AbstractResponse {
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.BOOLEAN;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
 
+public class MetadataResponse extends AbstractResponse {
     private static final String BROKERS_KEY_NAME = "brokers";
     private static final String TOPIC_METADATA_KEY_NAME = "topic_metadata";
 
@@ -50,9 +61,6 @@ public class MetadataResponse extends AbstractResponse {
 
     private static final String CLUSTER_ID_KEY_NAME = "cluster_id";
 
-    // topic level field names
-    private static final String TOPIC_ERROR_CODE_KEY_NAME = "topic_error_code";
-
     /**
      * Possible error codes:
      *
@@ -62,13 +70,9 @@ public class MetadataResponse extends AbstractResponse {
      * TopicAuthorizationFailed (29)
      */
 
-    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
-    private static final String PARTITION_ERROR_CODE_KEY_NAME = "partition_error_code";
-
     /**
      * Possible error codes:
      *
@@ -76,12 +80,95 @@ public class MetadataResponse extends AbstractResponse {
      * ReplicaNotAvailable (9)
      */
 
-    private static final String PARTITION_KEY_NAME = "partition_id";
     private static final String LEADER_KEY_NAME = "leader";
     private static final String REPLICAS_KEY_NAME = "replicas";
     private static final String ISR_KEY_NAME = "isr";
     private static final String OFFLINE_REPLICAS_KEY_NAME = "offline_replicas";
 
+    private static final Schema METADATA_BROKER_V0 = new Schema(
+            new Field(NODE_ID_KEY_NAME, INT32, "The broker id."),
+            new Field(HOST_KEY_NAME, STRING, "The hostname of the broker."),
+            new Field(PORT_KEY_NAME, INT32, "The port on which the broker accepts requests."));
+
+    private static final Schema PARTITION_METADATA_V0 = new Schema(
+            ERROR_CODE,
+            PARTITION_ID,
+            new Field(LEADER_KEY_NAME, INT32, "The id of the broker acting as leader for this partition."),
+            new Field(REPLICAS_KEY_NAME, new ArrayOf(INT32), "The set of all nodes that host this partition."),
+            new Field(ISR_KEY_NAME, new ArrayOf(INT32), "The set of nodes that are in sync with the leader for this partition."));
+
+    private static final Schema TOPIC_METADATA_V0 = new Schema(
+            ERROR_CODE,
+            TOPIC_NAME,
+            new Field(PARTITION_METADATA_KEY_NAME, new ArrayOf(PARTITION_METADATA_V0), "Metadata for each partition of the topic."));
+
+    private static final Schema METADATA_RESPONSE_V0 = new Schema(
+            new Field(BROKERS_KEY_NAME, new ArrayOf(METADATA_BROKER_V0), "Host and port information for all brokers."),
+            new Field(TOPIC_METADATA_KEY_NAME, new ArrayOf(TOPIC_METADATA_V0)));
+
+    private static final Schema METADATA_BROKER_V1 = new Schema(
+            new Field(NODE_ID_KEY_NAME, INT32, "The broker id."),
+            new Field(HOST_KEY_NAME, STRING, "The hostname of the broker."),
+            new Field(PORT_KEY_NAME, INT32, "The port on which the broker accepts requests."),
+            new Field(RACK_KEY_NAME, NULLABLE_STRING, "The rack of the broker."));
+
+    private static final Schema PARTITION_METADATA_V1 = PARTITION_METADATA_V0;
+
+    // PARTITION_METADATA_V2 added a per-partition offline_replicas field. This field specifies the list of replicas that are offline.
+    private static final Schema PARTITION_METADATA_V2 = new Schema(
+            ERROR_CODE,
+            PARTITION_ID,
+            new Field(LEADER_KEY_NAME, INT32, "The id of the broker acting as leader for this partition."),
+            new Field(REPLICAS_KEY_NAME, new ArrayOf(INT32), "The set of all nodes that host this partition."),
+            new Field(ISR_KEY_NAME, new ArrayOf(INT32), "The set of nodes that are in sync with the leader for this partition."),
+            new Field(OFFLINE_REPLICAS_KEY_NAME, new ArrayOf(INT32), "The set of offline replicas of this partition."));
+
+    private static final Schema TOPIC_METADATA_V1 = new Schema(
+            ERROR_CODE,
+            TOPIC_NAME,
+            new Field(IS_INTERNAL_KEY_NAME, BOOLEAN, "Indicates if the topic is considered a Kafka internal topic"),
+            new Field(PARTITION_METADATA_KEY_NAME, new ArrayOf(PARTITION_METADATA_V1), "Metadata for each partition of the topic."));
+
+    // TOPIC_METADATA_V2 added a per-partition offline_replicas field. This field specifies the list of replicas that are offline.
+    private static final Schema TOPIC_METADATA_V2 = new Schema(
+            ERROR_CODE,
+            TOPIC_NAME,
+            new Field(IS_INTERNAL_KEY_NAME, BOOLEAN, "Indicates if the topic is considered a Kafka internal topic"),
+            new Field(PARTITION_METADATA_KEY_NAME, new ArrayOf(PARTITION_METADATA_V2), "Metadata for each partition of the topic."));
+
+    private static final Schema METADATA_RESPONSE_V1 = new Schema(
+            new Field(BROKERS_KEY_NAME, new ArrayOf(METADATA_BROKER_V1), "Host and port information for all brokers."),
+            new Field(CONTROLLER_ID_KEY_NAME, INT32, "The broker id of the controller broker."),
+            new Field(TOPIC_METADATA_KEY_NAME, new ArrayOf(TOPIC_METADATA_V1)));
+
+    private static final Schema METADATA_RESPONSE_V2 = new Schema(
+            new Field(BROKERS_KEY_NAME, new ArrayOf(METADATA_BROKER_V1), "Host and port information for all brokers."),
+            new Field(CLUSTER_ID_KEY_NAME, NULLABLE_STRING, "The cluster id that this broker belongs to."),
+            new Field(CONTROLLER_ID_KEY_NAME, INT32, "The broker id of the controller broker."),
+            new Field(TOPIC_METADATA_KEY_NAME, new ArrayOf(TOPIC_METADATA_V1)));
+
+    private static final Schema METADATA_RESPONSE_V3 = new Schema(
+            THROTTLE_TIME_MS,
+            new Field(BROKERS_KEY_NAME, new ArrayOf(METADATA_BROKER_V1), "Host and port information for all brokers."),
+            new Field(CLUSTER_ID_KEY_NAME, NULLABLE_STRING, "The cluster id that this broker belongs to."),
+            new Field(CONTROLLER_ID_KEY_NAME, INT32, "The broker id of the controller broker."),
+            new Field(TOPIC_METADATA_KEY_NAME, new ArrayOf(TOPIC_METADATA_V1)));
+
+    private static final Schema METADATA_RESPONSE_V4 = METADATA_RESPONSE_V3;
+
+    // METADATA_RESPONSE_V5 added a per-partition offline_replicas field. This field specifies the list of replicas that are offline.
+    private static final Schema METADATA_RESPONSE_V5 = new Schema(
+            THROTTLE_TIME_MS,
+            new Field(BROKERS_KEY_NAME, new ArrayOf(METADATA_BROKER_V1), "Host and port information for all brokers."),
+            new Field(CLUSTER_ID_KEY_NAME, NULLABLE_STRING, "The cluster id that this broker belongs to."),
+            new Field(CONTROLLER_ID_KEY_NAME, INT32, "The broker id of the controller broker."),
+            new Field(TOPIC_METADATA_KEY_NAME, new ArrayOf(TOPIC_METADATA_V2)));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {METADATA_RESPONSE_V0, METADATA_RESPONSE_V1, METADATA_RESPONSE_V2, METADATA_RESPONSE_V3,
+            METADATA_RESPONSE_V4, METADATA_RESPONSE_V5};
+    }
+
     private final int throttleTimeMs;
     private final Collection<Node> brokers;
     private final Node controller;
@@ -104,7 +191,7 @@ public class MetadataResponse extends AbstractResponse {
     }
 
     public MetadataResponse(Struct struct) {
-        this.throttleTimeMs = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME;
+        this.throttleTimeMs = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME);
         Map<Integer, Node> brokers = new HashMap<>();
         Object[] brokerStructs = (Object[]) struct.get(BROKERS_KEY_NAME);
         for (Object brokerStruct : brokerStructs) {
@@ -135,8 +222,8 @@ public class MetadataResponse extends AbstractResponse {
         Object[] topicInfos = (Object[]) struct.get(TOPIC_METADATA_KEY_NAME);
         for (Object topicInfoObj : topicInfos) {
             Struct topicInfo = (Struct) topicInfoObj;
-            Errors topicError = Errors.forCode(topicInfo.getShort(TOPIC_ERROR_CODE_KEY_NAME));
-            String topic = topicInfo.getString(TOPIC_KEY_NAME);
+            Errors topicError = Errors.forCode(topicInfo.get(ERROR_CODE));
+            String topic = topicInfo.get(TOPIC_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;
@@ -146,8 +233,8 @@ public class MetadataResponse extends AbstractResponse {
             Object[] partitionInfos = (Object[]) topicInfo.get(PARTITION_METADATA_KEY_NAME);
             for (Object partitionInfoObj : partitionInfos) {
                 Struct partitionInfo = (Struct) partitionInfoObj;
-                Errors partitionError = Errors.forCode(partitionInfo.getShort(PARTITION_ERROR_CODE_KEY_NAME));
-                int partition = partitionInfo.getInt(PARTITION_KEY_NAME);
+                Errors partitionError = Errors.forCode(partitionInfo.get(ERROR_CODE));
+                int partition = partitionInfo.get(PARTITION_ID);
                 int leader = partitionInfo.getInt(LEADER_KEY_NAME);
                 Node leaderNode = leader == -1 ? null : brokers.get(leader);
 
@@ -400,8 +487,7 @@ public class MetadataResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.METADATA.responseSchema(version));
-        if (struct.hasField(THROTTLE_TIME_KEY_NAME))
-            struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
+        struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs);
         List<Struct> brokerArray = new ArrayList<>();
         for (Node node : brokers) {
             Struct broker = struct.instance(BROKERS_KEY_NAME);
@@ -426,8 +512,8 @@ public class MetadataResponse extends AbstractResponse {
         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());
+            topicData.set(TOPIC_NAME, metadata.topic);
+            topicData.set(ERROR_CODE, metadata.error.code());
             // This field only exists in v1+
             if (topicData.hasField(IS_INTERNAL_KEY_NAME))
                 topicData.set(IS_INTERNAL_KEY_NAME, metadata.isInternal());
@@ -435,8 +521,8 @@ public class MetadataResponse extends AbstractResponse {
             List<Struct> partitionMetadataArray = new ArrayList<>(metadata.partitionMetadata.size());
             for (PartitionMetadata partitionMetadata : metadata.partitionMetadata()) {
                 Struct partitionData = topicData.instance(PARTITION_METADATA_KEY_NAME);
-                partitionData.set(PARTITION_ERROR_CODE_KEY_NAME, partitionMetadata.error.code());
-                partitionData.set(PARTITION_KEY_NAME, partitionMetadata.partition);
+                partitionData.set(ERROR_CODE, partitionMetadata.error.code());
+                partitionData.set(PARTITION_ID, partitionMetadata.partition);
                 partitionData.set(LEADER_KEY_NAME, partitionMetadata.leader.id());
                 ArrayList<Integer> replicas = new ArrayList<>(partitionMetadata.replicas.size());
                 for (Node node : partitionMetadata.replicas)

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java
index 4402c4d..696d967 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java
@@ -20,6 +20,8 @@ import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.errors.UnsupportedVersionException;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
 import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.CollectionUtils;
@@ -30,6 +32,13 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.INT64;
+import static org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 /**
  * This wrapper supports both v0 and v1 of OffsetCommitRequest.
  */
@@ -41,17 +50,69 @@ public class OffsetCommitRequest extends AbstractRequest {
     private static final String RETENTION_TIME_KEY_NAME = "retention_time";
 
     // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
     private static final String PARTITIONS_KEY_NAME = "partitions";
 
     // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
     private static final String COMMIT_OFFSET_KEY_NAME = "offset";
     private static final String METADATA_KEY_NAME = "metadata";
 
     @Deprecated
     private static final String TIMESTAMP_KEY_NAME = "timestamp";         // for v0, v1
 
+    /* Offset commit api */
+    private static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V0 = new Schema(
+            PARTITION_ID,
+            new Field(COMMIT_OFFSET_KEY_NAME, INT64, "Message offset to be committed."),
+            new Field(METADATA_KEY_NAME, NULLABLE_STRING, "Any associated metadata the client wants to keep."));
+
+    private static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V1 = new Schema(
+            PARTITION_ID,
+            new Field(COMMIT_OFFSET_KEY_NAME, INT64, "Message offset to be committed."),
+            new Field(TIMESTAMP_KEY_NAME, INT64, "Timestamp of the commit"),
+            new Field(METADATA_KEY_NAME, NULLABLE_STRING, "Any associated metadata the client wants to keep."));
+
+    private static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V2 = new Schema(
+            PARTITION_ID,
+            new Field(COMMIT_OFFSET_KEY_NAME, INT64, "Message offset to be committed."),
+            new Field(METADATA_KEY_NAME, NULLABLE_STRING, "Any associated metadata the client wants to keep."));
+
+    private static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V0 = new Schema(
+            TOPIC_NAME,
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V0), "Partitions to commit offsets."));
+
+    private static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V1 = new Schema(
+            TOPIC_NAME,
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V1), "Partitions to commit offsets."));
+
+    private static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V2 = new Schema(
+            TOPIC_NAME,
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V2), "Partitions to commit offsets."));
+
+    private static final Schema OFFSET_COMMIT_REQUEST_V0 = new Schema(
+            new Field(GROUP_ID_KEY_NAME, STRING, "The group id."),
+            new Field(TOPICS_KEY_NAME, new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0), "Topics to commit offsets."));
+
+    private static final Schema OFFSET_COMMIT_REQUEST_V1 = new Schema(
+            new Field(GROUP_ID_KEY_NAME, STRING, "The group id."),
+            new Field(GENERATION_ID_KEY_NAME, INT32, "The generation of the group."),
+            new Field(MEMBER_ID_KEY_NAME, STRING, "The member id assigned by the group coordinator."),
+            new Field(TOPICS_KEY_NAME, new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V1), "Topics to commit offsets."));
+
+    private static final Schema OFFSET_COMMIT_REQUEST_V2 = new Schema(
+            new Field(GROUP_ID_KEY_NAME, STRING, "The group id."),
+            new Field(GENERATION_ID_KEY_NAME, INT32, "The generation of the consumer group."),
+            new Field(MEMBER_ID_KEY_NAME, STRING, "The consumer id assigned by the group coordinator."),
+            new Field(RETENTION_TIME_KEY_NAME, INT64, "Time period in ms to retain the offset."),
+            new Field(TOPICS_KEY_NAME, new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V2), "Topics to commit offsets."));
+
+    /* v3 request is same as v2. Throttle time has been added to response */
+    private static final Schema OFFSET_COMMIT_REQUEST_V3 = OFFSET_COMMIT_REQUEST_V2;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1, OFFSET_COMMIT_REQUEST_V2,
+            OFFSET_COMMIT_REQUEST_V3};
+    }
+
     // default values for the current version
     public static final int DEFAULT_GENERATION_ID = -1;
     public static final String DEFAULT_MEMBER_ID = "";
@@ -190,10 +251,10 @@ public class OffsetCommitRequest extends AbstractRequest {
         offsetData = new HashMap<>();
         for (Object topicDataObj : struct.getArray(TOPICS_KEY_NAME)) {
             Struct topicData = (Struct) topicDataObj;
-            String topic = topicData.getString(TOPIC_KEY_NAME);
+            String topic = topicData.get(TOPIC_NAME);
             for (Object partitionDataObj : topicData.getArray(PARTITIONS_KEY_NAME)) {
                 Struct partitionDataStruct = (Struct) partitionDataObj;
-                int partition = partitionDataStruct.getInt(PARTITION_KEY_NAME);
+                int partition = partitionDataStruct.get(PARTITION_ID);
                 long offset = partitionDataStruct.getLong(COMMIT_OFFSET_KEY_NAME);
                 String metadata = partitionDataStruct.getString(METADATA_KEY_NAME);
                 PartitionData partitionOffset;
@@ -219,12 +280,12 @@ public class OffsetCommitRequest extends AbstractRequest {
         List<Struct> topicArray = new ArrayList<>();
         for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
             Struct topicData = struct.instance(TOPICS_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
+            topicData.set(TOPIC_NAME, topicEntry.getKey());
             List<Struct> partitionArray = new ArrayList<>();
             for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
                 PartitionData fetchPartitionData = partitionEntry.getValue();
                 Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
+                partitionData.set(PARTITION_ID, partitionEntry.getKey());
                 partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset);
                 // Only for v1
                 if (partitionData.hasField(TIMESTAMP_KEY_NAME))

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java
index 782ffa5..0181eef 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java
@@ -19,6 +19,9 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.CollectionUtils;
 
@@ -28,18 +31,18 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+
 public class OffsetCommitResponse extends AbstractResponse {
 
     private static final String RESPONSES_KEY_NAME = "responses";
 
     // topic level fields
-    private static final String TOPIC_KEY_NAME = "topic";
     private static final String PARTITIONS_KEY_NAME = "partition_responses";
 
-    // partition level fields
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
-
     /**
      * Possible error codes:
      *
@@ -57,6 +60,32 @@ public class OffsetCommitResponse extends AbstractResponse {
      * GROUP_AUTHORIZATION_FAILED (30)
      */
 
+    private static final Schema OFFSET_COMMIT_RESPONSE_PARTITION_V0 = new Schema(
+            PARTITION_ID,
+            ERROR_CODE);
+
+    private static final Schema OFFSET_COMMIT_RESPONSE_TOPIC_V0 = new Schema(
+            TOPIC_NAME,
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(OFFSET_COMMIT_RESPONSE_PARTITION_V0)));
+
+    private static final Schema OFFSET_COMMIT_RESPONSE_V0 = new Schema(
+            new Field(RESPONSES_KEY_NAME, new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0)));
+
+
+    /* The response types for V0, V1 and V2 of OFFSET_COMMIT_REQUEST are the same. */
+    private static final Schema OFFSET_COMMIT_RESPONSE_V1 = OFFSET_COMMIT_RESPONSE_V0;
+    private static final Schema OFFSET_COMMIT_RESPONSE_V2 = OFFSET_COMMIT_RESPONSE_V0;
+
+    private static final Schema OFFSET_COMMIT_RESPONSE_V3 = new Schema(
+            THROTTLE_TIME_MS,
+            new Field(RESPONSES_KEY_NAME, new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0)));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V1, OFFSET_COMMIT_RESPONSE_V2,
+            OFFSET_COMMIT_RESPONSE_V3};
+    }
+
+
     private final Map<TopicPartition, Errors> responseData;
     private final int throttleTimeMs;
 
@@ -70,15 +99,15 @@ public class OffsetCommitResponse extends AbstractResponse {
     }
 
     public OffsetCommitResponse(Struct struct) {
-        this.throttleTimeMs = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME;
+        this.throttleTimeMs = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME);
         responseData = new HashMap<>();
         for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
             Struct topicResponse = (Struct) topicResponseObj;
-            String topic = topicResponse.getString(TOPIC_KEY_NAME);
+            String topic = topicResponse.get(TOPIC_NAME);
             for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
                 Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
-                Errors error = Errors.forCode(partitionResponse.getShort(ERROR_CODE_KEY_NAME));
+                int partition = partitionResponse.get(PARTITION_ID);
+                Errors error = Errors.forCode(partitionResponse.get(ERROR_CODE));
                 responseData.put(new TopicPartition(topic, partition), error);
             }
         }
@@ -87,19 +116,18 @@ public class OffsetCommitResponse extends AbstractResponse {
     @Override
     public Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.OFFSET_COMMIT.responseSchema(version));
-        if (struct.hasField(THROTTLE_TIME_KEY_NAME))
-            struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
+        struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs);
 
         Map<String, Map<Integer, Errors>> topicsData = CollectionUtils.groupDataByTopic(responseData);
         List<Struct> topicArray = new ArrayList<>();
         for (Map.Entry<String, Map<Integer, Errors>> entries: topicsData.entrySet()) {
             Struct topicData = struct.instance(RESPONSES_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, entries.getKey());
+            topicData.set(TOPIC_NAME, entries.getKey());
             List<Struct> partitionArray = new ArrayList<>();
             for (Map.Entry<Integer, Errors> partitionEntry : entries.getValue().entrySet()) {
                 Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
-                partitionData.set(ERROR_CODE_KEY_NAME, partitionEntry.getValue().code());
+                partitionData.set(PARTITION_ID, partitionEntry.getKey());
+                partitionData.set(ERROR_CODE, partitionEntry.getValue().code());
                 partitionArray.add(partitionData);
             }
             topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
index 15fdf57..6d8b959 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
@@ -20,6 +20,9 @@ import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.errors.UnsupportedVersionException;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.CollectionUtils;
 import org.apache.kafka.common.utils.Utils;
@@ -30,16 +33,53 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class OffsetFetchRequest extends AbstractRequest {
     private static final String GROUP_ID_KEY_NAME = "group_id";
     private static final String TOPICS_KEY_NAME = "topics";
 
     // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
     private static final String PARTITIONS_KEY_NAME = "partitions";
 
-    // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
+    /*
+     * Wire formats of version 0 and 1 are the same, but with different functionality.
+     * Wire format of version 2 is similar to version 1, with the exception of
+     * - accepting 'null' as list of topics
+     * - returning a top level error code
+     * Version 0 will read the offsets from ZK.
+     * Version 1 will read the offsets from Kafka.
+     * Version 2 will read the offsets from Kafka, and returns all associated topic partition offsets if
+     * a 'null' is passed instead of a list of specific topic partitions. It also returns a top level error code
+     * for group or coordinator level errors.
+     */
+    private static final Schema OFFSET_FETCH_REQUEST_PARTITION_V0 = new Schema(
+            PARTITION_ID);
+
+    private static final Schema OFFSET_FETCH_REQUEST_TOPIC_V0 = new Schema(
+            TOPIC_NAME,
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(OFFSET_FETCH_REQUEST_PARTITION_V0), "Partitions to fetch offsets."));
+
+    private static final Schema OFFSET_FETCH_REQUEST_V0 = new Schema(
+            new Field(GROUP_ID_KEY_NAME, STRING, "The consumer group id."),
+            new Field(TOPICS_KEY_NAME, new ArrayOf(OFFSET_FETCH_REQUEST_TOPIC_V0), "Topics to fetch offsets."));
+
+    private static final Schema OFFSET_FETCH_REQUEST_V1 = OFFSET_FETCH_REQUEST_V0;
+
+    private static final Schema OFFSET_FETCH_REQUEST_V2 = new Schema(
+            new Field(GROUP_ID_KEY_NAME, STRING, "The consumer group id."),
+            new Field(TOPICS_KEY_NAME, ArrayOf.nullable(OFFSET_FETCH_REQUEST_TOPIC_V0), "Topics to fetch offsets. If the " +
+                    "topic array is null fetch offsets for all topics."));
+
+    /* v3 request is the same as v2. Throttle time has been added to v3 response */
+    private static final Schema OFFSET_FETCH_REQUEST_V3 = OFFSET_FETCH_REQUEST_V2;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {OFFSET_FETCH_REQUEST_V0, OFFSET_FETCH_REQUEST_V1, OFFSET_FETCH_REQUEST_V2,
+            OFFSET_FETCH_REQUEST_V3};
+    }
 
     public static class Builder extends AbstractRequest.Builder<OffsetFetchRequest> {
         private static final List<TopicPartition> ALL_TOPIC_PARTITIONS = null;
@@ -102,10 +142,10 @@ public class OffsetFetchRequest extends AbstractRequest {
             partitions = new ArrayList<>();
             for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
                 Struct topicResponse = (Struct) topicResponseObj;
-                String topic = topicResponse.getString(TOPIC_KEY_NAME);
+                String topic = topicResponse.get(TOPIC_NAME);
                 for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
                     Struct partitionResponse = (Struct) partitionResponseObj;
-                    int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                    int partition = partitionResponse.get(PARTITION_ID);
                     partitions.add(new TopicPartition(topic, partition));
                 }
             }
@@ -177,11 +217,11 @@ public class OffsetFetchRequest extends AbstractRequest {
             List<Struct> topicArray = new ArrayList<>();
             for (Map.Entry<String, List<Integer>> entries : topicsData.entrySet()) {
                 Struct topicData = struct.instance(TOPICS_KEY_NAME);
-                topicData.set(TOPIC_KEY_NAME, entries.getKey());
+                topicData.set(TOPIC_NAME, entries.getKey());
                 List<Struct> partitionArray = new ArrayList<>();
                 for (Integer partitionId : entries.getValue()) {
                     Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                    partitionData.set(PARTITION_KEY_NAME, partitionId);
+                    partitionData.set(PARTITION_ID, partitionId);
                     partitionArray.add(partitionData);
                 }
                 topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
index 6315535..c3341e0 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
@@ -16,34 +16,71 @@
  */
 package org.apache.kafka.common.requests;
 
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.utils.CollectionUtils;
+
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.protocol.ApiKeys;
-import org.apache.kafka.common.protocol.Errors;
-import org.apache.kafka.common.protocol.types.Struct;
-import org.apache.kafka.common.utils.CollectionUtils;
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.INT64;
+import static org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING;
 
 public class OffsetFetchResponse extends AbstractResponse {
 
     private static final String RESPONSES_KEY_NAME = "responses";
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
 
     // topic level fields
-    private static final String TOPIC_KEY_NAME = "topic";
     private static final String PARTITIONS_KEY_NAME = "partition_responses";
 
     // partition level fields
-    private static final String PARTITION_KEY_NAME = "partition";
     private static final String COMMIT_OFFSET_KEY_NAME = "offset";
     private static final String METADATA_KEY_NAME = "metadata";
 
+    private static final Schema OFFSET_FETCH_RESPONSE_PARTITION_V0 = new Schema(
+            PARTITION_ID,
+            new Field(COMMIT_OFFSET_KEY_NAME, INT64, "Last committed message offset."),
+            new Field(METADATA_KEY_NAME, NULLABLE_STRING, "Any associated metadata the client wants to keep."),
+            ERROR_CODE);
+
+    private static final Schema OFFSET_FETCH_RESPONSE_TOPIC_V0 = new Schema(
+            TOPIC_NAME,
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(OFFSET_FETCH_RESPONSE_PARTITION_V0)));
+
+    private static final Schema OFFSET_FETCH_RESPONSE_V0 = new Schema(
+            new Field(RESPONSES_KEY_NAME, new ArrayOf(OFFSET_FETCH_RESPONSE_TOPIC_V0)));
+
+    private static final Schema OFFSET_FETCH_RESPONSE_V1 = OFFSET_FETCH_RESPONSE_V0;
+
+    private static final Schema OFFSET_FETCH_RESPONSE_V2 = new Schema(
+            new Field(RESPONSES_KEY_NAME, new ArrayOf(OFFSET_FETCH_RESPONSE_TOPIC_V0)),
+            ERROR_CODE);
+
+    /* v3 request is the same as v2. Throttle time has been added to v3 response */
+    private static final Schema OFFSET_FETCH_RESPONSE_V3 = new Schema(
+            THROTTLE_TIME_MS,
+            new Field(RESPONSES_KEY_NAME, new ArrayOf(OFFSET_FETCH_RESPONSE_TOPIC_V0)),
+            ERROR_CODE);
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {OFFSET_FETCH_RESPONSE_V0, OFFSET_FETCH_RESPONSE_V1, OFFSET_FETCH_RESPONSE_V2,
+            OFFSET_FETCH_RESPONSE_V3};
+    }
+
     public static final long INVALID_OFFSET = -1L;
     public static final String NO_METADATA = "";
     public static final PartitionData UNKNOWN_PARTITION = new PartitionData(INVALID_OFFSET, NO_METADATA,
@@ -62,8 +99,7 @@ public class OffsetFetchResponse extends AbstractResponse {
      *   - GROUP_AUTHORIZATION_FAILED (30)
      */
 
-    private static final List<Errors> PARTITION_ERRORS = Arrays.asList(
-            Errors.UNKNOWN_TOPIC_OR_PARTITION);
+    private static final List<Errors> PARTITION_ERRORS = Collections.singletonList(Errors.UNKNOWN_TOPIC_OR_PARTITION);
 
     private final Map<TopicPartition, PartitionData> responseData;
     private final Errors error;
@@ -107,18 +143,18 @@ public class OffsetFetchResponse extends AbstractResponse {
     }
 
     public OffsetFetchResponse(Struct struct) {
-        this.throttleTimeMs = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME;
+        this.throttleTimeMs = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME);
         Errors topLevelError = Errors.NONE;
         this.responseData = new HashMap<>();
         for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
             Struct topicResponse = (Struct) topicResponseObj;
-            String topic = topicResponse.getString(TOPIC_KEY_NAME);
+            String topic = topicResponse.get(TOPIC_NAME);
             for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
                 Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                int partition = partitionResponse.get(PARTITION_ID);
                 long offset = partitionResponse.getLong(COMMIT_OFFSET_KEY_NAME);
                 String metadata = partitionResponse.getString(METADATA_KEY_NAME);
-                Errors error = Errors.forCode(partitionResponse.getShort(ERROR_CODE_KEY_NAME));
+                Errors error = Errors.forCode(partitionResponse.get(ERROR_CODE));
                 if (error != Errors.NONE && !PARTITION_ERRORS.contains(error))
                     topLevelError = error;
                 PartitionData partitionData = new PartitionData(offset, metadata, error);
@@ -130,7 +166,7 @@ public class OffsetFetchResponse extends AbstractResponse {
         // for older versions there is no top-level error in the response and all errors are partition errors,
         // so if there is a group or coordinator error at the partition level use that as the top-level error.
         // this way clients can depend on the top-level error regardless of the offset fetch version.
-        this.error = struct.hasField(ERROR_CODE_KEY_NAME) ? Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME)) : topLevelError;
+        this.error = struct.hasField(ERROR_CODE) ? Errors.forCode(struct.get(ERROR_CODE)) : topLevelError;
     }
 
     public void maybeThrowFirstPartitionError() {
@@ -164,22 +200,21 @@ public class OffsetFetchResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.OFFSET_FETCH.responseSchema(version));
-        if (struct.hasField(THROTTLE_TIME_KEY_NAME))
-            struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
+        struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs);
 
         Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(responseData);
         List<Struct> topicArray = new ArrayList<>();
         for (Map.Entry<String, Map<Integer, PartitionData>> entries : topicsData.entrySet()) {
             Struct topicData = struct.instance(RESPONSES_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, entries.getKey());
+            topicData.set(TOPIC_NAME, entries.getKey());
             List<Struct> partitionArray = new ArrayList<>();
             for (Map.Entry<Integer, PartitionData> partitionEntry : entries.getValue().entrySet()) {
                 PartitionData fetchPartitionData = partitionEntry.getValue();
                 Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
-                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
+                partitionData.set(PARTITION_ID, partitionEntry.getKey());
                 partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset);
                 partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata);
-                partitionData.set(ERROR_CODE_KEY_NAME, fetchPartitionData.error.code());
+                partitionData.set(ERROR_CODE, fetchPartitionData.error.code());
                 partitionArray.add(partitionData);
             }
             topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
@@ -188,7 +223,7 @@ public class OffsetFetchResponse extends AbstractResponse {
         struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
 
         if (version > 1)
-            struct.set(ERROR_CODE_KEY_NAME, this.error.code());
+            struct.set(ERROR_CODE, this.error.code());
 
         return struct;
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochRequest.java
index fc31d75..b5fce78 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochRequest.java
@@ -19,6 +19,9 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.CollectionUtils;
 
@@ -28,12 +31,28 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+
 public class OffsetsForLeaderEpochRequest extends AbstractRequest {
-    public static final String TOPICS = "topics";
-    public static final String TOPIC = "topic";
-    public static final String PARTITIONS = "partitions";
-    public static final String PARTITION_ID = "partition_id";
-    public static final String LEADER_EPOCH = "leader_epoch";
+    private static final String TOPICS_KEY_NAME = "topics";
+    private static final String PARTITIONS_KEY_NAME = "partitions";
+    private static final String LEADER_EPOCH = "leader_epoch";
+
+    /* Offsets for Leader Epoch api */
+    private static final Schema OFFSET_FOR_LEADER_EPOCH_REQUEST_PARTITION_V0 = new Schema(
+            PARTITION_ID,
+            new Field("leader_epoch", INT32, "The epoch"));
+    private static final Schema OFFSET_FOR_LEADER_EPOCH_REQUEST_TOPIC_V0 = new Schema(
+            TOPIC_NAME,
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(OFFSET_FOR_LEADER_EPOCH_REQUEST_PARTITION_V0)));
+    private static final Schema OFFSET_FOR_LEADER_EPOCH_REQUEST_V0 = new Schema(
+            new Field(TOPICS_KEY_NAME, new ArrayOf(OFFSET_FOR_LEADER_EPOCH_REQUEST_TOPIC_V0), "An array of topics to get epochs for"));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{OFFSET_FOR_LEADER_EPOCH_REQUEST_V0};
+    }
 
     private Map<TopicPartition, Integer> epochsByPartition;
 
@@ -85,12 +104,12 @@ public class OffsetsForLeaderEpochRequest extends AbstractRequest {
     public OffsetsForLeaderEpochRequest(Struct struct, short version) {
         super(version);
         epochsByPartition = new HashMap<>();
-        for (Object topicAndEpochsObj : struct.getArray(TOPICS)) {
+        for (Object topicAndEpochsObj : struct.getArray(TOPICS_KEY_NAME)) {
             Struct topicAndEpochs = (Struct) topicAndEpochsObj;
-            String topic = topicAndEpochs.getString(TOPIC);
-            for (Object partitionAndEpochObj : topicAndEpochs.getArray(PARTITIONS)) {
+            String topic = topicAndEpochs.get(TOPIC_NAME);
+            for (Object partitionAndEpochObj : topicAndEpochs.getArray(PARTITIONS_KEY_NAME)) {
                 Struct partitionAndEpoch = (Struct) partitionAndEpochObj;
-                int partitionId = partitionAndEpoch.getInt(PARTITION_ID);
+                int partitionId = partitionAndEpoch.get(PARTITION_ID);
                 int epoch = partitionAndEpoch.getInt(LEADER_EPOCH);
                 TopicPartition tp = new TopicPartition(topic, partitionId);
                 epochsByPartition.put(tp, epoch);
@@ -110,19 +129,19 @@ public class OffsetsForLeaderEpochRequest extends AbstractRequest {
 
         List<Struct> topics = new ArrayList<>();
         for (Map.Entry<String, Map<Integer, Integer>> topicToEpochs : topicsToPartitionEpochs.entrySet()) {
-            Struct topicsStruct = requestStruct.instance(TOPICS);
-            topicsStruct.set(TOPIC, topicToEpochs.getKey());
+            Struct topicsStruct = requestStruct.instance(TOPICS_KEY_NAME);
+            topicsStruct.set(TOPIC_NAME, topicToEpochs.getKey());
             List<Struct> partitions = new ArrayList<>();
             for (Map.Entry<Integer, Integer> partitionEpoch : topicToEpochs.getValue().entrySet()) {
-                Struct partitionStruct = topicsStruct.instance(PARTITIONS);
+                Struct partitionStruct = topicsStruct.instance(PARTITIONS_KEY_NAME);
                 partitionStruct.set(PARTITION_ID, partitionEpoch.getKey());
                 partitionStruct.set(LEADER_EPOCH, partitionEpoch.getValue());
                 partitions.add(partitionStruct);
             }
-            topicsStruct.set(PARTITIONS, partitions.toArray());
+            topicsStruct.set(PARTITIONS_KEY_NAME, partitions.toArray());
             topics.add(topicsStruct);
         }
-        requestStruct.set(TOPICS, topics.toArray());
+        requestStruct.set(TOPICS_KEY_NAME, topics.toArray());
         return requestStruct;
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochResponse.java
index 4195b77..13d70b7 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochResponse.java
@@ -19,6 +19,9 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.CollectionUtils;
 
@@ -28,27 +31,44 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.INT64;
+
 public class OffsetsForLeaderEpochResponse extends AbstractResponse {
-    public static final String TOPICS = "topics";
-    public static final String TOPIC = "topic";
-    public static final String PARTITIONS = "partitions";
-    public static final String ERROR_CODE = "error_code";
-    public static final String PARTITION_ID = "partition_id";
-    public static final String END_OFFSET = "end_offset";
+    private static final String TOPICS_KEY_NAME = "topics";
+    private static final String PARTITIONS_KEY_NAME = "partitions";
+    private static final String END_OFFSET_KEY_NAME = "end_offset";
+
+    private static final Schema OFFSET_FOR_LEADER_EPOCH_RESPONSE_PARTITION_V0 = new Schema(
+            ERROR_CODE,
+            PARTITION_ID,
+            new Field(END_OFFSET_KEY_NAME, INT64, "The end offset"));
+    private static final Schema OFFSET_FOR_LEADER_EPOCH_RESPONSE_TOPIC_V0 = new Schema(
+            TOPIC_NAME,
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(OFFSET_FOR_LEADER_EPOCH_RESPONSE_PARTITION_V0)));
+    private static final Schema OFFSET_FOR_LEADER_EPOCH_RESPONSE_V0 = new Schema(
+            new Field(TOPICS_KEY_NAME, new ArrayOf(OFFSET_FOR_LEADER_EPOCH_RESPONSE_TOPIC_V0),
+                    "An array of topics for which we have leader offsets for some requested Partition Leader Epoch"));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{OFFSET_FOR_LEADER_EPOCH_RESPONSE_V0};
+    }
 
     private Map<TopicPartition, EpochEndOffset> epochEndOffsetsByPartition;
 
     public OffsetsForLeaderEpochResponse(Struct struct) {
         epochEndOffsetsByPartition = new HashMap<>();
-        for (Object topicAndEpocsObj : struct.getArray(TOPICS)) {
+        for (Object topicAndEpocsObj : struct.getArray(TOPICS_KEY_NAME)) {
             Struct topicAndEpochs = (Struct) topicAndEpocsObj;
-            String topic = topicAndEpochs.getString(TOPIC);
-            for (Object partitionAndEpochObj : topicAndEpochs.getArray(PARTITIONS)) {
+            String topic = topicAndEpochs.get(TOPIC_NAME);
+            for (Object partitionAndEpochObj : topicAndEpochs.getArray(PARTITIONS_KEY_NAME)) {
                 Struct partitionAndEpoch = (Struct) partitionAndEpochObj;
-                Errors error = Errors.forCode(partitionAndEpoch.getShort(ERROR_CODE));
-                int partitionId = partitionAndEpoch.getInt(PARTITION_ID);
+                Errors error = Errors.forCode(partitionAndEpoch.get(ERROR_CODE));
+                int partitionId = partitionAndEpoch.get(PARTITION_ID);
                 TopicPartition tp = new TopicPartition(topic, partitionId);
-                long endOffset = partitionAndEpoch.getLong(END_OFFSET);
+                long endOffset = partitionAndEpoch.getLong(END_OFFSET_KEY_NAME);
                 epochEndOffsetsByPartition.put(tp, new EpochEndOffset(error, endOffset));
             }
         }
@@ -74,21 +94,21 @@ public class OffsetsForLeaderEpochResponse extends AbstractResponse {
 
         List<Struct> topics = new ArrayList<>(endOffsetsByTopic.size());
         for (Map.Entry<String, Map<Integer, EpochEndOffset>> topicToPartitionEpochs : endOffsetsByTopic.entrySet()) {
-            Struct topicStruct = responseStruct.instance(TOPICS);
-            topicStruct.set(TOPIC, topicToPartitionEpochs.getKey());
+            Struct topicStruct = responseStruct.instance(TOPICS_KEY_NAME);
+            topicStruct.set(TOPIC_NAME, topicToPartitionEpochs.getKey());
             Map<Integer, EpochEndOffset> partitionEpochs = topicToPartitionEpochs.getValue();
             List<Struct> partitions = new ArrayList<>();
             for (Map.Entry<Integer, EpochEndOffset> partitionEndOffset : partitionEpochs.entrySet()) {
-                Struct partitionStruct = topicStruct.instance(PARTITIONS);
+                Struct partitionStruct = topicStruct.instance(PARTITIONS_KEY_NAME);
                 partitionStruct.set(ERROR_CODE, partitionEndOffset.getValue().error().code());
                 partitionStruct.set(PARTITION_ID, partitionEndOffset.getKey());
-                partitionStruct.set(END_OFFSET, partitionEndOffset.getValue().endOffset());
+                partitionStruct.set(END_OFFSET_KEY_NAME, partitionEndOffset.getValue().endOffset());
                 partitions.add(partitionStruct);
             }
-            topicStruct.set(PARTITIONS, partitions.toArray());
+            topicStruct.set(PARTITIONS_KEY_NAME, partitions.toArray());
             topics.add(topicStruct);
         }
-        responseStruct.set(TOPICS, topics.toArray());
+        responseStruct.set(TOPICS_KEY_NAME, topics.toArray());
         return responseStruct;
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
index 089d199..eac7661 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
@@ -20,11 +20,14 @@ import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.errors.UnsupportedVersionException;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.record.InvalidRecordException;
+import org.apache.kafka.common.record.MemoryRecords;
 import org.apache.kafka.common.record.MutableRecordBatch;
 import org.apache.kafka.common.record.RecordBatch;
-import org.apache.kafka.common.record.MemoryRecords;
 import org.apache.kafka.common.utils.CollectionUtils;
 import org.apache.kafka.common.utils.Utils;
 
@@ -36,6 +39,13 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.INT16;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING;
+import static org.apache.kafka.common.protocol.types.Type.RECORDS;
+
 public class ProduceRequest extends AbstractRequest {
     private static final String TRANSACTIONAL_ID_KEY_NAME = "transactional_id";
     private static final String ACKS_KEY_NAME = "acks";
@@ -43,13 +53,60 @@ public class ProduceRequest extends AbstractRequest {
     private static final String TOPIC_DATA_KEY_NAME = "topic_data";
 
     // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
     private static final String PARTITION_DATA_KEY_NAME = "data";
 
     // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
     private static final String RECORD_SET_KEY_NAME = "record_set";
 
+
+    private static final Schema TOPIC_PRODUCE_DATA_V0 = new Schema(
+            TOPIC_NAME,
+            new Field(PARTITION_DATA_KEY_NAME, new ArrayOf(new Schema(
+                    PARTITION_ID,
+                    new Field(RECORD_SET_KEY_NAME, RECORDS)))));
+
+    private static final Schema PRODUCE_REQUEST_V0 = new Schema(
+            new Field(ACKS_KEY_NAME, INT16, "The number of acknowledgments the producer requires the leader to have " +
+                    "received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for " +
+                    "only the leader and -1 for the full ISR."),
+            new Field(TIMEOUT_KEY_NAME, INT32, "The time to await a response in ms."),
+            new Field(TOPIC_DATA_KEY_NAME, new ArrayOf(TOPIC_PRODUCE_DATA_V0)));
+
+    /**
+     * The body of PRODUCE_REQUEST_V1 is the same as PRODUCE_REQUEST_V0.
+     * The version number is bumped up to indicate that the client supports quota throttle time field in the response.
+     */
+    private static final Schema PRODUCE_REQUEST_V1 = PRODUCE_REQUEST_V0;
+    /**
+     * The body of PRODUCE_REQUEST_V2 is the same as PRODUCE_REQUEST_V1.
+     * The version number is bumped up to indicate that message format V1 is used which has relative offset and
+     * timestamp.
+     */
+    private static final Schema PRODUCE_REQUEST_V2 = PRODUCE_REQUEST_V1;
+
+    // Produce request V3 adds the transactional id which is used for authorization when attempting to write
+    // transactional data. This version also adds support for message format V2.
+    private static final Schema PRODUCE_REQUEST_V3 = new Schema(
+            new Field(TRANSACTIONAL_ID_KEY_NAME, NULLABLE_STRING, "The transactional ID of the producer. This is used to " +
+                    "authorize transaction produce requests. This can be null for non-transactional producers."),
+            new Field(ACKS_KEY_NAME, INT16, "The number of acknowledgments the producer requires the leader to have " +
+                    "received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 " +
+                    "for only the leader and -1 for the full ISR."),
+            new Field(TIMEOUT_KEY_NAME, INT32, "The time to await a response in ms."),
+            new Field(TOPIC_DATA_KEY_NAME, new ArrayOf(TOPIC_PRODUCE_DATA_V0)));
+
+    /**
+     * The body of PRODUCE_REQUEST_V4 is the same as PRODUCE_REQUEST_V3.
+     * The version number is bumped up to indicate that the client supports KafkaStorageException.
+     * The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 3
+     */
+    private static final Schema PRODUCE_REQUEST_V4 = PRODUCE_REQUEST_V3;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {PRODUCE_REQUEST_V0, PRODUCE_REQUEST_V1, PRODUCE_REQUEST_V2, PRODUCE_REQUEST_V3,
+            PRODUCE_REQUEST_V4};
+    }
+
     public static class Builder extends AbstractRequest.Builder<ProduceRequest> {
         private final byte magic;
         private final short acks;
@@ -137,10 +194,10 @@ public class ProduceRequest extends AbstractRequest {
         partitionRecords = new HashMap<>();
         for (Object topicDataObj : struct.getArray(TOPIC_DATA_KEY_NAME)) {
             Struct topicData = (Struct) topicDataObj;
-            String topic = topicData.getString(TOPIC_KEY_NAME);
+            String topic = topicData.get(TOPIC_NAME);
             for (Object partitionResponseObj : topicData.getArray(PARTITION_DATA_KEY_NAME)) {
                 Struct partitionResponse = (Struct) partitionResponseObj;
-                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                int partition = partitionResponse.get(PARTITION_ID);
                 MemoryRecords records = (MemoryRecords) partitionResponse.getRecords(RECORD_SET_KEY_NAME);
                 validateRecords(version, records);
                 partitionRecords.put(new TopicPartition(topic, partition), records);
@@ -195,12 +252,12 @@ public class ProduceRequest extends AbstractRequest {
         List<Struct> topicDatas = new ArrayList<>(recordsByTopic.size());
         for (Map.Entry<String, Map<Integer, MemoryRecords>> topicEntry : recordsByTopic.entrySet()) {
             Struct topicData = struct.instance(TOPIC_DATA_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
+            topicData.set(TOPIC_NAME, topicEntry.getKey());
             List<Struct> partitionArray = new ArrayList<>();
             for (Map.Entry<Integer, MemoryRecords> partitionEntry : topicEntry.getValue().entrySet()) {
                 MemoryRecords records = partitionEntry.getValue();
                 Struct part = topicData.instance(PARTITION_DATA_KEY_NAME)
-                        .set(PARTITION_KEY_NAME, partitionEntry.getKey())
+                        .set(PARTITION_ID, partitionEntry.getKey())
                         .set(RECORD_SET_KEY_NAME, records);
                 partitionArray.add(part);
             }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
index fdfba8b..4786307 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
@@ -19,6 +19,9 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.record.RecordBatch;
 import org.apache.kafka.common.utils.CollectionUtils;
@@ -29,6 +32,12 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.INT64;
+
 /**
  * This wrapper supports both v0 and v1 of ProduceResponse.
  */
@@ -37,13 +46,8 @@ public class ProduceResponse extends AbstractResponse {
     private static final String RESPONSES_KEY_NAME = "responses";
 
     // topic level field names
-    private static final String TOPIC_KEY_NAME = "topic";
     private static final String PARTITION_RESPONSES_KEY_NAME = "partition_responses";
 
-    // partition level field names
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
-
     public static final long INVALID_OFFSET = -1L;
 
     /**
@@ -68,6 +72,55 @@ public class ProduceResponse extends AbstractResponse {
     private static final String BASE_OFFSET_KEY_NAME = "base_offset";
     private static final String LOG_APPEND_TIME_KEY_NAME = "log_append_time";
 
+    private static final Schema PRODUCE_RESPONSE_V0 = new Schema(
+            new Field(RESPONSES_KEY_NAME, new ArrayOf(new Schema(
+                    TOPIC_NAME,
+                    new Field(PARTITION_RESPONSES_KEY_NAME, new ArrayOf(new Schema(
+                            PARTITION_ID,
+                            ERROR_CODE,
+                            new Field(BASE_OFFSET_KEY_NAME, INT64))))))));
+
+    private static final Schema PRODUCE_RESPONSE_V1 = new Schema(
+            new Field(RESPONSES_KEY_NAME, new ArrayOf(new Schema(
+                    TOPIC_NAME,
+                    new Field(PARTITION_RESPONSES_KEY_NAME, new ArrayOf(new Schema(
+                            PARTITION_ID,
+                            ERROR_CODE,
+                            new Field(BASE_OFFSET_KEY_NAME, INT64))))))),
+            THROTTLE_TIME_MS);
+
+    /**
+     * PRODUCE_RESPONSE_V2 added a timestamp field in the per partition response status.
+     * The timestamp is log append time if the topic is configured to use log append time. Or it is NoTimestamp when create
+     * time is used for the topic.
+     */
+    private static final Schema PRODUCE_RESPONSE_V2 = new Schema(
+            new Field(RESPONSES_KEY_NAME, new ArrayOf(new Schema(
+                    TOPIC_NAME,
+                    new Field(PARTITION_RESPONSES_KEY_NAME, new ArrayOf(new Schema(
+                            PARTITION_ID,
+                            ERROR_CODE,
+                            new Field(BASE_OFFSET_KEY_NAME, INT64),
+                            new Field(LOG_APPEND_TIME_KEY_NAME, INT64, "The timestamp returned by broker after appending " +
+                                    "the messages. If CreateTime is used for the topic, the timestamp will be -1. " +
+                                    "If LogAppendTime is used for the topic, the timestamp will be " +
+                                    "the broker local time when the messages are appended."))))))),
+            THROTTLE_TIME_MS);
+
+    private static final Schema PRODUCE_RESPONSE_V3 = PRODUCE_RESPONSE_V2;
+
+    /**
+     * The body of PRODUCE_RESPONSE_V4 is the same as PRODUCE_RESPONSE_V3.
+     * The version number is bumped up to indicate that the client supports KafkaStorageException.
+     * The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 3
+     */
+    private static final Schema PRODUCE_RESPONSE_V4 = PRODUCE_RESPONSE_V3;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{PRODUCE_RESPONSE_V0, PRODUCE_RESPONSE_V1, PRODUCE_RESPONSE_V2, PRODUCE_RESPONSE_V3,
+            PRODUCE_RESPONSE_V4};
+    }
+
     private final Map<TopicPartition, PartitionResponse> responses;
     private final int throttleTime;
 
@@ -96,18 +149,18 @@ public class ProduceResponse extends AbstractResponse {
         responses = new HashMap<>();
         for (Object topicResponse : struct.getArray(RESPONSES_KEY_NAME)) {
             Struct topicRespStruct = (Struct) topicResponse;
-            String topic = topicRespStruct.getString(TOPIC_KEY_NAME);
+            String topic = topicRespStruct.get(TOPIC_NAME);
             for (Object partResponse : topicRespStruct.getArray(PARTITION_RESPONSES_KEY_NAME)) {
                 Struct partRespStruct = (Struct) partResponse;
-                int partition = partRespStruct.getInt(PARTITION_KEY_NAME);
-                Errors error = Errors.forCode(partRespStruct.getShort(ERROR_CODE_KEY_NAME));
+                int partition = partRespStruct.get(PARTITION_ID);
+                Errors error = Errors.forCode(partRespStruct.get(ERROR_CODE));
                 long offset = partRespStruct.getLong(BASE_OFFSET_KEY_NAME);
                 long logAppendTime = partRespStruct.getLong(LOG_APPEND_TIME_KEY_NAME);
                 TopicPartition tp = new TopicPartition(topic, partition);
                 responses.put(tp, new PartitionResponse(error, offset, logAppendTime));
             }
         }
-        this.throttleTime = struct.getInt(THROTTLE_TIME_KEY_NAME);
+        this.throttleTime = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME);
     }
 
     @Override
@@ -118,7 +171,7 @@ public class ProduceResponse extends AbstractResponse {
         List<Struct> topicDatas = new ArrayList<>(responseByTopic.size());
         for (Map.Entry<String, Map<Integer, PartitionResponse>> entry : responseByTopic.entrySet()) {
             Struct topicData = struct.instance(RESPONSES_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, entry.getKey());
+            topicData.set(TOPIC_NAME, entry.getKey());
             List<Struct> partitionArray = new ArrayList<>();
             for (Map.Entry<Integer, PartitionResponse> partitionEntry : entry.getValue().entrySet()) {
                 PartitionResponse part = partitionEntry.getValue();
@@ -130,8 +183,8 @@ public class ProduceResponse extends AbstractResponse {
                 if (errorCode == Errors.KAFKA_STORAGE_ERROR.code() && version <= 3)
                     errorCode = Errors.NOT_LEADER_FOR_PARTITION.code();
                 Struct partStruct = topicData.instance(PARTITION_RESPONSES_KEY_NAME)
-                        .set(PARTITION_KEY_NAME, partitionEntry.getKey())
-                        .set(ERROR_CODE_KEY_NAME, errorCode)
+                        .set(PARTITION_ID, partitionEntry.getKey())
+                        .set(ERROR_CODE, errorCode)
                         .set(BASE_OFFSET_KEY_NAME, part.baseOffset);
                 if (partStruct.hasField(LOG_APPEND_TIME_KEY_NAME))
                     partStruct.set(LOG_APPEND_TIME_KEY_NAME, part.logAppendTime);
@@ -141,9 +194,8 @@ public class ProduceResponse extends AbstractResponse {
             topicDatas.add(topicData);
         }
         struct.set(RESPONSES_KEY_NAME, topicDatas.toArray());
+        struct.setIfExists(THROTTLE_TIME_MS, throttleTime);
 
-        if (struct.hasField(THROTTLE_TIME_KEY_NAME))
-            struct.set(THROTTLE_TIME_KEY_NAME, throttleTime);
         return struct;
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/RequestContext.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestContext.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestContext.java
index 34bb3f5..5132202 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/RequestContext.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestContext.java
@@ -20,7 +20,6 @@ import org.apache.kafka.common.errors.InvalidRequestException;
 import org.apache.kafka.common.network.ListenerName;
 import org.apache.kafka.common.network.Send;
 import org.apache.kafka.common.protocol.ApiKeys;
-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.security.auth.KafkaPrincipal;
@@ -86,7 +85,7 @@ public class RequestContext {
     }
 
     private boolean isUnsupportedApiVersionsRequest() {
-        return header.apiKey() == API_VERSIONS && !Protocol.apiVersionSupported(API_VERSIONS.id, header.apiVersion());
+        return header.apiKey() == API_VERSIONS && !API_VERSIONS.isVersionSupported(header.apiVersion());
     }
 
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java
index 43b7baf..1284e7e 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java
@@ -18,13 +18,16 @@ package org.apache.kafka.common.requests;
 
 import org.apache.kafka.common.errors.InvalidRequestException;
 import org.apache.kafka.common.protocol.ApiKeys;
-import org.apache.kafka.common.protocol.Protocol;
+import org.apache.kafka.common.protocol.types.Field;
 import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
 
 import static java.util.Objects.requireNonNull;
+import static org.apache.kafka.common.protocol.types.Type.INT16;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING;
 
 /**
  * The header for a request in the Kafka protocol
@@ -35,6 +38,19 @@ public class RequestHeader extends AbstractRequestResponse {
     private static final String CLIENT_ID_FIELD_NAME = "client_id";
     private static final String CORRELATION_ID_FIELD_NAME = "correlation_id";
 
+    public static final Schema SCHEMA = new Schema(
+            new Field(API_KEY_FIELD_NAME, INT16, "The id of the request type."),
+            new Field(API_VERSION_FIELD_NAME, INT16, "The version of the API."),
+            new Field(CORRELATION_ID_FIELD_NAME, INT32, "A user-supplied integer value that will be passed back with the response"),
+            new Field(CLIENT_ID_FIELD_NAME, NULLABLE_STRING, "A user specified identifier for the client making the request.", ""));
+
+    // Version 0 of the controlled shutdown API used a non-standard request header (the clientId is missing).
+    // This can be removed once we drop support for that version.
+    private static final Schema CONTROLLED_SHUTDOWN_V0_SCHEMA = new Schema(
+            new Field(API_KEY_FIELD_NAME, INT16, "The id of the request type."),
+            new Field(API_VERSION_FIELD_NAME, INT16, "The version of the API."),
+            new Field(CORRELATION_ID_FIELD_NAME, INT32, "A user-supplied integer value that will be passed back with the response"));
+
     private final ApiKeys apiKey;
     private final short apiVersion;
     private final String clientId;
@@ -64,7 +80,7 @@ public class RequestHeader extends AbstractRequestResponse {
     }
 
     public Struct toStruct() {
-        Schema schema = Protocol.requestHeaderSchema(apiKey.id, apiVersion);
+        Schema schema = schema(apiKey.id, apiVersion);
         Struct struct = new Struct(schema);
         struct.set(API_KEY_FIELD_NAME, apiKey.id);
         struct.set(API_VERSION_FIELD_NAME, apiVersion);
@@ -100,7 +116,7 @@ public class RequestHeader extends AbstractRequestResponse {
         try {
             short apiKey = buffer.getShort();
             short apiVersion = buffer.getShort();
-            Schema schema = Protocol.requestHeaderSchema(apiKey, apiVersion);
+            Schema schema = schema(apiKey, apiVersion);
             buffer.rewind();
             return new RequestHeader(schema.read(buffer));
         } catch (InvalidRequestException e) {
@@ -140,4 +156,13 @@ public class RequestHeader extends AbstractRequestResponse {
         result = 31 * result + correlationId;
         return result;
     }
+
+
+    private static Schema schema(short apiKey, short version) {
+        if (apiKey == ApiKeys.CONTROLLED_SHUTDOWN.id && version == 0)
+            // This will be removed once we remove support for v0 of ControlledShutdownRequest, which
+            // depends on a non-standard request header (it does not have a clientId)
+            return CONTROLLED_SHUTDOWN_V0_SCHEMA;
+        return SCHEMA;
+    }
 }


[7/8] kafka git commit: MINOR: Move request/response schemas to the corresponding object representation

Posted by ij...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
----------------------------------------------------------------------
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 c7431d0..b5042c3 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
@@ -17,2058 +17,19 @@
 package org.apache.kafka.common.protocol;
 
 import org.apache.kafka.common.protocol.types.ArrayOf;
-import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.BoundField;
 import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.requests.RequestHeader;
+import org.apache.kafka.common.requests.ResponseHeader;
 
-import java.util.EnumSet;
-import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.atomic.AtomicReference;
-
-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;
-import static org.apache.kafka.common.protocol.types.Type.INT64;
-import static org.apache.kafka.common.protocol.types.Type.INT8;
-import static org.apache.kafka.common.protocol.types.Type.NULLABLE_BYTES;
-import static org.apache.kafka.common.protocol.types.Type.RECORDS;
-import static org.apache.kafka.common.protocol.types.Type.STRING;
-import static org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING;
 
 public class Protocol {
 
-    public static final Schema REQUEST_HEADER = new Schema(
-            new Field("api_key", INT16, "The id of the request type."),
-            new Field("api_version", INT16, "The version of the API."),
-            new Field("correlation_id", INT32, "A user-supplied integer value that will be passed back with the response"),
-            new Field("client_id", NULLABLE_STRING, "A user specified identifier for the client making the request.", ""));
-
-    // Version 0 of the controlled shutdown API used a non-standard request header (the clientId is missing).
-    // This can be removed once we drop support for that version.
-    public static final Schema CONTROLLED_SHUTDOWN_REQUEST_V0_HEADER = new Schema(
-            new Field("api_key", INT16, "The id of the request type."),
-            new Field("api_version", INT16, "The version of the API."),
-            new Field("correlation_id", INT32, "A user-supplied integer value that will be passed back with the response"));
-
-
-    public static final Schema RESPONSE_HEADER = new Schema(new Field("correlation_id",
-                                                                      INT32,
-                                                                      "The user-supplied value passed in with the request"));
-
-    /* Metadata api */
-
-    public static final Schema METADATA_REQUEST_V0 = new Schema(new Field("topics",
-                                                                          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 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."));
-
-    /* The v2 metadata request is the same as v1. An additional field for cluster id has been added to the v2 metadata response */
-    public static final Schema METADATA_REQUEST_V2 = METADATA_REQUEST_V1;
-
-    /* The v3 metadata request is the same as v1 and v2. An additional field for throttle time has been added to the v3 metadata response */
-    public static final Schema METADATA_REQUEST_V3 = METADATA_REQUEST_V2;
-
-    /* The v4 metadata request has an additional field for allowing auto topic creation. The response is the same as v3. */
-    public static final Schema METADATA_REQUEST_V4 = 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."),
-                                                                new Field("allow_auto_topic_creation",
-                                                                          BOOLEAN,
-                                                                          "If this and the broker config 'auto.create.topics.enable' are true, " +
-                                                                          "topics that don't exist will be created by the broker. " +
-                                                                          "Otherwise, no topics will be created by the broker."));
-
-    /* The v5 metadata request is the same as v4. An additional field for offline_replicas has been added to the v5 metadata response */
-    public static final Schema METADATA_REQUEST_V5 = METADATA_REQUEST_V4;
-
-    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,
-                                                             "The port on which the broker accepts requests."));
-
-    public static final Schema PARTITION_METADATA_V0 = new Schema(new Field("partition_error_code",
-                                                                            INT16,
-                                                                            "The error code for the partition, if any."),
-                                                                  new Field("partition_id",
-                                                                            INT32,
-                                                                            "The id of the partition."),
-                                                                  new Field("leader",
-                                                                            INT32,
-                                                                            "The id of the broker acting as leader for this partition."),
-                                                                  new Field("replicas",
-                                                                            new ArrayOf(INT32),
-                                                                            "The set of all nodes that host this partition."),
-                                                                  new Field("isr",
-                                                                            new ArrayOf(INT32),
-                                                                            "The set of nodes that are in sync with the leader for this partition."));
-
-    public static final Schema TOPIC_METADATA_V0 = 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("partition_metadata",
-                                                                        new ArrayOf(PARTITION_METADATA_V0),
-                                                                        "Metadata for each partition of the topic."));
-
-    public static final Schema METADATA_RESPONSE_V0 = new Schema(new Field("brokers",
-                                                                           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_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;
-
-    // PARTITION_METADATA_V2 added a per-partition offline_replicas field. This field specifies the list of replicas that are offline.
-    public static final Schema PARTITION_METADATA_V2 = new Schema(new Field("partition_error_code",
-                                                                            INT16,
-                                                                            "The error code for the partition, if any."),
-                                                                  new Field("partition_id",
-                                                                            INT32,
-                                                                            "The id of the partition."),
-                                                                  new Field("leader",
-                                                                            INT32,
-                                                                            "The id of the broker acting as leader for this partition."),
-                                                                  new Field("replicas",
-                                                                            new ArrayOf(INT32),
-                                                                            "The set of all nodes that host this partition."),
-                                                                  new Field("isr",
-                                                                            new ArrayOf(INT32),
-                                                                            "The set of nodes that are in sync with the leader for this partition."),
-                                                                  new Field("offline_replicas",
-                                                                            new ArrayOf(INT32),
-                                                                            "The set of offline replicas of this partition."));
-
-    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."));
-
-    // TOPIC_METADATA_V2 added a per-partition offline_replicas field. This field specifies the list of replicas that are offline.
-    public static final Schema TOPIC_METADATA_V2 = 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_V2),
-                                                                  "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_RESPONSE_V2 = new Schema(new Field("brokers", new ArrayOf(METADATA_BROKER_V1),
-                                                                    "Host and port information for all brokers."),
-                                                                 new Field("cluster_id", NULLABLE_STRING,
-                                                                     "The cluster id that this broker belongs to."),
-                                                                 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_RESPONSE_V3 = new Schema(
-         newThrottleTimeField(),
-         new Field("brokers", new ArrayOf(METADATA_BROKER_V1),
-            "Host and port information for all brokers."),
-         new Field("cluster_id", NULLABLE_STRING,
-             "The cluster id that this broker belongs to."),
-         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_RESPONSE_V4 = METADATA_RESPONSE_V3;
-
-    // METADATA_RESPONSE_V5 added a per-partition offline_replicas field. This field specifies the list of replicas that are offline.
-    public static final Schema METADATA_RESPONSE_V5 = new Schema(
-        newThrottleTimeField(),
-        new Field("brokers", new ArrayOf(METADATA_BROKER_V1),
-            "Host and port information for all brokers."),
-        new Field("cluster_id", NULLABLE_STRING,
-            "The cluster id that this broker belongs to."),
-        new Field("controller_id", INT32,
-            "The broker id of the controller broker."),
-        new Field("topic_metadata", new ArrayOf(TOPIC_METADATA_V2)));
-
-    public static final Schema[] METADATA_REQUEST = {METADATA_REQUEST_V0, METADATA_REQUEST_V1, METADATA_REQUEST_V2, METADATA_REQUEST_V3, METADATA_REQUEST_V4, METADATA_REQUEST_V5};
-    public static final Schema[] METADATA_RESPONSE = {METADATA_RESPONSE_V0, METADATA_RESPONSE_V1, METADATA_RESPONSE_V2, METADATA_RESPONSE_V3, METADATA_RESPONSE_V4, METADATA_RESPONSE_V5};
-
-    /* Produce api */
-
-    public static final Schema TOPIC_PRODUCE_DATA_V0 = new Schema(new Field("topic", STRING),
-                                                                  new Field("data", new ArrayOf(new Schema(new Field("partition", INT32),
-                                                                                                     new Field("record_set", RECORDS)))));
-
-    public static final Schema PRODUCE_REQUEST_V0 = new Schema(new Field("acks",
-                                                                   INT16,
-                                                                   "The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR."),
-                                                               new Field("timeout", INT32, "The time to await a response in ms."),
-                                                               new Field("topic_data", new ArrayOf(TOPIC_PRODUCE_DATA_V0)));
-
-    public static final Schema PRODUCE_RESPONSE_V0 = new Schema(new Field("responses",
-                                                                    new ArrayOf(new Schema(new Field("topic", STRING),
-                                                                                           new Field("partition_responses",
-                                                                                                     new ArrayOf(new Schema(new Field("partition",
-                                                                                                                                      INT32),
-                                                                                                                            new Field("error_code",
-                                                                                                                                      INT16),
-                                                                                                                            new Field("base_offset",
-                                                                                                                                      INT64))))))));
-    /**
-     * The body of PRODUCE_REQUEST_V1 is the same as PRODUCE_REQUEST_V0.
-     * The version number is bumped up to indicate that the client supports quota throttle time field in the response.
-     */
-    public static final Schema PRODUCE_REQUEST_V1 = PRODUCE_REQUEST_V0;
-    /**
-     * The body of PRODUCE_REQUEST_V2 is the same as PRODUCE_REQUEST_V1.
-     * The version number is bumped up to indicate that message format V1 is used which has relative offset and
-     * timestamp.
-     */
-    public static final Schema PRODUCE_REQUEST_V2 = PRODUCE_REQUEST_V1;
-
-    // Produce request V3 adds the transactional id which is used for authorization when attempting to write
-    // transactional data. This version also adds support for message format V2.
-    public static final Schema PRODUCE_REQUEST_V3 = new Schema(
-            new Field("transactional_id",
-                    NULLABLE_STRING,
-                    "The transactional ID of the producer. This is used to authorize transaction produce requests. " +
-                    "This can be null for non-transactional producers."),
-            new Field("acks",
-                    INT16,
-                    "The number of acknowledgments the producer requires the leader to have received before " +
-                    "considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader " +
-                    "and -1 for the full ISR."),
-            new Field("timeout", INT32, "The time to await a response in ms."),
-            new Field("topic_data", new ArrayOf(TOPIC_PRODUCE_DATA_V0)));
-
-    /**
-     * The body of PRODUCE_REQUEST_V4 is the same as PRODUCE_REQUEST_V3.
-     * The version number is bumped up to indicate that the client supports KafkaStorageException.
-     * The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 3
-     */
-    public static final Schema PRODUCE_REQUEST_V4 = PRODUCE_REQUEST_V3;
-
-    public static final Schema PRODUCE_RESPONSE_V1 = new Schema(new Field("responses",
-                                                                          new ArrayOf(new Schema(new Field("topic", STRING),
-                                                                                                 new Field("partition_responses",
-                                                                                                           new ArrayOf(new Schema(new Field("partition",
-                                                                                                                                            INT32),
-                                                                                                                                  new Field("error_code",
-                                                                                                                                            INT16),
-                                                                                                                                  new Field("base_offset",
-                                                                                                                                            INT64))))))),
-                                                                newThrottleTimeField());
-    /**
-     * PRODUCE_RESPONSE_V2 added a timestamp field in the per partition response status.
-     * The timestamp is log append time if the topic is configured to use log append time. Or it is NoTimestamp when create
-     * time is used for the topic.
-     */
-    public static final Schema PRODUCE_RESPONSE_V2 = new Schema(new Field("responses",
-                                                                new ArrayOf(new Schema(new Field("topic", STRING),
-                                                                                       new Field("partition_responses",
-                                                                                       new ArrayOf(new Schema(new Field("partition",
-                                                                                                                        INT32),
-                                                                                                              new Field("error_code",
-                                                                                                                        INT16),
-                                                                                                              new Field("base_offset",
-                                                                                                                        INT64),
-                                                                                                              new Field("log_append_time",
-                                                                                                                        INT64,
-                                                                                                                        "The timestamp returned by broker after appending the messages. " +
-                                                                                                                            "If CreateTime is used for the topic, the timestamp will be -1. " +
-                                                                                                                            "If LogAppendTime is used for the topic, the timestamp will be " +
-                                                                                                                            "the broker local time when the messages are appended."))))))),
-                                                                newThrottleTimeField());
-
-    public static final Schema PRODUCE_RESPONSE_V3 = PRODUCE_RESPONSE_V2;
-
-    /**
-     * The body of PRODUCE_RESPONSE_V4 is the same as PRODUCE_RESPONSE_V3.
-     * The version number is bumped up to indicate that the client supports KafkaStorageException.
-     * The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 3
-     */
-    public static final Schema PRODUCE_RESPONSE_V4 = PRODUCE_RESPONSE_V3;
-
-    public static final Schema[] PRODUCE_REQUEST = {PRODUCE_REQUEST_V0, PRODUCE_REQUEST_V1, PRODUCE_REQUEST_V2, PRODUCE_REQUEST_V3, PRODUCE_REQUEST_V4};
-    public static final Schema[] PRODUCE_RESPONSE = {PRODUCE_RESPONSE_V0, PRODUCE_RESPONSE_V1, PRODUCE_RESPONSE_V2, PRODUCE_RESPONSE_V3, PRODUCE_RESPONSE_V4};
-
-    /* Offset commit api */
-    public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                         INT32,
-                                                                                         "Topic partition id."),
-                                                                               new Field("offset",
-                                                                                         INT64,
-                                                                                         "Message offset to be committed."),
-                                                                               new Field("metadata",
-                                                                                         NULLABLE_STRING,
-                                                                                         "Any associated metadata the client wants to keep."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V1 = new Schema(new Field("partition",
-                                                                                         INT32,
-                                                                                         "Topic partition id."),
-                                                                               new Field("offset",
-                                                                                         INT64,
-                                                                                         "Message offset to be committed."),
-                                                                               new Field("timestamp",
-                                                                                         INT64,
-                                                                                         "Timestamp of the commit"),
-                                                                               new Field("metadata",
-                                                                                         NULLABLE_STRING,
-                                                                                         "Any associated metadata the client wants to keep."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V2 = new Schema(new Field("partition",
-                                                                                         INT32,
-                                                                                         "Topic partition id."),
-                                                                               new Field("offset",
-                                                                                         INT64,
-                                                                                         "Message offset to be committed."),
-                                                                               new Field("metadata",
-                                                                                         NULLABLE_STRING,
-                                                                                         "Any associated metadata the client wants to keep."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V0 = new Schema(new Field("topic",
-                                                                                     STRING,
-                                                                                     "Topic to commit."),
-                                                                           new Field("partitions",
-                                                                                     new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V0),
-                                                                                     "Partitions to commit offsets."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V1 = new Schema(new Field("topic",
-                                                                                     STRING,
-                                                                                     "Topic to commit."),
-                                                                           new Field("partitions",
-                                                                                     new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V1),
-                                                                                     "Partitions to commit offsets."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V2 = new Schema(new Field("topic",
-                                                                                     STRING,
-                                                                                     "Topic to commit."),
-                                                                           new Field("partitions",
-                                                                                     new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V2),
-                                                                                     "Partitions to commit offsets."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_V0 = new Schema(new Field("group_id",
-                                                                               STRING,
-                                                                               "The group id."),
-                                                                     new Field("topics",
-                                                                               new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0),
-                                                                               "Topics to commit offsets."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_V1 = new Schema(new Field("group_id",
-                                                                               STRING,
-                                                                               "The group id."),
-                                                                     new Field("group_generation_id",
-                                                                               INT32,
-                                                                               "The generation of the group."),
-                                                                     new Field("member_id",
-                                                                               STRING,
-                                                                               "The member id assigned by the group coordinator."),
-                                                                     new Field("topics",
-                                                                               new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V1),
-                                                                               "Topics to commit offsets."));
-
-    public static final Schema OFFSET_COMMIT_REQUEST_V2 = new Schema(new Field("group_id",
-                                                                               STRING,
-                                                                               "The group id."),
-                                                                     new Field("group_generation_id",
-                                                                               INT32,
-                                                                               "The generation of the consumer group."),
-                                                                     new Field("member_id",
-                                                                               STRING,
-                                                                               "The consumer id assigned by the group coordinator."),
-                                                                     new Field("retention_time",
-                                                                               INT64,
-                                                                               "Time period in ms to retain the offset."),
-                                                                     new Field("topics",
-                                                                               new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V2),
-                                                                               "Topics to commit offsets."));
-
-    /* v3 request is same as v2. Throttle time has been added to response */
-    public static final Schema OFFSET_COMMIT_REQUEST_V3 = OFFSET_COMMIT_REQUEST_V2;
-
-    public static final Schema OFFSET_COMMIT_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                          INT32,
-                                                                                          "Topic partition id."),
-                                                                                new Field("error_code",
-                                                                                          INT16));
-
-    public static final Schema OFFSET_COMMIT_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING),
-                                                                            new Field("partition_responses",
-                                                                                      new ArrayOf(OFFSET_COMMIT_RESPONSE_PARTITION_V0)));
-
-    public static final Schema OFFSET_COMMIT_RESPONSE_V0 = new Schema(new Field("responses",
-                                                                                new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0)));
-
-    public static final Schema[] OFFSET_COMMIT_REQUEST = {OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1, OFFSET_COMMIT_REQUEST_V2, OFFSET_COMMIT_REQUEST_V3};
-
-    /* The response types for V0, V1 and V2 of OFFSET_COMMIT_REQUEST are the same. */
-    public static final Schema OFFSET_COMMIT_RESPONSE_V1 = OFFSET_COMMIT_RESPONSE_V0;
-    public static final Schema OFFSET_COMMIT_RESPONSE_V2 = OFFSET_COMMIT_RESPONSE_V0;
-
-    public static final Schema OFFSET_COMMIT_RESPONSE_V3 = new Schema(
-            newThrottleTimeField(),
-            new Field("responses",
-                       new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0)));
-
-    public static final Schema[] OFFSET_COMMIT_RESPONSE = {OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V1, OFFSET_COMMIT_RESPONSE_V2, OFFSET_COMMIT_RESPONSE_V3};
-
-    /* Offset fetch api */
-
-    /*
-     * Wire formats of version 0 and 1 are the same, but with different functionality.
-     * Wire format of version 2 is similar to version 1, with the exception of
-     * - accepting 'null' as list of topics
-     * - returning a top level error code
-     * Version 0 will read the offsets from ZK.
-     * Version 1 will read the offsets from Kafka.
-     * Version 2 will read the offsets from Kafka, and returns all associated topic partition offsets if
-     * a 'null' is passed instead of a list of specific topic partitions. It also returns a top level error code
-     * for group or coordinator level errors.
-     */
-    public static final Schema OFFSET_FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                        INT32,
-                                                                                        "Topic partition id."));
-
-    public static final Schema OFFSET_FETCH_REQUEST_TOPIC_V0 = new Schema(new Field("topic",
-                                                                                    STRING,
-                                                                                    "Topic to fetch offset."),
-                                                                          new Field("partitions",
-                                                                                    new ArrayOf(OFFSET_FETCH_REQUEST_PARTITION_V0),
-                                                                                    "Partitions to fetch offsets."));
-
-    public static final Schema OFFSET_FETCH_REQUEST_V0 = new Schema(new Field("group_id",
-                                                                              STRING,
-                                                                              "The consumer group id."),
-                                                                    new Field("topics",
-                                                                              new ArrayOf(OFFSET_FETCH_REQUEST_TOPIC_V0),
-                                                                              "Topics to fetch offsets."));
-
-    public static final Schema OFFSET_FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                         INT32,
-                                                                                         "Topic partition id."),
-                                                                               new Field("offset",
-                                                                                         INT64,
-                                                                                         "Last committed message offset."),
-                                                                               new Field("metadata",
-                                                                                         NULLABLE_STRING,
-                                                                                         "Any associated metadata the client wants to keep."),
-                                                                               new Field("error_code", INT16));
-
-    public static final Schema OFFSET_FETCH_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING),
-                                                                           new Field("partition_responses",
-                                                                                     new ArrayOf(OFFSET_FETCH_RESPONSE_PARTITION_V0)));
-
-    public static final Schema OFFSET_FETCH_RESPONSE_V0 = new Schema(new Field("responses",
-                                                                               new ArrayOf(OFFSET_FETCH_RESPONSE_TOPIC_V0)));
-
-    public static final Schema OFFSET_FETCH_REQUEST_V1 = OFFSET_FETCH_REQUEST_V0;
-    public static final Schema OFFSET_FETCH_RESPONSE_V1 = OFFSET_FETCH_RESPONSE_V0;
-
-    public static final Schema OFFSET_FETCH_REQUEST_V2 = new Schema(new Field("group_id",
-                                                                              STRING,
-                                                                              "The consumer group id."),
-                                                                              new Field("topics",
-                                                                                        ArrayOf.nullable(OFFSET_FETCH_REQUEST_TOPIC_V0),
-                                                                                        "Topics to fetch offsets. If the topic array is null fetch offsets for all topics."));
-
-    public static final Schema OFFSET_FETCH_RESPONSE_V2 = new Schema(new Field("responses",
-                                                                               new ArrayOf(OFFSET_FETCH_RESPONSE_TOPIC_V0)),
-                                                                     new Field("error_code",
-                                                                               INT16));
-
-    /* v3 request is the same as v2. Throttle time has been added to v3 response */
-    public static final Schema OFFSET_FETCH_REQUEST_V3 = OFFSET_FETCH_REQUEST_V2;
-    public static final Schema OFFSET_FETCH_RESPONSE_V3 = new Schema(
-            newThrottleTimeField(),
-            new Field("responses",
-                    new ArrayOf(OFFSET_FETCH_RESPONSE_TOPIC_V0)),
-            new Field("error_code",
-                    INT16));
-
-    public static final Schema[] OFFSET_FETCH_REQUEST = {OFFSET_FETCH_REQUEST_V0, OFFSET_FETCH_REQUEST_V1, OFFSET_FETCH_REQUEST_V2, OFFSET_FETCH_REQUEST_V3};
-    public static final Schema[] OFFSET_FETCH_RESPONSE = {OFFSET_FETCH_RESPONSE_V0, OFFSET_FETCH_RESPONSE_V1, OFFSET_FETCH_RESPONSE_V2, OFFSET_FETCH_RESPONSE_V3};
-
-    /* List offset api */
-    public static final Schema LIST_OFFSET_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                       INT32,
-                                                                                       "Topic partition id."),
-                                                                             new Field("timestamp", INT64, "Timestamp."),
-                                                                             new Field("max_num_offsets",
-                                                                                       INT32,
-                                                                                       "Maximum offsets to return."));
-    public static final Schema LIST_OFFSET_REQUEST_PARTITION_V1 = new Schema(new Field("partition",
-                                                                                       INT32,
-                                                                                       "Topic partition id."),
-                                                                             new Field("timestamp",
-                                                                                       INT64,
-                                                                                       "The target timestamp for the partition."));
-
-    public static final Schema LIST_OFFSET_REQUEST_TOPIC_V0 = new Schema(new Field("topic",
-                                                                                   STRING,
-                                                                                   "Topic to list offset."),
-                                                                         new Field("partitions",
-                                                                                   new ArrayOf(LIST_OFFSET_REQUEST_PARTITION_V0),
-                                                                                   "Partitions to list offset."));
-    public static final Schema LIST_OFFSET_REQUEST_TOPIC_V1 = new Schema(new Field("topic",
-                                                                                   STRING,
-                                                                                   "Topic to list offset."),
-                                                                         new Field("partitions",
-                                                                                   new ArrayOf(LIST_OFFSET_REQUEST_PARTITION_V1),
-                                                                                   "Partitions to list offset."));
-
-    public static final Schema LIST_OFFSET_REQUEST_V0 = new Schema(new Field("replica_id",
-                                                                             INT32,
-                                                                             "Broker id of the follower. For normal consumers, use -1."),
-                                                                   new Field("topics",
-                                                                             new ArrayOf(LIST_OFFSET_REQUEST_TOPIC_V0),
-                                                                             "Topics to list offsets."));
-    public static final Schema LIST_OFFSET_REQUEST_V1 = new Schema(new Field("replica_id",
-                                                                             INT32,
-                                                                             "Broker id of the follower. For normal consumers, use -1."),
-                                                                   new Field("topics",
-                                                                             new ArrayOf(LIST_OFFSET_REQUEST_TOPIC_V1),
-                                                                             "Topics to list offsets."));
-
-    public static final Schema LIST_OFFSET_REQUEST_V2 = new Schema(
-            new Field("replica_id",
-                    INT32,
-                    "Broker id of the follower. For normal consumers, use -1."),
-            new Field("isolation_level",
-                    INT8,
-                    "This setting controls the visibility of transactional records. Using READ_UNCOMMITTED " +
-                            "(isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), " +
-                            "non-transactional and COMMITTED transactional records are visible. To be more concrete, " +
-                            "READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), " +
-                            "and enables the inclusion of the list of aborted transactions in the result, which allows " +
-                            "consumers to discard ABORTED transactional records"),
-            new Field("topics",
-                    new ArrayOf(LIST_OFFSET_REQUEST_TOPIC_V1),
-                    "Topics to list offsets."));;
-
-    public static final Schema LIST_OFFSET_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                        INT32,
-                                                                                        "Topic partition id."),
-                                                                              new Field("error_code", INT16),
-                                                                              new Field("offsets",
-                                                                                        new ArrayOf(INT64),
-                                                                                        "A list of offsets."));
-
-    public static final Schema LIST_OFFSET_RESPONSE_PARTITION_V1 = new Schema(new Field("partition",
-                                                                                        INT32,
-                                                                                        "Topic partition id."),
-                                                                              new Field("error_code", INT16),
-                                                                              new Field("timestamp",
-                                                                                        INT64,
-                                                                                        "The timestamp associated with the returned offset"),
-                                                                              new Field("offset",
-                                                                                        INT64,
-                                                                                        "offset found"));
-
-    public static final Schema LIST_OFFSET_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING),
-                                                                          new Field("partition_responses",
-                                                                                    new ArrayOf(LIST_OFFSET_RESPONSE_PARTITION_V0)));
-
-    public static final Schema LIST_OFFSET_RESPONSE_TOPIC_V1 = new Schema(new Field("topic", STRING),
-                                                                          new Field("partition_responses",
-                                                                                    new ArrayOf(LIST_OFFSET_RESPONSE_PARTITION_V1)));
-
-    public static final Schema LIST_OFFSET_RESPONSE_V0 = new Schema(new Field("responses",
-                                                                              new ArrayOf(LIST_OFFSET_RESPONSE_TOPIC_V0)));
-
-    public static final Schema LIST_OFFSET_RESPONSE_V1 = new Schema(new Field("responses",
-                                                                              new ArrayOf(LIST_OFFSET_RESPONSE_TOPIC_V1)));
-    public static final Schema LIST_OFFSET_RESPONSE_V2 = new Schema(
-            newThrottleTimeField(),
-            new Field("responses",
-                    new ArrayOf(LIST_OFFSET_RESPONSE_TOPIC_V1)));
-
-    public static final Schema[] LIST_OFFSET_REQUEST = {LIST_OFFSET_REQUEST_V0, LIST_OFFSET_REQUEST_V1, LIST_OFFSET_REQUEST_V2};
-    public static final Schema[] LIST_OFFSET_RESPONSE = {LIST_OFFSET_RESPONSE_V0, LIST_OFFSET_RESPONSE_V1, LIST_OFFSET_RESPONSE_V2};
-
-    /* Fetch api */
-    public static final Schema FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
-                                                                                 INT32,
-                                                                                 "Topic partition id."),
-                                                                       new Field("fetch_offset",
-                                                                                 INT64,
-                                                                                 "Message offset."),
-                                                                       new Field("max_bytes",
-                                                                                 INT32,
-                                                                                 "Maximum bytes to fetch."));
-
-    // FETCH_REQUEST_PARTITION_V1 added log_start_offset field - the earliest available offset of partition data that can be consumed.
-    public static final Schema FETCH_REQUEST_PARTITION_V5 = new Schema(new Field("partition",
-                                                                                 INT32,
-                                                                                 "Topic partition id."),
-                                                                       new Field("fetch_offset",
-                                                                                 INT64,
-                                                                                 "Message offset."),
-                                                                       new Field("log_start_offset",
-                                                                                 INT64,
-                                                                                 "Earliest available offset of the follower replica. " +
-                                                                                 "The field is only used when request is sent by follower. "),
-                                                                       new Field("max_bytes",
-                                                                                 INT32,
-                                                                                 "Maximum bytes to fetch."));
-
-    public static final Schema FETCH_REQUEST_TOPIC_V0 = new Schema(new Field("topic", STRING, "Topic to fetch."),
-                                                                   new Field("partitions",
-                                                                             new ArrayOf(FETCH_REQUEST_PARTITION_V0),
-                                                                             "Partitions to fetch."));
-
-    public static final Schema FETCH_REQUEST_TOPIC_V5 = new Schema(new Field("topic", STRING, "Topic to fetch."),
-                                                                   new Field("partitions",
-                                                                             new ArrayOf(FETCH_REQUEST_PARTITION_V5),
-                                                                             "Partitions to fetch."));
-
-    public static final Schema FETCH_REQUEST_V0 = new Schema(new Field("replica_id",
-                                                                       INT32,
-                                                                       "Broker id of the follower. For normal consumers, use -1."),
-                                                             new Field("max_wait_time",
-                                                                       INT32,
-                                                                       "Maximum time in ms to wait for the response."),
-                                                             new Field("min_bytes",
-                                                                       INT32,
-                                                                       "Minimum bytes to accumulate in the response."),
-                                                             new Field("topics",
-                                                                       new ArrayOf(FETCH_REQUEST_TOPIC_V0),
-                                                                       "Topics to fetch."));
-
-    // The V1 Fetch Request body is the same as V0.
-    // Only the version number is incremented to indicate a newer client
-    public static final Schema FETCH_REQUEST_V1 = FETCH_REQUEST_V0;
-    // The V2 Fetch Request body is the same as V1.
-    // Only the version number is incremented to indicate the client support message format V1 which uses
-    // relative offset and has timestamp.
-    public static final Schema FETCH_REQUEST_V2 = FETCH_REQUEST_V1;
-    // Fetch Request V3 added top level max_bytes field - the total size of partition data to accumulate in response.
-    // The partition ordering is now relevant - partitions will be processed in order they appear in request.
-    public static final Schema FETCH_REQUEST_V3 = new Schema(new Field("replica_id",
-                                                                       INT32,
-                                                                       "Broker id of the follower. For normal consumers, use -1."),
-                                                             new Field("max_wait_time",
-                                                                       INT32,
-                                                                       "Maximum time in ms to wait for the response."),
-                                                             new Field("min_bytes",
-                                                                       INT32,
-                                                                       "Minimum bytes to accumulate in the response."),
-                                                             new Field("max_bytes",
-                                                                       INT32,
-                                                                       "Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, " +
-                                                                       "if the first message in the first non-empty partition of the fetch is larger than this " +
-                                                                       "value, the message will still be returned to ensure that progress can be made."),
-                                                             new Field("topics",
-                                                                       new ArrayOf(FETCH_REQUEST_TOPIC_V0),
-                                                                       "Topics to fetch in the order provided."));
-
-    // The V4 Fetch Request adds the fetch isolation level and exposes magic v2 (via the response).
-    public static final Schema FETCH_REQUEST_V4 = new Schema(
-            new Field("replica_id",
-                    INT32,
-                    "Broker id of the follower. For normal consumers, use -1."),
-            new Field("max_wait_time",
-                    INT32,
-                    "Maximum time in ms to wait for the response."),
-            new Field("min_bytes",
-                    INT32,
-                    "Minimum bytes to accumulate in the response."),
-            new Field("max_bytes",
-                    INT32,
-                    "Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, " +
-                    "if the first message in the first non-empty partition of the fetch is larger than this " +
-                    "value, the message will still be returned to ensure that progress can be made."),
-            new Field("isolation_level",
-                    INT8,
-                    "This setting controls the visibility of transactional records. Using READ_UNCOMMITTED " +
-                    "(isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), " +
-                     "non-transactional and COMMITTED transactional records are visible. To be more concrete, " +
-                     "READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), " +
-                     "and enables the inclusion of the list of aborted transactions in the result, which allows " +
-                     "consumers to discard ABORTED transactional records"),
-            new Field("topics",
-                    new ArrayOf(FETCH_REQUEST_TOPIC_V0),
-                    "Topics to fetch in the order provided."));
-
-    // FETCH_REQUEST_V5 added a per-partition log_start_offset field - the earliest available offset of partition data that can be consumed.
-    public static final Schema FETCH_REQUEST_V5 = new Schema(
-            new Field("replica_id",
-                    INT32,
-                    "Broker id of the follower. For normal consumers, use -1."),
-            new Field("max_wait_time",
-                    INT32,
-                    "Maximum time in ms to wait for the response."),
-            new Field("min_bytes",
-                    INT32,
-                    "Minimum bytes to accumulate in the response."),
-            new Field("max_bytes",
-                    INT32,
-                    "Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, " +
-                    "if the first message in the first non-empty partition of the fetch is larger than this " +
-                    "value, the message will still be returned to ensure that progress can be made."),
-            new Field("isolation_level",
-                    INT8,
-                    "This setting controls the visibility of transactional records. Using READ_UNCOMMITTED " +
-                    "(isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), " +
-                     "non-transactional and COMMITTED transactional records are visible. To be more concrete, " +
-                     "READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), " +
-                     "and enables the inclusion of the list of aborted transactions in the result, which allows " +
-                     "consumers to discard ABORTED transactional records"),
-            new Field("topics",
-                    new ArrayOf(FETCH_REQUEST_TOPIC_V5),
-                    "Topics to fetch in the order provided."));
-
-    /**
-     * The body of FETCH_REQUEST_V6 is the same as FETCH_REQUEST_V5.
-     * The version number is bumped up to indicate that the client supports KafkaStorageException.
-     * The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 5
-     */
-    public static final Schema FETCH_REQUEST_V6 = FETCH_REQUEST_V5;
-
-    public static final Schema FETCH_RESPONSE_PARTITION_HEADER_V0 = new Schema(new Field("partition",
-                                                                                         INT32,
-                                                                                         "Topic partition id."),
-                                                                               new Field("error_code", INT16),
-                                                                               new Field("high_watermark",
-                                                                                         INT64,
-                                                                                         "Last committed offset."));
-    public static final Schema FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition_header", FETCH_RESPONSE_PARTITION_HEADER_V0),
-                                                                        new Field("record_set", RECORDS));
-
-    public static final Schema FETCH_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING),
-                                                                    new Field("partition_responses",
-                                                                              new ArrayOf(FETCH_RESPONSE_PARTITION_V0)));
-
-    public static final Schema FETCH_RESPONSE_V0 = new Schema(new Field("responses",
-                                                                        new ArrayOf(FETCH_RESPONSE_TOPIC_V0)));
-
-    public static final Schema FETCH_RESPONSE_V1 = new Schema(newThrottleTimeField(),
-                                                              new Field("responses",
-                                                                      new ArrayOf(FETCH_RESPONSE_TOPIC_V0)));
-    // Even though fetch response v2 has the same protocol as v1, the record set in the response is different. In v1,
-    // record set only includes messages of v0 (magic byte 0). In v2, record set can include messages of v0 and v1
-    // (magic byte 0 and 1). For details, see ByteBufferMessageSet.
-    public static final Schema FETCH_RESPONSE_V2 = FETCH_RESPONSE_V1;
-    public static final Schema FETCH_RESPONSE_V3 = FETCH_RESPONSE_V2;
-
-    // The v4 Fetch Response adds features for transactional consumption (the aborted transaction list and the
-    // last stable offset). It also exposes messages with magic v2 (along with older formats).
-    private static final Schema FETCH_RESPONSE_ABORTED_TRANSACTION_V4 = new Schema(
-            new Field("producer_id", INT64, "The producer id associated with the aborted transactions"),
-            new Field("first_offset", INT64, "The first offset in the aborted transaction"));
-
-    public static final Schema FETCH_RESPONSE_ABORTED_TRANSACTION_V5 = FETCH_RESPONSE_ABORTED_TRANSACTION_V4;
-
-    public static final Schema FETCH_RESPONSE_PARTITION_HEADER_V4 = new Schema(
-            new Field("partition",
-                    INT32,
-                    "Topic partition id."),
-            new Field("error_code", INT16),
-            new Field("high_watermark",
-                    INT64,
-                    "Last committed offset."),
-            new Field("last_stable_offset",
-                    INT64,
-                    "The last stable offset (or LSO) of the partition. This is the last offset such that the state " +
-                    "of all transactional records prior to this offset have been decided (ABORTED or COMMITTED)"),
-            new Field("aborted_transactions",
-                    ArrayOf.nullable(FETCH_RESPONSE_ABORTED_TRANSACTION_V4)));
-
-    // FETCH_RESPONSE_PARTITION_HEADER_V5 added log_start_offset field - the earliest available offset of partition data that can be consumed.
-    public static final Schema FETCH_RESPONSE_PARTITION_HEADER_V5 = new Schema(
-            new Field("partition",
-                    INT32,
-                    "Topic partition id."),
-            new Field("error_code", INT16),
-            new Field("high_watermark",
-                    INT64,
-                    "Last committed offset."),
-            new Field("last_stable_offset",
-                    INT64,
-                    "The last stable offset (or LSO) of the partition. This is the last offset such that the state " +
-                    "of all transactional records prior to this offset have been decided (ABORTED or COMMITTED)"),
-            new Field("log_start_offset",
-                    INT64,
-                    "Earliest available offset."),
-            new Field("aborted_transactions",
-                    ArrayOf.nullable(FETCH_RESPONSE_ABORTED_TRANSACTION_V5)));
-
-    public static final Schema FETCH_RESPONSE_PARTITION_V4 = new Schema(
-            new Field("partition_header", FETCH_RESPONSE_PARTITION_HEADER_V4),
-            new Field("record_set", RECORDS));
-
-    public static final Schema FETCH_RESPONSE_PARTITION_V5 = new Schema(
-            new Field("partition_header", FETCH_RESPONSE_PARTITION_HEADER_V5),
-            new Field("record_set", RECORDS));
-
-    public static final Schema FETCH_RESPONSE_TOPIC_V4 = new Schema(
-            new Field("topic", STRING),
-            new Field("partition_responses", new ArrayOf(FETCH_RESPONSE_PARTITION_V4)));
-
-    public static final Schema FETCH_RESPONSE_TOPIC_V5 = new Schema(
-            new Field("topic", STRING),
-            new Field("partition_responses", new ArrayOf(FETCH_RESPONSE_PARTITION_V5)));
-
-    public static final Schema FETCH_RESPONSE_V4 = new Schema(
-            newThrottleTimeField(),
-            new Field("responses", new ArrayOf(FETCH_RESPONSE_TOPIC_V4)));
-
-    public static final Schema FETCH_RESPONSE_V5 = new Schema(
-            newThrottleTimeField(),
-            new Field("responses", new ArrayOf(FETCH_RESPONSE_TOPIC_V5)));
-
-    /**
-     * The body of FETCH_RESPONSE_V6 is the same as FETCH_RESPONSE_V5.
-     * The version number is bumped up to indicate that the client supports KafkaStorageException.
-     * The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 5
-     */
-    public static final Schema FETCH_RESPONSE_V6 = FETCH_RESPONSE_V5;
-
-    public static final Schema[] FETCH_REQUEST = {FETCH_REQUEST_V0, FETCH_REQUEST_V1, FETCH_REQUEST_V2, FETCH_REQUEST_V3, FETCH_REQUEST_V4, FETCH_REQUEST_V5, FETCH_REQUEST_V6};
-    public static final Schema[] FETCH_RESPONSE = {FETCH_RESPONSE_V0, FETCH_RESPONSE_V1, FETCH_RESPONSE_V2, FETCH_RESPONSE_V3, FETCH_RESPONSE_V4, FETCH_RESPONSE_V5, FETCH_RESPONSE_V6};
-
-    /* List groups api */
-    public static final Schema LIST_GROUPS_REQUEST_V0 = new Schema();
-
-    /* v1 request is the same as v0. Throttle time has been added to response */
-    public static final Schema LIST_GROUPS_REQUEST_V1 = LIST_GROUPS_REQUEST_V0;
-
-    public static final Schema LIST_GROUPS_RESPONSE_GROUP_V0 = new Schema(new Field("group_id", STRING),
-                                                                          new Field("protocol_type", STRING));
-    public static final Schema LIST_GROUPS_RESPONSE_V0 = new Schema(new Field("error_code", INT16),
-                                                                    new Field("groups", new ArrayOf(LIST_GROUPS_RESPONSE_GROUP_V0)));
-    public static final Schema LIST_GROUPS_RESPONSE_V1 = new Schema(
-            newThrottleTimeField(),
-            new Field("error_code", INT16),
-            new Field("groups", new ArrayOf(LIST_GROUPS_RESPONSE_GROUP_V0)));
-
-    public static final Schema[] LIST_GROUPS_REQUEST = {LIST_GROUPS_REQUEST_V0, LIST_GROUPS_REQUEST_V1};
-    public static final Schema[] LIST_GROUPS_RESPONSE = {LIST_GROUPS_RESPONSE_V0, LIST_GROUPS_RESPONSE_V1};
-
-    /* Describe group api */
-    public static final Schema DESCRIBE_GROUPS_REQUEST_V0 = new Schema(new Field("group_ids",
-                                                                                 new ArrayOf(STRING),
-                                                                                 "List of groupIds to request metadata for (an empty groupId array will return empty group metadata)."));
-
-    /* v1 request is the same as v0. Throttle time has been added to response */
-    public static final Schema DESCRIBE_GROUPS_REQUEST_V1 = DESCRIBE_GROUPS_REQUEST_V0;
-
-    public static final Schema DESCRIBE_GROUPS_RESPONSE_MEMBER_V0 = new Schema(new Field("member_id",
-                                                                                         STRING,
-                                                                                         "The memberId assigned by the coordinator"),
-                                                                               new Field("client_id",
-                                                                                         STRING,
-                                                                                         "The client id used in the member's latest join group request"),
-                                                                               new Field("client_host",
-                                                                                         STRING,
-                                                                                         "The client host used in the request session corresponding to the member's join group."),
-                                                                               new Field("member_metadata",
-                                                                                         BYTES,
-                                                                                         "The metadata corresponding to the current group protocol in use (will only be present if the group is stable)."),
-                                                                               new Field("member_assignment",
-                                                                                         BYTES,
-                                                                                         "The current assignment provided by the group leader (will only be present if the group is stable)."));
-
-    public static final Schema DESCRIBE_GROUPS_RESPONSE_GROUP_METADATA_V0 = new Schema(new Field("error_code", INT16),
-                                                                                       new Field("group_id",
-                                                                                                 STRING),
-                                                                                       new Field("state",
-                                                                                                 STRING,
-                                                                                                 "The current state of the group (one of: Dead, Stable, AwaitingSync, PreparingRebalance, or empty if there is no active group)"),
-                                                                                       new Field("protocol_type",
-                                                                                                 STRING,
-                                                                                                 "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)"),
-                                                                                       new Field("members",
-                                                                                                 new ArrayOf(DESCRIBE_GROUPS_RESPONSE_MEMBER_V0),
-                                                                                                 "Current group members (only provided if the group is not Dead)"));
-
-    public static final Schema DESCRIBE_GROUPS_RESPONSE_V0 = new Schema(new Field("groups", new ArrayOf(DESCRIBE_GROUPS_RESPONSE_GROUP_METADATA_V0)));
-    public static final Schema DESCRIBE_GROUPS_RESPONSE_V1 = new Schema(
-            newThrottleTimeField(),
-            new Field("groups", new ArrayOf(DESCRIBE_GROUPS_RESPONSE_GROUP_METADATA_V0)));
-
-    public static final Schema[] DESCRIBE_GROUPS_REQUEST = {DESCRIBE_GROUPS_REQUEST_V0, DESCRIBE_GROUPS_REQUEST_V1};
-    public static final Schema[] DESCRIBE_GROUPS_RESPONSE = {DESCRIBE_GROUPS_RESPONSE_V0, DESCRIBE_GROUPS_RESPONSE_V1};
-
-    /* Find coordinator api */
-    public static final Schema FIND_COORDINATOR_REQUEST_V0 = new Schema(
-            new Field("group_id",
-                    STRING,
-                    "The unique group id."));
-
-    public static final Schema FIND_COORDINATOR_REQUEST_V1 = new Schema(
-            new Field("coordinator_key",
-                    STRING,
-                    "Id to use for finding the coordinator (for groups, this is the groupId, " +
-                            "for transactional producers, this is the transactional id)"),
-            new Field("coordinator_type",
-                    INT8,
-                    "The type of coordinator to find (0 = group, 1 = transaction)"));
-
-    public static final Schema FIND_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 FIND_COORDINATOR_RESPONSE_V0 = new Schema(
-            new Field("error_code", INT16),
-            new Field("coordinator",
-                    FIND_COORDINATOR_BROKER_V0,
-                    "Host and port information for the coordinator for a consumer group."));
-
-    public static final Schema FIND_COORDINATOR_RESPONSE_V1 = new Schema(
-            newThrottleTimeField(),
-            new Field("error_code", INT16),
-            new Field("error_message", NULLABLE_STRING),
-            new Field("coordinator",
-                    FIND_COORDINATOR_BROKER_V0,
-                    "Host and port information for the coordinator for a consumer group."));
-
-
-    public static final Schema[] FIND_COORDINATOR_REQUEST = {FIND_COORDINATOR_REQUEST_V0, FIND_COORDINATOR_REQUEST_V1};
-    public static final Schema[] FIND_COORDINATOR_RESPONSE = {FIND_COORDINATOR_RESPONSE_V0, FIND_COORDINATOR_RESPONSE_V1};
-
-    /* Controlled shutdown api */
-    public static final Schema CONTROLLED_SHUTDOWN_REQUEST_V0 = new Schema(new Field("broker_id",
-                                                                                     INT32,
-                                                                                     "The id of the broker for which controlled shutdown has been requested."));
-
-    public static final Schema CONTROLLED_SHUTDOWN_PARTITION_V0 = new Schema(new Field("topic", STRING),
-                                                                             new Field("partition",
-                                                                                       INT32,
-                                                                                       "Topic partition id."));
-
-    public static final Schema CONTROLLED_SHUTDOWN_RESPONSE_V0 = new Schema(new Field("error_code", INT16),
-                                                                            new Field("partitions_remaining",
-                                                                                      new ArrayOf(CONTROLLED_SHUTDOWN_PARTITION_V0),
-                                                                                      "The partitions that the broker still leads."));
-
-    public static final Schema CONTROLLED_SHUTDOWN_REQUEST_V1 = CONTROLLED_SHUTDOWN_REQUEST_V0;
-    public static final Schema CONTROLLED_SHUTDOWN_RESPONSE_V1 = CONTROLLED_SHUTDOWN_RESPONSE_V0;
-
-    public static final Schema[] CONTROLLED_SHUTDOWN_REQUEST = {CONTROLLED_SHUTDOWN_REQUEST_V0, CONTROLLED_SHUTDOWN_REQUEST_V1};
-    public static final Schema[] CONTROLLED_SHUTDOWN_RESPONSE = {CONTROLLED_SHUTDOWN_RESPONSE_V0, CONTROLLED_SHUTDOWN_RESPONSE_V1};
-
-    /* Join group api */
-    public static final Schema JOIN_GROUP_REQUEST_PROTOCOL_V0 = new Schema(new Field("protocol_name", STRING),
-                                                                           new Field("protocol_metadata", BYTES));
-
-    public static final Schema JOIN_GROUP_REQUEST_V0 = new Schema(new Field("group_id",
-                                                                            STRING,
-                                                                            "The group id."),
-                                                                  new Field("session_timeout",
-                                                                            INT32,
-                                                                            "The coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms."),
-                                                                  new Field("member_id",
-                                                                            STRING,
-                                                                            "The assigned consumer id or an empty string for a new consumer."),
-                                                                  new Field("protocol_type",
-                                                                            STRING,
-                                                                            "Unique name for class of protocols implemented by group"),
-                                                                  new Field("group_protocols",
-                                                                            new ArrayOf(JOIN_GROUP_REQUEST_PROTOCOL_V0),
-                                                                            "List of protocols that the member supports"));
-
-    public static final Schema JOIN_GROUP_REQUEST_V1 = new Schema(new Field("group_id",
-                                                                            STRING,
-                                                                            "The group id."),
-                                                                  new Field("session_timeout",
-                                                                            INT32,
-                                                                            "The coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms."),
-                                                                  new Field("rebalance_timeout",
-                                                                            INT32,
-                                                                            "The maximum time that the coordinator will wait for each member to rejoin when rebalancing the group"),
-                                                                  new Field("member_id",
-                                                                            STRING,
-                                                                            "The assigned consumer id or an empty string for a new consumer."),
-                                                                  new Field("protocol_type",
-                                                                            STRING,
-                                                                            "Unique name for class of protocols implemented by group"),
-                                                                  new Field("group_protocols",
-                                                                            new ArrayOf(JOIN_GROUP_REQUEST_PROTOCOL_V0),
-                                                                            "List of protocols that the member supports"));
-
-    /* v2 request is the same as v1. Throttle time has been added to response */
-    public static final Schema JOIN_GROUP_REQUEST_V2 = JOIN_GROUP_REQUEST_V1;
-
-    public static final Schema JOIN_GROUP_RESPONSE_MEMBER_V0 = new Schema(new Field("member_id", STRING),
-                                                                          new Field("member_metadata", BYTES));
-
-    public static final Schema JOIN_GROUP_RESPONSE_V0 = new Schema(new Field("error_code", INT16),
-                                                                   new Field("generation_id",
-                                                                             INT32,
-                                                                             "The generation of the consumer group."),
-                                                                   new Field("group_protocol",
-                                                                             STRING,
-                                                                             "The group protocol selected by the coordinator"),
-                                                                   new Field("leader_id",
-                                                                             STRING,
-                                                                             "The leader of the group"),
-                                                                   new Field("member_id",
-                                                                             STRING,
-                                                                             "The consumer id assigned by the group coordinator."),
-                                                                   new Field("members",
-                                                                             new ArrayOf(JOIN_GROUP_RESPONSE_MEMBER_V0)));
-
-    public static final Schema JOIN_GROUP_RESPONSE_V1 = JOIN_GROUP_RESPONSE_V0;
-
-    public static final Schema JOIN_GROUP_RESPONSE_V2 = new Schema(
-            newThrottleTimeField(),
-            new Field("error_code", INT16),
-            new Field("generation_id",
-                      INT32,
-                      "The generation of the consumer group."),
-            new Field("group_protocol",
-                      STRING,
-                      "The group protocol selected by the coordinator"),
-            new Field("leader_id",
-                      STRING,
-                      "The leader of the group"),
-            new Field("member_id",
-                      STRING,
-                      "The consumer id assigned by the group coordinator."),
-            new Field("members",
-                      new ArrayOf(JOIN_GROUP_RESPONSE_MEMBER_V0)));
-
-
-    public static final Schema[] JOIN_GROUP_REQUEST = {JOIN_GROUP_REQUEST_V0, JOIN_GROUP_REQUEST_V1, JOIN_GROUP_REQUEST_V2};
-    public static final Schema[] JOIN_GROUP_RESPONSE = {JOIN_GROUP_RESPONSE_V0, JOIN_GROUP_RESPONSE_V1, JOIN_GROUP_RESPONSE_V2};
-
-    /* SyncGroup api */
-    public static final Schema SYNC_GROUP_REQUEST_MEMBER_V0 = new Schema(new Field("member_id", STRING),
-                                                                         new Field("member_assignment", BYTES));
-    public static final Schema SYNC_GROUP_REQUEST_V0 = new Schema(new Field("group_id", STRING),
-                                                                  new Field("generation_id", INT32),
-                                                                  new Field("member_id", STRING),
-                                                                  new Field("group_assignment", new ArrayOf(SYNC_GROUP_REQUEST_MEMBER_V0)));
-
-    /* v1 request is the same as v0. Throttle time has been added to response */
-    public static final Schema SYNC_GROUP_REQUEST_V1 = SYNC_GROUP_REQUEST_V0;
-
-    public static final Schema SYNC_GROUP_RESPONSE_V0 = new Schema(new Field("error_code", INT16),
-                                                                   new Field("member_assignment", BYTES));
-    public static final Schema SYNC_GROUP_RESPONSE_V1 = new Schema(
-            newThrottleTimeField(),
-            new Field("error_code", INT16),
-            new Field("member_assignment", BYTES));
-    public static final Schema[] SYNC_GROUP_REQUEST = {SYNC_GROUP_REQUEST_V0, SYNC_GROUP_REQUEST_V1};
-    public static final Schema[] SYNC_GROUP_RESPONSE = {SYNC_GROUP_RESPONSE_V0, SYNC_GROUP_RESPONSE_V1};
-
-    /* Heartbeat api */
-    public static final Schema HEARTBEAT_REQUEST_V0 = new Schema(new Field("group_id", STRING, "The group id."),
-                                                                 new Field("group_generation_id",
-                                                                           INT32,
-                                                                           "The generation of the group."),
-                                                                 new Field("member_id",
-                                                                           STRING,
-                                                                           "The member id assigned by the group coordinator."));
-
-    /* v1 request is the same as v0. Throttle time has been added to response */
-    public static final Schema HEARTBEAT_REQUEST_V1 = HEARTBEAT_REQUEST_V0;
-
-    public static final Schema HEARTBEAT_RESPONSE_V0 = new Schema(new Field("error_code", INT16));
-    public static final Schema HEARTBEAT_RESPONSE_V1 = new Schema(
-            newThrottleTimeField(),
-            new Field("error_code", INT16));
-
-    public static final Schema[] HEARTBEAT_REQUEST = {HEARTBEAT_REQUEST_V0, HEARTBEAT_REQUEST_V1};
-    public static final Schema[] HEARTBEAT_RESPONSE = {HEARTBEAT_RESPONSE_V0, HEARTBEAT_RESPONSE_V1};
-
-    /* Leave group api */
-    public static final Schema LEAVE_GROUP_REQUEST_V0 = new Schema(new Field("group_id", STRING, "The group id."),
-                                                                   new Field("member_id",
-                                                                             STRING,
-                                                                             "The member id assigned by the group coordinator."));
-
-    /* v1 request is the same as v0. Throttle time has been added to response */
-    public static final Schema LEAVE_GROUP_REQUEST_V1 = LEAVE_GROUP_REQUEST_V0;
-
-    public static final Schema LEAVE_GROUP_RESPONSE_V0 = new Schema(new Field("error_code", INT16));
-    public static final Schema LEAVE_GROUP_RESPONSE_V1 = new Schema(
-            newThrottleTimeField(),
-            new Field("error_code", INT16));
-
-    public static final Schema[] LEAVE_GROUP_REQUEST = {LEAVE_GROUP_REQUEST_V0, LEAVE_GROUP_REQUEST_V1};
-    public static final Schema[] LEAVE_GROUP_RESPONSE = {LEAVE_GROUP_RESPONSE_V0, LEAVE_GROUP_RESPONSE_V1};
-
-    /* Leader and ISR api */
-    public static final Schema LEADER_AND_ISR_REQUEST_PARTITION_STATE_V0 =
-            new Schema(new Field("topic", STRING, "Topic name."),
-                       new Field("partition", INT32, "Topic partition id."),
-                       new Field("controller_epoch", INT32, "The controller epoch."),
-                       new Field("leader", INT32, "The broker id for the leader."),
-                       new Field("leader_epoch", INT32, "The leader epoch."),
-                       new Field("isr", new ArrayOf(INT32), "The in sync replica ids."),
-                       new Field("zk_version", INT32, "T

<TRUNCATED>

[8/8] kafka git commit: MINOR: Move request/response schemas to the corresponding object representation

Posted by ij...@apache.org.
MINOR: Move request/response schemas to the corresponding object representation

This refactor achieves the following:

1. Breaks up the increasingly unmanageable `Protocol` class and moves schemas closer to their actual usage.
2. Removes the need for redundant field identifiers maintained separately in `Protocol` and the respective request/response objects.
3. Provides a better mechanism for sharing common fields between different schemas (e.g. topics, partitions, error codes, etc.).
4. Adds convenience helpers to `Struct` for common patterns (such as setting a field only if it exists).

Author: Jason Gustafson <ja...@confluent.io>

Reviewers: Ismael Juma <is...@juma.me.uk>

Closes #3813 from hachikuji/protocol-schema-refactor


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/0cf77080
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/0cf77080
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/0cf77080

Branch: refs/heads/trunk
Commit: 0cf7708007b01faac5012d939f3c50db274f858d
Parents: a64fe2e
Author: Jason Gustafson <ja...@confluent.io>
Authored: Tue Sep 19 05:12:55 2017 +0100
Committer: Ismael Juma <is...@juma.me.uk>
Committed: Tue Sep 19 05:12:55 2017 +0100

----------------------------------------------------------------------
 checkstyle/import-control.xml                   |    1 +
 .../org/apache/kafka/clients/NetworkClient.java |    5 +-
 .../kafka/clients/consumer/StickyAssignor.java  |   22 +-
 .../apache/kafka/common/protocol/ApiKeys.java   |  249 ++-
 .../kafka/common/protocol/CommonFields.java     |   40 +
 .../kafka/common/protocol/ProtoUtils.java       |    6 +-
 .../apache/kafka/common/protocol/Protocol.java  | 2095 +-----------------
 .../kafka/common/protocol/types/BoundField.java |   37 +
 .../kafka/common/protocol/types/Field.java      |   72 +-
 .../kafka/common/protocol/types/Schema.java     |   46 +-
 .../kafka/common/protocol/types/Struct.java     |  129 +-
 .../kafka/common/requests/AbstractRequest.java  |    4 +-
 .../kafka/common/requests/AbstractResponse.java |    5 +-
 .../common/requests/AddOffsetsToTxnRequest.java |   16 +
 .../requests/AddOffsetsToTxnResponse.java       |   20 +-
 .../requests/AddPartitionsToTxnRequest.java     |   35 +-
 .../requests/AddPartitionsToTxnResponse.java    |   37 +-
 .../common/requests/AlterConfigsRequest.java    |   26 +
 .../common/requests/AlterConfigsResponse.java   |   27 +-
 .../common/requests/AlterReplicaDirRequest.java |   23 +-
 .../requests/AlterReplicaDirResponse.java       |   40 +-
 .../apache/kafka/common/requests/ApiError.java  |   20 +-
 .../common/requests/ApiVersionsRequest.java     |   10 +
 .../common/requests/ApiVersionsResponse.java    |   72 +-
 .../requests/ControlledShutdownRequest.java     |   20 +-
 .../requests/ControlledShutdownResponse.java    |   41 +-
 .../common/requests/CreateAclsRequest.java      |   31 +-
 .../common/requests/CreateAclsResponse.java     |   29 +-
 .../common/requests/CreateTopicsRequest.java    |   76 +-
 .../common/requests/CreateTopicsResponse.java   |   42 +-
 .../common/requests/DeleteAclsRequest.java      |   22 +
 .../common/requests/DeleteAclsResponse.java     |   55 +-
 .../common/requests/DeleteRecordsRequest.java   |   36 +-
 .../common/requests/DeleteRecordsResponse.java  |   47 +-
 .../common/requests/DeleteTopicsRequest.java    |   19 +
 .../common/requests/DeleteTopicsResponse.java   |   39 +-
 .../common/requests/DescribeAclsRequest.java    |   20 +
 .../common/requests/DescribeAclsResponse.java   |   52 +-
 .../common/requests/DescribeConfigsRequest.java |   18 +
 .../requests/DescribeConfigsResponse.java       |   65 +-
 .../common/requests/DescribeGroupsRequest.java  |   17 +
 .../common/requests/DescribeGroupsResponse.java |   48 +-
 .../common/requests/DescribeLogDirsRequest.java |   20 +-
 .../requests/DescribeLogDirsResponse.java       |   52 +-
 .../kafka/common/requests/EndTxnRequest.java    |   17 +
 .../kafka/common/requests/EndTxnResponse.java   |   20 +-
 .../kafka/common/requests/FetchRequest.java     |  108 +-
 .../kafka/common/requests/FetchResponse.java    |  132 +-
 .../common/requests/FindCoordinatorRequest.java |   17 +
 .../requests/FindCoordinatorResponse.java       |   57 +-
 .../kafka/common/requests/HeartbeatRequest.java |   17 +
 .../common/requests/HeartbeatResponse.java      |   23 +-
 .../common/requests/InitProducerIdRequest.java  |   13 +
 .../common/requests/InitProducerIdResponse.java |   29 +-
 .../kafka/common/requests/JoinGroupRequest.java |   39 +
 .../common/requests/JoinGroupResponse.java      |   70 +-
 .../common/requests/LeaderAndIsrRequest.java    |   64 +-
 .../common/requests/LeaderAndIsrResponse.java   |   42 +-
 .../common/requests/LeaveGroupRequest.java      |   18 +-
 .../common/requests/LeaveGroupResponse.java     |   23 +-
 .../common/requests/ListGroupsRequest.java      |   14 +-
 .../common/requests/ListGroupsResponse.java     |   47 +-
 .../common/requests/ListOffsetRequest.java      |   58 +-
 .../common/requests/ListOffsetResponse.java     |   65 +-
 .../kafka/common/requests/MetadataRequest.java  |   34 +
 .../kafka/common/requests/MetadataResponse.java |  126 +-
 .../common/requests/OffsetCommitRequest.java    |   73 +-
 .../common/requests/OffsetCommitResponse.java   |   56 +-
 .../common/requests/OffsetFetchRequest.java     |   54 +-
 .../common/requests/OffsetFetchResponse.java    |   79 +-
 .../requests/OffsetsForLeaderEpochRequest.java  |   47 +-
 .../requests/OffsetsForLeaderEpochResponse.java |   56 +-
 .../kafka/common/requests/ProduceRequest.java   |   71 +-
 .../kafka/common/requests/ProduceResponse.java  |   80 +-
 .../kafka/common/requests/RequestContext.java   |    3 +-
 .../kafka/common/requests/RequestHeader.java    |   31 +-
 .../kafka/common/requests/RequestUtils.java     |   59 +-
 .../kafka/common/requests/ResponseHeader.java   |   19 +-
 .../requests/SaslAuthenticateRequest.java       |   15 +-
 .../requests/SaslAuthenticateResponse.java      |   30 +-
 .../common/requests/SaslHandshakeRequest.java   |   22 +-
 .../common/requests/SaslHandshakeResponse.java  |   28 +-
 .../common/requests/StopReplicaRequest.java     |   31 +-
 .../common/requests/StopReplicaResponse.java    |   40 +-
 .../kafka/common/requests/SyncGroupRequest.java |   33 +-
 .../common/requests/SyncGroupResponse.java      |   29 +-
 .../common/requests/TxnOffsetCommitRequest.java |   47 +-
 .../requests/TxnOffsetCommitResponse.java       |   50 +-
 .../common/requests/UpdateMetadataRequest.java  |  174 +-
 .../common/requests/UpdateMetadataResponse.java |   21 +-
 .../common/requests/WriteTxnMarkersRequest.java |   40 +-
 .../requests/WriteTxnMarkersResponse.java       |   49 +-
 .../authenticator/SaslServerAuthenticator.java  |    5 +-
 .../apache/kafka/clients/NetworkClientTest.java |    5 +-
 .../kafka/clients/NodeApiVersionsTest.java      |    2 +-
 .../kafka/common/protocol/ApiKeysTest.java      |   13 +-
 .../kafka/common/protocol/ProtoUtilsTest.java   |    9 +-
 .../types/ProtocolSerializationTest.java        |   20 +-
 .../requests/ApiVersionsResponseTest.java       |    2 +-
 .../common/requests/RequestHeaderTest.java      |    6 +-
 .../common/requests/RequestResponseTest.java    |    2 +-
 .../authenticator/SaslAuthenticatorTest.java    |    7 +-
 .../controller/ControllerChannelManager.scala   |    4 +-
 .../group/GroupMetadataManager.scala            |    2 +-
 .../transaction/TransactionLog.scala            |    2 +-
 .../scala/kafka/log/ProducerStateManager.scala  |    2 +-
 .../scala/kafka/network/RequestChannel.scala    |    4 +-
 .../src/main/scala/kafka/server/KafkaApis.scala |    6 +-
 .../kafka/api/AuthorizerIntegrationTest.scala   |   18 +-
 .../kafka/server/ApiVersionsRequestTest.scala   |    2 +-
 .../unit/kafka/server/ApiVersionsTest.scala     |   14 +-
 .../unit/kafka/server/MetadataCacheTest.scala   |   12 +-
 .../unit/kafka/server/RequestQuotaTest.scala    |    6 +-
 113 files changed, 3243 insertions(+), 2956 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/checkstyle/import-control.xml
----------------------------------------------------------------------
diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index bea7e20..3329b2d 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -105,6 +105,7 @@
       <allow pkg="org.apache.kafka.common.errors" />
       <allow pkg="org.apache.kafka.common.protocol.types" />
       <allow pkg="org.apache.kafka.common.record" />
+      <allow pkg="org.apache.kafka.common.requests" />
     </subpackage>
 
     <subpackage name="record">

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
----------------------------------------------------------------------
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 f046696..c3c15df 100644
--- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
@@ -26,6 +26,7 @@ import org.apache.kafka.common.network.Selectable;
 import org.apache.kafka.common.network.Send;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.CommonFields;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.requests.AbstractRequest;
 import org.apache.kafka.common.requests.AbstractResponse;
@@ -570,8 +571,8 @@ public class NetworkClient implements KafkaClient {
         // Always expect the response version id to be the same as the request version id
         Struct responseBody = requestHeader.apiKey().parseResponse(requestHeader.apiVersion(), responseBuffer);
         correlate(requestHeader, responseHeader);
-        if (throttleTimeSensor != null && responseBody.hasField(AbstractResponse.THROTTLE_TIME_KEY_NAME))
-            throttleTimeSensor.record(responseBody.getInt(AbstractResponse.THROTTLE_TIME_KEY_NAME), now);
+        if (throttleTimeSensor != null && responseBody.hasField(CommonFields.THROTTLE_TIME_MS))
+            throttleTimeSensor.record(responseBody.get(CommonFields.THROTTLE_TIME_MS), now);
         return responseBody;
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java
index d7bfaf1..247b619 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java
@@ -16,6 +16,17 @@
  */
 package org.apache.kafka.clients.consumer;
 
+import org.apache.kafka.clients.consumer.internals.AbstractPartitionAssignor;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.apache.kafka.common.utils.CollectionUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.io.Serializable;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -31,17 +42,6 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeSet;
 
-import org.apache.kafka.clients.consumer.internals.AbstractPartitionAssignor;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.protocol.types.ArrayOf;
-import org.apache.kafka.common.protocol.types.Field;
-import org.apache.kafka.common.protocol.types.Schema;
-import org.apache.kafka.common.protocol.types.Struct;
-import org.apache.kafka.common.protocol.types.Type;
-import org.apache.kafka.common.utils.CollectionUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 /**
  * <p>The sticky assignor serves two purposes. First, it guarantees an assignment that is as balanced as possible, meaning either:
  * <ul>

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
----------------------------------------------------------------------
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 d37eddf..0e087eb 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
@@ -19,33 +19,117 @@ package org.apache.kafka.common.protocol;
 import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.SchemaException;
 import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
 import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.requests.AddOffsetsToTxnRequest;
+import org.apache.kafka.common.requests.AddOffsetsToTxnResponse;
+import org.apache.kafka.common.requests.AddPartitionsToTxnRequest;
+import org.apache.kafka.common.requests.AddPartitionsToTxnResponse;
+import org.apache.kafka.common.requests.AlterConfigsRequest;
+import org.apache.kafka.common.requests.AlterConfigsResponse;
+import org.apache.kafka.common.requests.AlterReplicaDirRequest;
+import org.apache.kafka.common.requests.AlterReplicaDirResponse;
+import org.apache.kafka.common.requests.ApiVersionsRequest;
+import org.apache.kafka.common.requests.ApiVersionsResponse;
+import org.apache.kafka.common.requests.ControlledShutdownRequest;
+import org.apache.kafka.common.requests.ControlledShutdownResponse;
+import org.apache.kafka.common.requests.CreateAclsRequest;
+import org.apache.kafka.common.requests.CreateAclsResponse;
+import org.apache.kafka.common.requests.CreateTopicsRequest;
+import org.apache.kafka.common.requests.CreateTopicsResponse;
+import org.apache.kafka.common.requests.DeleteAclsRequest;
+import org.apache.kafka.common.requests.DeleteAclsResponse;
+import org.apache.kafka.common.requests.DeleteRecordsRequest;
+import org.apache.kafka.common.requests.DeleteRecordsResponse;
+import org.apache.kafka.common.requests.DeleteTopicsRequest;
+import org.apache.kafka.common.requests.DeleteTopicsResponse;
+import org.apache.kafka.common.requests.DescribeAclsRequest;
+import org.apache.kafka.common.requests.DescribeAclsResponse;
+import org.apache.kafka.common.requests.DescribeConfigsRequest;
+import org.apache.kafka.common.requests.DescribeConfigsResponse;
+import org.apache.kafka.common.requests.DescribeGroupsRequest;
+import org.apache.kafka.common.requests.DescribeGroupsResponse;
+import org.apache.kafka.common.requests.DescribeLogDirsRequest;
+import org.apache.kafka.common.requests.DescribeLogDirsResponse;
+import org.apache.kafka.common.requests.EndTxnRequest;
+import org.apache.kafka.common.requests.EndTxnResponse;
+import org.apache.kafka.common.requests.FetchRequest;
+import org.apache.kafka.common.requests.FetchResponse;
+import org.apache.kafka.common.requests.FindCoordinatorRequest;
+import org.apache.kafka.common.requests.FindCoordinatorResponse;
+import org.apache.kafka.common.requests.HeartbeatRequest;
+import org.apache.kafka.common.requests.HeartbeatResponse;
+import org.apache.kafka.common.requests.InitProducerIdRequest;
+import org.apache.kafka.common.requests.InitProducerIdResponse;
+import org.apache.kafka.common.requests.JoinGroupRequest;
+import org.apache.kafka.common.requests.JoinGroupResponse;
+import org.apache.kafka.common.requests.LeaderAndIsrRequest;
+import org.apache.kafka.common.requests.LeaderAndIsrResponse;
+import org.apache.kafka.common.requests.LeaveGroupRequest;
+import org.apache.kafka.common.requests.LeaveGroupResponse;
+import org.apache.kafka.common.requests.ListGroupsRequest;
+import org.apache.kafka.common.requests.ListGroupsResponse;
+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.requests.OffsetCommitRequest;
+import org.apache.kafka.common.requests.OffsetCommitResponse;
+import org.apache.kafka.common.requests.OffsetFetchRequest;
+import org.apache.kafka.common.requests.OffsetFetchResponse;
+import org.apache.kafka.common.requests.OffsetsForLeaderEpochRequest;
+import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse;
+import org.apache.kafka.common.requests.ProduceRequest;
+import org.apache.kafka.common.requests.ProduceResponse;
+import org.apache.kafka.common.requests.SaslAuthenticateRequest;
+import org.apache.kafka.common.requests.SaslAuthenticateResponse;
+import org.apache.kafka.common.requests.SaslHandshakeRequest;
+import org.apache.kafka.common.requests.SaslHandshakeResponse;
+import org.apache.kafka.common.requests.StopReplicaRequest;
+import org.apache.kafka.common.requests.StopReplicaResponse;
+import org.apache.kafka.common.requests.SyncGroupRequest;
+import org.apache.kafka.common.requests.SyncGroupResponse;
+import org.apache.kafka.common.requests.TxnOffsetCommitRequest;
+import org.apache.kafka.common.requests.TxnOffsetCommitResponse;
+import org.apache.kafka.common.requests.UpdateMetadataRequest;
+import org.apache.kafka.common.requests.UpdateMetadataResponse;
+import org.apache.kafka.common.requests.WriteTxnMarkersRequest;
+import org.apache.kafka.common.requests.WriteTxnMarkersResponse;
 
 import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.kafka.common.protocol.types.Type.BYTES;
+import static org.apache.kafka.common.protocol.types.Type.NULLABLE_BYTES;
+import static org.apache.kafka.common.protocol.types.Type.RECORDS;
 
 /**
  * Identifiers for all the Kafka APIs
  */
 public enum ApiKeys {
-    PRODUCE(0, "Produce"),
-    FETCH(1, "Fetch"),
-    LIST_OFFSETS(2, "Offsets"),
-    METADATA(3, "Metadata"),
-    LEADER_AND_ISR(4, "LeaderAndIsr", true),
-    STOP_REPLICA(5, "StopReplica", true),
-    UPDATE_METADATA_KEY(6, "UpdateMetadata", true),
-    CONTROLLED_SHUTDOWN_KEY(7, "ControlledShutdown", true),
-    OFFSET_COMMIT(8, "OffsetCommit"),
-    OFFSET_FETCH(9, "OffsetFetch"),
-    FIND_COORDINATOR(10, "FindCoordinator"),
-    JOIN_GROUP(11, "JoinGroup"),
-    HEARTBEAT(12, "Heartbeat"),
-    LEAVE_GROUP(13, "LeaveGroup"),
-    SYNC_GROUP(14, "SyncGroup"),
-    DESCRIBE_GROUPS(15, "DescribeGroups"),
-    LIST_GROUPS(16, "ListGroups"),
-    SASL_HANDSHAKE(17, "SaslHandshake"),
-    API_VERSIONS(18, "ApiVersions") {
+    PRODUCE(0, "Produce", ProduceRequest.schemaVersions(), ProduceResponse.schemaVersions()),
+    FETCH(1, "Fetch", FetchRequest.schemaVersions(), FetchResponse.schemaVersions()),
+    LIST_OFFSETS(2, "ListOffsets", ListOffsetRequest.schemaVersions(), ListOffsetResponse.schemaVersions()),
+    METADATA(3, "Metadata", MetadataRequest.schemaVersions(), MetadataResponse.schemaVersions()),
+    LEADER_AND_ISR(4, "LeaderAndIsr", true, LeaderAndIsrRequest.schemaVersions(), LeaderAndIsrResponse.schemaVersions()),
+    STOP_REPLICA(5, "StopReplica", true, StopReplicaRequest.schemaVersions(), StopReplicaResponse.schemaVersions()),
+    UPDATE_METADATA(6, "UpdateMetadata", true, UpdateMetadataRequest.schemaVersions(),
+            UpdateMetadataResponse.schemaVersions()),
+    CONTROLLED_SHUTDOWN(7, "ControlledShutdown", true, ControlledShutdownRequest.schemaVersions(),
+            ControlledShutdownResponse.schemaVersions()),
+    OFFSET_COMMIT(8, "OffsetCommit", OffsetCommitRequest.schemaVersions(), OffsetCommitResponse.schemaVersions()),
+    OFFSET_FETCH(9, "OffsetFetch", OffsetFetchRequest.schemaVersions(), OffsetFetchResponse.schemaVersions()),
+    FIND_COORDINATOR(10, "FindCoordinator", FindCoordinatorRequest.schemaVersions(),
+            FindCoordinatorResponse.schemaVersions()),
+    JOIN_GROUP(11, "JoinGroup", JoinGroupRequest.schemaVersions(), JoinGroupResponse.schemaVersions()),
+    HEARTBEAT(12, "Heartbeat", HeartbeatRequest.schemaVersions(), HeartbeatResponse.schemaVersions()),
+    LEAVE_GROUP(13, "LeaveGroup", LeaveGroupRequest.schemaVersions(), LeaveGroupResponse.schemaVersions()),
+    SYNC_GROUP(14, "SyncGroup", SyncGroupRequest.schemaVersions(), SyncGroupResponse.schemaVersions()),
+    DESCRIBE_GROUPS(15, "DescribeGroups", DescribeGroupsRequest.schemaVersions(),
+            DescribeGroupsResponse.schemaVersions()),
+    LIST_GROUPS(16, "ListGroups", ListGroupsRequest.schemaVersions(), ListGroupsResponse.schemaVersions()),
+    SASL_HANDSHAKE(17, "SaslHandshake", SaslHandshakeRequest.schemaVersions(), SaslHandshakeResponse.schemaVersions()),
+    API_VERSIONS(18, "ApiVersions", ApiVersionsRequest.schemaVersions(), ApiVersionsResponse.schemaVersions()) {
         @Override
         public Struct parseResponse(short version, ByteBuffer buffer) {
             // Fallback to version 0 for ApiVersions response. If a client sends an ApiVersionsRequest
@@ -54,24 +138,37 @@ public enum ApiKeys {
             return parseResponse(version, buffer, (short) 0);
         }
     },
-    CREATE_TOPICS(19, "CreateTopics"),
-    DELETE_TOPICS(20, "DeleteTopics"),
-    DELETE_RECORDS(21, "DeleteRecords"),
-    INIT_PRODUCER_ID(22, "InitProducerId"),
-    OFFSET_FOR_LEADER_EPOCH(23, "OffsetForLeaderEpoch", true),
-    ADD_PARTITIONS_TO_TXN(24, "AddPartitionsToTxn", false, RecordBatch.MAGIC_VALUE_V2),
-    ADD_OFFSETS_TO_TXN(25, "AddOffsetsToTxn", false, RecordBatch.MAGIC_VALUE_V2),
-    END_TXN(26, "EndTxn", false, RecordBatch.MAGIC_VALUE_V2),
-    WRITE_TXN_MARKERS(27, "WriteTxnMarkers", true, RecordBatch.MAGIC_VALUE_V2),
-    TXN_OFFSET_COMMIT(28, "TxnOffsetCommit", false, RecordBatch.MAGIC_VALUE_V2),
-    DESCRIBE_ACLS(29, "DescribeAcls"),
-    CREATE_ACLS(30, "CreateAcls"),
-    DELETE_ACLS(31, "DeleteAcls"),
-    DESCRIBE_CONFIGS(32, "DescribeConfigs"),
-    ALTER_CONFIGS(33, "AlterConfigs"),
-    ALTER_REPLICA_DIR(34, "AlterReplicaDir"),
-    DESCRIBE_LOG_DIRS(35, "DescribeLogDirs"),
-    SASL_AUTHENTICATE(36, "SaslAuthenticate");
+    CREATE_TOPICS(19, "CreateTopics", CreateTopicsRequest.schemaVersions(), CreateTopicsResponse.schemaVersions()),
+    DELETE_TOPICS(20, "DeleteTopics", DeleteTopicsRequest.schemaVersions(), DeleteTopicsResponse.schemaVersions()),
+    DELETE_RECORDS(21, "DeleteRecords", DeleteRecordsRequest.schemaVersions(), DeleteRecordsResponse.schemaVersions()),
+    INIT_PRODUCER_ID(22, "InitProducerId", InitProducerIdRequest.schemaVersions(),
+            InitProducerIdResponse.schemaVersions()),
+    OFFSET_FOR_LEADER_EPOCH(23, "OffsetForLeaderEpoch", true, OffsetsForLeaderEpochRequest.schemaVersions(),
+            OffsetsForLeaderEpochResponse.schemaVersions()),
+    ADD_PARTITIONS_TO_TXN(24, "AddPartitionsToTxn", false, RecordBatch.MAGIC_VALUE_V2,
+            AddPartitionsToTxnRequest.schemaVersions(), AddPartitionsToTxnResponse.schemaVersions()),
+    ADD_OFFSETS_TO_TXN(25, "AddOffsetsToTxn", false, RecordBatch.MAGIC_VALUE_V2, AddOffsetsToTxnRequest.schemaVersions(),
+            AddOffsetsToTxnResponse.schemaVersions()),
+    END_TXN(26, "EndTxn", false, RecordBatch.MAGIC_VALUE_V2, EndTxnRequest.schemaVersions(),
+            EndTxnResponse.schemaVersions()),
+    WRITE_TXN_MARKERS(27, "WriteTxnMarkers", true, RecordBatch.MAGIC_VALUE_V2, WriteTxnMarkersRequest.schemaVersions(),
+            WriteTxnMarkersResponse.schemaVersions()),
+    TXN_OFFSET_COMMIT(28, "TxnOffsetCommit", false, RecordBatch.MAGIC_VALUE_V2, TxnOffsetCommitRequest.schemaVersions(),
+            TxnOffsetCommitResponse.schemaVersions()),
+    DESCRIBE_ACLS(29, "DescribeAcls", DescribeAclsRequest.schemaVersions(), DescribeAclsResponse.schemaVersions()),
+    CREATE_ACLS(30, "CreateAcls", CreateAclsRequest.schemaVersions(), CreateAclsResponse.schemaVersions()),
+    DELETE_ACLS(31, "DeleteAcls", DeleteAclsRequest.schemaVersions(), DeleteAclsResponse.schemaVersions()),
+    DESCRIBE_CONFIGS(32, "DescribeConfigs", DescribeConfigsRequest.schemaVersions(),
+            DescribeConfigsResponse.schemaVersions()),
+    ALTER_CONFIGS(33, "AlterConfigs", AlterConfigsRequest.schemaVersions(),
+            AlterConfigsResponse.schemaVersions()),
+    ALTER_REPLICA_DIR(34, "AlterReplicaDir", AlterReplicaDirRequest.schemaVersions(),
+            AlterReplicaDirResponse.schemaVersions()),
+    DESCRIBE_LOG_DIRS(35, "DescribeLogDirs", DescribeLogDirsRequest.schemaVersions(),
+            DescribeLogDirsResponse.schemaVersions()),
+    SASL_AUTHENTICATE(36, "SaslAuthenticate", SaslAuthenticateRequest.schemaVersions(),
+            SaslAuthenticateResponse.schemaVersions());
+
 
     private static final ApiKeys[] ID_TO_TYPE;
     private static final int MIN_API_KEY = 0;
@@ -100,21 +197,48 @@ public enum ApiKeys {
     /** indicates the minimum required inter broker magic required to support the API */
     public final byte minRequiredInterBrokerMagic;
 
-    ApiKeys(int id, String name) {
-        this(id, name, false);
+    public final Schema[] requestSchemas;
+    public final Schema[] responseSchemas;
+    public final boolean requiresDelayedAllocation;
+
+    ApiKeys(int id, String name, Schema[] requestSchemas, Schema[] responseSchemas) {
+        this(id, name, false, requestSchemas, responseSchemas);
     }
 
-    ApiKeys(int id, String name, boolean clusterAction) {
-        this(id, name, clusterAction, RecordBatch.MAGIC_VALUE_V0);
+    ApiKeys(int id, String name, boolean clusterAction, Schema[] requestSchemas, Schema[] responseSchemas) {
+        this(id, name, clusterAction, RecordBatch.MAGIC_VALUE_V0, requestSchemas, responseSchemas);
     }
 
-    ApiKeys(int id, String name, boolean clusterAction, byte minRequiredInterBrokerMagic) {
+    ApiKeys(int id, String name, boolean clusterAction, byte minRequiredInterBrokerMagic,
+            Schema[] requestSchemas, Schema[] responseSchemas) {
         if (id < 0)
             throw new IllegalArgumentException("id must not be negative, id: " + id);
         this.id = (short) id;
         this.name = name;
         this.clusterAction = clusterAction;
         this.minRequiredInterBrokerMagic = minRequiredInterBrokerMagic;
+
+        if (requestSchemas.length != responseSchemas.length)
+            throw new IllegalStateException(requestSchemas.length + " request versions for api " + name
+                    + " but " + responseSchemas.length + " response versions.");
+
+        for (int i = 0; i < requestSchemas.length; ++i) {
+            if (requestSchemas[i] == null)
+                throw new IllegalStateException("Request schema for api " + name + " for version " + i + " is null");
+            if (responseSchemas[i] == null)
+                throw new IllegalStateException("Response schema for api " + name + " for version " + i + " is null");
+        }
+
+        boolean requestRetainsBufferReference = false;
+        for (Schema requestVersionSchema : requestSchemas) {
+            if (retainsBufferReference(requestVersionSchema)) {
+                requestRetainsBufferReference = true;
+                break;
+            }
+        }
+        this.requiresDelayedAllocation = requestRetainsBufferReference;
+        this.requestSchemas = requestSchemas;
+        this.responseSchemas = responseSchemas;
     }
 
     public static ApiKeys forId(int id) {
@@ -129,23 +253,19 @@ public enum ApiKeys {
     }
 
     public short latestVersion() {
-        if (id >= Protocol.CURR_VERSION.length)
-            throw new IllegalArgumentException("Latest version for API key " + this + " is not defined");
-        return Protocol.CURR_VERSION[id];
+        return (short) (requestSchemas.length - 1);
     }
 
     public short oldestVersion() {
-        if (id >= Protocol.MIN_VERSIONS.length)
-            throw new IllegalArgumentException("Oldest version for API key " + this + " is not defined");
-        return Protocol.MIN_VERSIONS[id];
+        return 0;
     }
 
     public Schema requestSchema(short version) {
-        return schemaFor(Protocol.REQUESTS, version);
+        return schemaFor(requestSchemas, version);
     }
 
     public Schema responseSchema(short version) {
-        return schemaFor(Protocol.RESPONSES, version);
+        return schemaFor(responseSchemas, version);
     }
 
     public Struct parseRequest(short version, ByteBuffer buffer) {
@@ -169,18 +289,16 @@ public enum ApiKeys {
         }
     }
 
-    private Schema schemaFor(Schema[][] schemas, short version) {
-        if (id > schemas.length)
-            throw new IllegalArgumentException("No schema available for API key " + this);
-        if (version < 0 || version > latestVersion())
+    private Schema schemaFor(Schema[] versions, short version) {
+        if (!isVersionSupported(version))
             throw new IllegalArgumentException("Invalid version for API key " + this + ": " + version);
-
-        Schema[] versions = schemas[id];
-        if (versions[version] == null)
-            throw new IllegalArgumentException("Unsupported version for API key " + this + ": " + version);
         return versions[version];
     }
 
+    public boolean isVersionSupported(short apiVersion) {
+        return apiVersion >= oldestVersion() && apiVersion <= latestVersion();
+    }
+
     private static String toHtml() {
         final StringBuilder b = new StringBuilder();
         b.append("<table class=\"data-table\"><tbody>\n");
@@ -206,4 +324,19 @@ public enum ApiKeys {
         System.out.println(toHtml());
     }
 
+    private static boolean retainsBufferReference(Schema schema) {
+        final AtomicReference<Boolean> foundBufferReference = new AtomicReference<>(Boolean.FALSE);
+        SchemaVisitor detector = new SchemaVisitorAdapter() {
+            @Override
+            public void visit(Type field) {
+                if (field == BYTES || field == NULLABLE_BYTES || field == RECORDS) {
+                    foundBufferReference.set(Boolean.TRUE);
+                }
+            }
+        };
+        foundBufferReference.set(Boolean.FALSE);
+        ProtoUtils.walk(schema, detector);
+        return foundBufferReference.get();
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java b/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java
new file mode 100644
index 0000000..e12cde4
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java
@@ -0,0 +1,40 @@
+/*
+ * 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.protocol;
+
+import org.apache.kafka.common.protocol.types.Field;
+
+public class CommonFields {
+    public static final Field.Int32 THROTTLE_TIME_MS = new Field.Int32("throttle_time_ms",
+            "Duration in milliseconds for which the request was throttled due to quota violation (Zero if the " +
+                    "request did not violate any quota)", 0);
+    public static final Field.Str TOPIC_NAME = new Field.Str("topic", "Name of topic");
+    public static final Field.Int32 PARTITION_ID = new Field.Int32("partition", "Topic partition id");
+    public static final Field.Int16 ERROR_CODE = new Field.Int16("error_code", "Response error code");
+    public static final Field.NullableStr ERROR_MESSAGE = new Field.NullableStr("error_message", "Response error message");
+
+    // ACL Apis
+    public static final Field.Int8 RESOURCE_TYPE = new Field.Int8("resource_type", "The resource type");
+    public static final Field.Str RESOURCE_NAME = new Field.Str("resource_name", "The resource name");
+    public static final Field.NullableStr RESOURCE_NAME_FILTER = new Field.NullableStr("resource_name", "The resource name filter");
+    public static final Field.Str PRINCIPAL = new Field.Str("principal", "The ACL principal");
+    public static final Field.NullableStr PRINCIPAL_FILTER = new Field.NullableStr("principal", "The ACL principal filter");
+    public static final Field.Str HOST = new Field.Str("host", "The ACL host");
+    public static final Field.NullableStr HOST_FILTER = new Field.NullableStr("host", "The ACL host filter");
+    public static final Field.Int8 OPERATION = new Field.Int8("operation", "The ACL operation");
+    public static final Field.Int8 PERMISSION_TYPE = new Field.Int8("permission_type", "The ACL permission type");
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/protocol/ProtoUtils.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ProtoUtils.java b/clients/src/main/java/org/apache/kafka/common/protocol/ProtoUtils.java
index 5d39dff..f9be12c 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/ProtoUtils.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/ProtoUtils.java
@@ -17,7 +17,7 @@
 package org.apache.kafka.common.protocol;
 
 import org.apache.kafka.common.protocol.types.ArrayOf;
-import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.BoundField;
 import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Type;
 
@@ -33,8 +33,8 @@ public class ProtoUtils {
         if (node instanceof Schema) {
             Schema schema = (Schema) node;
             visitor.visit(schema);
-            for (Field f : schema.fields()) {
-                handleNode(f.type, visitor);
+            for (BoundField f : schema.fields()) {
+                handleNode(f.def.type, visitor);
             }
         } else if (node instanceof ArrayOf) {
             ArrayOf array = (ArrayOf) node;


[6/8] kafka git commit: MINOR: Move request/response schemas to the corresponding object representation

Posted by ij...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/protocol/types/BoundField.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/BoundField.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/BoundField.java
new file mode 100644
index 0000000..b031b4f
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/BoundField.java
@@ -0,0 +1,37 @@
+/*
+ * 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.protocol.types;
+
+/**
+ * A field definition bound to a particular schema.
+ */
+public class BoundField {
+    public final Field def;
+    final int index;
+    final Schema schema;
+
+    public BoundField(Field def, Schema schema, int index) {
+        this.def = def;
+        this.schema = schema;
+        this.index = index;
+    }
+
+    @Override
+    public String toString() {
+        return def.name + ":" + def.type;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java
index 29a89d4..8da848b 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java
@@ -16,63 +16,67 @@
  */
 package org.apache.kafka.common.protocol.types;
 
-/**
- * A field in a schema
- */
 public class Field {
-
-    public static final Object NO_DEFAULT = new Object();
-
-    final int index;
     public final String name;
+    public final String docString;
     public final Type type;
+    public final boolean hasDefaultValue;
     public final Object defaultValue;
-    public final String doc;
-    final Schema schema;
 
-    /**
-     * Create the field.
-     *
-     * @throws SchemaException If the default value is not primitive and the validation fails
-     */
-    public Field(int index, String name, Type type, String doc, Object defaultValue, Schema schema) {
-        this.index = index;
+    public Field(String name, Type type, String docString, boolean hasDefaultValue, Object defaultValue) {
         this.name = name;
+        this.docString = docString;
         this.type = type;
-        this.doc = doc;
+        this.hasDefaultValue = hasDefaultValue;
         this.defaultValue = defaultValue;
-        this.schema = schema;
-        if (defaultValue != NO_DEFAULT)
+
+        if (hasDefaultValue)
             type.validate(defaultValue);
     }
 
-    public Field(int index, String name, Type type, String doc, Object defaultValue) {
-        this(index, name, type, doc, defaultValue, null);
+    public Field(String name, Type type, String docString) {
+        this(name, type, docString, false, null);
     }
 
-    public Field(String name, Type type, String doc, Object defaultValue) {
-        this(-1, name, type, doc, defaultValue);
+    public Field(String name, Type type, String docString, Object defaultValue) {
+        this(name, type, docString, true, defaultValue);
     }
 
-    public Field(String name, Type type, String doc) {
-        this(name, type, doc, NO_DEFAULT);
+    public Field(String name, Type type) {
+        this(name, type, null, false, null);
     }
 
-    public Field(String name, Type type) {
-        this(name, type, "");
+    public static class Int8 extends Field {
+        public Int8(String name, String docString) {
+            super(name, Type.INT8, docString, false, null);
+        }
     }
 
-    public Type type() {
-        return type;
+    public static class Int32 extends Field {
+        public Int32(String name, String docString) {
+            super(name, Type.INT32, docString, false, null);
+        }
+
+        public Int32(String name, String docString, int defaultValue) {
+            super(name, Type.INT32, docString, true, defaultValue);
+        }
     }
 
-    public Schema schema() {
-        return schema;
+    public static class Int16 extends Field {
+        public Int16(String name, String docString) {
+            super(name, Type.INT16, docString, false, null);
+        }
     }
 
+    public static class Str extends Field {
+        public Str(String name, String docString) {
+            super(name, Type.STRING, docString, false, null);
+        }
+    }
 
-    @Override
-    public String toString() {
-        return name + ":" + type;
+    public static class NullableStr extends Field {
+        public NullableStr(String name, String docString) {
+            super(name, Type.NULLABLE_STRING, docString, false, null);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java
index a9c08aa..187e14b 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java
@@ -25,8 +25,8 @@ import java.util.Map;
  */
 public class Schema extends Type {
 
-    private final Field[] fields;
-    private final Map<String, Field> fieldsByName;
+    private final BoundField[] fields;
+    private final Map<String, BoundField> fieldsByName;
 
     /**
      * Construct the schema with a given list of its field values
@@ -34,14 +34,14 @@ public class Schema extends Type {
      * @throws SchemaException If the given list have duplicate fields
      */
     public Schema(Field... fs) {
-        this.fields = new Field[fs.length];
+        this.fields = new BoundField[fs.length];
         this.fieldsByName = new HashMap<>();
         for (int i = 0; i < this.fields.length; i++) {
-            Field field = fs[i];
-            if (fieldsByName.containsKey(field.name))
-                throw new SchemaException("Schema contains a duplicate field: " + field.name);
-            this.fields[i] = new Field(i, field.name, field.type, field.doc, field.defaultValue, this);
-            this.fieldsByName.put(fs[i].name, this.fields[i]);
+            Field def = fs[i];
+            if (fieldsByName.containsKey(def.name))
+                throw new SchemaException("Schema contains a duplicate field: " + def.name);
+            this.fields[i] = new BoundField(def, this, i);
+            this.fieldsByName.put(def.name, this.fields[i]);
         }
     }
 
@@ -51,12 +51,12 @@ public class Schema extends Type {
     @Override
     public void write(ByteBuffer buffer, Object o) {
         Struct r = (Struct) o;
-        for (Field field : fields) {
+        for (BoundField field : fields) {
             try {
-                Object value = field.type().validate(r.get(field));
-                field.type.write(buffer, value);
+                Object value = field.def.type.validate(r.get(field));
+                field.def.type.write(buffer, value);
             } catch (Exception e) {
-                throw new SchemaException("Error writing field '" + field.name + "': " +
+                throw new SchemaException("Error writing field '" + field.def.name + "': " +
                                           (e.getMessage() == null ? e.getClass().getName() : e.getMessage()));
             }
         }
@@ -70,9 +70,9 @@ public class Schema extends Type {
         Object[] objects = new Object[fields.length];
         for (int i = 0; i < fields.length; i++) {
             try {
-                objects[i] = fields[i].type.read(buffer);
+                objects[i] = fields[i].def.type.read(buffer);
             } catch (Exception e) {
-                throw new SchemaException("Error reading field '" + fields[i].name + "': " +
+                throw new SchemaException("Error reading field '" + fields[i].def.name + "': " +
                                           (e.getMessage() == null ? e.getClass().getName() : e.getMessage()));
             }
         }
@@ -86,11 +86,11 @@ public class Schema extends Type {
     public int sizeOf(Object o) {
         int size = 0;
         Struct r = (Struct) o;
-        for (Field field : fields) {
+        for (BoundField field : fields) {
             try {
-                size += field.type.sizeOf(r.get(field));
+                size += field.def.type.sizeOf(r.get(field));
             } catch (Exception e) {
-                throw new SchemaException("Error computing size for field '" + field.name + "': " +
+                throw new SchemaException("Error computing size for field '" + field.def.name + "': " +
                         (e.getMessage() == null ? e.getClass().getName() : e.getMessage()));
             }
         }
@@ -110,7 +110,7 @@ public class Schema extends Type {
      * @param slot The slot at which this field sits
      * @return The field
      */
-    public Field get(int slot) {
+    public BoundField get(int slot) {
         return this.fields[slot];
     }
 
@@ -120,14 +120,14 @@ public class Schema extends Type {
      * @param name The name of the field
      * @return The field
      */
-    public Field get(String name) {
+    public BoundField get(String name) {
         return this.fieldsByName.get(name);
     }
 
     /**
      * Get all the fields in this schema
      */
-    public Field[] fields() {
+    public BoundField[] fields() {
         return this.fields;
     }
 
@@ -151,11 +151,11 @@ public class Schema extends Type {
     public Struct validate(Object item) {
         try {
             Struct struct = (Struct) item;
-            for (Field field : fields) {
+            for (BoundField field : fields) {
                 try {
-                    field.type.validate(struct.get(field));
+                    field.def.type.validate(struct.get(field));
                 } catch (SchemaException e) {
-                    throw new SchemaException("Invalid value for field '" + field.name + "': " + e.getMessage());
+                    throw new SchemaException("Invalid value for field '" + field.def.name + "': " + e.getMessage());
                 }
             }
             return struct;

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
----------------------------------------------------------------------
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 c42390b..b3e9975 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
@@ -51,16 +51,16 @@ public class Struct {
      * @param field The field for which to get the default value
      * @throws SchemaException if the field has no value and has no default.
      */
-    private Object getFieldOrDefault(Field field) {
+    private Object getFieldOrDefault(BoundField field) {
         Object value = this.values[field.index];
         if (value != null)
             return value;
-        else if (field.defaultValue != Field.NO_DEFAULT)
-            return field.defaultValue;
-        else if (field.type.isNullable())
+        else if (field.def.hasDefaultValue)
+            return field.def.defaultValue;
+        else if (field.def.type.isNullable())
             return null;
         else
-            throw new SchemaException("Missing value for field '" + field.name + "' which has no default value.");
+            throw new SchemaException("Missing value for field '" + field.def.name + "' which has no default value.");
     }
 
     /**
@@ -70,11 +70,43 @@ public class Struct {
      * @return The value for that field.
      * @throws SchemaException if the field has no value and has no default.
      */
-    public Object get(Field field) {
+    public Object get(BoundField field) {
         validateField(field);
         return getFieldOrDefault(field);
     }
 
+    public Byte get(Field.Int8 field) {
+        return getByte(field.name);
+    }
+
+    public Integer get(Field.Int32 field) {
+        return getInt(field.name);
+    }
+
+    public Short get(Field.Int16 field) {
+        return getShort(field.name);
+    }
+
+    public String get(Field.Str field) {
+        return getString(field.name);
+    }
+
+    public String get(Field.NullableStr field) {
+        return getString(field.name);
+    }
+
+    public Integer getOrElse(Field.Int32 field, int alternative) {
+        if (hasField(field.name))
+            return getInt(field.name);
+        return alternative;
+    }
+
+    public String getOrElse(Field.NullableStr field, String alternative) {
+        if (hasField(field.name))
+            return getString(field.name);
+        return alternative;
+    }
+
     /**
      * Get the record value for the field with the given name by doing a hash table lookup (slower!)
      *
@@ -83,7 +115,7 @@ public class Struct {
      * @throws SchemaException If no such field exists
      */
     public Object get(String name) {
-        Field field = schema.get(name);
+        BoundField field = schema.get(name);
         if (field == null)
             throw new SchemaException("No such field: " + name);
         return getFieldOrDefault(field);
@@ -98,7 +130,11 @@ public class Struct {
         return schema.get(name) != null;
     }
 
-    public Struct getStruct(Field field) {
+    public boolean hasField(Field def) {
+        return schema.get(def.name) != null;
+    }
+
+    public Struct getStruct(BoundField field) {
         return (Struct) get(field);
     }
 
@@ -106,7 +142,7 @@ public class Struct {
         return (Struct) get(name);
     }
 
-    public Byte getByte(Field field) {
+    public Byte getByte(BoundField field) {
         return (Byte) get(field);
     }
 
@@ -118,7 +154,7 @@ public class Struct {
         return (Records) get(name);
     }
 
-    public Short getShort(Field field) {
+    public Short getShort(BoundField field) {
         return (Short) get(field);
     }
 
@@ -126,7 +162,7 @@ public class Struct {
         return (Short) get(name);
     }
 
-    public Integer getInt(Field field) {
+    public Integer getInt(BoundField field) {
         return (Integer) get(field);
     }
 
@@ -138,7 +174,7 @@ public class Struct {
         return (Long) get(name);
     }
 
-    public Long getLong(Field field) {
+    public Long getLong(BoundField field) {
         return (Long) get(field);
     }
 
@@ -146,7 +182,7 @@ public class Struct {
         return (Long) get(name);
     }
 
-    public Object[] getArray(Field field) {
+    public Object[] getArray(BoundField field) {
         return (Object[]) get(field);
     }
 
@@ -154,7 +190,7 @@ public class Struct {
         return (Object[]) get(name);
     }
 
-    public String getString(Field field) {
+    public String getString(BoundField field) {
         return (String) get(field);
     }
 
@@ -162,7 +198,7 @@ public class Struct {
         return (String) get(name);
     }
 
-    public Boolean getBoolean(Field field) {
+    public Boolean getBoolean(BoundField field) {
         return (Boolean) get(field);
     }
 
@@ -170,7 +206,7 @@ public class Struct {
         return (Boolean) get(name);
     }
 
-    public ByteBuffer getBytes(Field field) {
+    public ByteBuffer getBytes(BoundField field) {
         Object result = get(field);
         if (result instanceof byte[])
             return ByteBuffer.wrap((byte[]) result);
@@ -191,7 +227,7 @@ public class Struct {
      * @param value The value
      * @throws SchemaException If the validation of the field failed
      */
-    public Struct set(Field field, Object value) {
+    public Struct set(BoundField field, Object value) {
         validateField(field);
         this.values[field.index] = value;
         return this;
@@ -205,13 +241,40 @@ public class Struct {
      * @throws SchemaException If the field is not known
      */
     public Struct set(String name, Object value) {
-        Field field = this.schema.get(name);
+        BoundField field = this.schema.get(name);
         if (field == null)
             throw new SchemaException("Unknown field: " + name);
         this.values[field.index] = value;
         return this;
     }
 
+    public Struct set(Field.Str def, String value) {
+        return set(def.name, value);
+    }
+
+    public Struct set(Field.NullableStr def, String value) {
+        return set(def.name, value);
+    }
+
+    public Struct set(Field.Int8 def, byte value) {
+        return set(def.name, value);
+    }
+
+    public Struct set(Field.Int32 def, int value) {
+        return set(def.name, value);
+    }
+
+    public Struct set(Field.Int16 def, short value) {
+        return set(def.name, value);
+    }
+
+    public Struct setIfExists(Field def, Object value) {
+        BoundField field = this.schema.get(def.name);
+        if (field != null)
+            this.values[field.index] = value;
+        return this;
+    }
+
     /**
      * 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
@@ -221,15 +284,15 @@ public class Struct {
      * @return The struct
      * @throws SchemaException If the given field is not a container type
      */
-    public Struct instance(Field field) {
+    public Struct instance(BoundField field) {
         validateField(field);
-        if (field.type() instanceof Schema) {
-            return new Struct((Schema) field.type());
-        } else if (field.type() instanceof ArrayOf) {
-            ArrayOf array = (ArrayOf) field.type();
+        if (field.def.type instanceof Schema) {
+            return new Struct((Schema) field.def.type);
+        } else if (field.def.type instanceof ArrayOf) {
+            ArrayOf array = (ArrayOf) field.def.type;
             return new Struct((Schema) array.type());
         } else {
-            throw new SchemaException("Field '" + field.name + "' is not a container type, it is of type " + field.type());
+            throw new SchemaException("Field '" + field.def.name + "' is not a container type, it is of type " + field.def.type);
         }
     }
 
@@ -270,9 +333,9 @@ public class Struct {
      *
      * @throws SchemaException If validation fails
      */
-    private void validateField(Field field) {
+    private void validateField(BoundField field) {
         if (this.schema != field.schema)
-            throw new SchemaException("Attempt to access field '" + field.name + "' from a different schema instance.");
+            throw new SchemaException("Attempt to access field '" + field.def.name + "' from a different schema instance.");
         if (field.index > values.length)
             throw new SchemaException("Invalid field index: " + field.index);
     }
@@ -291,10 +354,10 @@ public class Struct {
         StringBuilder b = new StringBuilder();
         b.append('{');
         for (int i = 0; i < this.values.length; i++) {
-            Field f = this.schema.get(i);
-            b.append(f.name);
+            BoundField f = this.schema.get(i);
+            b.append(f.def.name);
             b.append('=');
-            if (f.type() instanceof ArrayOf && this.values[i] != null) {
+            if (f.def.type instanceof ArrayOf && this.values[i] != null) {
                 Object[] arrayValue = (Object[]) this.values[i];
                 b.append('[');
                 for (int j = 0; j < arrayValue.length; j++) {
@@ -317,8 +380,8 @@ public class Struct {
         final int prime = 31;
         int result = 1;
         for (int i = 0; i < this.values.length; i++) {
-            Field f = this.schema.get(i);
-            if (f.type() instanceof ArrayOf) {
+            BoundField f = this.schema.get(i);
+            if (f.def.type instanceof ArrayOf) {
                 if (this.get(f) != null) {
                     Object[] arrayObject = (Object[]) this.get(f);
                     for (Object arrayItem: arrayObject)
@@ -346,9 +409,9 @@ public class Struct {
         if (schema != other.schema)
             return false;
         for (int i = 0; i < this.values.length; i++) {
-            Field f = this.schema.get(i);
+            BoundField f = this.schema.get(i);
             boolean result;
-            if (f.type() instanceof ArrayOf) {
+            if (f.def.type instanceof ArrayOf) {
                 result = Arrays.equals((Object[]) this.get(f), (Object[]) other.get(f));
             } else {
                 Object thisField = this.get(f);

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
----------------------------------------------------------------------
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 34fda50..1f1418f 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
@@ -133,9 +133,9 @@ public abstract class AbstractRequest extends AbstractRequestResponse {
                 return new SyncGroupRequest(struct, apiVersion);
             case STOP_REPLICA:
                 return new StopReplicaRequest(struct, apiVersion);
-            case CONTROLLED_SHUTDOWN_KEY:
+            case CONTROLLED_SHUTDOWN:
                 return new ControlledShutdownRequest(struct, apiVersion);
-            case UPDATE_METADATA_KEY:
+            case UPDATE_METADATA:
                 return new UpdateMetadataRequest(struct, apiVersion);
             case LEADER_AND_ISR:
                 return new LeaderAndIsrRequest(struct, apiVersion);

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java
index 95d1ef9..b6cb8fb 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java
@@ -24,7 +24,6 @@ import org.apache.kafka.common.protocol.types.Struct;
 import java.nio.ByteBuffer;
 
 public abstract class AbstractResponse extends AbstractRequestResponse {
-    public static final String THROTTLE_TIME_KEY_NAME = "throttle_time_ms";
     public static final int DEFAULT_THROTTLE_TIME = 0;
 
     protected Send toSend(String destination, ResponseHeader header, short apiVersion) {
@@ -66,9 +65,9 @@ public abstract class AbstractResponse extends AbstractRequestResponse {
                 return new SyncGroupResponse(struct);
             case STOP_REPLICA:
                 return new StopReplicaResponse(struct);
-            case CONTROLLED_SHUTDOWN_KEY:
+            case CONTROLLED_SHUTDOWN:
                 return new ControlledShutdownResponse(struct);
-            case UPDATE_METADATA_KEY:
+            case UPDATE_METADATA:
                 return new UpdateMetadataResponse(struct);
             case LEADER_AND_ISR:
                 return new LeaderAndIsrResponse(struct);

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java
index 36b290f..e3e4d79 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java
@@ -18,16 +18,32 @@ 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.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
 
+import static org.apache.kafka.common.protocol.types.Type.INT16;
+import static org.apache.kafka.common.protocol.types.Type.INT64;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class AddOffsetsToTxnRequest extends AbstractRequest {
     private static final String TRANSACTIONAL_ID_KEY_NAME = "transactional_id";
     private static final String PRODUCER_ID_KEY_NAME = "producer_id";
     private static final String EPOCH_KEY_NAME = "producer_epoch";
     private static final String CONSUMER_GROUP_ID_KEY_NAME = "consumer_group_id";
 
+    private static final Schema ADD_OFFSETS_TO_TXN_REQUEST_V0 = new Schema(
+            new Field(TRANSACTIONAL_ID_KEY_NAME, STRING, "The transactional id corresponding to the transaction."),
+            new Field(PRODUCER_ID_KEY_NAME, INT64, "Current producer id in use by the transactional id."),
+            new Field(EPOCH_KEY_NAME, INT16, "Current epoch associated with the producer id."),
+            new Field(CONSUMER_GROUP_ID_KEY_NAME, STRING, "Consumer group id whose offsets should be included in the transaction."));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{ADD_OFFSETS_TO_TXN_REQUEST_V0};
+    }
+
     public static class Builder extends AbstractRequest.Builder<AddOffsetsToTxnRequest> {
         private final String transactionalId;
         private final long producerId;

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java
index 981a234..10dc279 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java
@@ -18,12 +18,22 @@ 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.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+
 public class AddOffsetsToTxnResponse extends AbstractResponse {
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
+    private static final Schema ADD_OFFSETS_TO_TXN_RESPONSE_V0 = new Schema(
+            THROTTLE_TIME_MS,
+            ERROR_CODE);
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{ADD_OFFSETS_TO_TXN_RESPONSE_V0};
+    }
 
     // Possible error codes:
     //   NotCoordinator
@@ -44,8 +54,8 @@ public class AddOffsetsToTxnResponse extends AbstractResponse {
     }
 
     public AddOffsetsToTxnResponse(Struct struct) {
-        this.throttleTimeMs = struct.getInt(THROTTLE_TIME_KEY_NAME);
-        this.error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME));
+        this.throttleTimeMs = struct.get(THROTTLE_TIME_MS);
+        this.error = Errors.forCode(struct.get(ERROR_CODE));
     }
 
     public int throttleTimeMs() {
@@ -59,8 +69,8 @@ public class AddOffsetsToTxnResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.ADD_OFFSETS_TO_TXN.responseSchema(version));
-        struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
-        struct.set(ERROR_CODE_KEY_NAME, error.code());
+        struct.set(THROTTLE_TIME_MS, throttleTimeMs);
+        struct.set(ERROR_CODE, error.code());
         return struct;
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java
index 6fe034c..c195e24 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java
@@ -19,6 +19,9 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.CollectionUtils;
 
@@ -28,14 +31,32 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.INT16;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.INT64;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class AddPartitionsToTxnRequest extends AbstractRequest {
     private static final String TRANSACTIONAL_ID_KEY_NAME = "transactional_id";
     private static final String PRODUCER_ID_KEY_NAME = "producer_id";
     private static final String PRODUCER_EPOCH_KEY_NAME = "producer_epoch";
-    private static final String TOPIC_PARTITIONS_KEY_NAME = "topics";
-    private static final String TOPIC_KEY_NAME = "topic";
+    private static final String TOPICS_KEY_NAME = "topics";
     private static final String PARTITIONS_KEY_NAME = "partitions";
 
+    private static final Schema ADD_PARTITIONS_TO_TXN_REQUEST_V0 = new Schema(
+            new Field(TRANSACTIONAL_ID_KEY_NAME, STRING, "The transactional id corresponding to the transaction."),
+            new Field(PRODUCER_ID_KEY_NAME, INT64, "Current producer id in use by the transactional id."),
+            new Field(PRODUCER_EPOCH_KEY_NAME, INT16, "Current epoch associated with the producer id."),
+            new Field(TOPICS_KEY_NAME, new ArrayOf(new Schema(
+                    TOPIC_NAME,
+                    new Field(PARTITIONS_KEY_NAME, new ArrayOf(INT32)))),
+                    "The partitions to add to the transaction."));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{ADD_PARTITIONS_TO_TXN_REQUEST_V0};
+    }
+
     public static class Builder extends AbstractRequest.Builder<AddPartitionsToTxnRequest> {
         private final String transactionalId;
         private final long producerId;
@@ -93,10 +114,10 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
         this.producerEpoch = struct.getShort(PRODUCER_EPOCH_KEY_NAME);
 
         List<TopicPartition> partitions = new ArrayList<>();
-        Object[] topicPartitionsArray = struct.getArray(TOPIC_PARTITIONS_KEY_NAME);
+        Object[] topicPartitionsArray = struct.getArray(TOPICS_KEY_NAME);
         for (Object topicPartitionObj : topicPartitionsArray) {
             Struct topicPartitionStruct = (Struct) topicPartitionObj;
-            String topic = topicPartitionStruct.getString(TOPIC_KEY_NAME);
+            String topic = topicPartitionStruct.get(TOPIC_NAME);
             for (Object partitionObj : topicPartitionStruct.getArray(PARTITIONS_KEY_NAME)) {
                 partitions.add(new TopicPartition(topic, (Integer) partitionObj));
             }
@@ -131,13 +152,13 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
         Object[] partitionsArray = new Object[mappedPartitions.size()];
         int i = 0;
         for (Map.Entry<String, List<Integer>> topicAndPartitions : mappedPartitions.entrySet()) {
-            Struct topicPartitionsStruct = struct.instance(TOPIC_PARTITIONS_KEY_NAME);
-            topicPartitionsStruct.set(TOPIC_KEY_NAME, topicAndPartitions.getKey());
+            Struct topicPartitionsStruct = struct.instance(TOPICS_KEY_NAME);
+            topicPartitionsStruct.set(TOPIC_NAME, topicAndPartitions.getKey());
             topicPartitionsStruct.set(PARTITIONS_KEY_NAME, topicAndPartitions.getValue().toArray());
             partitionsArray[i++] = topicPartitionsStruct;
         }
 
-        struct.set(TOPIC_PARTITIONS_KEY_NAME, partitionsArray);
+        struct.set(TOPICS_KEY_NAME, partitionsArray);
         return struct;
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java
index f05310a..e9f6088 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java
@@ -19,6 +19,9 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.CollectionUtils;
 
@@ -28,13 +31,27 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+
 public class AddPartitionsToTxnResponse extends AbstractResponse {
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
     private static final String ERRORS_KEY_NAME = "errors";
-    private static final String TOPIC_NAME = "topic";
-    private static final String PARTITION = "partition";
     private static final String PARTITION_ERRORS = "partition_errors";
 
+    private static final Schema ADD_PARTITIONS_TO_TXN_RESPONSE_V0 = new Schema(
+            THROTTLE_TIME_MS,
+            new Field(ERRORS_KEY_NAME, new ArrayOf(new Schema(
+                    TOPIC_NAME,
+                    new Field(PARTITION_ERRORS, new ArrayOf(new Schema(
+                            PARTITION_ID,
+                            ERROR_CODE)))))));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{ADD_PARTITIONS_TO_TXN_RESPONSE_V0};
+    }
+
     private final int throttleTimeMs;
 
     // Possible error codes:
@@ -56,15 +73,15 @@ public class AddPartitionsToTxnResponse extends AbstractResponse {
     }
 
     public AddPartitionsToTxnResponse(Struct struct) {
-        this.throttleTimeMs = struct.getInt(THROTTLE_TIME_KEY_NAME);
+        this.throttleTimeMs = struct.get(THROTTLE_TIME_MS);
         errors = new HashMap<>();
         for (Object topic : struct.getArray(ERRORS_KEY_NAME)) {
             Struct topicStruct = (Struct) topic;
-            final String topicName = topicStruct.getString(TOPIC_NAME);
+            final String topicName = topicStruct.get(TOPIC_NAME);
             for (Object partition : topicStruct.getArray(PARTITION_ERRORS)) {
                 Struct partitionStruct = (Struct) partition;
-                TopicPartition topicPartition = new TopicPartition(topicName, partitionStruct.getInt(PARTITION));
-                errors.put(topicPartition, Errors.forCode(partitionStruct.getShort(ERROR_CODE_KEY_NAME)));
+                TopicPartition topicPartition = new TopicPartition(topicName, partitionStruct.get(PARTITION_ID));
+                errors.put(topicPartition, Errors.forCode(partitionStruct.get(ERROR_CODE)));
             }
         }
     }
@@ -80,7 +97,7 @@ public class AddPartitionsToTxnResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.ADD_PARTITIONS_TO_TXN.responseSchema(version));
-        struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
+        struct.set(THROTTLE_TIME_MS, throttleTimeMs);
 
         Map<String, Map<Integer, Errors>> errorsByTopic = CollectionUtils.groupDataByTopic(errors);
         List<Struct> topics = new ArrayList<>(errorsByTopic.size());
@@ -90,8 +107,8 @@ public class AddPartitionsToTxnResponse extends AbstractResponse {
             List<Struct> partitionArray = new ArrayList<>();
             for (Map.Entry<Integer, Errors> partitionErrors : entry.getValue().entrySet()) {
                 final Struct partitionData = topicErrorCodes.instance(PARTITION_ERRORS)
-                        .set(PARTITION, partitionErrors.getKey())
-                        .set(ERROR_CODE_KEY_NAME, partitionErrors.getValue().code());
+                        .set(PARTITION_ID, partitionErrors.getKey())
+                        .set(ERROR_CODE, partitionErrors.getValue().code());
                 partitionArray.add(partitionData);
 
             }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsRequest.java
index a964f85..14b39ae 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsRequest.java
@@ -18,6 +18,9 @@
 package org.apache.kafka.common.requests;
 
 import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
@@ -27,6 +30,11 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.types.Type.BOOLEAN;
+import static org.apache.kafka.common.protocol.types.Type.INT8;
+import static org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class AlterConfigsRequest extends AbstractRequest {
 
     private static final String RESOURCES_KEY_NAME = "resources";
@@ -38,6 +46,24 @@ public class AlterConfigsRequest extends AbstractRequest {
     private static final String CONFIG_NAME = "config_name";
     private static final String CONFIG_VALUE = "config_value";
 
+    private static final Schema CONFIG_ENTRY = new Schema(
+            new Field(CONFIG_NAME, STRING, "Configuration name"),
+            new Field(CONFIG_VALUE, NULLABLE_STRING, "Configuration value"));
+
+    private static final Schema ALTER_CONFIGS_REQUEST_RESOURCE_V0 = new Schema(
+            new Field(RESOURCE_TYPE_KEY_NAME, INT8),
+            new Field(RESOURCE_NAME_KEY_NAME, STRING),
+            new Field(CONFIG_ENTRIES_KEY_NAME, new ArrayOf(CONFIG_ENTRY)));
+
+    private static final Schema ALTER_CONFIGS_REQUEST_V0 = new Schema(
+            new Field(RESOURCES_KEY_NAME, new ArrayOf(ALTER_CONFIGS_REQUEST_RESOURCE_V0),
+                    "An array of resources to update with the provided configs."),
+            new Field(VALIDATE_ONLY_KEY_NAME, BOOLEAN));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {ALTER_CONFIGS_REQUEST_V0};
+    }
+
     public static class Config {
         private final Collection<ConfigEntry> entries;
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsResponse.java
index 3a3eb9a..df9416e 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsResponse.java
@@ -18,6 +18,9 @@
 package org.apache.kafka.common.requests;
 
 import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
@@ -26,12 +29,32 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_MESSAGE;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.types.Type.INT8;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class AlterConfigsResponse extends AbstractResponse {
 
     private static final String RESOURCES_KEY_NAME = "resources";
     private static final String RESOURCE_TYPE_KEY_NAME = "resource_type";
     private static final String RESOURCE_NAME_KEY_NAME = "resource_name";
 
+    private static final Schema ALTER_CONFIGS_RESPONSE_ENTITY_V0 = new Schema(
+            ERROR_CODE,
+            ERROR_MESSAGE,
+            new Field(RESOURCE_TYPE_KEY_NAME, INT8),
+            new Field(RESOURCE_NAME_KEY_NAME, STRING));
+
+    private static final Schema ALTER_CONFIGS_RESPONSE_V0 = new Schema(
+            THROTTLE_TIME_MS,
+            new Field(RESOURCES_KEY_NAME, new ArrayOf(ALTER_CONFIGS_RESPONSE_ENTITY_V0)));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{ALTER_CONFIGS_RESPONSE_V0};
+    }
+
     private final int throttleTimeMs;
     private final Map<Resource, ApiError> errors;
 
@@ -42,7 +65,7 @@ public class AlterConfigsResponse extends AbstractResponse {
     }
 
     public AlterConfigsResponse(Struct struct) {
-        throttleTimeMs = struct.getInt(THROTTLE_TIME_KEY_NAME);
+        throttleTimeMs = struct.get(THROTTLE_TIME_MS);
         Object[] resourcesArray = struct.getArray(RESOURCES_KEY_NAME);
         errors = new HashMap<>(resourcesArray.length);
         for (Object resourceObj : resourcesArray) {
@@ -65,7 +88,7 @@ public class AlterConfigsResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.ALTER_CONFIGS.responseSchema(version));
-        struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
+        struct.set(THROTTLE_TIME_MS, throttleTimeMs);
         List<Struct> resourceStructs = new ArrayList<>(errors.size());
         for (Map.Entry<Resource, ApiError> entry : errors.entrySet()) {
             Struct resourceStruct = struct.instance(RESOURCES_KEY_NAME);

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaDirRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaDirRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaDirRequest.java
index 2c2401b..7e58fd6 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaDirRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaDirRequest.java
@@ -20,14 +20,21 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.CollectionUtils;
+
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
 
 public class AlterReplicaDirRequest extends AbstractRequest {
 
@@ -39,9 +46,19 @@ public class AlterReplicaDirRequest extends AbstractRequest {
     private static final String TOPICS_KEY_NAME = "topics";
 
     // topic level key names
-    private static final String TOPIC_KEY_NAME = "topic";
     private static final String PARTITIONS_KEY_NAME = "partitions";
 
+    private static final Schema ALTER_REPLICA_DIR_REQUEST_V0 = new Schema(
+            new Field("log_dirs", new ArrayOf(new Schema(
+                    new Field("log_dir", STRING, "The absolute log directory path."),
+                    new Field("topics", new ArrayOf(new Schema(
+                            TOPIC_NAME,
+                            new Field("partitions", new ArrayOf(INT32), "List of partition ids of the topic."))))))));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{ALTER_REPLICA_DIR_REQUEST_V0};
+    }
+
     private final Map<TopicPartition, String> partitionDirs;
 
     public static class Builder extends AbstractRequest.Builder<AlterReplicaDirRequest> {
@@ -76,7 +93,7 @@ public class AlterReplicaDirRequest extends AbstractRequest {
             String logDir = logDirStruct.getString(LOG_DIR_KEY_NAME);
             for (Object topicStructObj : logDirStruct.getArray(TOPICS_KEY_NAME)) {
                 Struct topicStruct = (Struct) topicStructObj;
-                String topic = topicStruct.getString(TOPIC_KEY_NAME);
+                String topic = topicStruct.get(TOPIC_NAME);
                 for (Object partitionObj : topicStruct.getArray(PARTITIONS_KEY_NAME)) {
                     int partition = (Integer) partitionObj;
                     partitionDirs.put(new TopicPartition(topic, partition), logDir);
@@ -108,7 +125,7 @@ public class AlterReplicaDirRequest extends AbstractRequest {
             List<Struct> topicStructArray = new ArrayList<>();
             for (Map.Entry<String, List<Integer>> topicEntry: CollectionUtils.groupDataByTopic(logDirEntry.getValue()).entrySet()) {
                 Struct topicStruct = logDirStruct.instance(TOPICS_KEY_NAME);
-                topicStruct.set(TOPIC_KEY_NAME, topicEntry.getKey());
+                topicStruct.set(TOPIC_NAME, topicEntry.getKey());
                 topicStruct.set(PARTITIONS_KEY_NAME, topicEntry.getValue().toArray());
                 topicStructArray.add(topicStruct);
             }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaDirResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaDirResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaDirResponse.java
index f97f9a0..ed00b75 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaDirResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaDirResponse.java
@@ -20,14 +20,23 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.CollectionUtils;
+
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+
 
 public class AlterReplicaDirResponse extends AbstractResponse {
 
@@ -35,12 +44,19 @@ public class AlterReplicaDirResponse extends AbstractResponse {
     private static final String TOPICS_KEY_NAME = "topics";
 
     // topic level key names
-    private static final String TOPIC_KEY_NAME = "topic";
     private static final String PARTITIONS_KEY_NAME = "partitions";
 
-    // partition level key names
-    private static final String PARTITION_KEY_NAME = "partition";
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
+    private static final Schema ALTER_REPLICA_DIR_RESPONSE_V0 = new Schema(
+            THROTTLE_TIME_MS,
+            new Field("topics", new ArrayOf(new Schema(
+                    TOPIC_NAME,
+                    new Field("partitions", new ArrayOf(new Schema(
+                            PARTITION_ID,
+                            ERROR_CODE)))))));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{ALTER_REPLICA_DIR_RESPONSE_V0};
+    }
 
     /**
      * Possible error code:
@@ -54,15 +70,15 @@ public class AlterReplicaDirResponse extends AbstractResponse {
     private final int throttleTimeMs;
 
     public AlterReplicaDirResponse(Struct struct) {
-        throttleTimeMs = struct.getInt(THROTTLE_TIME_KEY_NAME);
+        throttleTimeMs = struct.get(THROTTLE_TIME_MS);
         responses = new HashMap<>();
         for (Object topicStructObj : struct.getArray(TOPICS_KEY_NAME)) {
             Struct topicStruct = (Struct) topicStructObj;
-            String topic = topicStruct.getString(TOPIC_KEY_NAME);
+            String topic = topicStruct.get(TOPIC_NAME);
             for (Object partitionStructObj : topicStruct.getArray(PARTITIONS_KEY_NAME)) {
                 Struct partitionStruct = (Struct) partitionStructObj;
-                int partition = partitionStruct.getInt(PARTITION_KEY_NAME);
-                Errors error = Errors.forCode(partitionStruct.getShort(ERROR_CODE_KEY_NAME));
+                int partition = partitionStruct.get(PARTITION_ID);
+                Errors error = Errors.forCode(partitionStruct.get(ERROR_CODE));
                 responses.put(new TopicPartition(topic, partition), error);
             }
         }
@@ -79,18 +95,18 @@ public class AlterReplicaDirResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.ALTER_REPLICA_DIR.responseSchema(version));
-        struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
+        struct.set(THROTTLE_TIME_MS, throttleTimeMs);
         Map<String, Map<Integer, Errors>> responsesByTopic = CollectionUtils.groupDataByTopic(responses);
         List<Struct> topicStructArray = new ArrayList<>();
         for (Map.Entry<String, Map<Integer, Errors>> responsesByTopicEntry : responsesByTopic.entrySet()) {
             Struct topicStruct = struct.instance(TOPICS_KEY_NAME);
-            topicStruct.set(TOPIC_KEY_NAME, responsesByTopicEntry.getKey());
+            topicStruct.set(TOPIC_NAME, responsesByTopicEntry.getKey());
             List<Struct> partitionStructArray = new ArrayList<>();
             for (Map.Entry<Integer, Errors> responsesByPartitionEntry : responsesByTopicEntry.getValue().entrySet()) {
                 Struct partitionStruct = topicStruct.instance(PARTITIONS_KEY_NAME);
                 Errors response = responsesByPartitionEntry.getValue();
-                partitionStruct.set(PARTITION_KEY_NAME, responsesByPartitionEntry.getKey());
-                partitionStruct.set(ERROR_CODE_KEY_NAME, response.code());
+                partitionStruct.set(PARTITION_ID, responsesByPartitionEntry.getKey());
+                partitionStruct.set(ERROR_CODE, response.code());
                 partitionStructArray.add(partitionStruct);
             }
             topicStruct.set(PARTITIONS_KEY_NAME, partitionStructArray.toArray());

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/ApiError.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiError.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiError.java
index d712123..dad21b3 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ApiError.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiError.java
@@ -21,6 +21,9 @@ import org.apache.kafka.common.errors.ApiException;
 import org.apache.kafka.common.protocol.Errors;
 import org.apache.kafka.common.protocol.types.Struct;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_MESSAGE;
+
 /**
  * Encapsulates an error code (via the Errors enum) and an optional message. Generally, the optional message is only
  * defined if it adds information over the default message associated with the error code.
@@ -31,9 +34,6 @@ public class ApiError {
 
     public static final ApiError NONE = new ApiError(Errors.NONE, null);
 
-    private static final String CODE_KEY_NAME = "error_code";
-    private static final String MESSAGE_KEY_NAME = "error_message";
-
     private final Errors error;
     private final String message;
 
@@ -45,12 +45,9 @@ public class ApiError {
     }
 
     public ApiError(Struct struct) {
-        error = Errors.forCode(struct.getShort(CODE_KEY_NAME));
+        error = Errors.forCode(struct.get(ERROR_CODE));
         // In some cases, the error message field was introduced in newer version
-        if (struct.hasField(MESSAGE_KEY_NAME))
-            message = struct.getString(MESSAGE_KEY_NAME);
-        else
-            message = null;
+        message = struct.getOrElse(ERROR_MESSAGE, null);
     }
 
     public ApiError(Errors error, String message) {
@@ -59,10 +56,9 @@ public class ApiError {
     }
 
     public void write(Struct struct) {
-        struct.set(CODE_KEY_NAME, error.code());
-        // In some cases, the error message field was introduced in a newer protocol API version
-        if (struct.hasField(MESSAGE_KEY_NAME) && message != null && error != Errors.NONE)
-            struct.set(MESSAGE_KEY_NAME, message);
+        struct.set(ERROR_CODE, error.code());
+        if (error != Errors.NONE)
+            struct.setIfExists(ERROR_MESSAGE, message);
     }
 
     public boolean is(Errors error) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java
----------------------------------------------------------------------
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
index 025ef6c..22daf6c 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java
@@ -18,12 +18,22 @@ 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.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 API_VERSIONS_REQUEST_V0 = new Schema();
+
+    /* v1 request is the same as v0. Throttle time has been added to response */
+    private static final Schema API_VERSIONS_REQUEST_V1 = API_VERSIONS_REQUEST_V0;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{API_VERSIONS_REQUEST_V0, API_VERSIONS_REQUEST_V1};
+    }
+
     public static class Builder extends AbstractRequest.Builder<ApiVersionsRequest> {
 
         public Builder() {

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
----------------------------------------------------------------------
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
index 5a48c93..6a0418f 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java
@@ -18,6 +18,9 @@ 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.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.record.RecordBatch;
 
@@ -28,15 +31,35 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-public class ApiVersionsResponse extends AbstractResponse {
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.types.Type.INT16;
 
-    public static final ApiVersionsResponse API_VERSIONS_RESPONSE = createApiVersionsResponse(DEFAULT_THROTTLE_TIME, RecordBatch.CURRENT_MAGIC_VALUE);
-    private static final String THROTTLE_TIME_KEY_NAME = "throttle_time_ms";
-    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";
+public class ApiVersionsResponse extends AbstractResponse {
+    private static final String API_VERSIONS_KEY_NAME = "api_versions";
+    private static final String API_KEY_NAME = "api_key";
+    private static final String MIN_VERSION_KEY_NAME = "min_version";
+    private static final String MAX_VERSION_KEY_NAME = "max_version";
+
+    private static final Schema API_VERSIONS_V0 = new Schema(
+            new Field(API_KEY_NAME, INT16, "API key."),
+            new Field(MIN_VERSION_KEY_NAME, INT16, "Minimum supported version."),
+            new Field(MAX_VERSION_KEY_NAME, INT16, "Maximum supported version."));
+
+    private static final Schema API_VERSIONS_RESPONSE_V0 = new Schema(
+            ERROR_CODE,
+            new Field(API_VERSIONS_KEY_NAME, new ArrayOf(API_VERSIONS_V0), "API versions supported by the broker."));
+    private static final Schema API_VERSIONS_RESPONSE_V1 = new Schema(
+            ERROR_CODE,
+            new Field(API_VERSIONS_KEY_NAME, new ArrayOf(API_VERSIONS_V0), "API versions supported by the broker."),
+            THROTTLE_TIME_MS);
+
+    // initialized lazily to avoid circular initialization dependence with ApiKeys
+    private static volatile ApiVersionsResponse defaultApiVersionsResponse;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{API_VERSIONS_RESPONSE_V0, API_VERSIONS_RESPONSE_V1};
+    }
 
     /**
      * Possible error codes:
@@ -83,8 +106,8 @@ public class ApiVersionsResponse extends AbstractResponse {
     }
 
     public ApiVersionsResponse(Struct struct) {
-        this.throttleTimeMs = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME;
-        this.error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME));
+        this.throttleTimeMs = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME);
+        this.error = Errors.forCode(struct.get(ERROR_CODE));
         List<ApiVersion> tempApiVersions = new ArrayList<>();
         for (Object apiVersionsObj : struct.getArray(API_VERSIONS_KEY_NAME)) {
             Struct apiVersionStruct = (Struct) apiVersionsObj;
@@ -99,9 +122,8 @@ public class ApiVersionsResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.API_VERSIONS.responseSchema(version));
-        if (struct.hasField(THROTTLE_TIME_KEY_NAME))
-            struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
-        struct.set(ERROR_CODE_KEY_NAME, error.code());
+        struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs);
+        struct.set(ERROR_CODE, error.code());
         List<Struct> apiVersionList = new ArrayList<>();
         for (ApiVersion apiVersion : apiKeyToApiVersion.values()) {
             Struct apiVersionStruct = struct.instance(API_VERSIONS_KEY_NAME);
@@ -116,7 +138,7 @@ public class ApiVersionsResponse extends AbstractResponse {
 
     public static ApiVersionsResponse apiVersionsResponse(int throttleTimeMs, byte maxMagic) {
         if (maxMagic == RecordBatch.CURRENT_MAGIC_VALUE && throttleTimeMs == DEFAULT_THROTTLE_TIME) {
-            return API_VERSIONS_RESPONSE;
+            return defaultApiVersionsResponse();
         }
         return createApiVersionsResponse(throttleTimeMs, maxMagic);
     }
@@ -141,22 +163,28 @@ public class ApiVersionsResponse extends AbstractResponse {
         return new ApiVersionsResponse(ApiKeys.API_VERSIONS.parseResponse(version, buffer));
     }
 
+    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;
+    }
+
     public static ApiVersionsResponse createApiVersionsResponse(int throttleTimeMs, final byte minMagic) {
-        List<ApiVersion> versionList = new ArrayList<>();
+        List<ApiVersionsResponse.ApiVersion> versionList = new ArrayList<>();
         for (ApiKeys apiKey : ApiKeys.values()) {
             if (apiKey.minRequiredInterBrokerMagic <= minMagic) {
-                versionList.add(new ApiVersion(apiKey));
+                versionList.add(new ApiVersionsResponse.ApiVersion(apiKey));
             }
         }
         return new ApiVersionsResponse(throttleTimeMs, Errors.NONE, 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;
+    public static ApiVersionsResponse defaultApiVersionsResponse() {
+        if (defaultApiVersionsResponse == null)
+            defaultApiVersionsResponse = createApiVersionsResponse(DEFAULT_THROTTLE_TIME, RecordBatch.CURRENT_MAGIC_VALUE);
+        return defaultApiVersionsResponse;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java
index 1b49c6a..c77bd13 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java
@@ -19,14 +19,26 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
 import java.util.Collections;
 
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+
 public class ControlledShutdownRequest extends AbstractRequest {
     private static final String BROKER_ID_KEY_NAME = "broker_id";
 
+    private static final Schema CONTROLLED_SHUTDOWN_REQUEST_V0 = new Schema(
+            new Field(BROKER_ID_KEY_NAME, INT32, "The id of the broker for which controlled shutdown has been requested."));
+    private static final Schema CONTROLLED_SHUTDOWN_REQUEST_V1 = CONTROLLED_SHUTDOWN_REQUEST_V0;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {CONTROLLED_SHUTDOWN_REQUEST_V0, CONTROLLED_SHUTDOWN_REQUEST_V1};
+    }
+
     public static class Builder extends AbstractRequest.Builder<ControlledShutdownRequest> {
         private final int brokerId;
 
@@ -35,7 +47,7 @@ public class ControlledShutdownRequest extends AbstractRequest {
         }
 
         public Builder(int brokerId, Short desiredVersion) {
-            super(ApiKeys.CONTROLLED_SHUTDOWN_KEY, desiredVersion);
+            super(ApiKeys.CONTROLLED_SHUTDOWN, desiredVersion);
             this.brokerId = brokerId;
         }
 
@@ -74,7 +86,7 @@ public class ControlledShutdownRequest extends AbstractRequest {
                 return new ControlledShutdownResponse(Errors.forException(e), Collections.<TopicPartition>emptySet());
             default:
                 throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
-                        versionId, this.getClass().getSimpleName(), ApiKeys.CONTROLLED_SHUTDOWN_KEY.latestVersion()));
+                        versionId, this.getClass().getSimpleName(), ApiKeys.CONTROLLED_SHUTDOWN.latestVersion()));
         }
     }
 
@@ -84,12 +96,12 @@ public class ControlledShutdownRequest extends AbstractRequest {
 
     public static ControlledShutdownRequest parse(ByteBuffer buffer, short version) {
         return new ControlledShutdownRequest(
-                ApiKeys.CONTROLLED_SHUTDOWN_KEY.parseRequest(version, buffer), version);
+                ApiKeys.CONTROLLED_SHUTDOWN.parseRequest(version, buffer), version);
     }
 
     @Override
     protected Struct toStruct() {
-        Struct struct = new Struct(ApiKeys.CONTROLLED_SHUTDOWN_KEY.requestSchema(version()));
+        Struct struct = new Struct(ApiKeys.CONTROLLED_SHUTDOWN.requestSchema(version()));
         struct.set(BROKER_ID_KEY_NAME, brokerId);
         return struct;
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java
index 00973f0..e0b3860 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java
@@ -19,6 +19,9 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
@@ -27,13 +30,28 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+
 public class ControlledShutdownResponse extends AbstractResponse {
 
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
     private static final String PARTITIONS_REMAINING_KEY_NAME = "partitions_remaining";
 
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String PARTITION_KEY_NAME = "partition";
+    private static final Schema CONTROLLED_SHUTDOWN_PARTITION_V0 = new Schema(
+            TOPIC_NAME,
+            PARTITION_ID);
+
+    private static final Schema CONTROLLED_SHUTDOWN_RESPONSE_V0 = new Schema(
+            ERROR_CODE,
+            new Field("partitions_remaining", new ArrayOf(CONTROLLED_SHUTDOWN_PARTITION_V0), "The partitions " +
+                    "that the broker still leads."));
+
+    private static final Schema CONTROLLED_SHUTDOWN_RESPONSE_V1 = CONTROLLED_SHUTDOWN_RESPONSE_V0;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{CONTROLLED_SHUTDOWN_RESPONSE_V0, CONTROLLED_SHUTDOWN_RESPONSE_V1};
+    }
 
     /**
      * Possible error codes:
@@ -52,12 +70,12 @@ public class ControlledShutdownResponse extends AbstractResponse {
     }
 
     public ControlledShutdownResponse(Struct struct) {
-        error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME));
+        error = Errors.forCode(struct.get(ERROR_CODE));
         Set<TopicPartition> partitions = new HashSet<>();
         for (Object topicPartitionObj : struct.getArray(PARTITIONS_REMAINING_KEY_NAME)) {
             Struct topicPartition = (Struct) topicPartitionObj;
-            String topic = topicPartition.getString(TOPIC_KEY_NAME);
-            int partition = topicPartition.getInt(PARTITION_KEY_NAME);
+            String topic = topicPartition.get(TOPIC_NAME);
+            int partition = topicPartition.get(PARTITION_ID);
             partitions.add(new TopicPartition(topic, partition));
         }
         partitionsRemaining = partitions;
@@ -72,20 +90,19 @@ public class ControlledShutdownResponse extends AbstractResponse {
     }
 
     public static ControlledShutdownResponse parse(ByteBuffer buffer, short version) {
-        return new ControlledShutdownResponse(ApiKeys.CONTROLLED_SHUTDOWN_KEY.parseResponse(version, buffer));
+        return new ControlledShutdownResponse(ApiKeys.CONTROLLED_SHUTDOWN.parseResponse(version, buffer));
     }
 
     @Override
     protected Struct toStruct(short version) {
-        Struct struct = new Struct(ApiKeys.CONTROLLED_SHUTDOWN_KEY.responseSchema(version));
-
-        struct.set(ERROR_CODE_KEY_NAME, error.code());
+        Struct struct = new Struct(ApiKeys.CONTROLLED_SHUTDOWN.responseSchema(version));
+        struct.set(ERROR_CODE, error.code());
 
         List<Struct> partitionsRemainingList = new ArrayList<>(partitionsRemaining.size());
         for (TopicPartition topicPartition : partitionsRemaining) {
             Struct topicPartitionStruct = struct.instance(PARTITIONS_REMAINING_KEY_NAME);
-            topicPartitionStruct.set(TOPIC_KEY_NAME, topicPartition.topic());
-            topicPartitionStruct.set(PARTITION_KEY_NAME, topicPartition.partition());
+            topicPartitionStruct.set(TOPIC_NAME, topicPartition.topic());
+            topicPartitionStruct.set(PARTITION_ID, topicPartition.partition());
             partitionsRemainingList.add(topicPartitionStruct);
         }
         struct.set(PARTITIONS_REMAINING_KEY_NAME, partitionsRemainingList.toArray());

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsRequest.java
index 3598d4f..d281b3b 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsRequest.java
@@ -20,6 +20,9 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.acl.AccessControlEntry;
 import org.apache.kafka.common.acl.AclBinding;
 import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.resource.Resource;
 import org.apache.kafka.common.utils.Utils;
@@ -28,8 +31,28 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
+import static org.apache.kafka.common.protocol.CommonFields.HOST;
+import static org.apache.kafka.common.protocol.CommonFields.OPERATION;
+import static org.apache.kafka.common.protocol.CommonFields.PERMISSION_TYPE;
+import static org.apache.kafka.common.protocol.CommonFields.PRINCIPAL;
+import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_NAME;
+import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_TYPE;
+
 public class CreateAclsRequest extends AbstractRequest {
-    private final static String CREATIONS = "creations";
+    private final static String CREATIONS_KEY_NAME = "creations";
+
+    private static final Schema CREATE_ACLS_REQUEST_V0 = new Schema(
+            new Field(CREATIONS_KEY_NAME, new ArrayOf(new Schema(
+                    RESOURCE_TYPE,
+                    RESOURCE_NAME,
+                    PRINCIPAL,
+                    HOST,
+                    OPERATION,
+                    PERMISSION_TYPE))));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{CREATE_ACLS_REQUEST_V0};
+    }
 
     public static class AclCreation {
         private final AclBinding acl;
@@ -88,7 +111,7 @@ public class CreateAclsRequest extends AbstractRequest {
     public CreateAclsRequest(Struct struct, short version) {
         super(version);
         this.aclCreations = new ArrayList<>();
-        for (Object creationStructObj : struct.getArray(CREATIONS)) {
+        for (Object creationStructObj : struct.getArray(CREATIONS_KEY_NAME)) {
             Struct creationStruct = (Struct) creationStructObj;
             aclCreations.add(AclCreation.fromStruct(creationStruct));
         }
@@ -99,11 +122,11 @@ public class CreateAclsRequest extends AbstractRequest {
         Struct struct = new Struct(ApiKeys.CREATE_ACLS.requestSchema(version()));
         List<Struct> requests = new ArrayList<>();
         for (AclCreation creation : aclCreations) {
-            Struct creationStruct = struct.instance(CREATIONS);
+            Struct creationStruct = struct.instance(CREATIONS_KEY_NAME);
             creation.setStructFields(creationStruct);
             requests.add(creationStruct);
         }
-        struct.set(CREATIONS, requests.toArray());
+        struct.set(CREATIONS_KEY_NAME, requests.toArray());
         return struct;
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsResponse.java
index 1fc75da..836215e 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsResponse.java
@@ -17,14 +17,31 @@
 package org.apache.kafka.common.requests;
 
 import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+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.List;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_MESSAGE;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+
 public class CreateAclsResponse extends AbstractResponse {
-    private final static String CREATION_RESPONSES = "creation_responses";
+    private final static String CREATION_RESPONSES_KEY_NAME = "creation_responses";
+
+    private static final Schema CREATE_ACLS_RESPONSE_V0 = new Schema(
+            THROTTLE_TIME_MS,
+            new Field(CREATION_RESPONSES_KEY_NAME, new ArrayOf(new Schema(
+                    ERROR_CODE,
+                    ERROR_MESSAGE))));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{CREATE_ACLS_RESPONSE_V0};
+    }
 
     public static class AclCreationResponse {
         private final ApiError error;
@@ -53,9 +70,9 @@ public class CreateAclsResponse extends AbstractResponse {
     }
 
     public CreateAclsResponse(Struct struct) {
-        this.throttleTimeMs = struct.getInt(THROTTLE_TIME_KEY_NAME);
+        this.throttleTimeMs = struct.get(THROTTLE_TIME_MS);
         this.aclCreationResponses = new ArrayList<>();
-        for (Object responseStructObj : struct.getArray(CREATION_RESPONSES)) {
+        for (Object responseStructObj : struct.getArray(CREATION_RESPONSES_KEY_NAME)) {
             Struct responseStruct = (Struct) responseStructObj;
             ApiError error = new ApiError(responseStruct);
             this.aclCreationResponses.add(new AclCreationResponse(error));
@@ -65,14 +82,14 @@ public class CreateAclsResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.CREATE_ACLS.responseSchema(version));
-        struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
+        struct.set(THROTTLE_TIME_MS, throttleTimeMs);
         List<Struct> responseStructs = new ArrayList<>();
         for (AclCreationResponse response : aclCreationResponses) {
-            Struct responseStruct = struct.instance(CREATION_RESPONSES);
+            Struct responseStruct = struct.instance(CREATION_RESPONSES_KEY_NAME);
             response.error.write(responseStruct);
             responseStructs.add(responseStruct);
         }
-        struct.set(CREATION_RESPONSES, responseStructs.toArray());
+        struct.set(CREATION_RESPONSES_KEY_NAME, responseStructs.toArray());
         return struct;
     }
 


[5/8] kafka git commit: MINOR: Move request/response schemas to the corresponding object representation

Posted by ij...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java
index def4c85..3a1de51 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java
@@ -18,6 +18,9 @@ package org.apache.kafka.common.requests;
 
 import org.apache.kafka.common.errors.UnsupportedVersionException;
 import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
@@ -29,21 +32,68 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.BOOLEAN;
+import static org.apache.kafka.common.protocol.types.Type.INT16;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class CreateTopicsRequest extends AbstractRequest {
     private static final String REQUESTS_KEY_NAME = "create_topic_requests";
 
     private static final String TIMEOUT_KEY_NAME = "timeout";
     private static final String VALIDATE_ONLY_KEY_NAME = "validate_only";
-    private static final String TOPIC_KEY_NAME = "topic";
     private static final String NUM_PARTITIONS_KEY_NAME = "num_partitions";
     private static final String REPLICATION_FACTOR_KEY_NAME = "replication_factor";
     private static final String REPLICA_ASSIGNMENT_KEY_NAME = "replica_assignment";
-    private static final String REPLICA_ASSIGNMENT_PARTITION_ID_KEY_NAME = "partition_id";
     private static final String REPLICA_ASSIGNMENT_REPLICAS_KEY_NAME = "replicas";
 
-    private static final String CONFIG_KEY_KEY_NAME = "config_name";
+    private static final String CONFIG_NAME_KEY_NAME = "config_name";
     private static final String CONFIG_VALUE_KEY_NAME = "config_value";
-    private static final String CONFIGS_KEY_NAME = "config_entries";
+    private static final String CONFIG_ENTRIES_KEY_NAME = "config_entries";
+
+    private static final Schema CONFIG_ENTRY = new Schema(
+            new Field(CONFIG_NAME_KEY_NAME, STRING, "Configuration name"),
+            new Field(CONFIG_VALUE_KEY_NAME, NULLABLE_STRING, "Configuration value"));
+
+    private static final Schema PARTITION_REPLICA_ASSIGNMENT_ENTRY = new Schema(
+            PARTITION_ID,
+            new Field(REPLICA_ASSIGNMENT_REPLICAS_KEY_NAME, new ArrayOf(INT32), "The set of all nodes that should " +
+                    "host this partition. The first replica in the list is the preferred leader."));
+
+    private static final Schema SINGLE_CREATE_TOPIC_REQUEST_V0 = new Schema(
+            TOPIC_NAME,
+            new Field(NUM_PARTITIONS_KEY_NAME, INT32, "Number of partitions to be created. -1 indicates unset."),
+            new Field(REPLICATION_FACTOR_KEY_NAME, INT16, "Replication factor for the topic. -1 indicates unset."),
+            new Field(REPLICA_ASSIGNMENT_KEY_NAME, new ArrayOf(PARTITION_REPLICA_ASSIGNMENT_ENTRY),
+                    "Replica assignment among kafka brokers for this topic partitions. If this is set num_partitions " +
+                            "and replication_factor must be unset."),
+            new Field(CONFIG_ENTRIES_KEY_NAME, new ArrayOf(CONFIG_ENTRY), "Topic level configuration for topic to be set."));
+
+    private static final Schema SINGLE_CREATE_TOPIC_REQUEST_V1 = SINGLE_CREATE_TOPIC_REQUEST_V0;
+
+    private static final Schema CREATE_TOPICS_REQUEST_V0 = new Schema(
+            new Field(REQUESTS_KEY_NAME, new ArrayOf(SINGLE_CREATE_TOPIC_REQUEST_V0),
+                    "An array of single topic creation requests. Can not have multiple entries for the same topic."),
+            new Field(TIMEOUT_KEY_NAME, INT32, "The time in ms to wait for a topic to be completely created on the " +
+                    "controller node. Values <= 0 will trigger topic creation and return immediately"));
+
+    private static final Schema CREATE_TOPICS_REQUEST_V1 = new Schema(
+            new Field(REQUESTS_KEY_NAME, new ArrayOf(SINGLE_CREATE_TOPIC_REQUEST_V1), "An array of single " +
+                    "topic creation requests. Can not have multiple entries for the same topic."),
+            new Field(TIMEOUT_KEY_NAME, INT32, "The time in ms to wait for a topic to be completely created on the " +
+                    "controller node. Values <= 0 will trigger topic creation and return immediately"),
+            new Field(VALIDATE_ONLY_KEY_NAME, BOOLEAN, "If this is true, the request will be validated, but the " +
+                    "topic won't be created."));
+
+    /* v2 request is the same as v1. Throttle time has been added to the response */
+    private static final Schema CREATE_TOPICS_REQUEST_V2 = CREATE_TOPICS_REQUEST_V1;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{CREATE_TOPICS_REQUEST_V0, CREATE_TOPICS_REQUEST_V1, CREATE_TOPICS_REQUEST_V2};
+    }
 
     public static final class TopicDetails {
         public final int numPartitions;
@@ -157,7 +207,7 @@ public class CreateTopicsRequest extends AbstractRequest {
 
         for (Object requestStructObj : requestStructs) {
             Struct singleRequestStruct = (Struct) requestStructObj;
-            String topic = singleRequestStruct.getString(TOPIC_KEY_NAME);
+            String topic = singleRequestStruct.get(TOPIC_NAME);
 
             if (topics.containsKey(topic))
                 duplicateTopics.add(topic);
@@ -171,7 +221,7 @@ public class CreateTopicsRequest extends AbstractRequest {
             for (Object assignmentStructObj : assignmentsArray) {
                 Struct assignmentStruct = (Struct) assignmentStructObj;
 
-                Integer partitionId = assignmentStruct.getInt(REPLICA_ASSIGNMENT_PARTITION_ID_KEY_NAME);
+                Integer partitionId = assignmentStruct.get(PARTITION_ID);
 
                 Object[] replicasArray = assignmentStruct.getArray(REPLICA_ASSIGNMENT_REPLICAS_KEY_NAME);
                 List<Integer> replicas = new ArrayList<>(replicasArray.length);
@@ -182,12 +232,12 @@ public class CreateTopicsRequest extends AbstractRequest {
                 partitionReplicaAssignments.put(partitionId, replicas);
             }
 
-            Object[] configArray = singleRequestStruct.getArray(CONFIGS_KEY_NAME);
+            Object[] configArray = singleRequestStruct.getArray(CONFIG_ENTRIES_KEY_NAME);
             Map<String, String> configs = new HashMap<>(configArray.length);
             for (Object configStructObj : configArray) {
                 Struct configStruct = (Struct) configStructObj;
 
-                String key = configStruct.getString(CONFIG_KEY_KEY_NAME);
+                String key = configStruct.getString(CONFIG_NAME_KEY_NAME);
                 String value = configStruct.getString(CONFIG_VALUE_KEY_NAME);
 
                 configs.put(key, value);
@@ -262,7 +312,7 @@ public class CreateTopicsRequest extends AbstractRequest {
             String topic = entry.getKey();
             TopicDetails args = entry.getValue();
 
-            singleRequestStruct.set(TOPIC_KEY_NAME, topic);
+            singleRequestStruct.set(TOPIC_NAME, topic);
             singleRequestStruct.set(NUM_PARTITIONS_KEY_NAME, args.numPartitions);
             singleRequestStruct.set(REPLICATION_FACTOR_KEY_NAME, args.replicationFactor);
 
@@ -270,7 +320,7 @@ public class CreateTopicsRequest extends AbstractRequest {
             List<Struct> replicaAssignmentsStructs = new ArrayList<>(args.replicasAssignments.size());
             for (Map.Entry<Integer, List<Integer>> partitionReplicaAssignment : args.replicasAssignments.entrySet()) {
                 Struct replicaAssignmentStruct = singleRequestStruct.instance(REPLICA_ASSIGNMENT_KEY_NAME);
-                replicaAssignmentStruct.set(REPLICA_ASSIGNMENT_PARTITION_ID_KEY_NAME, partitionReplicaAssignment.getKey());
+                replicaAssignmentStruct.set(PARTITION_ID, partitionReplicaAssignment.getKey());
                 replicaAssignmentStruct.set(REPLICA_ASSIGNMENT_REPLICAS_KEY_NAME, partitionReplicaAssignment.getValue().toArray());
                 replicaAssignmentsStructs.add(replicaAssignmentStruct);
             }
@@ -279,12 +329,12 @@ public class CreateTopicsRequest extends AbstractRequest {
             // configs
             List<Struct> configsStructs = new ArrayList<>(args.configs.size());
             for (Map.Entry<String, String> configEntry : args.configs.entrySet()) {
-                Struct configStruct = singleRequestStruct.instance(CONFIGS_KEY_NAME);
-                configStruct.set(CONFIG_KEY_KEY_NAME, configEntry.getKey());
+                Struct configStruct = singleRequestStruct.instance(CONFIG_ENTRIES_KEY_NAME);
+                configStruct.set(CONFIG_NAME_KEY_NAME, configEntry.getKey());
                 configStruct.set(CONFIG_VALUE_KEY_NAME, configEntry.getValue());
                 configsStructs.add(configStruct);
             }
-            singleRequestStruct.set(CONFIGS_KEY_NAME, configsStructs.toArray());
+            singleRequestStruct.set(CONFIG_ENTRIES_KEY_NAME, configsStructs.toArray());
             createTopicRequestStructs.add(singleRequestStruct);
         }
         struct.set(REQUESTS_KEY_NAME, createTopicRequestStructs.toArray());

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java
index c34265d..b3c052b 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java
@@ -18,6 +18,9 @@ package org.apache.kafka.common.requests;
 
 
 import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
@@ -26,9 +29,37 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_MESSAGE;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+
 public class CreateTopicsResponse extends AbstractResponse {
     private static final String TOPIC_ERRORS_KEY_NAME = "topic_errors";
-    private static final String TOPIC_KEY_NAME = "topic";
+
+    private static final Schema TOPIC_ERROR_CODE = new Schema(
+            TOPIC_NAME,
+            ERROR_CODE);
+
+    // Improves on TOPIC_ERROR_CODE by adding an error_message to complement the error_code
+    private static final Schema TOPIC_ERROR = new Schema(
+            TOPIC_NAME,
+            ERROR_CODE,
+            ERROR_MESSAGE);
+
+    private static final Schema CREATE_TOPICS_RESPONSE_V0 = new Schema(
+            new Field(TOPIC_ERRORS_KEY_NAME, new ArrayOf(TOPIC_ERROR_CODE), "An array of per topic error codes."));
+
+    private static final Schema CREATE_TOPICS_RESPONSE_V1 = new Schema(
+            new Field(TOPIC_ERRORS_KEY_NAME, new ArrayOf(TOPIC_ERROR), "An array of per topic errors."));
+
+    private static final Schema CREATE_TOPICS_RESPONSE_V2 = new Schema(
+            THROTTLE_TIME_MS,
+            new Field(TOPIC_ERRORS_KEY_NAME, new ArrayOf(TOPIC_ERROR), "An array of per topic errors."));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{CREATE_TOPICS_RESPONSE_V0, CREATE_TOPICS_RESPONSE_V1, CREATE_TOPICS_RESPONSE_V2};
+    }
 
     /**
      * Possible error codes:
@@ -62,24 +93,23 @@ public class CreateTopicsResponse extends AbstractResponse {
         Map<String, ApiError> errors = new HashMap<>();
         for (Object topicErrorStructObj : topicErrorStructs) {
             Struct topicErrorStruct = (Struct) topicErrorStructObj;
-            String topic = topicErrorStruct.getString(TOPIC_KEY_NAME);
+            String topic = topicErrorStruct.get(TOPIC_NAME);
             errors.put(topic, new ApiError(topicErrorStruct));
         }
 
-        this.throttleTimeMs = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME;
+        this.throttleTimeMs = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME);
         this.errors = errors;
     }
 
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.CREATE_TOPICS.responseSchema(version));
-        if (struct.hasField(THROTTLE_TIME_KEY_NAME))
-            struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
+        struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs);
 
         List<Struct> topicErrorsStructs = new ArrayList<>(errors.size());
         for (Map.Entry<String, ApiError> topicError : errors.entrySet()) {
             Struct topicErrorsStruct = struct.instance(TOPIC_ERRORS_KEY_NAME);
-            topicErrorsStruct.set(TOPIC_KEY_NAME, topicError.getKey());
+            topicErrorsStruct.set(TOPIC_NAME, topicError.getKey());
             topicError.getValue().write(topicErrorsStruct);
             topicErrorsStructs.add(topicErrorsStruct);
         }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsRequest.java
index c05bec6..2d50ea6 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsRequest.java
@@ -19,6 +19,9 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.acl.AccessControlEntryFilter;
 import org.apache.kafka.common.acl.AclBindingFilter;
 import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.resource.ResourceFilter;
 import org.apache.kafka.common.utils.Utils;
@@ -29,10 +32,29 @@ import java.util.Collections;
 import java.util.List;
 
 import static org.apache.kafka.common.protocol.ApiKeys.DELETE_ACLS;
+import static org.apache.kafka.common.protocol.CommonFields.HOST_FILTER;
+import static org.apache.kafka.common.protocol.CommonFields.OPERATION;
+import static org.apache.kafka.common.protocol.CommonFields.PERMISSION_TYPE;
+import static org.apache.kafka.common.protocol.CommonFields.PRINCIPAL_FILTER;
+import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_NAME_FILTER;
+import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_TYPE;
 
 public class DeleteAclsRequest extends AbstractRequest {
     private final static String FILTERS = "filters";
 
+    private static final Schema DELETE_ACLS_REQUEST_V0 = new Schema(
+            new Field(FILTERS, new ArrayOf(new Schema(
+                    RESOURCE_TYPE,
+                    RESOURCE_NAME_FILTER,
+                    PRINCIPAL_FILTER,
+                    HOST_FILTER,
+                    OPERATION,
+                    PERMISSION_TYPE))));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{DELETE_ACLS_REQUEST_V0};
+    }
+
     public static class Builder extends AbstractRequest.Builder<DeleteAclsRequest> {
         private final List<AclBindingFilter> filters;
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsResponse.java
index 973aa8e..0857287 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsResponse.java
@@ -19,6 +19,9 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.acl.AccessControlEntry;
 import org.apache.kafka.common.acl.AclBinding;
 import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.resource.Resource;
 import org.apache.kafka.common.utils.Utils;
@@ -30,10 +33,42 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_MESSAGE;
+import static org.apache.kafka.common.protocol.CommonFields.HOST;
+import static org.apache.kafka.common.protocol.CommonFields.OPERATION;
+import static org.apache.kafka.common.protocol.CommonFields.PERMISSION_TYPE;
+import static org.apache.kafka.common.protocol.CommonFields.PRINCIPAL;
+import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_NAME;
+import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_TYPE;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+
 public class DeleteAclsResponse extends AbstractResponse {
     public static final Logger log = LoggerFactory.getLogger(DeleteAclsResponse.class);
-    private final static String FILTER_RESPONSES = "filter_responses";
-    private final static String MATCHING_ACLS = "matching_acls";
+    private final static String FILTER_RESPONSES_KEY_NAME = "filter_responses";
+    private final static String MATCHING_ACLS_KEY_NAME = "matching_acls";
+
+    private static final Schema MATCHING_ACL = new Schema(
+            ERROR_CODE,
+            ERROR_MESSAGE,
+            RESOURCE_TYPE,
+            RESOURCE_NAME,
+            PRINCIPAL,
+            HOST,
+            OPERATION,
+            PERMISSION_TYPE);
+
+    private static final Schema DELETE_ACLS_RESPONSE_V0 = new Schema(
+            THROTTLE_TIME_MS,
+            new Field(FILTER_RESPONSES_KEY_NAME,
+                    new ArrayOf(new Schema(
+                            ERROR_CODE,
+                            ERROR_MESSAGE,
+                            new Field(MATCHING_ACLS_KEY_NAME, new ArrayOf(MATCHING_ACL), "The matching ACLs")))));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{DELETE_ACLS_RESPONSE_V0};
+    }
 
     public static class AclDeletionResult {
         private final ApiError error;
@@ -99,13 +134,13 @@ public class DeleteAclsResponse extends AbstractResponse {
     }
 
     public DeleteAclsResponse(Struct struct) {
-        this.throttleTimeMs = struct.getInt(THROTTLE_TIME_KEY_NAME);
+        this.throttleTimeMs = struct.get(THROTTLE_TIME_MS);
         this.responses = new ArrayList<>();
-        for (Object responseStructObj : struct.getArray(FILTER_RESPONSES)) {
+        for (Object responseStructObj : struct.getArray(FILTER_RESPONSES_KEY_NAME)) {
             Struct responseStruct = (Struct) responseStructObj;
             ApiError error = new ApiError(responseStruct);
             List<AclDeletionResult> deletions = new ArrayList<>();
-            for (Object matchingAclStructObj : responseStruct.getArray(MATCHING_ACLS)) {
+            for (Object matchingAclStructObj : responseStruct.getArray(MATCHING_ACLS_KEY_NAME)) {
                 Struct matchingAclStruct = (Struct) matchingAclStructObj;
                 ApiError matchError = new ApiError(matchingAclStruct);
                 AccessControlEntry entry = RequestUtils.aceFromStructFields(matchingAclStruct);
@@ -119,23 +154,23 @@ public class DeleteAclsResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.DELETE_ACLS.responseSchema(version));
-        struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
+        struct.set(THROTTLE_TIME_MS, throttleTimeMs);
         List<Struct> responseStructs = new ArrayList<>();
         for (AclFilterResponse response : responses) {
-            Struct responseStruct = struct.instance(FILTER_RESPONSES);
+            Struct responseStruct = struct.instance(FILTER_RESPONSES_KEY_NAME);
             response.error.write(responseStruct);
             List<Struct> deletionStructs = new ArrayList<>();
             for (AclDeletionResult deletion : response.deletions()) {
-                Struct deletionStruct = responseStruct.instance(MATCHING_ACLS);
+                Struct deletionStruct = responseStruct.instance(MATCHING_ACLS_KEY_NAME);
                 deletion.error.write(deletionStruct);
                 RequestUtils.resourceSetStructFields(deletion.acl().resource(), deletionStruct);
                 RequestUtils.aceSetStructFields(deletion.acl().entry(), deletionStruct);
                 deletionStructs.add(deletionStruct);
             }
-            responseStruct.set(MATCHING_ACLS, deletionStructs.toArray(new Struct[0]));
+            responseStruct.set(MATCHING_ACLS_KEY_NAME, deletionStructs.toArray(new Struct[0]));
             responseStructs.add(responseStruct);
         }
-        struct.set(FILTER_RESPONSES, responseStructs.toArray());
+        struct.set(FILTER_RESPONSES_KEY_NAME, responseStructs.toArray());
         return struct;
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsRequest.java
index fcd9836..350238e 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsRequest.java
@@ -20,14 +20,23 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.CollectionUtils;
+
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.INT64;
+
 public class DeleteRecordsRequest extends AbstractRequest {
 
     public static final long HIGH_WATERMARK = -1L;
@@ -37,13 +46,28 @@ public class DeleteRecordsRequest extends AbstractRequest {
     private static final String TIMEOUT_KEY_NAME = "timeout";
 
     // topic level key names
-    private static final String TOPIC_KEY_NAME = "topic";
     private static final String PARTITIONS_KEY_NAME = "partitions";
 
     // partition level key names
-    private static final String PARTITION_KEY_NAME = "partition";
     private static final String OFFSET_KEY_NAME = "offset";
 
+
+    private static final Schema DELETE_RECORDS_REQUEST_PARTITION_V0 = new Schema(
+            PARTITION_ID,
+            new Field(OFFSET_KEY_NAME, INT64, "The offset before which the messages will be deleted."));
+
+    private static final Schema DELETE_RECORDS_REQUEST_TOPIC_V0 = new Schema(
+            TOPIC_NAME,
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(DELETE_RECORDS_REQUEST_PARTITION_V0)));
+
+    private static final Schema DELETE_RECORDS_REQUEST_V0 = new Schema(
+            new Field(TOPICS_KEY_NAME, new ArrayOf(DELETE_RECORDS_REQUEST_TOPIC_V0)),
+            new Field(TIMEOUT_KEY_NAME, INT32, "The maximum time to await a response in ms."));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{DELETE_RECORDS_REQUEST_V0};
+    }
+
     private final int timeout;
     private final Map<TopicPartition, Long> partitionOffsets;
 
@@ -79,10 +103,10 @@ public class DeleteRecordsRequest extends AbstractRequest {
         partitionOffsets = new HashMap<>();
         for (Object topicStructObj : struct.getArray(TOPICS_KEY_NAME)) {
             Struct topicStruct = (Struct) topicStructObj;
-            String topic = topicStruct.getString(TOPIC_KEY_NAME);
+            String topic = topicStruct.get(TOPIC_NAME);
             for (Object partitionStructObj : topicStruct.getArray(PARTITIONS_KEY_NAME)) {
                 Struct partitionStruct = (Struct) partitionStructObj;
-                int partition = partitionStruct.getInt(PARTITION_KEY_NAME);
+                int partition = partitionStruct.get(PARTITION_ID);
                 long offset = partitionStruct.getLong(OFFSET_KEY_NAME);
                 partitionOffsets.put(new TopicPartition(topic, partition), offset);
             }
@@ -103,11 +127,11 @@ public class DeleteRecordsRequest extends AbstractRequest {
         List<Struct> topicStructArray = new ArrayList<>();
         for (Map.Entry<String, Map<Integer, Long>> offsetsByTopicEntry : offsetsByTopic.entrySet()) {
             Struct topicStruct = struct.instance(TOPICS_KEY_NAME);
-            topicStruct.set(TOPIC_KEY_NAME, offsetsByTopicEntry.getKey());
+            topicStruct.set(TOPIC_NAME, offsetsByTopicEntry.getKey());
             List<Struct> partitionStructArray = new ArrayList<>();
             for (Map.Entry<Integer, Long> offsetsByPartitionEntry : offsetsByTopicEntry.getValue().entrySet()) {
                 Struct partitionStruct = topicStruct.instance(PARTITIONS_KEY_NAME);
-                partitionStruct.set(PARTITION_KEY_NAME, offsetsByPartitionEntry.getKey());
+                partitionStruct.set(PARTITION_ID, offsetsByPartitionEntry.getKey());
                 partitionStruct.set(OFFSET_KEY_NAME, offsetsByPartitionEntry.getValue());
                 partitionStructArray.add(partitionStruct);
             }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsResponse.java
index f19f933..aeea1cd 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsResponse.java
@@ -20,14 +20,24 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.CollectionUtils;
+
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.INT64;
+
 public class DeleteRecordsResponse extends AbstractResponse {
 
     public static final long INVALID_LOW_WATERMARK = -1L;
@@ -36,13 +46,27 @@ public class DeleteRecordsResponse extends AbstractResponse {
     private static final String TOPICS_KEY_NAME = "topics";
 
     // topic level key names
-    private static final String TOPIC_KEY_NAME = "topic";
     private static final String PARTITIONS_KEY_NAME = "partitions";
 
     // partition level key names
-    private static final String PARTITION_KEY_NAME = "partition";
     private static final String LOW_WATERMARK_KEY_NAME = "low_watermark";
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
+
+    private static final Schema DELETE_RECORDS_RESPONSE_PARTITION_V0 = new Schema(
+            PARTITION_ID,
+            new Field(LOW_WATERMARK_KEY_NAME, INT64, "Smallest available offset of all live replicas"),
+            ERROR_CODE);
+
+    private static final Schema DELETE_RECORDS_RESPONSE_TOPIC_V0 = new Schema(
+            TOPIC_NAME,
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(DELETE_RECORDS_RESPONSE_PARTITION_V0)));
+
+    private static final Schema DELETE_RECORDS_RESPONSE_V0 = new Schema(
+            THROTTLE_TIME_MS,
+            new Field("topics", new ArrayOf(DELETE_RECORDS_RESPONSE_TOPIC_V0)));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{DELETE_RECORDS_RESPONSE_V0};
+    }
 
     private final int throttleTimeMs;
     private final Map<TopicPartition, PartitionResponse> responses;
@@ -81,16 +105,16 @@ public class DeleteRecordsResponse extends AbstractResponse {
     }
 
     public DeleteRecordsResponse(Struct struct) {
-        this.throttleTimeMs = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME;
+        this.throttleTimeMs = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME);
         responses = new HashMap<>();
         for (Object topicStructObj : struct.getArray(TOPICS_KEY_NAME)) {
             Struct topicStruct = (Struct) topicStructObj;
-            String topic = topicStruct.getString(TOPIC_KEY_NAME);
+            String topic = topicStruct.get(TOPIC_NAME);
             for (Object partitionStructObj : topicStruct.getArray(PARTITIONS_KEY_NAME)) {
                 Struct partitionStruct = (Struct) partitionStructObj;
-                int partition = partitionStruct.getInt(PARTITION_KEY_NAME);
+                int partition = partitionStruct.get(PARTITION_ID);
                 long lowWatermark = partitionStruct.getLong(LOW_WATERMARK_KEY_NAME);
-                Errors error = Errors.forCode(partitionStruct.getShort(ERROR_CODE_KEY_NAME));
+                Errors error = Errors.forCode(partitionStruct.get(ERROR_CODE));
                 responses.put(new TopicPartition(topic, partition), new PartitionResponse(lowWatermark, error));
             }
         }
@@ -107,20 +131,19 @@ public class DeleteRecordsResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.DELETE_RECORDS.responseSchema(version));
-        if (struct.hasField(THROTTLE_TIME_KEY_NAME))
-            struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
+        struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs);
         Map<String, Map<Integer, PartitionResponse>> responsesByTopic = CollectionUtils.groupDataByTopic(responses);
         List<Struct> topicStructArray = new ArrayList<>();
         for (Map.Entry<String, Map<Integer, PartitionResponse>> responsesByTopicEntry : responsesByTopic.entrySet()) {
             Struct topicStruct = struct.instance(TOPICS_KEY_NAME);
-            topicStruct.set(TOPIC_KEY_NAME, responsesByTopicEntry.getKey());
+            topicStruct.set(TOPIC_NAME, responsesByTopicEntry.getKey());
             List<Struct> partitionStructArray = new ArrayList<>();
             for (Map.Entry<Integer, PartitionResponse> responsesByPartitionEntry : responsesByTopicEntry.getValue().entrySet()) {
                 Struct partitionStruct = topicStruct.instance(PARTITIONS_KEY_NAME);
                 PartitionResponse response = responsesByPartitionEntry.getValue();
-                partitionStruct.set(PARTITION_KEY_NAME, responsesByPartitionEntry.getKey());
+                partitionStruct.set(PARTITION_ID, responsesByPartitionEntry.getKey());
                 partitionStruct.set(LOW_WATERMARK_KEY_NAME, response.lowWatermark);
-                partitionStruct.set(ERROR_CODE_KEY_NAME, response.error.code());
+                partitionStruct.set(ERROR_CODE, response.error.code());
                 partitionStructArray.add(partitionStruct);
             }
             topicStruct.set(PARTITIONS_KEY_NAME, partitionStructArray.toArray());

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java
index 2ea8c21..4696b50 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java
@@ -18,6 +18,9 @@ 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.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.Utils;
 
@@ -27,10 +30,26 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class DeleteTopicsRequest extends AbstractRequest {
     private static final String TOPICS_KEY_NAME = "topics";
     private static final String TIMEOUT_KEY_NAME = "timeout";
 
+    /* DeleteTopic api */
+    private static final Schema DELETE_TOPICS_REQUEST_V0 = new Schema(
+            new Field(TOPICS_KEY_NAME, new ArrayOf(STRING), "An array of topics to be deleted."),
+            new Field(TIMEOUT_KEY_NAME, INT32, "The time in ms to wait for a topic to be completely deleted on the " +
+                    "controller node. Values <= 0 will trigger topic deletion and return immediately"));
+
+    /* v1 request is the same as v0. Throttle time has been added to the response */
+    private static final Schema DELETE_TOPICS_REQUEST_V1 = DELETE_TOPICS_REQUEST_V0;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{DELETE_TOPICS_REQUEST_V0, DELETE_TOPICS_REQUEST_V1};
+    }
+
     private final Set<String> topics;
     private final Integer timeout;
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java
index 3f11167..9c84c11 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java
@@ -18,6 +18,9 @@ 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.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
@@ -26,10 +29,29 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+
 public class DeleteTopicsResponse extends AbstractResponse {
     private static final String TOPIC_ERROR_CODES_KEY_NAME = "topic_error_codes";
-    private static final String TOPIC_KEY_NAME = "topic";
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
+
+    private static final Schema TOPIC_ERROR_CODE = new Schema(
+            TOPIC_NAME,
+            ERROR_CODE);
+
+    private static final Schema DELETE_TOPICS_RESPONSE_V0 = new Schema(
+            new Field(TOPIC_ERROR_CODES_KEY_NAME,
+                    new ArrayOf(TOPIC_ERROR_CODE), "An array of per topic error codes."));
+
+    private static final Schema DELETE_TOPICS_RESPONSE_V1 = new Schema(
+            THROTTLE_TIME_MS,
+            new Field(TOPIC_ERROR_CODES_KEY_NAME, new ArrayOf(TOPIC_ERROR_CODE), "An array of per topic error codes."));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{DELETE_TOPICS_RESPONSE_V0, DELETE_TOPICS_RESPONSE_V1};
+    }
+
 
     /**
      * Possible error codes:
@@ -52,13 +74,13 @@ public class DeleteTopicsResponse extends AbstractResponse {
     }
 
     public DeleteTopicsResponse(Struct struct) {
-        this.throttleTimeMs = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME;
+        this.throttleTimeMs = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME);
         Object[] topicErrorCodesStructs = struct.getArray(TOPIC_ERROR_CODES_KEY_NAME);
         Map<String, Errors> errors = new HashMap<>();
         for (Object topicErrorCodeStructObj : topicErrorCodesStructs) {
             Struct topicErrorCodeStruct = (Struct) topicErrorCodeStructObj;
-            String topic = topicErrorCodeStruct.getString(TOPIC_KEY_NAME);
-            Errors error = Errors.forCode(topicErrorCodeStruct.getShort(ERROR_CODE_KEY_NAME));
+            String topic = topicErrorCodeStruct.get(TOPIC_NAME);
+            Errors error = Errors.forCode(topicErrorCodeStruct.get(ERROR_CODE));
             errors.put(topic, error);
         }
 
@@ -68,13 +90,12 @@ public class DeleteTopicsResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.DELETE_TOPICS.responseSchema(version));
-        if (struct.hasField(THROTTLE_TIME_KEY_NAME))
-            struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
+        struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs);
         List<Struct> topicErrorCodeStructs = new ArrayList<>(errors.size());
         for (Map.Entry<String, Errors> topicError : errors.entrySet()) {
             Struct topicErrorCodeStruct = struct.instance(TOPIC_ERROR_CODES_KEY_NAME);
-            topicErrorCodeStruct.set(TOPIC_KEY_NAME, topicError.getKey());
-            topicErrorCodeStruct.set(ERROR_CODE_KEY_NAME, topicError.getValue().code());
+            topicErrorCodeStruct.set(TOPIC_NAME, topicError.getKey());
+            topicErrorCodeStruct.set(ERROR_CODE, topicError.getValue().code());
             topicErrorCodeStructs.add(topicErrorCodeStruct);
         }
         struct.set(TOPIC_ERROR_CODES_KEY_NAME, topicErrorCodeStructs.toArray());

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsRequest.java
index 58ce539..1bacac7 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsRequest.java
@@ -20,13 +20,33 @@ import org.apache.kafka.common.acl.AccessControlEntryFilter;
 import org.apache.kafka.common.acl.AclBinding;
 import org.apache.kafka.common.acl.AclBindingFilter;
 import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.resource.ResourceFilter;
 
 import java.nio.ByteBuffer;
 import java.util.Collections;
 
+import static org.apache.kafka.common.protocol.CommonFields.HOST_FILTER;
+import static org.apache.kafka.common.protocol.CommonFields.OPERATION;
+import static org.apache.kafka.common.protocol.CommonFields.PERMISSION_TYPE;
+import static org.apache.kafka.common.protocol.CommonFields.PRINCIPAL_FILTER;
+import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_NAME_FILTER;
+import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_TYPE;
+
 public class DescribeAclsRequest extends AbstractRequest {
+    private static final Schema DESCRIBE_ACLS_REQUEST_V0 = new Schema(
+            RESOURCE_TYPE,
+            RESOURCE_NAME_FILTER,
+            PRINCIPAL_FILTER,
+            HOST_FILTER,
+            OPERATION,
+            PERMISSION_TYPE);
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{DESCRIBE_ACLS_REQUEST_V0};
+    }
+
     public static class Builder extends AbstractRequest.Builder<DescribeAclsRequest> {
         private final AclBindingFilter filter;
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsResponse.java
index 993a45f..f8b9695 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsResponse.java
@@ -20,6 +20,9 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.acl.AccessControlEntry;
 import org.apache.kafka.common.acl.AclBinding;
 import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.resource.Resource;
 
@@ -30,9 +33,38 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_MESSAGE;
+import static org.apache.kafka.common.protocol.CommonFields.HOST;
+import static org.apache.kafka.common.protocol.CommonFields.OPERATION;
+import static org.apache.kafka.common.protocol.CommonFields.PERMISSION_TYPE;
+import static org.apache.kafka.common.protocol.CommonFields.PRINCIPAL;
+import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_NAME;
+import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_TYPE;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+
 public class DescribeAclsResponse extends AbstractResponse {
-    private final static String RESOURCES = "resources";
-    private final static String ACLS = "acls";
+    private final static String RESOURCES_KEY_NAME = "resources";
+    private final static String ACLS_KEY_NAME = "acls";
+
+    private static final Schema DESCRIBE_ACLS_RESOURCE = new Schema(
+            RESOURCE_TYPE,
+            RESOURCE_NAME,
+            new Field(ACLS_KEY_NAME, new ArrayOf(new Schema(
+                    PRINCIPAL,
+                    HOST,
+                    OPERATION,
+                    PERMISSION_TYPE))));
+
+    private static final Schema DESCRIBE_ACLS_RESPONSE_V0 = new Schema(
+            THROTTLE_TIME_MS,
+            ERROR_CODE,
+            ERROR_MESSAGE,
+            new Field(RESOURCES_KEY_NAME, new ArrayOf(DESCRIBE_ACLS_RESOURCE), "The resources and their associated ACLs."));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{DESCRIBE_ACLS_RESPONSE_V0};
+    }
 
     private final int throttleTimeMs;
     private final ApiError error;
@@ -45,13 +77,13 @@ public class DescribeAclsResponse extends AbstractResponse {
     }
 
     public DescribeAclsResponse(Struct struct) {
-        this.throttleTimeMs = struct.getInt(THROTTLE_TIME_KEY_NAME);
+        this.throttleTimeMs = struct.get(THROTTLE_TIME_MS);
         this.error = new ApiError(struct);
         this.acls = new ArrayList<>();
-        for (Object resourceStructObj : struct.getArray(RESOURCES)) {
+        for (Object resourceStructObj : struct.getArray(RESOURCES_KEY_NAME)) {
             Struct resourceStruct = (Struct) resourceStructObj;
             Resource resource = RequestUtils.resourceFromStructFields(resourceStruct);
-            for (Object aclDataStructObj : resourceStruct.getArray(ACLS)) {
+            for (Object aclDataStructObj : resourceStruct.getArray(ACLS_KEY_NAME)) {
                 Struct aclDataStruct = (Struct) aclDataStructObj;
                 AccessControlEntry entry = RequestUtils.aceFromStructFields(aclDataStruct);
                 this.acls.add(new AclBinding(resource, entry));
@@ -62,7 +94,7 @@ public class DescribeAclsResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.DESCRIBE_ACLS.responseSchema(version));
-        struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
+        struct.set(THROTTLE_TIME_MS, throttleTimeMs);
         error.write(struct);
 
         Map<Resource, List<AccessControlEntry>> resourceToData = new HashMap<>();
@@ -78,18 +110,18 @@ public class DescribeAclsResponse extends AbstractResponse {
         List<Struct> resourceStructs = new ArrayList<>();
         for (Map.Entry<Resource, List<AccessControlEntry>> tuple : resourceToData.entrySet()) {
             Resource resource = tuple.getKey();
-            Struct resourceStruct = struct.instance(RESOURCES);
+            Struct resourceStruct = struct.instance(RESOURCES_KEY_NAME);
             RequestUtils.resourceSetStructFields(resource, resourceStruct);
             List<Struct> dataStructs = new ArrayList<>();
             for (AccessControlEntry entry : tuple.getValue()) {
-                Struct dataStruct = resourceStruct.instance(ACLS);
+                Struct dataStruct = resourceStruct.instance(ACLS_KEY_NAME);
                 RequestUtils.aceSetStructFields(entry, dataStruct);
                 dataStructs.add(dataStruct);
             }
-            resourceStruct.set(ACLS, dataStructs.toArray());
+            resourceStruct.set(ACLS_KEY_NAME, dataStructs.toArray());
             resourceStructs.add(resourceStruct);
         }
-        struct.set(RESOURCES, resourceStructs.toArray());
+        struct.set(RESOURCES_KEY_NAME, resourceStructs.toArray());
         return struct;
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsRequest.java
index 64fae0e..74e25f4 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsRequest.java
@@ -17,6 +17,9 @@
 package org.apache.kafka.common.requests;
 
 import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
@@ -27,6 +30,9 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.types.Type.INT8;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class DescribeConfigsRequest extends AbstractRequest {
 
     private static final String RESOURCES_KEY_NAME = "resources";
@@ -34,6 +40,18 @@ public class DescribeConfigsRequest extends AbstractRequest {
     private static final String RESOURCE_NAME_KEY_NAME = "resource_name";
     private static final String CONFIG_NAMES_KEY_NAME = "config_names";
 
+    private static final Schema DESCRIBE_CONFIGS_REQUEST_RESOURCE_V0 = new Schema(
+            new Field(RESOURCE_TYPE_KEY_NAME, INT8),
+            new Field(RESOURCE_NAME_KEY_NAME, STRING),
+            new Field(CONFIG_NAMES_KEY_NAME, ArrayOf.nullable(STRING)));
+
+    private static final Schema DESCRIBE_CONFIGS_REQUEST_V0 = new Schema(
+            new Field(RESOURCES_KEY_NAME, new ArrayOf(DESCRIBE_CONFIGS_REQUEST_RESOURCE_V0), "An array of config resources to be returned."));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{DESCRIBE_CONFIGS_REQUEST_V0};
+    }
+
     public static class Builder extends AbstractRequest.Builder {
         private final Map<Resource, Collection<String>> resourceToConfigNames;
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsResponse.java
index 8694e1f..9b2289d 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsResponse.java
@@ -18,6 +18,9 @@
 package org.apache.kafka.common.requests;
 
 import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
@@ -27,6 +30,14 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_MESSAGE;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.types.Type.BOOLEAN;
+import static org.apache.kafka.common.protocol.types.Type.INT8;
+import static org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class DescribeConfigsResponse extends AbstractResponse {
 
     private static final String RESOURCES_KEY_NAME = "resources";
@@ -36,11 +47,31 @@ public class DescribeConfigsResponse extends AbstractResponse {
 
     private static final String CONFIG_ENTRIES_KEY_NAME = "config_entries";
 
-    private static final String CONFIG_NAME = "config_name";
-    private static final String CONFIG_VALUE = "config_value";
-    private static final String IS_SENSITIVE = "is_sensitive";
-    private static final String IS_DEFAULT = "is_default";
-    private static final String READ_ONLY = "read_only";
+    private static final String CONFIG_NAME_KEY_NAME = "config_name";
+    private static final String CONFIG_VALUE_KEY_NAME = "config_value";
+    private static final String IS_SENSITIVE_KEY_NAME = "is_sensitive";
+    private static final String IS_DEFAULT_KEY_NAME = "is_default";
+    private static final String READ_ONLY_KEY_NAME = "read_only";
+
+    private static final Schema DESCRIBE_CONFIGS_RESPONSE_ENTITY_V0 = new Schema(
+            ERROR_CODE,
+            ERROR_MESSAGE,
+            new Field(RESOURCE_TYPE_KEY_NAME, INT8),
+            new Field(RESOURCE_NAME_KEY_NAME, STRING),
+            new Field(CONFIG_ENTRIES_KEY_NAME, new ArrayOf(new Schema(
+                    new Field(CONFIG_NAME_KEY_NAME, STRING),
+                    new Field(CONFIG_VALUE_KEY_NAME, NULLABLE_STRING),
+                    new Field(READ_ONLY_KEY_NAME, BOOLEAN),
+                    new Field(IS_DEFAULT_KEY_NAME, BOOLEAN),
+                    new Field(IS_SENSITIVE_KEY_NAME, BOOLEAN)))));
+
+    private static final Schema DESCRIBE_CONFIGS_RESPONSE_V0 = new Schema(
+            THROTTLE_TIME_MS,
+            new Field(RESOURCES_KEY_NAME, new ArrayOf(DESCRIBE_CONFIGS_RESPONSE_ENTITY_V0)));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{DESCRIBE_CONFIGS_RESPONSE_V0};
+    }
 
     public static class Config {
         private final ApiError error;
@@ -105,7 +136,7 @@ public class DescribeConfigsResponse extends AbstractResponse {
     }
 
     public DescribeConfigsResponse(Struct struct) {
-        throttleTimeMs = struct.getInt(THROTTLE_TIME_KEY_NAME);
+        throttleTimeMs = struct.get(THROTTLE_TIME_MS);
         Object[] resourcesArray = struct.getArray(RESOURCES_KEY_NAME);
         configs = new HashMap<>(resourcesArray.length);
         for (Object resourceObj : resourcesArray) {
@@ -120,11 +151,11 @@ public class DescribeConfigsResponse extends AbstractResponse {
             List<ConfigEntry> configEntries = new ArrayList<>(configEntriesArray.length);
             for (Object configEntriesObj: configEntriesArray) {
                 Struct configEntriesStruct = (Struct) configEntriesObj;
-                String configName = configEntriesStruct.getString(CONFIG_NAME);
-                String configValue = configEntriesStruct.getString(CONFIG_VALUE);
-                boolean isSensitive = configEntriesStruct.getBoolean(IS_SENSITIVE);
-                boolean isDefault = configEntriesStruct.getBoolean(IS_DEFAULT);
-                boolean readOnly = configEntriesStruct.getBoolean(READ_ONLY);
+                String configName = configEntriesStruct.getString(CONFIG_NAME_KEY_NAME);
+                String configValue = configEntriesStruct.getString(CONFIG_VALUE_KEY_NAME);
+                boolean isSensitive = configEntriesStruct.getBoolean(IS_SENSITIVE_KEY_NAME);
+                boolean isDefault = configEntriesStruct.getBoolean(IS_DEFAULT_KEY_NAME);
+                boolean readOnly = configEntriesStruct.getBoolean(READ_ONLY_KEY_NAME);
                 configEntries.add(new ConfigEntry(configName, configValue, isSensitive, isDefault, readOnly));
             }
             Config config = new Config(error, configEntries);
@@ -147,7 +178,7 @@ public class DescribeConfigsResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.DESCRIBE_CONFIGS.responseSchema(version));
-        struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
+        struct.set(THROTTLE_TIME_MS, throttleTimeMs);
         List<Struct> resourceStructs = new ArrayList<>(configs.size());
         for (Map.Entry<Resource, Config> entry : configs.entrySet()) {
             Struct resourceStruct = struct.instance(RESOURCES_KEY_NAME);
@@ -162,11 +193,11 @@ public class DescribeConfigsResponse extends AbstractResponse {
             List<Struct> configEntryStructs = new ArrayList<>(config.entries.size());
             for (ConfigEntry configEntry : config.entries) {
                 Struct configEntriesStruct = resourceStruct.instance(CONFIG_ENTRIES_KEY_NAME);
-                configEntriesStruct.set(CONFIG_NAME, configEntry.name);
-                configEntriesStruct.set(CONFIG_VALUE, configEntry.value);
-                configEntriesStruct.set(IS_SENSITIVE, configEntry.isSensitive);
-                configEntriesStruct.set(IS_DEFAULT, configEntry.isDefault);
-                configEntriesStruct.set(READ_ONLY, configEntry.readOnly);
+                configEntriesStruct.set(CONFIG_NAME_KEY_NAME, configEntry.name);
+                configEntriesStruct.set(CONFIG_VALUE_KEY_NAME, configEntry.value);
+                configEntriesStruct.set(IS_SENSITIVE_KEY_NAME, configEntry.isSensitive);
+                configEntriesStruct.set(IS_DEFAULT_KEY_NAME, configEntry.isDefault);
+                configEntriesStruct.set(READ_ONLY_KEY_NAME, configEntry.readOnly);
                 configEntryStructs.add(configEntriesStruct);
             }
             resourceStruct.set(CONFIG_ENTRIES_KEY_NAME, configEntryStructs.toArray(new Struct[0]));

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java
index b43e254..56117da 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java
@@ -18,6 +18,9 @@ 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.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.Utils;
 
@@ -25,9 +28,23 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class DescribeGroupsRequest extends AbstractRequest {
     private static final String GROUP_IDS_KEY_NAME = "group_ids";
 
+    /* Describe group api */
+    private static final Schema DESCRIBE_GROUPS_REQUEST_V0 = new Schema(
+            new Field(GROUP_IDS_KEY_NAME, new ArrayOf(STRING), "List of groupIds to request metadata for (an " +
+                    "empty groupId array will return empty group metadata)."));
+
+    /* v1 request is the same as v0. Throttle time has been added to response */
+    private static final Schema DESCRIBE_GROUPS_REQUEST_V1 = DESCRIBE_GROUPS_REQUEST_V0;
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{DESCRIBE_GROUPS_REQUEST_V0, DESCRIBE_GROUPS_REQUEST_V1};
+    }
+
     public static class Builder extends AbstractRequest.Builder<DescribeGroupsRequest> {
         private final List<String> groupIds;
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java
index 0e1d6bd..9241165 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java
@@ -18,6 +18,9 @@ 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.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
@@ -27,11 +30,15 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.types.Type.BYTES;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class DescribeGroupsResponse extends AbstractResponse {
 
     private static final String GROUPS_KEY_NAME = "groups";
 
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
     private static final String GROUP_ID_KEY_NAME = "group_id";
     private static final String GROUP_STATE_KEY_NAME = "state";
     private static final String PROTOCOL_TYPE_KEY_NAME = "protocol_type";
@@ -44,6 +51,36 @@ public class DescribeGroupsResponse extends AbstractResponse {
     private static final String MEMBER_METADATA_KEY_NAME = "member_metadata";
     private static final String MEMBER_ASSIGNMENT_KEY_NAME = "member_assignment";
 
+    private static final Schema DESCRIBE_GROUPS_RESPONSE_MEMBER_V0 = new Schema(
+            new Field(MEMBER_ID_KEY_NAME, STRING, "The memberId assigned by the coordinator"),
+            new Field(CLIENT_ID_KEY_NAME, STRING, "The client id used in the member's latest join group request"),
+            new Field(CLIENT_HOST_KEY_NAME, STRING, "The client host used in the request session corresponding to the " +
+                    "member's join group."),
+            new Field(MEMBER_METADATA_KEY_NAME, BYTES, "The metadata corresponding to the current group protocol in " +
+                    "use (will only be present if the group is stable)."),
+            new Field(MEMBER_ASSIGNMENT_KEY_NAME, BYTES, "The current assignment provided by the group leader " +
+                    "(will only be present if the group is stable)."));
+
+    private static final Schema DESCRIBE_GROUPS_RESPONSE_GROUP_METADATA_V0 = new Schema(
+            ERROR_CODE,
+            new Field(GROUP_ID_KEY_NAME, STRING),
+            new Field(GROUP_STATE_KEY_NAME, STRING, "The current state of the group (one of: Dead, Stable, AwaitingSync, " +
+                    "PreparingRebalance, or empty if there is no active group)"),
+            new Field(PROTOCOL_TYPE_KEY_NAME, STRING, "The current group protocol type (will be empty if there is no active group)"),
+            new Field(PROTOCOL_KEY_NAME, STRING, "The current group protocol (only provided if the group is Stable)"),
+            new Field(MEMBERS_KEY_NAME, new ArrayOf(DESCRIBE_GROUPS_RESPONSE_MEMBER_V0), "Current group members " +
+                    "(only provided if the group is not Dead)"));
+
+    private static final Schema DESCRIBE_GROUPS_RESPONSE_V0 = new Schema(
+            new Field(GROUPS_KEY_NAME, new ArrayOf(DESCRIBE_GROUPS_RESPONSE_GROUP_METADATA_V0)));
+    private static final Schema DESCRIBE_GROUPS_RESPONSE_V1 = new Schema(
+            THROTTLE_TIME_MS,
+            new Field(GROUPS_KEY_NAME, new ArrayOf(DESCRIBE_GROUPS_RESPONSE_GROUP_METADATA_V0)));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[] {DESCRIBE_GROUPS_RESPONSE_V0, DESCRIBE_GROUPS_RESPONSE_V1};
+    }
+
     public static final String UNKNOWN_STATE = "";
     public static final String UNKNOWN_PROTOCOL_TYPE = "";
     public static final String UNKNOWN_PROTOCOL = "";
@@ -70,13 +107,13 @@ public class DescribeGroupsResponse extends AbstractResponse {
     }
 
     public DescribeGroupsResponse(Struct struct) {
-        this.throttleTimeMs = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME;
+        this.throttleTimeMs = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME);
         this.groups = new HashMap<>();
         for (Object groupObj : struct.getArray(GROUPS_KEY_NAME)) {
             Struct groupStruct = (Struct) groupObj;
 
             String groupId = groupStruct.getString(GROUP_ID_KEY_NAME);
-            Errors error = Errors.forCode(groupStruct.getShort(ERROR_CODE_KEY_NAME));
+            Errors error = Errors.forCode(groupStruct.get(ERROR_CODE));
             String state = groupStruct.getString(GROUP_STATE_KEY_NAME);
             String protocolType = groupStruct.getString(PROTOCOL_TYPE_KEY_NAME);
             String protocol = groupStruct.getString(PROTOCOL_KEY_NAME);
@@ -209,15 +246,14 @@ public class DescribeGroupsResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.DESCRIBE_GROUPS.responseSchema(version));
-        if (struct.hasField(THROTTLE_TIME_KEY_NAME))
-            struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
+        struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs);
 
         List<Struct> groupStructs = new ArrayList<>();
         for (Map.Entry<String, GroupMetadata> groupEntry : groups.entrySet()) {
             Struct groupStruct = struct.instance(GROUPS_KEY_NAME);
             GroupMetadata group = groupEntry.getValue();
             groupStruct.set(GROUP_ID_KEY_NAME, groupEntry.getKey());
-            groupStruct.set(ERROR_CODE_KEY_NAME, group.error.code());
+            groupStruct.set(ERROR_CODE, group.error.code());
             groupStruct.set(GROUP_STATE_KEY_NAME, group.state);
             groupStruct.set(PROTOCOL_TYPE_KEY_NAME, group.protocolType);
             groupStruct.set(PROTOCOL_KEY_NAME, group.protocol);

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsRequest.java
index 338d684..0169da5 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsRequest.java
@@ -19,8 +19,12 @@ package org.apache.kafka.common.requests;
 
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.requests.DescribeLogDirsResponse.LogDirInfo;
+
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -29,6 +33,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
 
 public class DescribeLogDirsRequest extends AbstractRequest {
 
@@ -36,9 +42,17 @@ public class DescribeLogDirsRequest extends AbstractRequest {
     private static final String TOPICS_KEY_NAME = "topics";
 
     // topic level key names
-    private static final String TOPIC_KEY_NAME = "topic";
     private static final String PARTITIONS_KEY_NAME = "partitions";
 
+    private static final Schema DESCRIBE_LOG_DIRS_REQUEST_V0 = new Schema(
+            new Field("topics", ArrayOf.nullable(new Schema(
+                    TOPIC_NAME,
+                    new Field("partitions", new ArrayOf(INT32), "List of partition ids of the topic.")))));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{DESCRIBE_LOG_DIRS_REQUEST_V0};
+    }
+
     private final Set<TopicPartition> topicPartitions;
 
     public static class Builder extends AbstractRequest.Builder<DescribeLogDirsRequest> {
@@ -75,7 +89,7 @@ public class DescribeLogDirsRequest extends AbstractRequest {
             topicPartitions = new HashSet<>();
             for (Object topicStructObj : struct.getArray(TOPICS_KEY_NAME)) {
                 Struct topicStruct = (Struct) topicStructObj;
-                String topic = topicStruct.getString(TOPIC_KEY_NAME);
+                String topic = topicStruct.get(TOPIC_NAME);
                 for (Object partitionObj : topicStruct.getArray(PARTITIONS_KEY_NAME)) {
                     int partition = (Integer) partitionObj;
                     topicPartitions.add(new TopicPartition(topic, partition));
@@ -109,7 +123,7 @@ public class DescribeLogDirsRequest extends AbstractRequest {
         List<Struct> topicStructArray = new ArrayList<>();
         for (Map.Entry<String, List<Integer>> partitionsByTopicEntry : partitionsByTopic.entrySet()) {
             Struct topicStruct = struct.instance(TOPICS_KEY_NAME);
-            topicStruct.set(TOPIC_KEY_NAME, partitionsByTopicEntry.getKey());
+            topicStruct.set(TOPIC_NAME, partitionsByTopicEntry.getKey());
             topicStruct.set(PARTITIONS_KEY_NAME, partitionsByTopicEntry.getValue().toArray());
             topicStructArray.add(topicStruct);
         }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java
index f6b31ae..e35056e 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java
@@ -20,14 +20,26 @@ package org.apache.kafka.common.requests;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.utils.CollectionUtils;
+
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME;
+import static org.apache.kafka.common.protocol.types.Type.BOOLEAN;
+import static org.apache.kafka.common.protocol.types.Type.INT64;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 
 public class DescribeLogDirsResponse extends AbstractResponse {
 
@@ -37,40 +49,58 @@ public class DescribeLogDirsResponse extends AbstractResponse {
     private static final String LOG_DIRS_KEY_NAME = "log_dirs";
 
     // dir level key names
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
     private static final String LOG_DIR_KEY_NAME = "log_dir";
     private static final String TOPICS_KEY_NAME = "topics";
 
     // topic level key names
-    private static final String TOPIC_KEY_NAME = "topic";
     private static final String PARTITIONS_KEY_NAME = "partitions";
 
     // partition level key names
-    private static final String PARTITION_KEY_NAME = "partition";
     private static final String SIZE_KEY_NAME = "size";
     private static final String OFFSET_LAG_KEY_NAME = "offset_lag";
     private static final String IS_FUTURE_KEY_NAME = "is_future";
 
+    private static final Schema DESCRIBE_LOG_DIRS_RESPONSE_V0 = new Schema(
+            THROTTLE_TIME_MS,
+            new Field("log_dirs", new ArrayOf(new Schema(
+                    ERROR_CODE,
+                    new Field("log_dir", STRING, "The absolute log directory path."),
+                    new Field("topics", new ArrayOf(new Schema(
+                            TOPIC_NAME,
+                            new Field("partitions", new ArrayOf(new Schema(
+                                    PARTITION_ID,
+                                    new Field("size", INT64, "The size of the log segments of the partition in bytes."),
+                                    new Field("offset_lag", INT64, "The lag of the log's LEO w.r.t. partition's HW " +
+                                            "(if it is the current log for the partition) or current replica's LEO " +
+                                            "(if it is the future log for the partition)"),
+                                    new Field("is_future", BOOLEAN, "True if this log is created by " +
+                                            "AlterReplicaDirRequest and will replace the current log of the replica " +
+                                            "in the future.")))))))))));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{DESCRIBE_LOG_DIRS_RESPONSE_V0};
+    }
+
     private final int throttleTimeMs;
     private final Map<String, LogDirInfo> logDirInfos;
 
     public DescribeLogDirsResponse(Struct struct) {
-        throttleTimeMs = struct.getInt(THROTTLE_TIME_KEY_NAME);
+        throttleTimeMs = struct.get(THROTTLE_TIME_MS);
         logDirInfos = new HashMap<>();
 
         for (Object logDirStructObj : struct.getArray(LOG_DIRS_KEY_NAME)) {
             Struct logDirStruct = (Struct) logDirStructObj;
-            Errors error = Errors.forCode(logDirStruct.getShort(ERROR_CODE_KEY_NAME));
+            Errors error = Errors.forCode(logDirStruct.get(ERROR_CODE));
             String logDir = logDirStruct.getString(LOG_DIR_KEY_NAME);
             Map<TopicPartition, ReplicaInfo> replicaInfos = new HashMap<>();
 
             for (Object topicStructObj : logDirStruct.getArray(TOPICS_KEY_NAME)) {
                 Struct topicStruct = (Struct) topicStructObj;
-                String topic = topicStruct.getString(TOPIC_KEY_NAME);
+                String topic = topicStruct.get(TOPIC_NAME);
 
                 for (Object partitionStructObj : topicStruct.getArray(PARTITIONS_KEY_NAME)) {
                     Struct partitionStruct = (Struct) partitionStructObj;
-                    int partition = partitionStruct.getInt(PARTITION_KEY_NAME);
+                    int partition = partitionStruct.get(PARTITION_ID);
                     long size = partitionStruct.getLong(SIZE_KEY_NAME);
                     long offsetLag = partitionStruct.getLong(OFFSET_LAG_KEY_NAME);
                     boolean isFuture = partitionStruct.getBoolean(IS_FUTURE_KEY_NAME);
@@ -94,25 +124,25 @@ public class DescribeLogDirsResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.DESCRIBE_LOG_DIRS.responseSchema(version));
-        struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
+        struct.set(THROTTLE_TIME_MS, throttleTimeMs);
         List<Struct> logDirStructArray = new ArrayList<>();
         for (Map.Entry<String, LogDirInfo> logDirInfosEntry : logDirInfos.entrySet()) {
             LogDirInfo logDirInfo = logDirInfosEntry.getValue();
             Struct logDirStruct = struct.instance(LOG_DIRS_KEY_NAME);
-            logDirStruct.set(ERROR_CODE_KEY_NAME, logDirInfo.error.code());
+            logDirStruct.set(ERROR_CODE, logDirInfo.error.code());
             logDirStruct.set(LOG_DIR_KEY_NAME, logDirInfosEntry.getKey());
 
             Map<String, Map<Integer, ReplicaInfo>> replicaInfosByTopic = CollectionUtils.groupDataByTopic(logDirInfo.replicaInfos);
             List<Struct> topicStructArray = new ArrayList<>();
             for (Map.Entry<String, Map<Integer, ReplicaInfo>> replicaInfosByTopicEntry : replicaInfosByTopic.entrySet()) {
                 Struct topicStruct = logDirStruct.instance(TOPICS_KEY_NAME);
-                topicStruct.set(TOPIC_KEY_NAME, replicaInfosByTopicEntry.getKey());
+                topicStruct.set(TOPIC_NAME, replicaInfosByTopicEntry.getKey());
                 List<Struct> partitionStructArray = new ArrayList<>();
 
                 for (Map.Entry<Integer, ReplicaInfo> replicaInfosByPartitionEntry : replicaInfosByTopicEntry.getValue().entrySet()) {
                     Struct partitionStruct = topicStruct.instance(PARTITIONS_KEY_NAME);
                     ReplicaInfo replicaInfo = replicaInfosByPartitionEntry.getValue();
-                    partitionStruct.set(PARTITION_KEY_NAME, replicaInfosByPartitionEntry.getKey());
+                    partitionStruct.set(PARTITION_ID, replicaInfosByPartitionEntry.getKey());
                     partitionStruct.set(SIZE_KEY_NAME, replicaInfo.size);
                     partitionStruct.set(OFFSET_LAG_KEY_NAME, replicaInfo.offsetLag);
                     partitionStruct.set(IS_FUTURE_KEY_NAME, replicaInfo.isFuture);

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java
index 01d73b2..243e9f5 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java
@@ -18,16 +18,33 @@ 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.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
 
+import static org.apache.kafka.common.protocol.types.Type.BOOLEAN;
+import static org.apache.kafka.common.protocol.types.Type.INT16;
+import static org.apache.kafka.common.protocol.types.Type.INT64;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
 public class EndTxnRequest extends AbstractRequest {
     private static final String TRANSACTIONAL_ID_KEY_NAME = "transactional_id";
     private static final String PRODUCER_ID_KEY_NAME = "producer_id";
     private static final String PRODUCER_EPOCH_KEY_NAME = "producer_epoch";
     private static final String TRANSACTION_RESULT_KEY_NAME = "transaction_result";
 
+    private static final Schema END_TXN_REQUEST_V0 = new Schema(
+            new Field(TRANSACTIONAL_ID_KEY_NAME, STRING, "The transactional id corresponding to the transaction."),
+            new Field(PRODUCER_ID_KEY_NAME, INT64, "Current producer id in use by the transactional id."),
+            new Field(PRODUCER_EPOCH_KEY_NAME, INT16, "Current epoch associated with the producer id."),
+            new Field(TRANSACTION_RESULT_KEY_NAME, BOOLEAN, "The result of the transaction (0 = ABORT, 1 = COMMIT)"));
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{END_TXN_REQUEST_V0};
+    }
+
     public static class Builder extends AbstractRequest.Builder<EndTxnRequest> {
         private final String transactionalId;
         private final long producerId;

http://git-wip-us.apache.org/repos/asf/kafka/blob/0cf77080/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java
index 47a6623..a3bae58 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java
@@ -18,12 +18,22 @@ 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.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 import java.nio.ByteBuffer;
 
+import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE;
+import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;
+
 public class EndTxnResponse extends AbstractResponse {
-    private static final String ERROR_CODE_KEY_NAME = "error_code";
+    private static final Schema END_TXN_RESPONSE_V0 = new Schema(
+            THROTTLE_TIME_MS,
+            ERROR_CODE);
+
+    public static Schema[] schemaVersions() {
+        return new Schema[]{END_TXN_RESPONSE_V0};
+    }
 
     // Possible error codes:
     //   NotCoordinator
@@ -43,8 +53,8 @@ public class EndTxnResponse extends AbstractResponse {
     }
 
     public EndTxnResponse(Struct struct) {
-        this.throttleTimeMs = struct.getInt(THROTTLE_TIME_KEY_NAME);
-        this.error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME));
+        this.throttleTimeMs = struct.get(THROTTLE_TIME_MS);
+        this.error = Errors.forCode(struct.get(ERROR_CODE));
     }
 
     public int throttleTimeMs() {
@@ -58,8 +68,8 @@ public class EndTxnResponse extends AbstractResponse {
     @Override
     protected Struct toStruct(short version) {
         Struct struct = new Struct(ApiKeys.END_TXN.responseSchema(version));
-        struct.set(THROTTLE_TIME_KEY_NAME, throttleTimeMs);
-        struct.set(ERROR_CODE_KEY_NAME, error.code());
+        struct.set(THROTTLE_TIME_MS, throttleTimeMs);
+        struct.set(ERROR_CODE, error.code());
         return struct;
     }