You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by jj...@apache.org on 2014/08/06 00:59:58 UTC
[01/37] git commit: KAFKA-1096 An old controller coming out of long
GC could update its epoch to the latest controller's epoch;
reviewed by Neha Narkhede
Repository: kafka
Updated Branches:
refs/heads/transactional_messaging 6b0ae4bba -> 7a67a7226
KAFKA-1096 An old controller coming out of long GC could update its epoch to the latest controller's epoch; reviewed by Neha Narkhede
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/62f20870
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/62f20870
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/62f20870
Branch: refs/heads/transactional_messaging
Commit: 62f208704fe7fb08f085d569a26e780e0050dba0
Parents: 6b0ae4b
Author: Sriharsha Chintalapani <sc...@hortonworks.com>
Authored: Thu Jun 26 16:16:03 2014 -0700
Committer: Neha Narkhede <ne...@gmail.com>
Committed: Thu Jun 26 16:16:15 2014 -0700
----------------------------------------------------------------------
.../kafka/controller/KafkaController.scala | 53 +++++---------------
1 file changed, 12 insertions(+), 41 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/62f20870/core/src/main/scala/kafka/controller/KafkaController.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala
index 94bbd33..a7a21df 100644
--- a/core/src/main/scala/kafka/controller/KafkaController.scala
+++ b/core/src/main/scala/kafka/controller/KafkaController.scala
@@ -169,8 +169,6 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt
private val preferredReplicaPartitionLeaderSelector = new PreferredReplicaPartitionLeaderSelector(controllerContext)
private val controlledShutdownPartitionLeaderSelector = new ControlledShutdownLeaderSelector(controllerContext)
private val brokerRequestBatch = new ControllerBrokerRequestBatch(this)
- registerControllerChangedListener()
-
newGauge(
"ActiveControllerCount",
new Gauge[Int] {
@@ -298,6 +296,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt
def onControllerFailover() {
if(isRunning) {
info("Broker %d starting become controller state transition".format(config.brokerId))
+ //read controller epoch from zk
+ readControllerEpochFromZookeeper()
// increment the controller epoch
incrementControllerEpoch(zkClient)
// before reading source of truth from zookeeper, register the listeners to get broker/topic callbacks
@@ -346,6 +346,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt
controllerContext.controllerChannelManager.shutdown()
controllerContext.controllerChannelManager = null
}
+ controllerContext.epoch=0
+ controllerContext.epochZkVersion=0
brokerState.newState(RunningAsBroker)
}
}
@@ -875,8 +877,14 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt
zkClient.subscribeDataChanges(ZkUtils.PreferredReplicaLeaderElectionPath, new PreferredReplicaElectionListener(this))
}
- private def registerControllerChangedListener() {
- zkClient.subscribeDataChanges(ZkUtils.ControllerEpochPath, new ControllerEpochListener(this))
+ private def readControllerEpochFromZookeeper() {
+ // initialize the controller epoch and zk version by reading from zookeeper
+ if(ZkUtils.pathExists(controllerContext.zkClient, ZkUtils.ControllerEpochPath)) {
+ val epochData = ZkUtils.readData(controllerContext.zkClient, ZkUtils.ControllerEpochPath)
+ controllerContext.epoch = epochData._1.toInt
+ controllerContext.epochZkVersion = epochData._2.getVersion
+ info("Initialized controller epoch to %d and zk version %d".format(controllerContext.epoch, controllerContext.epochZkVersion))
+ }
}
def removePartitionFromReassignedPartitions(topicAndPartition: TopicAndPartition) {
@@ -1275,43 +1283,6 @@ class PreferredReplicaElectionListener(controller: KafkaController) extends IZkD
}
}
-class ControllerEpochListener(controller: KafkaController) extends IZkDataListener with Logging {
- this.logIdent = "[ControllerEpochListener on " + controller.config.brokerId + "]: "
- val controllerContext = controller.controllerContext
- readControllerEpochFromZookeeper()
-
- /**
- * Invoked when a controller updates the epoch value
- * @throws Exception On any error.
- */
- @throws(classOf[Exception])
- def handleDataChange(dataPath: String, data: Object) {
- debug("Controller epoch listener fired with new epoch " + data.toString)
- inLock(controllerContext.controllerLock) {
- // read the epoch path to get the zk version
- readControllerEpochFromZookeeper()
- }
- }
-
- /**
- * @throws Exception
- * On any error.
- */
- @throws(classOf[Exception])
- def handleDataDeleted(dataPath: String) {
- }
-
- private def readControllerEpochFromZookeeper() {
- // initialize the controller epoch and zk version by reading from zookeeper
- if(ZkUtils.pathExists(controllerContext.zkClient, ZkUtils.ControllerEpochPath)) {
- val epochData = ZkUtils.readData(controllerContext.zkClient, ZkUtils.ControllerEpochPath)
- controllerContext.epoch = epochData._1.toInt
- controllerContext.epochZkVersion = epochData._2.getVersion
- info("Initialized controller epoch to %d and zk version %d".format(controllerContext.epoch, controllerContext.epochZkVersion))
- }
- }
-}
-
case class ReassignedPartitionsContext(var newReplicas: Seq[Int] = Seq.empty,
var isrChangeListener: ReassignedPartitionsIsrChangeListener = null)
[02/37] git commit: KAFKA-1491;
Always read coordinator information in consumer metadata response;
reviewed by Neha Narkhede.
Posted by jj...@apache.org.
KAFKA-1491; Always read coordinator information in consumer metadata response; reviewed by Neha Narkhede.
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/c4b95641
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/c4b95641
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/c4b95641
Branch: refs/heads/transactional_messaging
Commit: c4b95641ea295478f9480b264652b7b52d8f71c5
Parents: 62f2087
Author: Joel Koshy <jj...@gmail.com>
Authored: Fri Jun 27 11:09:18 2014 -0700
Committer: Joel Koshy <jj...@gmail.com>
Committed: Fri Jun 27 11:09:18 2014 -0700
----------------------------------------------------------------------
core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala | 3 ++-
core/src/main/scala/kafka/client/ClientUtils.scala | 5 +++++
2 files changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/c4b95641/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
index f8cf6c3..c72ca14 100644
--- a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
+++ b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
@@ -29,8 +29,9 @@ object ConsumerMetadataResponse {
def readFrom(buffer: ByteBuffer) = {
val correlationId = buffer.getInt
val errorCode = buffer.getShort
+ val broker = Broker.readFrom(buffer)
val coordinatorOpt = if (errorCode == ErrorMapping.NoError)
- Some(Broker.readFrom(buffer))
+ Some(broker)
else
None
http://git-wip-us.apache.org/repos/asf/kafka/blob/c4b95641/core/src/main/scala/kafka/client/ClientUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala
index ba5fbdc..ce7ede3 100644
--- a/core/src/main/scala/kafka/client/ClientUtils.scala
+++ b/core/src/main/scala/kafka/client/ClientUtils.scala
@@ -160,6 +160,11 @@ object ClientUtils extends Logging{
debug("Consumer metadata response: " + consumerMetadataResponse.toString)
if (consumerMetadataResponse.errorCode == ErrorMapping.NoError)
coordinatorOpt = consumerMetadataResponse.coordinatorOpt
+ else {
+ debug("Query to %s:%d to locate offset manager for %s failed - will retry in %d milliseconds."
+ .format(queryChannel.host, queryChannel.port, group, retryBackOffMs))
+ Thread.sleep(retryBackOffMs)
+ }
}
catch {
case ioe: IOException =>
[33/37] git commit: kafka-1562;
kafka-topics.sh alter add partitions resets cleanup.policy;
patched by Jonathan Natkins; reviewed by Jun Rao
Posted by jj...@apache.org.
kafka-1562; kafka-topics.sh alter add partitions resets cleanup.policy; patched by Jonathan Natkins; reviewed by Jun Rao
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/f8d521a9
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/f8d521a9
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/f8d521a9
Branch: refs/heads/transactional_messaging
Commit: f8d521a9616134d78966419e5cf2aa73e8d6a5c5
Parents: a01a101
Author: Jonathan Natkins <na...@wibidata.com>
Authored: Mon Aug 4 07:21:25 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Mon Aug 4 07:21:25 2014 -0700
----------------------------------------------------------------------
.../src/main/scala/kafka/admin/AdminUtils.scala | 10 +++-
.../main/scala/kafka/admin/TopicCommand.scala | 4 +-
.../unit/kafka/admin/TopicCommandTest.scala | 63 ++++++++++++++++++++
3 files changed, 73 insertions(+), 4 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/f8d521a9/core/src/main/scala/kafka/admin/AdminUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala
index b5d8714..94c5332 100644
--- a/core/src/main/scala/kafka/admin/AdminUtils.scala
+++ b/core/src/main/scala/kafka/admin/AdminUtils.scala
@@ -96,8 +96,14 @@ object AdminUtils extends Logging {
* @param numPartitions Number of partitions to be set
* @param replicaAssignmentStr Manual replica assignment
* @param checkBrokerAvailable Ignore checking if assigned replica broker is available. Only used for testing
+ * @param config Pre-existing properties that should be preserved
*/
- def addPartitions(zkClient: ZkClient, topic: String, numPartitions: Int = 1, replicaAssignmentStr: String = "", checkBrokerAvailable: Boolean = true) {
+ def addPartitions(zkClient: ZkClient,
+ topic: String,
+ numPartitions: Int = 1,
+ replicaAssignmentStr: String = "",
+ checkBrokerAvailable: Boolean = true,
+ config: Properties = new Properties) {
val existingPartitionsReplicaList = ZkUtils.getReplicaAssignmentForTopics(zkClient, List(topic))
if (existingPartitionsReplicaList.size == 0)
throw new AdminOperationException("The topic %s does not exist".format(topic))
@@ -124,7 +130,7 @@ object AdminUtils extends Logging {
val partitionReplicaList = existingPartitionsReplicaList.map(p => p._1.partition -> p._2)
// add the new list
partitionReplicaList ++= newPartitionReplicaList
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaList, update = true)
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaList, config, true)
}
def getManualReplicaAssignment(replicaAssignmentList: String, availableBrokerList: Set[Int], startPartitionId: Int, checkBrokerAvailable: Boolean = true): Map[Int, List[Int]] = {
http://git-wip-us.apache.org/repos/asf/kafka/blob/f8d521a9/core/src/main/scala/kafka/admin/TopicCommand.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala
index 8d5c2e7..003a09c 100644
--- a/core/src/main/scala/kafka/admin/TopicCommand.scala
+++ b/core/src/main/scala/kafka/admin/TopicCommand.scala
@@ -95,11 +95,11 @@ object TopicCommand {
def alterTopic(zkClient: ZkClient, opts: TopicCommandOptions) {
val topics = getTopics(zkClient, opts)
topics.foreach { topic =>
+ val configs = AdminUtils.fetchTopicConfig(zkClient, topic)
if(opts.options.has(opts.configOpt) || opts.options.has(opts.deleteConfigOpt)) {
val configsToBeAdded = parseTopicConfigsToBeAdded(opts)
val configsToBeDeleted = parseTopicConfigsToBeDeleted(opts)
// compile the final set of configs
- val configs = AdminUtils.fetchTopicConfig(zkClient, topic)
configs.putAll(configsToBeAdded)
configsToBeDeleted.foreach(config => configs.remove(config))
AdminUtils.changeTopicConfig(zkClient, topic, configs)
@@ -113,7 +113,7 @@ object TopicCommand {
"logic or ordering of the messages will be affected")
val nPartitions = opts.options.valueOf(opts.partitionsOpt).intValue
val replicaAssignmentStr = opts.options.valueOf(opts.replicaAssignmentOpt)
- AdminUtils.addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr)
+ AdminUtils.addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr, config = configs)
println("Adding partitions succeeded!")
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/f8d521a9/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala
new file mode 100644
index 0000000..ac6dd20
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.admin
+
+import junit.framework.Assert._
+import org.junit.Test
+import org.scalatest.junit.JUnit3Suite
+import kafka.utils.Logging
+import kafka.utils.TestUtils
+import kafka.zk.ZooKeeperTestHarness
+import kafka.server.KafkaConfig
+import kafka.admin.TopicCommand.TopicCommandOptions
+import kafka.utils.ZkUtils
+
+class TopicCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Logging {
+
+ @Test
+ def testConfigPreservationAcrossPartitionAlteration() {
+ val topic = "test"
+ val numPartitionsOriginal = 1
+ val cleanupKey = "cleanup.policy"
+ val cleanupVal = "compact"
+ // create brokers
+ val brokers = List(0, 1, 2)
+ TestUtils.createBrokersInZk(zkClient, brokers)
+ // create the topic
+ val createOpts = new TopicCommandOptions(Array("--partitions", numPartitionsOriginal.toString,
+ "--replication-factor", "1",
+ "--config", cleanupKey + "=" + cleanupVal,
+ "--topic", topic))
+ TopicCommand.createTopic(zkClient, createOpts)
+ val props = AdminUtils.fetchTopicConfig(zkClient, topic)
+ assertTrue("Properties after creation don't contain " + cleanupKey, props.containsKey(cleanupKey))
+ assertTrue("Properties after creation have incorrect value", props.getProperty(cleanupKey).equals(cleanupVal))
+
+ // pre-create the topic config changes path to avoid a NoNodeException
+ ZkUtils.createPersistentPath(zkClient, ZkUtils.TopicConfigChangesPath)
+
+ // modify the topic to add new partitions
+ val numPartitionsModified = 3
+ val alterOpts = new TopicCommandOptions(Array("--partitions", numPartitionsModified.toString,
+ "--config", cleanupKey + "=" + cleanupVal,
+ "--topic", topic))
+ TopicCommand.alterTopic(zkClient, alterOpts)
+ val newProps = AdminUtils.fetchTopicConfig(zkClient, topic)
+ assertTrue("Updated properties do not contain " + cleanupKey, newProps.containsKey(cleanupKey))
+ assertTrue("Updated properties have incorrect value", newProps.getProperty(cleanupKey).equals(cleanupVal))
+ }
+}
\ No newline at end of file
[24/37] git commit: kafka-1533;
transient unit test failure in ProducerFailureHandlingTest;
reviewed by Guozhang Wang; reviewed by Jun Rao
Posted by jj...@apache.org.
kafka-1533; transient unit test failure in ProducerFailureHandlingTest; reviewed by Guozhang Wang; reviewed by Jun Rao
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/ff05e9b3
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/ff05e9b3
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/ff05e9b3
Branch: refs/heads/transactional_messaging
Commit: ff05e9b3616a222e29a42f6e8fdf41945a417f41
Parents: 014b700
Author: Guozhang Wang <gu...@linkedin.com>
Authored: Tue Jul 22 14:14:19 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Tue Jul 22 14:14:19 2014 -0700
----------------------------------------------------------------------
.../org/apache/kafka/clients/NetworkClient.java | 13 ++-
.../clients/producer/internals/Metadata.java | 7 ++
.../kafka/api/ProducerFailureHandlingTest.scala | 88 ++++++++------------
.../kafka/api/ProducerSendTest.scala | 56 +++++--------
.../integration/KafkaServerTestHarness.scala | 2 +
.../test/scala/unit/kafka/utils/TestUtils.scala | 2 +-
6 files changed, 78 insertions(+), 90 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/ff05e9b3/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 d8f9ce6..eea270a 100644
--- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
@@ -77,6 +77,9 @@ public class NetworkClient implements KafkaClient {
/* true iff there is a metadata request that has been sent and for which we have not yet received a response */
private boolean metadataFetchInProgress;
+ /* the last timestamp when no broker node is available to connect */
+ private long lastNoNodeAvailableMs;
+
public NetworkClient(Selectable selector,
Metadata metadata,
String clientId,
@@ -94,6 +97,7 @@ public class NetworkClient implements KafkaClient {
this.correlation = 0;
this.nodeIndexOffset = new Random().nextInt(Integer.MAX_VALUE);
this.metadataFetchInProgress = false;
+ this.lastNoNodeAvailableMs = 0;
}
/**
@@ -162,7 +166,10 @@ public class NetworkClient implements KafkaClient {
}
// should we update our metadata?
- long metadataTimeout = metadata.timeToNextUpdate(now);
+ long timeToNextMetadataUpdate = metadata.timeToNextUpdate(now);
+ long timeToNextReconnectAttempt = this.lastNoNodeAvailableMs + metadata.refreshBackoff() - now;
+ // if there is no node available to connect, back off refreshing metadata
+ long metadataTimeout = Math.max(timeToNextMetadataUpdate, timeToNextReconnectAttempt);
if (!this.metadataFetchInProgress && metadataTimeout == 0)
maybeUpdateMetadata(sends, now);
@@ -354,6 +361,8 @@ public class NetworkClient implements KafkaClient {
Node node = this.leastLoadedNode(now);
if (node == null) {
log.debug("Give up sending metadata request since no node is available");
+ // mark the timestamp for no node available to connect
+ this.lastNoNodeAvailableMs = now;
return;
}
@@ -367,7 +376,7 @@ public class NetworkClient implements KafkaClient {
this.inFlightRequests.add(metadataRequest);
} else if (connectionStates.canConnect(node.id(), now)) {
// we don't have a connection to this node right now, make one
- log.debug("Give up sending metadata request to node {} since it is either not connected or cannot have more in flight requests", node.id());
+ log.debug("Init connection to node {} for sending metadata request in the next iteration", node.id());
initiateConnect(node, now);
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ff05e9b3/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
index 4aa5b01..1d30f9e 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
@@ -139,4 +139,11 @@ public final class Metadata {
public synchronized long lastUpdate() {
return this.lastRefreshMs;
}
+
+ /**
+ * The metadata refresh backoff in ms
+ */
+ public long refreshBackoff() {
+ return refreshBackoffMs;
+ }
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ff05e9b3/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
index 15fd5bc..789e74c 100644
--- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
@@ -21,26 +21,31 @@ import org.scalatest.junit.JUnit3Suite
import org.junit.Test
import org.junit.Assert._
-import java.util.{Random, Properties}
+import java.util.Random
import java.lang.Integer
import java.util.concurrent.{TimeoutException, TimeUnit, ExecutionException}
-import kafka.server.{KafkaConfig, KafkaServer}
-import kafka.utils.{ShutdownableThread, Utils, TestUtils}
-import kafka.zk.ZooKeeperTestHarness
+import kafka.server.KafkaConfig
+import kafka.utils.{TestZKUtils, ShutdownableThread, TestUtils}
+import kafka.integration.KafkaServerTestHarness
import kafka.consumer.SimpleConsumer
import org.apache.kafka.common.KafkaException
import org.apache.kafka.clients.producer._
-class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness {
- private val brokerId1 = 0
- private val brokerId2 = 1
- private val ports = TestUtils.choosePorts(2)
- private val (port1, port2) = (ports(0), ports(1))
- private var server1: KafkaServer = null
- private var server2: KafkaServer = null
- private var servers = List.empty[KafkaServer]
+class ProducerFailureHandlingTest extends JUnit3Suite with KafkaServerTestHarness {
+ private val producerBufferSize = 30000
+ private val serverMessageMaxBytes = producerBufferSize/2
+
+ val numServers = 2
+ val configs =
+ for(props <- TestUtils.createBrokerConfigs(numServers, false))
+ yield new KafkaConfig(props) {
+ override val zkConnect = TestZKUtils.zookeeperConnect
+ override val autoCreateTopicsEnable = false
+ override val messageMaxBytes = serverMessageMaxBytes
+ }
+
private var consumer1: SimpleConsumer = null
private var consumer2: SimpleConsumer = null
@@ -50,32 +55,19 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
private var producer3: KafkaProducer = null
private var producer4: KafkaProducer = null
- private val props1 = TestUtils.createBrokerConfig(brokerId1, port1, false)
- private val props2 = TestUtils.createBrokerConfig(brokerId2, port2, false)
- props1.put("auto.create.topics.enable", "false")
- props2.put("auto.create.topics.enable", "false")
- private val config1 = new KafkaConfig(props1)
- private val config2 = new KafkaConfig(props2)
- private val brokerList = TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))
-
- private val bufferSize = 2 * config1.messageMaxBytes
-
private val topic1 = "topic-1"
private val topic2 = "topic-2"
override def setUp() {
super.setUp()
- server1 = TestUtils.createServer(config1)
- server2 = TestUtils.createServer(config2)
- servers = List(server1,server2)
// TODO: we need to migrate to new consumers when 0.9 is final
- consumer1 = new SimpleConsumer("localhost", port1, 100, 1024*1024, "")
- consumer2 = new SimpleConsumer("localhost", port2, 100, 1024*1024, "")
+ consumer1 = new SimpleConsumer("localhost", configs(0).port, 100, 1024*1024, "")
+ consumer2 = new SimpleConsumer("localhost", configs(1).port, 100, 1024*1024, "")
- producer1 = TestUtils.createNewProducer(brokerList, acks = 0, blockOnBufferFull = false, bufferSize = bufferSize);
- producer2 = TestUtils.createNewProducer(brokerList, acks = 1, blockOnBufferFull = false, bufferSize = bufferSize)
- producer3 = TestUtils.createNewProducer(brokerList, acks = -1, blockOnBufferFull = false, bufferSize = bufferSize)
+ producer1 = TestUtils.createNewProducer(brokerList, acks = 0, blockOnBufferFull = false, bufferSize = producerBufferSize);
+ producer2 = TestUtils.createNewProducer(brokerList, acks = 1, blockOnBufferFull = false, bufferSize = producerBufferSize)
+ producer3 = TestUtils.createNewProducer(brokerList, acks = -1, blockOnBufferFull = false, bufferSize = producerBufferSize)
}
override def tearDown() {
@@ -87,9 +79,6 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
if (producer3 != null) producer3.close
if (producer4 != null) producer4.close
- server1.shutdown; Utils.rm(server1.config.logDirs)
- server2.shutdown; Utils.rm(server2.config.logDirs)
-
super.tearDown()
}
@@ -102,7 +91,7 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
TestUtils.createTopic(zkClient, topic1, 1, 2, servers)
// send a too-large record
- val record = new ProducerRecord(topic1, null, "key".getBytes, new Array[Byte](config1.messageMaxBytes + 1))
+ val record = new ProducerRecord(topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1))
assertEquals("Returned metadata should have offset -1", producer1.send(record).get.offset, -1L)
}
@@ -115,7 +104,7 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
TestUtils.createTopic(zkClient, topic1, 1, 2, servers)
// send a too-large record
- val record = new ProducerRecord(topic1, null, "key".getBytes, new Array[Byte](config1.messageMaxBytes + 1))
+ val record = new ProducerRecord(topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1))
intercept[ExecutionException] {
producer2.send(record).get
}
@@ -149,7 +138,7 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
TestUtils.createTopic(zkClient, topic1, 1, 2, servers)
// producer with incorrect broker list
- producer4 = TestUtils.createNewProducer("localhost:8686,localhost:4242", acks = 1, blockOnBufferFull = false, bufferSize = bufferSize)
+ producer4 = TestUtils.createNewProducer("localhost:8686,localhost:4242", acks = 1, blockOnBufferFull = false, bufferSize = producerBufferSize)
// send a record with incorrect broker list
val record = new ProducerRecord(topic1, null, "key".getBytes, "value".getBytes)
@@ -175,8 +164,7 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
// stop IO threads and request handling, but leave networking operational
// any requests should be accepted and queue up, but not handled
- server1.requestHandlerPool.shutdown()
- server2.requestHandlerPool.shutdown()
+ servers.foreach(server => server.requestHandlerPool.shutdown())
producer1.send(record1).get(5000, TimeUnit.MILLISECONDS)
@@ -186,11 +174,11 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
// TODO: expose producer configs after creating them
// send enough messages to get buffer full
- val msgSize = 10000
+ val tooManyRecords = 10
+ val msgSize = producerBufferSize / tooManyRecords
val value = new Array[Byte](msgSize)
new Random().nextBytes(value)
val record2 = new ProducerRecord(topic1, null, "key".getBytes, value)
- val tooManyRecords = bufferSize / ("key".getBytes.length + value.length)
intercept[KafkaException] {
for (i <- 1 to tooManyRecords)
@@ -269,17 +257,13 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
// rolling bounce brokers
for (i <- 0 until 2) {
- server1.shutdown()
- server1.awaitShutdown()
- server1.startup
-
- Thread.sleep(2000)
+ for (server <- servers) {
+ server.shutdown()
+ server.awaitShutdown()
+ server.startup
- server2.shutdown()
- server2.awaitShutdown()
- server2.startup
-
- Thread.sleep(2000)
+ Thread.sleep(2000)
+ }
// Make sure the producer do not see any exception
// in returned metadata due to broker failures
@@ -298,7 +282,7 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
// double check that the leader info has been propagated after consecutive bounces
val leader = TestUtils.waitUntilMetadataIsPropagated(servers, topic1, partition)
- val fetchResponse = if(leader == server1.config.brokerId) {
+ val fetchResponse = if(leader == configs(0).brokerId) {
consumer1.fetch(new FetchRequestBuilder().addFetch(topic1, partition, 0, Int.MaxValue).build()).messageSet(topic1, partition)
} else {
consumer2.fetch(new FetchRequestBuilder().addFetch(topic1, partition, 0, Int.MaxValue).build()).messageSet(topic1, partition)
@@ -317,7 +301,7 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
var sent = 0
var failed = false
- val producer = TestUtils.createNewProducer(brokerList, bufferSize = bufferSize, retries = 10)
+ val producer = TestUtils.createNewProducer(brokerList, bufferSize = producerBufferSize, retries = 10)
override def doWork(): Unit = {
val responses =
http://git-wip-us.apache.org/repos/asf/kafka/blob/ff05e9b3/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala
index 34a7db4..d407af9 100644
--- a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala
@@ -17,7 +17,6 @@
package kafka.api.test
-import java.util.Properties
import java.lang.{Integer, IllegalArgumentException}
import org.apache.kafka.clients.producer._
@@ -25,53 +24,41 @@ import org.scalatest.junit.JUnit3Suite
import org.junit.Test
import org.junit.Assert._
-import kafka.server.{KafkaConfig, KafkaServer}
-import kafka.utils.{Utils, TestUtils}
-import kafka.zk.ZooKeeperTestHarness
+import kafka.server.KafkaConfig
+import kafka.utils.{TestZKUtils, TestUtils}
import kafka.consumer.SimpleConsumer
import kafka.api.FetchRequestBuilder
import kafka.message.Message
+import kafka.integration.KafkaServerTestHarness
-class ProducerSendTest extends JUnit3Suite with ZooKeeperTestHarness {
- private val brokerId1 = 0
- private val brokerId2 = 1
- private val ports = TestUtils.choosePorts(2)
- private val (port1, port2) = (ports(0), ports(1))
- private var server1: KafkaServer = null
- private var server2: KafkaServer = null
- private var servers = List.empty[KafkaServer]
+class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness {
+ val numServers = 2
+ val configs =
+ for(props <- TestUtils.createBrokerConfigs(numServers, false))
+ yield new KafkaConfig(props) {
+ override val zkConnect = TestZKUtils.zookeeperConnect
+ override val numPartitions = 4
+ }
private var consumer1: SimpleConsumer = null
private var consumer2: SimpleConsumer = null
- private val props1 = TestUtils.createBrokerConfig(brokerId1, port1, false)
- private val props2 = TestUtils.createBrokerConfig(brokerId2, port2, false)
- props1.put("num.partitions", "4")
- props2.put("num.partitions", "4")
- private val config1 = new KafkaConfig(props1)
- private val config2 = new KafkaConfig(props2)
-
private val topic = "topic"
private val numRecords = 100
override def setUp() {
super.setUp()
- // set up 2 brokers with 4 partitions each
- server1 = TestUtils.createServer(config1)
- server2 = TestUtils.createServer(config2)
- servers = List(server1,server2)
// TODO: we need to migrate to new consumers when 0.9 is final
- consumer1 = new SimpleConsumer("localhost", port1, 100, 1024*1024, "")
- consumer2 = new SimpleConsumer("localhost", port2, 100, 1024*1024, "")
+ consumer1 = new SimpleConsumer("localhost", configs(0).port, 100, 1024*1024, "")
+ consumer2 = new SimpleConsumer("localhost", configs(1).port, 100, 1024*1024, "")
}
override def tearDown() {
- server1.shutdown
- server2.shutdown
- Utils.rm(server1.config.logDirs)
- Utils.rm(server2.config.logDirs)
+ consumer1.close()
+ consumer2.close()
+
super.tearDown()
}
@@ -90,7 +77,7 @@ class ProducerSendTest extends JUnit3Suite with ZooKeeperTestHarness {
*/
@Test
def testSendOffset() {
- var producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)))
+ var producer = TestUtils.createNewProducer(brokerList)
val callback = new CheckErrorCallback
@@ -146,7 +133,7 @@ class ProducerSendTest extends JUnit3Suite with ZooKeeperTestHarness {
*/
@Test
def testClose() {
- var producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)))
+ var producer = TestUtils.createNewProducer(brokerList)
try {
// create topic
@@ -182,7 +169,7 @@ class ProducerSendTest extends JUnit3Suite with ZooKeeperTestHarness {
*/
@Test
def testSendToPartition() {
- var producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)))
+ var producer = TestUtils.createNewProducer(brokerList)
try {
// create topic
@@ -209,7 +196,7 @@ class ProducerSendTest extends JUnit3Suite with ZooKeeperTestHarness {
}
// make sure the fetched messages also respect the partitioning and ordering
- val fetchResponse1 = if(leader1.get == server1.config.brokerId) {
+ val fetchResponse1 = if(leader1.get == configs(0).brokerId) {
consumer1.fetch(new FetchRequestBuilder().addFetch(topic, partition, 0, Int.MaxValue).build())
} else {
consumer2.fetch(new FetchRequestBuilder().addFetch(topic, partition, 0, Int.MaxValue).build())
@@ -237,8 +224,7 @@ class ProducerSendTest extends JUnit3Suite with ZooKeeperTestHarness {
*/
@Test
def testAutoCreateTopic() {
- var producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)),
- retries = 5)
+ var producer = TestUtils.createNewProducer(brokerList, retries = 5)
try {
// Send a message to auto-create the topic
http://git-wip-us.apache.org/repos/asf/kafka/blob/ff05e9b3/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
index 194dd70..3cf7c9b 100644
--- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
+++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
@@ -30,11 +30,13 @@ trait KafkaServerTestHarness extends JUnit3Suite with ZooKeeperTestHarness {
val configs: List[KafkaConfig]
var servers: List[KafkaServer] = null
+ var brokerList: String = null
override def setUp() {
super.setUp
if(configs.size <= 0)
throw new KafkaException("Must suply at least one server config.")
+ brokerList = TestUtils.getBrokerListStrFromConfigs(configs)
servers = configs.map(TestUtils.createServer(_))
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ff05e9b3/core/src/test/scala/unit/kafka/utils/TestUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index 3faa884..4d01d25 100644
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -385,7 +385,7 @@ object TestUtils extends Logging {
producerProps.put(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, blockOnBufferFull.toString)
producerProps.put(ProducerConfig.BUFFER_MEMORY_CONFIG, bufferSize.toString)
producerProps.put(ProducerConfig.RETRIES_CONFIG, retries.toString)
- producerProps.put(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, "1000")
+ producerProps.put(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, "100")
producerProps.put(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG, "200")
return new KafkaProducer(producerProps)
}
[19/37] kafka-1462;
Add new request and response formats for the new consumer and
coordinator communication; patched by Jun Rao;
reviewed by Guozhang Wang and Jay Kreps
Posted by jj...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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 2652c32..7d90fce 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
@@ -12,6 +12,7 @@
*/
package org.apache.kafka.common.requests;
+import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
@@ -20,50 +21,112 @@ import java.util.Map;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
import org.apache.kafka.common.protocol.types.Struct;
-public class MetadataResponse {
+public class MetadataResponse extends AbstractRequestResponse {
+ private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.METADATA.id);
+ private static String BROKERS_KEY_NAME = "brokers";
+ private static String TOPIC_METATDATA_KEY_NAME = "topic_metadata";
+
+ // broker level field names
+ private static String NODE_ID_KEY_NAME = "node_id";
+ private static String HOST_KEY_NAME = "host";
+ private static String PORT_KEY_NAME = "port";
+
+ // topic level field names
+ private static String TOPIC_ERROR_CODE_KEY_NAME = "topic_error_code";
+ private static String TOPIC_KEY_NAME = "topic";
+ private static String PARTITION_METADATA_KEY_NAME = "partition_metadata";
+
+ // partition level field names
+ private static String PARTITION_ERROR_CODE_KEY_NAME = "partition_error_code";
+ private static String PARTITION_KEY_NAME = "partition_id";
+ private static String LEADER_KEY_NAME = "leader";
+ private static String REPLICAS_KEY_NAME = "replicas";
+ private static String ISR_KEY_NAME = "isr";
private final Cluster cluster;
private final Map<String, Errors> errors;
- public MetadataResponse(Cluster cluster, Map<String, Errors> errors) {
+ public MetadataResponse(Cluster cluster) {
+ super(new Struct(curSchema));
+
+ List<Struct> brokerArray = new ArrayList<Struct>();
+ for (Node node: cluster.nodes()) {
+ Struct broker = struct.instance(BROKERS_KEY_NAME);
+ broker.set(NODE_ID_KEY_NAME, node.id());
+ broker.set(HOST_KEY_NAME, node.host());
+ broker.set(PORT_KEY_NAME, node.port());
+ brokerArray.add(broker);
+ }
+ struct.set(BROKERS_KEY_NAME, brokerArray.toArray());
+
+ List<Struct> topicArray = new ArrayList<Struct>();
+ for (String topic: cluster.topics()) {
+ Struct topicData = struct.instance(TOPIC_METATDATA_KEY_NAME);
+ topicData.set(TOPIC_ERROR_CODE_KEY_NAME, (short)0); // no error
+ topicData.set(TOPIC_KEY_NAME, topic);
+ List<Struct> partitionArray = new ArrayList<Struct>();
+ for (PartitionInfo fetchPartitionData : cluster.partitionsForTopic(topic)) {
+ Struct partitionData = topicData.instance(PARTITION_METADATA_KEY_NAME);
+ partitionData.set(PARTITION_ERROR_CODE_KEY_NAME, (short)0); // no error
+ partitionData.set(PARTITION_KEY_NAME, fetchPartitionData.partition());
+ partitionData.set(LEADER_KEY_NAME, fetchPartitionData.leader().id());
+ ArrayList<Integer> replicas = new ArrayList<Integer>();
+ for (Node node: fetchPartitionData.replicas())
+ replicas.add(node.id());
+ partitionData.set(REPLICAS_KEY_NAME, replicas.toArray());
+ ArrayList<Integer> isr = new ArrayList<Integer>();
+ for (Node node: fetchPartitionData.inSyncReplicas())
+ isr.add(node.id());
+ partitionData.set(ISR_KEY_NAME, isr.toArray());
+ partitionArray.add(partitionData);
+ }
+ topicData.set(PARTITION_METADATA_KEY_NAME, partitionArray.toArray());
+ topicArray.add(topicData);
+ }
+ struct.set(TOPIC_METATDATA_KEY_NAME, topicArray.toArray());
+
this.cluster = cluster;
- this.errors = errors;
+ this.errors = new HashMap<String, Errors>();
}
public MetadataResponse(Struct struct) {
+ super(struct);
Map<String, Errors> errors = new HashMap<String, Errors>();
Map<Integer, Node> brokers = new HashMap<Integer, Node>();
- Object[] brokerStructs = (Object[]) struct.get("brokers");
+ Object[] brokerStructs = (Object[]) struct.get(BROKERS_KEY_NAME);
for (int i = 0; i < brokerStructs.length; i++) {
Struct broker = (Struct) brokerStructs[i];
- int nodeId = (Integer) broker.get("node_id");
- String host = (String) broker.get("host");
- int port = (Integer) broker.get("port");
+ int nodeId = broker.getInt(NODE_ID_KEY_NAME);
+ String host = broker.getString(HOST_KEY_NAME);
+ int port = broker.getInt(PORT_KEY_NAME);
brokers.put(nodeId, new Node(nodeId, host, port));
}
List<PartitionInfo> partitions = new ArrayList<PartitionInfo>();
- Object[] topicInfos = (Object[]) struct.get("topic_metadata");
+ Object[] topicInfos = (Object[]) struct.get(TOPIC_METATDATA_KEY_NAME);
for (int i = 0; i < topicInfos.length; i++) {
Struct topicInfo = (Struct) topicInfos[i];
- short topicError = topicInfo.getShort("topic_error_code");
- String topic = topicInfo.getString("topic");
+ short topicError = topicInfo.getShort(TOPIC_ERROR_CODE_KEY_NAME);
+ String topic = topicInfo.getString(TOPIC_KEY_NAME);
if (topicError == Errors.NONE.code()) {
- Object[] partitionInfos = (Object[]) topicInfo.get("partition_metadata");
+ Object[] partitionInfos = (Object[]) topicInfo.get(PARTITION_METADATA_KEY_NAME);
for (int j = 0; j < partitionInfos.length; j++) {
Struct partitionInfo = (Struct) partitionInfos[j];
- short partError = partitionInfo.getShort("partition_error_code");
+ short partError = partitionInfo.getShort(PARTITION_ERROR_CODE_KEY_NAME);
if (partError == Errors.NONE.code()) {
- int partition = partitionInfo.getInt("partition_id");
- int leader = partitionInfo.getInt("leader");
+ int partition = partitionInfo.getInt(PARTITION_KEY_NAME);
+ int leader = partitionInfo.getInt(LEADER_KEY_NAME);
Node leaderNode = leader == -1 ? null : brokers.get(leader);
- Object[] replicas = (Object[]) partitionInfo.get("replicas");
+ Object[] replicas = (Object[]) partitionInfo.get(REPLICAS_KEY_NAME);
Node[] replicaNodes = new Node[replicas.length];
for (int k = 0; k < replicas.length; k++)
replicaNodes[k] = brokers.get(replicas[k]);
- Object[] isr = (Object[]) partitionInfo.get("isr");
+ Object[] isr = (Object[]) partitionInfo.get(ISR_KEY_NAME);
Node[] isrNodes = new Node[isr.length];
for (int k = 0; k < isr.length; k++)
isrNodes[k] = brokers.get(isr[k]);
@@ -86,4 +149,7 @@ public class MetadataResponse {
return this.cluster;
}
+ public static MetadataResponse parse(ByteBuffer buffer) {
+ return new MetadataResponse(((Struct) curSchema.read(buffer)));
+ }
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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
new file mode 100644
index 0000000..3ee5cba
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java
@@ -0,0 +1,180 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+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;
+
+/**
+ * This wrapper supports both v0 and v1 of OffsetCommitRequest.
+ */
+public class OffsetCommitRequest extends AbstractRequestResponse {
+ public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_COMMIT.id);
+ private static String GROUP_ID_KEY_NAME = "group_id";
+ private static String GENERATION_ID_KEY_NAME = "group_generation_id";
+ private static String CONSUMER_ID_KEY_NAME = "consumer_id";
+ private static String TOPICS_KEY_NAME = "topics";
+
+ // topic level field names
+ private static String TOPIC_KEY_NAME = "topic";
+ private static String PARTITIONS_KEY_NAME = "partitions";
+
+ // partition level field names
+ private static String PARTITION_KEY_NAME = "partition";
+ private static String COMMIT_OFFSET_KEY_NAME = "offset";
+ private static String TIMESTAMP_KEY_NAME = "timestamp";
+ private static String METADATA_KEY_NAME = "metadata";
+
+ public static final int DEFAULT_GENERATION_ID = -1;
+ public static final String DEFAULT_CONSUMER_ID = "";
+
+ private final String groupId;
+ private final int generationId;
+ private final String consumerId;
+ private final Map<TopicPartition, PartitionData> offsetData;
+
+ public static final class PartitionData {
+ public final long offset;
+ public final long timestamp;
+ public final String metadata;
+
+ public PartitionData(long offset, long timestamp, String metadata) {
+ this.offset = offset;
+ this.timestamp = timestamp;
+ this.metadata = metadata;
+ }
+ }
+
+ /**
+ * Constructor for version 0.
+ * @param groupId
+ * @param offsetData
+ */
+ @Deprecated
+ public OffsetCommitRequest(String groupId, Map<TopicPartition, PartitionData> offsetData) {
+ super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 0)));
+ initCommonFields(groupId, offsetData);
+ this.groupId = groupId;
+ this.generationId = DEFAULT_GENERATION_ID;
+ this.consumerId = DEFAULT_CONSUMER_ID;
+ this.offsetData = offsetData;
+ }
+
+ /**
+ * Constructor for version 1.
+ * @param groupId
+ * @param generationId
+ * @param consumerId
+ * @param offsetData
+ */
+ public OffsetCommitRequest(String groupId, int generationId, String consumerId, Map<TopicPartition, PartitionData> offsetData) {
+ super(new Struct(curSchema));
+
+ initCommonFields(groupId, offsetData);
+ struct.set(GENERATION_ID_KEY_NAME, generationId);
+ struct.set(CONSUMER_ID_KEY_NAME, consumerId);
+ this.groupId = groupId;
+ this.generationId = generationId;
+ this.consumerId = consumerId;
+ this.offsetData = offsetData;
+ }
+
+ private void initCommonFields(String groupId, Map<TopicPartition, PartitionData> offsetData) {
+ Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(offsetData);
+
+ struct.set(GROUP_ID_KEY_NAME, groupId);
+ List<Struct> topicArray = new ArrayList<Struct>();
+ for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
+ Struct topicData = struct.instance(TOPICS_KEY_NAME);
+ topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
+ List<Struct> partitionArray = new ArrayList<Struct>();
+ 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(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset);
+ partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionData.timestamp);
+ partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata);
+ partitionArray.add(partitionData);
+ }
+ topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+ topicArray.add(topicData);
+ }
+ struct.set(TOPICS_KEY_NAME, topicArray.toArray());
+ }
+
+ public OffsetCommitRequest(Struct struct) {
+ super(struct);
+ offsetData = new HashMap<TopicPartition, PartitionData>();
+ for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
+ Struct topicResponse = (Struct) topicResponseObj;
+ String topic = topicResponse.getString(TOPIC_KEY_NAME);
+ for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+ Struct partitionResponse = (Struct) partitionResponseObj;
+ int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+ long offset = partitionResponse.getLong(COMMIT_OFFSET_KEY_NAME);
+ long timestamp = partitionResponse.getLong(TIMESTAMP_KEY_NAME);
+ String metadata = partitionResponse.getString(METADATA_KEY_NAME);
+ PartitionData partitionData = new PartitionData(offset, timestamp, metadata);
+ offsetData.put(new TopicPartition(topic, partition), partitionData);
+ }
+ }
+ groupId = struct.getString(GROUP_ID_KEY_NAME);
+ // This field only exists in v1.
+ if (struct.hasField(GENERATION_ID_KEY_NAME))
+ generationId = struct.getInt(GENERATION_ID_KEY_NAME);
+ else
+ generationId = DEFAULT_GENERATION_ID;
+
+ // This field only exists in v1.
+ if (struct.hasField(CONSUMER_ID_KEY_NAME))
+ consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
+ else
+ consumerId = DEFAULT_CONSUMER_ID;
+ }
+
+ public String groupId() {
+ return groupId;
+ }
+
+ public int generationId() {
+ return generationId;
+ }
+
+ public String consumerId() {
+ return consumerId;
+ }
+
+ public Map<TopicPartition, PartitionData> offsetData() {
+ return offsetData;
+ }
+
+ public static OffsetCommitRequest parse(ByteBuffer buffer, int versionId) {
+ Schema schema = ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, versionId);
+ return new OffsetCommitRequest(((Struct) schema.read(buffer)));
+ }
+
+ public static OffsetCommitRequest parse(ByteBuffer buffer) {
+ return new OffsetCommitRequest(((Struct) curSchema.read(buffer)));
+ }
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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
new file mode 100644
index 0000000..711232a
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+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;
+
+public class OffsetCommitResponse extends AbstractRequestResponse {
+ public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_COMMIT.id);
+ private static String RESPONSES_KEY_NAME = "responses";
+
+ // topic level fields
+ private static String TOPIC_KEY_NAME = "topic";
+ private static String PARTITIONS_KEY_NAME = "partition_responses";
+
+ // partition level fields
+ private static String PARTITION_KEY_NAME = "partition";
+ private static String ERROR_CODE_KEY_NAME = "error_code";
+
+ private final Map<TopicPartition, Short> responseData;
+
+ public OffsetCommitResponse(Map<TopicPartition, Short> responseData) {
+ super(new Struct(curSchema));
+
+ Map<String, Map<Integer, Short>> topicsData = CollectionUtils.groupDataByTopic(responseData);
+
+ List<Struct> topicArray = new ArrayList<Struct>();
+ for (Map.Entry<String, Map<Integer, Short>> entries: topicsData.entrySet()) {
+ Struct topicData = struct.instance(RESPONSES_KEY_NAME);
+ topicData.set(TOPIC_KEY_NAME, entries.getKey());
+ List<Struct> partitionArray = new ArrayList<Struct>();
+ for (Map.Entry<Integer, Short> 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());
+ partitionArray.add(partitionData);
+ }
+ topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+ topicArray.add(topicData);
+ }
+ struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
+ this.responseData = responseData;
+ }
+
+ public OffsetCommitResponse(Struct struct) {
+ super(struct);
+ responseData = new HashMap<TopicPartition, Short>();
+ for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
+ Struct topicResponse = (Struct) topicResponseObj;
+ String topic = topicResponse.getString(TOPIC_KEY_NAME);
+ for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+ Struct partitionResponse = (Struct) partitionResponseObj;
+ int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+ short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME);
+ responseData.put(new TopicPartition(topic, partition), errorCode);
+ }
+ }
+ }
+
+ public Map<TopicPartition, Short> responseData() {
+ return responseData;
+ }
+
+ public static OffsetCommitResponse parse(ByteBuffer buffer) {
+ return new OffsetCommitResponse(((Struct) curSchema.read(buffer)));
+ }
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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
new file mode 100644
index 0000000..90d5135
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * This wrapper supports both v0 and v1 of OffsetFetchRequest.
+ */
+public class OffsetFetchRequest extends AbstractRequestResponse {
+ public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_FETCH.id);
+ private static String GROUP_ID_KEY_NAME = "group_id";
+ private static String TOPICS_KEY_NAME = "topics";
+
+ // topic level field names
+ private static String TOPIC_KEY_NAME = "topic";
+ private static String PARTITIONS_KEY_NAME = "partitions";
+
+ // partition level field names
+ private static String PARTITION_KEY_NAME = "partition";
+
+ public static final int DEFAULT_GENERATION_ID = -1;
+ public static final String DEFAULT_CONSUMER_ID = "";
+
+ private final String groupId;
+ private final List<TopicPartition> partitions;
+
+ public OffsetFetchRequest(String groupId, List<TopicPartition> partitions) {
+ super(new Struct(curSchema));
+
+ Map<String, List<Integer>> topicsData = CollectionUtils.groupDataByTopic(partitions);
+
+ struct.set(GROUP_ID_KEY_NAME, groupId);
+ List<Struct> topicArray = new ArrayList<Struct>();
+ for (Map.Entry<String, List<Integer>> entries: topicsData.entrySet()) {
+ Struct topicData = struct.instance(TOPICS_KEY_NAME);
+ topicData.set(TOPIC_KEY_NAME, entries.getKey());
+ List<Struct> partitionArray = new ArrayList<Struct>();
+ for (Integer partiitonId : entries.getValue()) {
+ Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+ partitionData.set(PARTITION_KEY_NAME, partiitonId);
+ partitionArray.add(partitionData);
+ }
+ topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+ topicArray.add(topicData);
+ }
+ struct.set(TOPICS_KEY_NAME, topicArray.toArray());
+ this.groupId = groupId;
+ this.partitions = partitions;
+ }
+
+ public OffsetFetchRequest(Struct struct) {
+ super(struct);
+ partitions = new ArrayList<TopicPartition>();
+ for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
+ Struct topicResponse = (Struct) topicResponseObj;
+ String topic = topicResponse.getString(TOPIC_KEY_NAME);
+ for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+ Struct partitionResponse = (Struct) partitionResponseObj;
+ int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+ partitions.add(new TopicPartition(topic, partition));
+ }
+ }
+ groupId = struct.getString(GROUP_ID_KEY_NAME);
+ }
+
+ public String groupId() {
+ return groupId;
+ }
+
+ public List<TopicPartition> partitions() {
+ return partitions;
+ }
+
+ public static OffsetFetchRequest parse(ByteBuffer buffer) {
+ return new OffsetFetchRequest(((Struct) curSchema.read(buffer)));
+ }
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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
new file mode 100644
index 0000000..6b7c269
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
@@ -0,0 +1,107 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+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;
+
+public class OffsetFetchResponse extends AbstractRequestResponse {
+ public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_FETCH.id);
+ private static String RESPONSES_KEY_NAME = "responses";
+
+ // topic level fields
+ private static String TOPIC_KEY_NAME = "topic";
+ private static String PARTITIONS_KEY_NAME = "partition_responses";
+
+ // partition level fields
+ private static String PARTITION_KEY_NAME = "partition";
+ private static String COMMIT_OFFSET_KEY_NAME = "offset";
+ private static String METADATA_KEY_NAME = "metadata";
+ private static String ERROR_CODE_KEY_NAME = "error_code";
+
+ private final Map<TopicPartition,PartitionData> responseData;
+
+ public static final class PartitionData {
+ public final long offset;
+ public final String metadata;
+ public final short errorCode;
+
+ public PartitionData(long offset, String metadata, short errorCode) {
+ this.offset = offset;
+ this.metadata = metadata;
+ this.errorCode = errorCode;
+ }
+ }
+
+ public OffsetFetchResponse(Map<TopicPartition, PartitionData> responseData) {
+ super(new Struct(curSchema));
+
+ Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(responseData);
+
+ List<Struct> topicArray = new ArrayList<Struct>();
+ for (Map.Entry<String, Map<Integer, PartitionData>> entries: topicsData.entrySet()) {
+ Struct topicData = struct.instance(RESPONSES_KEY_NAME);
+ topicData.set(TOPIC_KEY_NAME, entries.getKey());
+ List<Struct> partitionArray = new ArrayList<Struct>();
+ 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(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset);
+ partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata);
+ partitionData.set(ERROR_CODE_KEY_NAME, fetchPartitionData.errorCode);
+ partitionArray.add(partitionData);
+ }
+ topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+ topicArray.add(topicData);
+ }
+ struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
+ this.responseData = responseData;
+ }
+
+ public OffsetFetchResponse(Struct struct) {
+ super(struct);
+ responseData = new HashMap<TopicPartition, PartitionData>();
+ for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
+ Struct topicResponse = (Struct) topicResponseObj;
+ String topic = topicResponse.getString(TOPIC_KEY_NAME);
+ for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+ Struct partitionResponse = (Struct) partitionResponseObj;
+ int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+ long offset = partitionResponse.getLong(COMMIT_OFFSET_KEY_NAME);
+ String metadata = partitionResponse.getString(METADATA_KEY_NAME);
+ short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME);
+ PartitionData partitionData = new PartitionData(offset, metadata, errorCode);
+ responseData.put(new TopicPartition(topic, partition), partitionData);
+ }
+ }
+ }
+
+ public Map<TopicPartition, PartitionData> responseData() {
+ return responseData;
+ }
+
+ public static OffsetFetchResponse parse(ByteBuffer buffer) {
+ return new OffsetFetchResponse(((Struct) curSchema.read(buffer)));
+ }
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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 6036f6a..3dbba8a 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
@@ -1,71 +1,105 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+
package org.apache.kafka.common.requests;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+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 org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.protocol.ApiKeys;
-import org.apache.kafka.common.protocol.ProtoUtils;
-import org.apache.kafka.common.protocol.types.Struct;
-import org.apache.kafka.common.record.MemoryRecords;
+public class ProduceRequest extends AbstractRequestResponse {
+ public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id);
+ private static String ACKS_KEY_NAME = "acks";
+ private static String TIMEOUT_KEY_NAME = "timeout";
+ private static String TOPIC_DATA_KEY_NAME = "topic_data";
+
+ // topic level field names
+ private static String TOPIC_KEY_NAME = "topic";
+ private static String PARTITION_DATA_KEY_NAME = "data";
-public class ProduceRequest {
+ // partition level field names
+ private static String PARTITION_KEY_NAME = "partition";
+ private static String RECORD_SET_KEY_NAME = "record_set";
private final short acks;
private final int timeout;
- private final Map<String, List<PartitionRecords>> records;
+ private final Map<TopicPartition, ByteBuffer> partitionRecords;
- public ProduceRequest(short acks, int timeout) {
+ public ProduceRequest(short acks, int timeout, Map<TopicPartition, ByteBuffer> partitionRecords) {
+ super(new Struct(curSchema));
+ Map<String, Map<Integer, ByteBuffer>> recordsByTopic = CollectionUtils.groupDataByTopic(partitionRecords);
+ struct.set(ACKS_KEY_NAME, acks);
+ struct.set(TIMEOUT_KEY_NAME, timeout);
+ List<Struct> topicDatas = new ArrayList<Struct>(recordsByTopic.size());
+ for (Map.Entry<String, Map<Integer, ByteBuffer>> entry : recordsByTopic.entrySet()) {
+ Struct topicData = struct.instance(TOPIC_DATA_KEY_NAME);
+ topicData.set(TOPIC_KEY_NAME, entry.getKey());
+ List<Struct> partitionArray = new ArrayList<Struct>();
+ for (Map.Entry<Integer, ByteBuffer> partitionEntry : entry.getValue().entrySet()) {
+ ByteBuffer buffer = partitionEntry.getValue().duplicate();
+ Struct part = topicData.instance(PARTITION_DATA_KEY_NAME)
+ .set(PARTITION_KEY_NAME, partitionEntry.getKey())
+ .set(RECORD_SET_KEY_NAME, buffer);
+ partitionArray.add(part);
+ }
+ topicData.set(PARTITION_DATA_KEY_NAME, partitionArray.toArray());
+ topicDatas.add(topicData);
+ }
+ struct.set(TOPIC_DATA_KEY_NAME, topicDatas.toArray());
this.acks = acks;
this.timeout = timeout;
- this.records = new HashMap<String, List<PartitionRecords>>();
+ this.partitionRecords = partitionRecords;
}
- public void add(TopicPartition tp, MemoryRecords recs) {
- List<PartitionRecords> found = this.records.get(tp.topic());
- if (found == null) {
- found = new ArrayList<PartitionRecords>();
- records.put(tp.topic(), found);
+ public ProduceRequest(Struct struct) {
+ super(struct);
+ partitionRecords = new HashMap<TopicPartition, ByteBuffer>();
+ for (Object topicDataObj : struct.getArray(TOPIC_DATA_KEY_NAME)) {
+ Struct topicData = (Struct) topicDataObj;
+ String topic = topicData.getString(TOPIC_KEY_NAME);
+ for (Object partitionResponseObj : topicData.getArray(PARTITION_DATA_KEY_NAME)) {
+ Struct partitionResponse = (Struct) partitionResponseObj;
+ int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+ ByteBuffer records = partitionResponse.getBytes(RECORD_SET_KEY_NAME);
+ partitionRecords.put(new TopicPartition(topic, partition), records);
+ }
}
- found.add(new PartitionRecords(tp, recs));
+ acks = struct.getShort(ACKS_KEY_NAME);
+ timeout = struct.getInt(TIMEOUT_KEY_NAME);
}
- public Struct toStruct() {
- Struct produce = new Struct(ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id));
- produce.set("acks", acks);
- produce.set("timeout", timeout);
- List<Struct> topicDatas = new ArrayList<Struct>(records.size());
- for (Map.Entry<String, List<PartitionRecords>> entry : records.entrySet()) {
- Struct topicData = produce.instance("topic_data");
- topicData.set("topic", entry.getKey());
- List<PartitionRecords> parts = entry.getValue();
- Object[] partitionData = new Object[parts.size()];
- for (int i = 0; i < parts.size(); i++) {
- ByteBuffer buffer = parts.get(i).records.buffer();
- buffer.flip();
- Struct part = topicData.instance("data")
- .set("partition", parts.get(i).topicPartition.partition())
- .set("record_set", buffer);
- partitionData[i] = part;
- }
- topicData.set("data", partitionData);
- topicDatas.add(topicData);
- }
- produce.set("topic_data", topicDatas.toArray());
- return produce;
+ public short acks() {
+ return acks;
}
- private static final class PartitionRecords {
- public final TopicPartition topicPartition;
- public final MemoryRecords records;
+ public int timeout() {
+ return timeout;
+ }
- public PartitionRecords(TopicPartition topicPartition, MemoryRecords records) {
- this.topicPartition = topicPartition;
- this.records = records;
- }
+ public Map<TopicPartition, ByteBuffer> partitionRecords() {
+ return partitionRecords;
}
+ public static ProduceRequest parse(ByteBuffer buffer) {
+ return new ProduceRequest(((Struct) curSchema.read(buffer)));
+ }
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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 6cf4fb7..5220464 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
@@ -12,67 +12,83 @@
*/
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.ProtoUtils;
+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 org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.protocol.types.Struct;
+public class ProduceResponse extends AbstractRequestResponse {
+ private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id);
+ private static String RESPONSES_KEY_NAME = "responses";
+
+ // topic level field names
+ private static String TOPIC_KEY_NAME = "topic";
+ private static String PARTITION_RESPONSES_KEY_NAME = "partition_responses";
-public class ProduceResponse {
+ // partition level field names
+ private static String PARTITION_KEY_NAME = "partition";
+ private static String ERROR_CODE_KEY_NAME = "error_code";
+ private static String BASE_OFFSET_KEY_NAME = "base_offset";
private final Map<TopicPartition, PartitionResponse> responses;
- public ProduceResponse() {
- this.responses = new HashMap<TopicPartition, PartitionResponse>();
+ public ProduceResponse(Map<TopicPartition, PartitionResponse> responses) {
+ super(new Struct(curSchema));
+ Map<String, Map<Integer, PartitionResponse>> responseByTopic = CollectionUtils.groupDataByTopic(responses);
+ List<Struct> topicDatas = new ArrayList<Struct>(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());
+ List<Struct> partitionArray = new ArrayList<Struct>();
+ for (Map.Entry<Integer, PartitionResponse> partitionEntry : entry.getValue().entrySet()) {
+ PartitionResponse part = partitionEntry.getValue();
+ Struct partStruct = topicData.instance(PARTITION_RESPONSES_KEY_NAME)
+ .set(PARTITION_KEY_NAME, partitionEntry.getKey())
+ .set(ERROR_CODE_KEY_NAME, part.errorCode)
+ .set(BASE_OFFSET_KEY_NAME, part.baseOffset);
+ partitionArray.add(partStruct);
+ }
+ topicData.set(PARTITION_RESPONSES_KEY_NAME, partitionArray.toArray());
+ topicDatas.add(topicData);
+ }
+ struct.set(RESPONSES_KEY_NAME, topicDatas.toArray());
+ this.responses = responses;
}
public ProduceResponse(Struct struct) {
+ super(struct);
responses = new HashMap<TopicPartition, PartitionResponse>();
- for (Object topicResponse : (Object[]) struct.get("responses")) {
+ for (Object topicResponse : struct.getArray("responses")) {
Struct topicRespStruct = (Struct) topicResponse;
- String topic = (String) topicRespStruct.get("topic");
- for (Object partResponse : (Object[]) topicRespStruct.get("partition_responses")) {
+ String topic = topicRespStruct.getString("topic");
+ for (Object partResponse : topicRespStruct.getArray("partition_responses")) {
Struct partRespStruct = (Struct) partResponse;
- int partition = (Integer) partRespStruct.get("partition");
- short errorCode = (Short) partRespStruct.get("error_code");
- long offset = (Long) partRespStruct.get("base_offset");
+ int partition = partRespStruct.getInt("partition");
+ short errorCode = partRespStruct.getShort("error_code");
+ long offset = partRespStruct.getLong("base_offset");
TopicPartition tp = new TopicPartition(topic, partition);
- responses.put(tp, new PartitionResponse(partition, errorCode, offset));
+ responses.put(tp, new PartitionResponse(errorCode, offset));
}
}
}
- public void addResponse(TopicPartition tp, int partition, short error, long baseOffset) {
- this.responses.put(tp, new PartitionResponse(partition, error, baseOffset));
- }
-
public Map<TopicPartition, PartitionResponse> responses() {
return this.responses;
}
- @Override
- public String toString() {
- StringBuilder b = new StringBuilder();
- b.append('{');
- boolean isFirst = true;
- for (Map.Entry<TopicPartition, PartitionResponse> entry : responses.entrySet()) {
- if (isFirst)
- isFirst = false;
- else
- b.append(',');
- b.append(entry.getKey() + " : " + entry.getValue());
- }
- b.append('}');
- return b.toString();
- }
-
- public static class PartitionResponse {
- public int partitionId;
+ public static final class PartitionResponse {
public short errorCode;
public long baseOffset;
- public PartitionResponse(int partitionId, short errorCode, long baseOffset) {
- this.partitionId = partitionId;
+ public PartitionResponse(short errorCode, long baseOffset) {
this.errorCode = errorCode;
this.baseOffset = baseOffset;
}
@@ -81,9 +97,7 @@ public class ProduceResponse {
public String toString() {
StringBuilder b = new StringBuilder();
b.append('{');
- b.append("pid: ");
- b.append(partitionId);
- b.append(",error: ");
+ b.append("error: ");
b.append(errorCode);
b.append(",offset: ");
b.append(baseOffset);
@@ -91,4 +105,8 @@ public class ProduceResponse {
return b.toString();
}
}
+
+ public static ProduceResponse parse(ByteBuffer buffer) {
+ return new ProduceResponse(((Struct) curSchema.read(buffer)));
+ }
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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 66cc2fe..f459a2a 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
@@ -24,18 +24,24 @@ import org.apache.kafka.common.protocol.types.Struct;
/**
* The header for a request in the Kafka protocol
*/
-public class RequestHeader {
+public class RequestHeader extends AbstractRequestResponse {
private static Field API_KEY_FIELD = REQUEST_HEADER.get("api_key");
private static Field API_VERSION_FIELD = REQUEST_HEADER.get("api_version");
private static Field CLIENT_ID_FIELD = REQUEST_HEADER.get("client_id");
private static Field CORRELATION_ID_FIELD = REQUEST_HEADER.get("correlation_id");
- private final Struct header;
+ private final short apiKey;
+ private final short apiVersion;
+ private final String clientId;
+ private final int correlationId;
public RequestHeader(Struct header) {
- super();
- this.header = header;
+ super(header);
+ apiKey = struct.getShort(API_KEY_FIELD);
+ apiVersion = struct.getShort(API_VERSION_FIELD);
+ clientId = struct.getString(CLIENT_ID_FIELD);
+ correlationId = struct.getInt(CORRELATION_ID_FIELD);
}
public RequestHeader(short apiKey, String client, int correlation) {
@@ -43,43 +49,34 @@ public class RequestHeader {
}
public RequestHeader(short apiKey, short version, String client, int correlation) {
- this(new Struct(Protocol.REQUEST_HEADER));
- this.header.set(API_KEY_FIELD, apiKey);
- this.header.set(API_VERSION_FIELD, version);
- this.header.set(CLIENT_ID_FIELD, client);
- this.header.set(CORRELATION_ID_FIELD, correlation);
+ super(new Struct(Protocol.REQUEST_HEADER));
+ struct.set(API_KEY_FIELD, apiKey);
+ struct.set(API_VERSION_FIELD, version);
+ struct.set(CLIENT_ID_FIELD, client);
+ struct.set(CORRELATION_ID_FIELD, correlation);
+ this.apiKey = apiKey;
+ this.apiVersion = version;
+ this.clientId = client;
+ this.correlationId = correlation;
}
public short apiKey() {
- return (Short) this.header.get(API_KEY_FIELD);
+ return apiKey;
}
public short apiVersion() {
- return (Short) this.header.get(API_VERSION_FIELD);
+ return apiVersion;
}
public String clientId() {
- return (String) this.header.get(CLIENT_ID_FIELD);
+ return clientId;
}
public int correlationId() {
- return (Integer) this.header.get(CORRELATION_ID_FIELD);
+ return correlationId;
}
public static RequestHeader parse(ByteBuffer buffer) {
return new RequestHeader((Struct) Protocol.REQUEST_HEADER.read(buffer));
}
-
- public void writeTo(ByteBuffer buffer) {
- header.writeTo(buffer);
- }
-
- public int sizeOf() {
- return header.sizeOf();
- }
-
- @Override
- public String toString() {
- return header.toString();
- }
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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 257b828..dd63853 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
@@ -28,31 +28,25 @@ import org.apache.kafka.common.protocol.types.Struct;
/**
* A response header in the kafka protocol.
*/
-public class ResponseHeader {
+public class ResponseHeader extends AbstractRequestResponse {
private static Field CORRELATION_KEY_FIELD = RESPONSE_HEADER.get("correlation_id");
- private final Struct header;
+ private final int correlationId;
public ResponseHeader(Struct header) {
- this.header = header;
+ super(header);
+ correlationId = struct.getInt(CORRELATION_KEY_FIELD);
}
public ResponseHeader(int correlationId) {
- this(new Struct(Protocol.RESPONSE_HEADER));
- this.header.set(CORRELATION_KEY_FIELD, correlationId);
+ super(new Struct(Protocol.RESPONSE_HEADER));
+ struct.set(CORRELATION_KEY_FIELD, correlationId);
+ this.correlationId = correlationId;
}
public int correlationId() {
- return (Integer) header.get(CORRELATION_KEY_FIELD);
- }
-
- public void writeTo(ByteBuffer buffer) {
- header.writeTo(buffer);
- }
-
- public int sizeOf() {
- return header.sizeOf();
+ return correlationId;
}
public static ResponseHeader parse(ByteBuffer buffer) {
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java b/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java
new file mode 100644
index 0000000..ba38637
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.utils;
+
+import org.apache.kafka.common.TopicPartition;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class CollectionUtils {
+ /**
+ * group data by topic
+ * @param data Data to be partitioned
+ * @param <T> Partition data type
+ * @return partitioned data
+ */
+ public static <T> Map<String, Map<Integer, T>> groupDataByTopic(Map<TopicPartition, T> data) {
+ Map<String, Map<Integer, T>> dataByTopic = new HashMap<String, Map<Integer, T>>();
+ for (Map.Entry<TopicPartition, T> entry: data.entrySet()) {
+ String topic = entry.getKey().topic();
+ int partition = entry.getKey().partition();
+ Map<Integer, T> topicData = dataByTopic.get(topic);
+ if (topicData == null) {
+ topicData = new HashMap<Integer, T>();
+ dataByTopic.put(topic, topicData);
+ }
+ topicData.put(partition, entry.getValue());
+ }
+ return dataByTopic;
+ }
+
+ /**
+ * group partitions by topic
+ * @param partitions
+ * @return partitions per topic
+ */
+ public static Map<String, List<Integer>> groupDataByTopic(List<TopicPartition> partitions) {
+ Map<String, List<Integer>> partitionsByTopic = new HashMap<String, List<Integer>>();
+ for (TopicPartition tp: partitions) {
+ String topic = tp.topic();
+ List<Integer> topicData = partitionsByTopic.get(topic);
+ if (topicData == null) {
+ topicData = new ArrayList<Integer>();
+ partitionsByTopic.put(topic, topicData);
+ }
+ topicData.add(tp.partition());
+ }
+ return partitionsByTopic;
+ }
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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 2f98192..1a55242 100644
--- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
@@ -7,11 +7,13 @@ import static org.junit.Assert.assertTrue;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Collections;
import java.util.List;
import org.apache.kafka.clients.producer.internals.Metadata;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.network.NetworkReceive;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ProtoUtils;
@@ -68,7 +70,7 @@ public class NetworkClientTest {
@Test
public void testSimpleRequestResponse() {
- ProduceRequest produceRequest = new ProduceRequest((short) 1, 1000);
+ ProduceRequest produceRequest = new ProduceRequest((short) 1, 1000, Collections.<TopicPartition, ByteBuffer>emptyMap());
RequestHeader reqHeader = client.nextRequestHeader(ApiKeys.PRODUCE);
RequestSend send = new RequestSend(node.id(), reqHeader, produceRequest.toStruct());
ClientRequest request = new ClientRequest(time.milliseconds(), true, send, null);
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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
new file mode 100644
index 0000000..df37fc6
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
@@ -0,0 +1,173 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.junit.Test;
+
+import java.lang.reflect.Method;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class RequestResponseTest {
+
+ @Test
+ public void testSerialization() throws Exception{
+ List<AbstractRequestResponse> requestList = Arrays.asList(
+ createRequestHeader(),
+ createResponseHeader(),
+ createConsumerMetadataRequest(),
+ createConsumerMetadataResponse(),
+ createFetchRequest(),
+ createFetchResponse(),
+ createHeartBeatRequest(),
+ createHeartBeatResponse(),
+ createJoinGroupRequest(),
+ createJoinGroupResponse(),
+ createListOffsetRequest(),
+ createListOffsetResponse(),
+ createMetadataRequest(),
+ createMetadataResponse(),
+ createOffsetCommitRequest(),
+ createOffsetCommitResponse(),
+ createOffsetFetchRequest(),
+ createOffsetFetchResponse(),
+ createProduceRequest(),
+ createProduceResponse());
+
+ for (AbstractRequestResponse req: requestList) {
+ ByteBuffer buffer = ByteBuffer.allocate(req.sizeOf());
+ req.writeTo(buffer);
+ buffer.rewind();
+ Method deserializer = req.getClass().getDeclaredMethod("parse", ByteBuffer.class);
+ AbstractRequestResponse deserialized = (AbstractRequestResponse) deserializer.invoke(null, buffer);
+ assertEquals("The original and deserialized of " + req.getClass().getSimpleName() + " should be the same.", req, deserialized);
+ assertEquals("The original and deserialized of " + req.getClass().getSimpleName() + " should have the same hashcode.",
+ req.hashCode(), deserialized.hashCode());
+ }
+ }
+
+ private AbstractRequestResponse createRequestHeader() {
+ return new RequestHeader((short)10, (short)1, "", 10);
+ }
+
+ private AbstractRequestResponse createResponseHeader() {
+ return new ResponseHeader(10);
+ }
+
+ private AbstractRequestResponse createConsumerMetadataRequest() {
+ return new ConsumerMetadataRequest("test-group");
+ }
+
+ private AbstractRequestResponse createConsumerMetadataResponse() {
+ return new ConsumerMetadataResponse((short)1, new Node(10, "host1", 2014));
+ }
+
+ private AbstractRequestResponse createFetchRequest() {
+ Map<TopicPartition, FetchRequest.PartitionData> fetchData = new HashMap<TopicPartition, FetchRequest.PartitionData>();
+ fetchData.put(new TopicPartition("test1", 0), new FetchRequest.PartitionData(100, 1000000));
+ fetchData.put(new TopicPartition("test2", 0), new FetchRequest.PartitionData(200, 1000000));
+ return new FetchRequest(-1, 100, 100000, fetchData);
+ }
+
+ private AbstractRequestResponse createFetchResponse() {
+ Map<TopicPartition, FetchResponse.PartitionData> responseData = new HashMap<TopicPartition, FetchResponse.PartitionData>();
+ responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData((short)0, 1000000, ByteBuffer.allocate(10)));
+ return new FetchResponse(responseData);
+ }
+
+ private AbstractRequestResponse createHeartBeatRequest() {
+ return new HeartbeatRequest("group1", 1, "consumer1");
+ }
+
+ private AbstractRequestResponse createHeartBeatResponse() {
+ return new HeartbeatResponse((short)0);
+ }
+
+ private AbstractRequestResponse createJoinGroupRequest() {
+ return new JoinGroupRequest("group1", 30000, Arrays.asList("topic1"), "consumer1", "strategy1");
+ }
+
+ private AbstractRequestResponse createJoinGroupResponse() {
+ return new JoinGroupResponse((short)0, 1, "consumer1", Arrays.asList(new TopicPartition("test11", 1), new TopicPartition("test2", 1)));
+ }
+
+ private AbstractRequestResponse createListOffsetRequest() {
+ Map<TopicPartition, ListOffsetRequest.PartitionData> offsetData = new HashMap<TopicPartition, ListOffsetRequest.PartitionData>();
+ offsetData.put(new TopicPartition("test", 0), new ListOffsetRequest.PartitionData(1000000L, 10));
+ return new ListOffsetRequest(-1, offsetData);
+ }
+
+ private AbstractRequestResponse createListOffsetResponse() {
+ Map<TopicPartition, ListOffsetResponse.PartitionData> responseData = new HashMap<TopicPartition, ListOffsetResponse.PartitionData>();
+ responseData.put(new TopicPartition("test", 0), new ListOffsetResponse.PartitionData((short)0, Arrays.asList(100L)));
+ return new ListOffsetResponse(responseData);
+ }
+
+ private AbstractRequestResponse createMetadataRequest() {
+ return new MetadataRequest(Arrays.asList("topic1"));
+ }
+
+ private AbstractRequestResponse createMetadataResponse() {
+ Node node = new Node(1, "host1", 1001);
+ Node[] replicas = new Node[1];
+ replicas[0] = node;
+ Node[] isr = new Node[1];
+ isr[0] = node;
+ Cluster cluster = new Cluster(Arrays.asList(node), Arrays.asList(new PartitionInfo("topic1", 1, node, replicas, isr)));
+ return new MetadataResponse(cluster);
+ }
+
+ private AbstractRequestResponse createOffsetCommitRequest() {
+ Map<TopicPartition, OffsetCommitRequest.PartitionData> commitData = new HashMap<TopicPartition, OffsetCommitRequest.PartitionData>();
+ commitData.put(new TopicPartition("test", 0), new OffsetCommitRequest.PartitionData(100, 1000000, ""));
+ return new OffsetCommitRequest("group1", 100, "consumer1", commitData);
+ }
+
+ private AbstractRequestResponse createOffsetCommitResponse() {
+ Map<TopicPartition, Short> responseData = new HashMap<TopicPartition, Short>();
+ responseData.put(new TopicPartition("test", 0), (short)0);
+ return new OffsetCommitResponse(responseData);
+ }
+
+ private AbstractRequestResponse createOffsetFetchRequest() {
+ return new OffsetFetchRequest("group1", Arrays.asList(new TopicPartition("test11", 1)));
+ }
+
+ private AbstractRequestResponse createOffsetFetchResponse() {
+ Map<TopicPartition, OffsetFetchResponse.PartitionData> responseData = new HashMap<TopicPartition, OffsetFetchResponse.PartitionData>();
+ responseData.put(new TopicPartition("test", 0), new OffsetFetchResponse.PartitionData(100L, "", (short)0));
+ return new OffsetFetchResponse(responseData);
+ }
+
+ private AbstractRequestResponse createProduceRequest() {
+ Map<TopicPartition, ByteBuffer> produceData = new HashMap<TopicPartition, ByteBuffer>();
+ produceData.put(new TopicPartition("test", 0), ByteBuffer.allocate(10));
+ return new ProduceRequest((short)0, 5000, produceData);
+ }
+
+ private AbstractRequestResponse createProduceResponse() {
+ Map<TopicPartition, ProduceResponse.PartitionResponse> responseData = new HashMap<TopicPartition, ProduceResponse.PartitionResponse>();
+ responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse((short) 0, 10000));
+ return new ProduceResponse(responseData);
+ }
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala
index dfad6e6..6d00ed0 100644
--- a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala
+++ b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala
@@ -41,9 +41,9 @@ object ConsumerMetadataRequest {
case class ConsumerMetadataRequest(group: String,
versionId: Short = ConsumerMetadataRequest.CurrentVersion,
- override val correlationId: Int = 0,
+ correlationId: Int = 0,
clientId: String = ConsumerMetadataRequest.DefaultClientId)
- extends RequestOrResponse(Some(RequestKeys.ConsumerMetadataKey), correlationId) {
+ extends RequestOrResponse(Some(RequestKeys.ConsumerMetadataKey)) {
def sizeInBytes =
2 + /* versionId */
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
index c72ca14..84f6017 100644
--- a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
+++ b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
@@ -40,8 +40,8 @@ object ConsumerMetadataResponse {
}
-case class ConsumerMetadataResponse (coordinatorOpt: Option[Broker], errorCode: Short, override val correlationId: Int = 0)
- extends RequestOrResponse(correlationId = correlationId) {
+case class ConsumerMetadataResponse (coordinatorOpt: Option[Broker], errorCode: Short, correlationId: Int = 0)
+ extends RequestOrResponse() {
def sizeInBytes =
4 + /* correlationId */
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala b/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala
index 7dacb20..5be393a 100644
--- a/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala
+++ b/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala
@@ -38,9 +38,9 @@ object ControlledShutdownRequest extends Logging {
}
case class ControlledShutdownRequest(val versionId: Short,
- override val correlationId: Int,
+ val correlationId: Int,
val brokerId: Int)
- extends RequestOrResponse(Some(RequestKeys.ControlledShutdownKey), correlationId){
+ extends RequestOrResponse(Some(RequestKeys.ControlledShutdownKey)){
def this(correlationId: Int, brokerId: Int) =
this(ControlledShutdownRequest.CurrentVersion, correlationId, brokerId)
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala b/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala
index 46ec3db..5e0a1cf 100644
--- a/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala
+++ b/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala
@@ -39,10 +39,10 @@ object ControlledShutdownResponse {
}
-case class ControlledShutdownResponse(override val correlationId: Int,
+case class ControlledShutdownResponse(val correlationId: Int,
val errorCode: Short = ErrorMapping.NoError,
val partitionsRemaining: Set[TopicAndPartition])
- extends RequestOrResponse(correlationId = correlationId) {
+ extends RequestOrResponse() {
def sizeInBytes(): Int ={
var size =
4 /* correlation id */ +
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/FetchRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/FetchRequest.scala b/core/src/main/scala/kafka/api/FetchRequest.scala
index a8b73ac..55a5982 100644
--- a/core/src/main/scala/kafka/api/FetchRequest.scala
+++ b/core/src/main/scala/kafka/api/FetchRequest.scala
@@ -60,13 +60,13 @@ object FetchRequest {
}
case class FetchRequest private[kafka] (versionId: Short = FetchRequest.CurrentVersion,
- override val correlationId: Int = FetchRequest.DefaultCorrelationId,
+ correlationId: Int = FetchRequest.DefaultCorrelationId,
clientId: String = ConsumerConfig.DefaultClientId,
replicaId: Int = Request.OrdinaryConsumerId,
maxWait: Int = FetchRequest.DefaultMaxWait,
minBytes: Int = FetchRequest.DefaultMinBytes,
requestInfo: Map[TopicAndPartition, PartitionFetchInfo])
- extends RequestOrResponse(Some(RequestKeys.FetchKey), correlationId) {
+ extends RequestOrResponse(Some(RequestKeys.FetchKey)) {
/**
* Partitions the request info into a map of maps (one for each topic).
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala b/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala
new file mode 100644
index 0000000..fb022e8
--- /dev/null
+++ b/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+
+package kafka.api
+
+import java.nio.ByteBuffer
+import org.apache.kafka.common.requests.AbstractRequestResponse
+
+private[kafka] abstract class GenericRequestOrResponseAndHeader(val header: AbstractRequestResponse,
+ val body: AbstractRequestResponse,
+ val name: String,
+ override val requestId: Option[Short] = None)
+ extends RequestOrResponse(requestId) {
+
+ def writeTo(buffer: ByteBuffer) {
+ header.writeTo(buffer)
+ body.writeTo(buffer)
+ }
+
+ def sizeInBytes(): Int = {
+ header.sizeOf() + body.sizeOf();
+ }
+
+ override def toString(): String = {
+ describe(true)
+ }
+
+ override def describe(details: Boolean): String = {
+ val strBuffer = new StringBuilder
+ strBuffer.append("Name: " + name)
+ strBuffer.append("; header: " + header.toString)
+ strBuffer.append("; body: " + body.toString)
+ strBuffer.toString()
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala b/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala
new file mode 100644
index 0000000..932418b
--- /dev/null
+++ b/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+
+package kafka.api
+
+import java.nio.ByteBuffer
+import kafka.network.{BoundedByteBufferSend, RequestChannel}
+import kafka.common.ErrorMapping
+import kafka.network.RequestChannel.Response
+import org.apache.kafka.common.requests.{HeartbeatResponse, ResponseHeader, HeartbeatRequest, RequestHeader}
+
+object HeartbeatRequestAndHeader {
+ def readFrom(buffer: ByteBuffer): HeartbeatRequestAndHeader = {
+ val header = RequestHeader.parse(buffer)
+ val body = HeartbeatRequest.parse(buffer)
+ new HeartbeatRequestAndHeader(header, body)
+ }
+}
+
+case class HeartbeatRequestAndHeader(override val header: RequestHeader, override val body: HeartbeatRequest)
+ extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), Some(RequestKeys.HeartbeatKey)) {
+
+ override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
+ val errorResponseHeader = new ResponseHeader(header.correlationId)
+ val errorResponseBody = new HeartbeatResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
+ val errorHeartBeatResponseAndHeader = new HeartbeatResponseAndHeader(errorResponseHeader, errorResponseBody)
+ requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorHeartBeatResponseAndHeader)))
+ }
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala b/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala
new file mode 100644
index 0000000..556f38d
--- /dev/null
+++ b/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package kafka.api
+
+import org.apache.kafka.common.requests.{ResponseHeader, HeartbeatResponse}
+import java.nio.ByteBuffer
+
+object HeartbeatResponseAndHeader {
+ def readFrom(buffer: ByteBuffer): HeartbeatResponseAndHeader = {
+ val header = ResponseHeader.parse(buffer)
+ val body = HeartbeatResponse.parse(buffer)
+ new HeartbeatResponseAndHeader(header, body)
+ }
+}
+
+case class HeartbeatResponseAndHeader(override val header: ResponseHeader, override val body: HeartbeatResponse)
+ extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), None) {
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala b/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala
new file mode 100644
index 0000000..9aea28c
--- /dev/null
+++ b/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala
@@ -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 kafka.api
+
+import java.nio.ByteBuffer
+import kafka.network.{BoundedByteBufferSend, RequestChannel}
+import kafka.common.ErrorMapping
+import org.apache.kafka.common.requests._
+import kafka.network.RequestChannel.Response
+import scala.Some
+
+object JoinGroupRequestAndHeader {
+ def readFrom(buffer: ByteBuffer): JoinGroupRequestAndHeader = {
+ val header = RequestHeader.parse(buffer)
+ val body = JoinGroupRequest.parse(buffer)
+ new JoinGroupRequestAndHeader(header, body)
+ }
+}
+
+case class JoinGroupRequestAndHeader(override val header: RequestHeader, override val body: JoinGroupRequest)
+ extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), Some(RequestKeys.JoinGroupKey)) {
+
+ override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
+ val errorResponseHeader = new ResponseHeader(header.correlationId)
+ val errorResponseBody = new JoinGroupResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
+ val errorHeartBeatResponseAndHeader = new JoinGroupResponseAndHeader(errorResponseHeader, errorResponseBody)
+ requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorHeartBeatResponseAndHeader)))
+ }
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala b/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala
new file mode 100644
index 0000000..7389ae6
--- /dev/null
+++ b/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package kafka.api
+
+import org.apache.kafka.common.requests.{JoinGroupResponse, ResponseHeader}
+import java.nio.ByteBuffer
+
+object JoinGroupResponseAndHeader {
+ def readFrom(buffer: ByteBuffer): JoinGroupResponseAndHeader = {
+ val header = ResponseHeader.parse(buffer)
+ val body = JoinGroupResponse.parse(buffer)
+ new JoinGroupResponseAndHeader(header, body)
+ }
+}
+
+case class JoinGroupResponseAndHeader(override val header: ResponseHeader, override val body: JoinGroupResponse)
+ extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), None) {
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala
index 3e40817..4ff7e8f 100644
--- a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala
+++ b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala
@@ -129,13 +129,13 @@ object LeaderAndIsrRequest {
}
case class LeaderAndIsrRequest (versionId: Short,
- override val correlationId: Int,
+ correlationId: Int,
clientId: String,
controllerId: Int,
controllerEpoch: Int,
partitionStateInfos: Map[(String, Int), PartitionStateInfo],
leaders: Set[Broker])
- extends RequestOrResponse(Some(RequestKeys.LeaderAndIsrKey), correlationId) {
+ extends RequestOrResponse(Some(RequestKeys.LeaderAndIsrKey)) {
def this(partitionStateInfos: Map[(String, Int), PartitionStateInfo], leaders: Set[Broker], controllerId: Int,
controllerEpoch: Int, correlationId: Int, clientId: String) = {
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala b/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala
index f636444..22ce48a 100644
--- a/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala
+++ b/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala
@@ -41,10 +41,10 @@ object LeaderAndIsrResponse {
}
-case class LeaderAndIsrResponse(override val correlationId: Int,
+case class LeaderAndIsrResponse(correlationId: Int,
responseMap: Map[(String, Int), Short],
errorCode: Short = ErrorMapping.NoError)
- extends RequestOrResponse(correlationId = correlationId) {
+ extends RequestOrResponse() {
def sizeInBytes(): Int ={
var size =
4 /* correlation id */ +
[21/37] git commit: KAFKA-1180 WhiteList topic filter gets a
NullPointerException on complex Regex patch by Joe Stein,
reviewed by Joel Koshy
Posted by jj...@apache.org.
KAFKA-1180 WhiteList topic filter gets a NullPointerException on complex Regex patch by Joe Stein, reviewed by Joel Koshy
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/592678e4
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/592678e4
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/592678e4
Branch: refs/heads/transactional_messaging
Commit: 592678e4d72151940002ffb1a367ec433b27f2ef
Parents: fc0e03f
Author: Joe Stein <jo...@stealth.ly>
Authored: Sat Jul 19 21:00:22 2014 -0400
Committer: Joe Stein <jo...@stealth.ly>
Committed: Sat Jul 19 21:00:22 2014 -0400
----------------------------------------------------------------------
.../main/scala/kafka/consumer/TopicCount.scala | 4 +--
core/src/main/scala/kafka/utils/Utils.scala | 24 ++++++++++++++++++
.../unit/kafka/consumer/TopicFilterTest.scala | 26 ++++++++++++++++++++
3 files changed, 52 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/592678e4/core/src/main/scala/kafka/consumer/TopicCount.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/consumer/TopicCount.scala b/core/src/main/scala/kafka/consumer/TopicCount.scala
index c793110..8b0ae57 100644
--- a/core/src/main/scala/kafka/consumer/TopicCount.scala
+++ b/core/src/main/scala/kafka/consumer/TopicCount.scala
@@ -19,7 +19,7 @@ package kafka.consumer
import scala.collection._
import org.I0Itec.zkclient.ZkClient
-import kafka.utils.{Json, ZKGroupDirs, ZkUtils, Logging}
+import kafka.utils.{Json, ZKGroupDirs, ZkUtils, Logging, Utils}
import kafka.common.KafkaException
private[kafka] trait TopicCount {
@@ -127,7 +127,7 @@ private[kafka] class WildcardTopicCount(zkClient: ZkClient,
makeConsumerThreadIdsPerTopic(consumerIdString, Map(wildcardTopics.map((_, numStreams)): _*))
}
- def getTopicCountMap = Map(topicFilter.regex -> numStreams)
+ def getTopicCountMap = Map(Utils.JSONEscapeString(topicFilter.regex) -> numStreams)
def pattern: String = {
topicFilter match {
http://git-wip-us.apache.org/repos/asf/kafka/blob/592678e4/core/src/main/scala/kafka/utils/Utils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala
index 63d3dda..6576adf 100644
--- a/core/src/main/scala/kafka/utils/Utils.scala
+++ b/core/src/main/scala/kafka/utils/Utils.scala
@@ -545,4 +545,28 @@ object Utils extends Logging {
lock.unlock()
}
}
+
+ //JSON strings need to be escaped based on ECMA-404 standard http://json.org
+ def JSONEscapeString (s : String) : String = {
+ s.map {
+ case '"' => "\\\""
+ case '\\' => "\\\\"
+ case '/' => "\\/"
+ case '\b' => "\\b"
+ case '\f' => "\\f"
+ case '\n' => "\\n"
+ case '\r' => "\\r"
+ case '\t' => "\\t"
+ /* We'll unicode escape any control characters. These include:
+ * 0x0 -> 0x1f : ASCII Control (C0 Control Codes)
+ * 0x7f : ASCII DELETE
+ * 0x80 -> 0x9f : C1 Control Codes
+ *
+ * Per RFC4627, section 2.5, we're not technically required to
+ * encode the C1 codes, but we do to be safe.
+ */
+ case c if ((c >= '\u0000' && c <= '\u001f') || (c >= '\u007f' && c <= '\u009f')) => "\\u%04x".format(c: Int)
+ case c => c
+ }.mkString
+ }
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/592678e4/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala
index d903a6f..4f124af 100644
--- a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala
+++ b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala
@@ -43,6 +43,10 @@ class TopicFilterTest extends JUnitSuite {
val topicFilter3 = new Whitelist("white_listed-topic.+")
assertTrue(topicFilter3.isTopicAllowed("white_listed-topic1", excludeInternalTopics = true))
assertFalse(topicFilter3.isTopicAllowed("black1", excludeInternalTopics = true))
+
+ val topicFilter4 = new Whitelist("test-(?!bad\\b)[\\w]+")
+ assertTrue(topicFilter4.isTopicAllowed("test-good", excludeInternalTopics = true))
+ assertFalse(topicFilter4.isTopicAllowed("test-bad", excludeInternalTopics = true))
}
@Test
@@ -56,4 +60,26 @@ class TopicFilterTest extends JUnitSuite {
assertFalse(topicFilter1.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = true))
assertTrue(topicFilter1.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = false))
}
+
+ @Test
+ def testWildcardTopicCountGetTopicCountMapEscapeJson() {
+ def getTopicCountMapKey(regex: String): String = {
+ val topicCount = new WildcardTopicCount(null, "consumerId", new Whitelist(regex), 1, true)
+ topicCount.getTopicCountMap.head._1
+ }
+ //lets make sure that the JSON strings are escaping as we expect
+ //if they are not then when they get saved to zookeeper and read back out they will be broken on parse
+ assertEquals("-\\\"-", getTopicCountMapKey("-\"-"))
+ assertEquals("-\\\\-", getTopicCountMapKey("-\\-"))
+ assertEquals("-\\/-", getTopicCountMapKey("-/-"))
+ assertEquals("-\\\\b-", getTopicCountMapKey("-\\b-"))
+ assertEquals("-\\\\f-", getTopicCountMapKey("-\\f-"))
+ assertEquals("-\\\\n-", getTopicCountMapKey("-\\n-"))
+ assertEquals("-\\\\r-", getTopicCountMapKey("-\\r-"))
+ assertEquals("-\\\\t-", getTopicCountMapKey("-\\t-"))
+ assertEquals("-\\\\u0000-", getTopicCountMapKey("-\\u0000-"))
+ assertEquals("-\\\\u001f-", getTopicCountMapKey("-\\u001f-"))
+ assertEquals("-\\\\u007f-", getTopicCountMapKey("-\\u007f-"))
+ assertEquals("-\\\\u009f-", getTopicCountMapKey("-\\u009f-"))
+ }
}
\ No newline at end of file
[13/37] git commit: KAFKA-1258 Automatically delete temporary
directories with a shutdown hook during tests to avoid leaking temp files.
Patch by Manikumar Reddy.
Posted by jj...@apache.org.
KAFKA-1258 Automatically delete temporary directories with a shutdown hook during tests to avoid leaking temp files. Patch by Manikumar Reddy.
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/4b3d03e8
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/4b3d03e8
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/4b3d03e8
Branch: refs/heads/transactional_messaging
Commit: 4b3d03e8e0d660b27c7823c03dde075d6ce8348b
Parents: 83a9aa5
Author: Jay Kreps <ja...@gmail.com>
Authored: Sat Jul 12 12:49:04 2014 -0700
Committer: Jay Kreps <ja...@gmail.com>
Committed: Sat Jul 12 12:49:04 2014 -0700
----------------------------------------------------------------------
core/src/test/scala/unit/kafka/utils/TestUtils.scala | 7 +++++++
1 file changed, 7 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/4b3d03e8/core/src/test/scala/unit/kafka/utils/TestUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index 57b2bd5..3faa884 100644
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -84,6 +84,13 @@ object TestUtils extends Logging {
val f = new File(IoTmpDir, "kafka-" + random.nextInt(1000000))
f.mkdirs()
f.deleteOnExit()
+
+ Runtime.getRuntime().addShutdownHook(new Thread() {
+ override def run() = {
+ Utils.rm(f)
+ }
+ })
+
f
}
[37/37] git commit: KAFKA-1485 Upgrade to Zookeeper 3.4.6 patch by
Gwen Shapira reviewed by Joe Stein
Posted by jj...@apache.org.
KAFKA-1485 Upgrade to Zookeeper 3.4.6 patch by Gwen Shapira reviewed by Joe Stein
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/7a67a722
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/7a67a722
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/7a67a722
Branch: refs/heads/transactional_messaging
Commit: 7a67a72261bcb5b09d9defb2fd95a65c89e75172
Parents: 09690e3
Author: vagrant <vagrant@precise64.(none)>
Authored: Tue Aug 5 22:24:03 2014 +0000
Committer: vagrant <vagrant@precise64.(none)>
Committed: Tue Aug 5 22:24:03 2014 +0000
----------------------------------------------------------------------
build.gradle | 2 +-
core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala | 4 +++-
2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/7a67a722/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index a72905d..9cbedb8 100644
--- a/build.gradle
+++ b/build.gradle
@@ -215,7 +215,7 @@ project(':core') {
dependencies {
compile project(':clients')
compile "org.scala-lang:scala-library:$scalaVersion"
- compile 'org.apache.zookeeper:zookeeper:3.3.4'
+ compile 'org.apache.zookeeper:zookeeper:3.4.6'
compile 'com.101tec:zkclient:0.3'
compile 'com.yammer.metrics:metrics-core:2.2.0'
compile 'net.sf.jopt-simple:jopt-simple:3.2'
http://git-wip-us.apache.org/repos/asf/kafka/blob/7a67a722/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala b/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala
index d883bde..3021a8c 100644
--- a/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala
+++ b/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala
@@ -19,6 +19,7 @@ package kafka.zk
import org.apache.zookeeper.server.ZooKeeperServer
import org.apache.zookeeper.server.NIOServerCnxn
+import org.apache.zookeeper.server.NIOServerCnxnFactory
import kafka.utils.TestUtils
import java.net.InetSocketAddress
import kafka.utils.Utils
@@ -29,7 +30,8 @@ class EmbeddedZookeeper(val connectString: String) {
val tickTime = 500
val zookeeper = new ZooKeeperServer(snapshotDir, logDir, tickTime)
val port = connectString.split(":")(1).toInt
- val factory = new NIOServerCnxn.Factory(new InetSocketAddress("127.0.0.1", port))
+ val factory = new NIOServerCnxnFactory()
+ factory.configure(new InetSocketAddress("127.0.0.1", port),0)
factory.startup(zookeeper)
def shutdown() {
[12/37] git commit: KAFKA-1515 Producer can hang during metadata
updates. Patch by Guozhang.
Posted by jj...@apache.org.
KAFKA-1515 Producer can hang during metadata updates. Patch by Guozhang.
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/83a9aa55
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/83a9aa55
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/83a9aa55
Branch: refs/heads/transactional_messaging
Commit: 83a9aa55d340f6b2720394a49f01a88509e17e52
Parents: 8034390
Author: Jay Kreps <ja...@gmail.com>
Authored: Fri Jul 11 13:08:24 2014 -0700
Committer: Jay Kreps <ja...@gmail.com>
Committed: Fri Jul 11 13:08:24 2014 -0700
----------------------------------------------------------------------
.../org/apache/kafka/clients/NetworkClient.java | 33 +++++++++++++-------
.../clients/producer/internals/Metadata.java | 2 +-
.../main/scala/kafka/tools/MirrorMaker.scala | 31 +++++++++---------
3 files changed, 37 insertions(+), 29 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/83a9aa55/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 f739279..d8f9ce6 100644
--- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
@@ -122,16 +122,21 @@ public class NetworkClient implements KafkaClient {
*/
@Override
public boolean isReady(Node node, long now) {
- return isReady(node.id(), now);
- }
-
- private boolean isReady(int node, long now) {
+ int nodeId = node.id();
if (!this.metadataFetchInProgress && this.metadata.timeToNextUpdate(now) == 0)
// if we need to update our metadata now declare all requests unready to make metadata requests first priority
return false;
else
// otherwise we are ready if we are connected and can send more requests
- return connectionStates.isConnected(node) && inFlightRequests.canSendMore(node);
+ return isSendable(nodeId);
+ }
+
+ /**
+ * Are we connected and ready and able to send more requests to the given node?
+ * @param node The node
+ */
+ private boolean isSendable(int node) {
+ return connectionStates.isConnected(node) && inFlightRequests.canSendMore(node);
}
/**
@@ -146,21 +151,21 @@ public class NetworkClient implements KafkaClient {
public List<ClientResponse> poll(List<ClientRequest> requests, long timeout, long now) {
List<NetworkSend> sends = new ArrayList<NetworkSend>();
- // should we update our metadata?
- long metadataTimeout = metadata.timeToNextUpdate(now);
- if (!this.metadataFetchInProgress && metadataTimeout == 0)
- maybeUpdateMetadata(sends, now);
-
for (int i = 0; i < requests.size(); i++) {
ClientRequest request = requests.get(i);
int nodeId = request.request().destination();
- if (!isReady(nodeId, now))
+ if (!isSendable(nodeId))
throw new IllegalStateException("Attempt to send a request to node " + nodeId + " which is not ready.");
this.inFlightRequests.add(request);
sends.add(request.request());
}
+ // should we update our metadata?
+ long metadataTimeout = metadata.timeToNextUpdate(now);
+ if (!this.metadataFetchInProgress && metadataTimeout == 0)
+ maybeUpdateMetadata(sends, now);
+
// do the I/O
try {
this.selector.poll(Math.min(timeout, metadataTimeout), sends);
@@ -347,9 +352,12 @@ public class NetworkClient implements KafkaClient {
*/
private void maybeUpdateMetadata(List<NetworkSend> sends, long now) {
Node node = this.leastLoadedNode(now);
- if (node == null)
+ if (node == null) {
+ log.debug("Give up sending metadata request since no node is available");
return;
+ }
+ log.debug("Trying to send metadata request to node {}", node.id());
if (connectionStates.isConnected(node.id()) && inFlightRequests.canSendMore(node.id())) {
Set<String> topics = metadata.topics();
this.metadataFetchInProgress = true;
@@ -359,6 +367,7 @@ public class NetworkClient implements KafkaClient {
this.inFlightRequests.add(metadataRequest);
} else if (connectionStates.canConnect(node.id(), now)) {
// we don't have a connection to this node right now, make one
+ log.debug("Give up sending metadata request to node {} since it is either not connected or cannot have more in flight requests", node.id());
initiateConnect(node, now);
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/83a9aa55/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
index 140237f..4aa5b01 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
@@ -130,7 +130,7 @@ public final class Metadata {
this.version += 1;
this.cluster = cluster;
notifyAll();
- log.debug("Updated cluster metadata to {}", cluster);
+ log.debug("Updated cluster metadata version {} to {}", this.version, this.cluster);
}
/**
http://git-wip-us.apache.org/repos/asf/kafka/blob/83a9aa55/core/src/main/scala/kafka/tools/MirrorMaker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala
index 4f06e34..555d751 100644
--- a/core/src/main/scala/kafka/tools/MirrorMaker.scala
+++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala
@@ -116,19 +116,6 @@ object MirrorMaker extends Logging {
val numStreams = options.valueOf(numStreamsOpt).intValue()
val bufferSize = options.valueOf(bufferSizeOpt).intValue()
- val useNewProducer = options.has(useNewProducerOpt)
- val producerProps = Utils.loadProps(options.valueOf(producerConfigOpt))
-
- // create producer threads
- val clientId = producerProps.getProperty("client.id", "")
- val producers = (1 to numProducers).map(i => {
- producerProps.setProperty("client.id", clientId + "-" + i)
- if (useNewProducer)
- new NewShinyProducer(producerProps)
- else
- new OldProducer(producerProps)
- })
-
// create consumer streams
connectors = options.valuesOf(consumerConfigOpt).toList
.map(cfg => new ConsumerConfig(Utils.loadProps(cfg)))
@@ -138,8 +125,21 @@ object MirrorMaker extends Logging {
// create a data channel btw the consumers and the producers
val mirrorDataChannel = new DataChannel(bufferSize, numConsumers, numProducers)
- producerThreads = producers.zipWithIndex.map(producerAndIndex => new ProducerThread(mirrorDataChannel, producerAndIndex._1, producerAndIndex._2))
+ // create producer threads
+ val useNewProducer = options.has(useNewProducerOpt)
+ val producerProps = Utils.loadProps(options.valueOf(producerConfigOpt))
+ val clientId = producerProps.getProperty("client.id", "")
+ producerThreads = (0 until numProducers).map(i => {
+ producerProps.setProperty("client.id", clientId + "-" + i)
+ val producer =
+ if (useNewProducer)
+ new NewShinyProducer(producerProps)
+ else
+ new OldProducer(producerProps)
+ new ProducerThread(mirrorDataChannel, producer, i)
+ })
+ // create consumer threads
val filterSpec = if (options.has(whitelistOpt))
new Whitelist(options.valueOf(whitelistOpt))
else
@@ -153,7 +153,7 @@ object MirrorMaker extends Logging {
fatal("Unable to create stream - shutting down mirror maker.")
connectors.foreach(_.shutdown)
}
- consumerThreads = streams.zipWithIndex.map(streamAndIndex => new ConsumerThread(streamAndIndex._1, mirrorDataChannel, producers, streamAndIndex._2))
+ consumerThreads = streams.zipWithIndex.map(streamAndIndex => new ConsumerThread(streamAndIndex._1, mirrorDataChannel, streamAndIndex._2))
assert(consumerThreads.size == numConsumers)
Runtime.getRuntime.addShutdownHook(new Thread() {
@@ -233,7 +233,6 @@ object MirrorMaker extends Logging {
class ConsumerThread(stream: KafkaStream[Array[Byte], Array[Byte]],
mirrorDataChannel: DataChannel,
- producers: Seq[BaseProducer],
threadId: Int)
extends Thread with Logging with KafkaMetricsGroup {
[07/37] git commit: kafka-1513; Cleanup KafkaServerStartable code;
patched by Evgeny Vereshchagin; reviewed by Jun Rao
Posted by jj...@apache.org.
kafka-1513; Cleanup KafkaServerStartable code; patched by Evgeny Vereshchagin; reviewed by Jun Rao
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/b1a1cae8
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/b1a1cae8
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/b1a1cae8
Branch: refs/heads/transactional_messaging
Commit: b1a1cae880817332f172dccaa5e3e4a69228074f
Parents: 2a4718c
Author: Evgeny Vereshchagin <ev...@ya.ru>
Authored: Wed Jul 2 22:09:26 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Wed Jul 2 22:09:26 2014 -0700
----------------------------------------------------------------------
core/src/main/scala/kafka/server/KafkaServerStartable.scala | 8 +-------
1 file changed, 1 insertion(+), 7 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/b1a1cae8/core/src/main/scala/kafka/server/KafkaServerStartable.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaServerStartable.scala b/core/src/main/scala/kafka/server/KafkaServerStartable.scala
index cef3b84..28658bb 100644
--- a/core/src/main/scala/kafka/server/KafkaServerStartable.scala
+++ b/core/src/main/scala/kafka/server/KafkaServerStartable.scala
@@ -21,13 +21,7 @@ import kafka.utils.Logging
class KafkaServerStartable(val serverConfig: KafkaConfig) extends Logging {
- private var server : KafkaServer = null
-
- init
-
- private def init() {
- server = new KafkaServer(serverConfig)
- }
+ private val server = new KafkaServer(serverConfig)
def startup() {
try {
[34/37] git commit: kafka-1571; MetadataTest hangs; patched by Jun Rao;
reviewed by Guozhang Wang
Posted by jj...@apache.org.
kafka-1571; MetadataTest hangs; patched by Jun Rao; reviewed by Guozhang Wang
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/1d2e776d
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/1d2e776d
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/1d2e776d
Branch: refs/heads/transactional_messaging
Commit: 1d2e776d8c4ffbe59e40985ecfc3b010d99519d1
Parents: f8d521a
Author: Jun Rao <ju...@gmail.com>
Authored: Mon Aug 4 21:19:49 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Mon Aug 4 21:19:49 2014 -0700
----------------------------------------------------------------------
.../apache/kafka/clients/producer/MetadataTest.java | 14 +++++++++-----
1 file changed, 9 insertions(+), 5 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/1d2e776d/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
index 543304c..4547bfc 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
@@ -17,6 +17,7 @@ import static org.junit.Assert.assertTrue;
import org.apache.kafka.clients.producer.internals.Metadata;
import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.test.TestUtils;
import org.junit.Test;
@@ -40,9 +41,7 @@ public class MetadataTest {
Thread t2 = asyncFetch(topic);
assertTrue("Awaiting update", t1.isAlive());
assertTrue("Awaiting update", t2.isAlive());
- // keep updating the metadata until no need to
- while (metadata.timeToNextUpdate(time) == 0)
- metadata.update(TestUtils.singletonCluster(topic, 1), time);
+ metadata.update(TestUtils.singletonCluster(topic, 1), time);
t1.join();
t2.join();
assertFalse("No update needed.", metadata.timeToNextUpdate(time) == 0);
@@ -53,8 +52,13 @@ public class MetadataTest {
private Thread asyncFetch(final String topic) {
Thread thread = new Thread() {
public void run() {
- while (metadata.fetch().partitionsForTopic(topic) == null)
- metadata.awaitUpdate(metadata.requestUpdate(), Long.MAX_VALUE);
+ while (metadata.fetch().partitionsForTopic(topic) == null) {
+ try {
+ metadata.awaitUpdate(metadata.requestUpdate(), refreshBackoffMs);
+ } catch(TimeoutException e) {
+ // let it go
+ }
+ }
}
};
thread.start();
[06/37] git commit: kafka-1503;
all partitions are using same broker as their leader after broker is
down; patched by Jianwen Wang; reviewed by Guozhang Wang and Jun Rao
Posted by jj...@apache.org.
kafka-1503; all partitions are using same broker as their leader after broker is down; patched by Jianwen Wang; reviewed by Guozhang Wang and Jun Rao
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/2a4718c1
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/2a4718c1
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/2a4718c1
Branch: refs/heads/transactional_messaging
Commit: 2a4718c1a7e8e1566c5c87468779fdd1f95fe3bc
Parents: b8d87d0
Author: Jianwen Wang <Ja...@gmail.com>
Authored: Wed Jul 2 21:54:42 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Wed Jul 2 21:54:42 2014 -0700
----------------------------------------------------------------------
.../main/scala/kafka/controller/PartitionLeaderSelector.scala | 5 +++--
1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/2a4718c1/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala
index d3b25fa..4a31c72 100644
--- a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala
+++ b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala
@@ -83,7 +83,8 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext, confi
new LeaderAndIsr(newLeader, currentLeaderEpoch + 1, List(newLeader), currentLeaderIsrZkPathVersion + 1)
}
case false =>
- val newLeader = liveBrokersInIsr.head
+ val liveReplicasInIsr = liveAssignedReplicas.filter(r => liveBrokersInIsr.contains(r))
+ val newLeader = liveReplicasInIsr.head
debug("Some broker in ISR is alive for %s. Select %d from ISR %s to be the leader."
.format(topicAndPartition, newLeader, liveBrokersInIsr.mkString(",")))
new LeaderAndIsr(newLeader, currentLeaderEpoch + 1, liveBrokersInIsr.toList, currentLeaderIsrZkPathVersion + 1)
@@ -210,4 +211,4 @@ class NoOpLeaderSelector(controllerContext: ControllerContext) extends Partition
warn("I should never have been asked to perform leader election, returning the current LeaderAndIsr and replica assignment.")
(currentLeaderAndIsr, controllerContext.partitionReplicaAssignment(topicAndPartition))
}
-}
\ No newline at end of file
+}
[28/37] git commit: kafka-1549;
dead brokers coming in the TopicMetadataResponse; patched by Nicu Marasoiu;
reviewed by Jun Rao
Posted by jj...@apache.org.
kafka-1549; dead brokers coming in the TopicMetadataResponse; patched by Nicu Marasoiu; reviewed by Jun Rao
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/7f2278fb
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/7f2278fb
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/7f2278fb
Branch: refs/heads/transactional_messaging
Commit: 7f2278fb9de08141f21b017ba66752857dcdeba4
Parents: d9e5080
Author: Nicu Marasoiu <nm...@adobe.com>
Authored: Sun Jul 27 09:59:08 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Sun Jul 27 09:59:08 2014 -0700
----------------------------------------------------------------------
.../main/scala/kafka/cluster/Partition.scala | 27 ++++++++++----------
.../main/scala/kafka/server/MetadataCache.scala | 17 ++++++------
core/src/main/scala/kafka/utils/Utils.scala | 11 +++++---
3 files changed, 29 insertions(+), 26 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/7f2278fb/core/src/main/scala/kafka/cluster/Partition.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala
index f2ca856..134aef9 100644
--- a/core/src/main/scala/kafka/cluster/Partition.scala
+++ b/core/src/main/scala/kafka/cluster/Partition.scala
@@ -18,8 +18,7 @@ package kafka.cluster
import kafka.common._
import kafka.admin.AdminUtils
-import kafka.utils.{ReplicationUtils, Pool, Time, Logging}
-import kafka.utils.Utils.inLock
+import kafka.utils._
import kafka.api.{PartitionStateInfo, LeaderAndIsr}
import kafka.log.LogConfig
import kafka.server.{OffsetManager, ReplicaManager}
@@ -29,7 +28,7 @@ import kafka.message.ByteBufferMessageSet
import java.io.IOException
import java.util.concurrent.locks.ReentrantReadWriteLock
-import scala.Some
+import kafka.utils.Utils.{inReadLock,inWriteLock}
import scala.collection._
import com.yammer.metrics.core.Gauge
@@ -73,7 +72,7 @@ class Partition(val topic: String,
)
def isUnderReplicated(): Boolean = {
- inLock(leaderIsrUpdateLock.readLock()) {
+ inReadLock(leaderIsrUpdateLock) {
leaderReplicaIfLocal() match {
case Some(_) =>
inSyncReplicas.size < assignedReplicas.size
@@ -115,7 +114,7 @@ class Partition(val topic: String,
}
def leaderReplicaIfLocal(): Option[Replica] = {
- inLock(leaderIsrUpdateLock.readLock()) {
+ inReadLock(leaderIsrUpdateLock) {
leaderReplicaIdOpt match {
case Some(leaderReplicaId) =>
if (leaderReplicaId == localBrokerId)
@@ -141,7 +140,7 @@ class Partition(val topic: String,
def delete() {
// need to hold the lock to prevent appendMessagesToLeader() from hitting I/O exceptions due to log being deleted
- inLock(leaderIsrUpdateLock.writeLock()) {
+ inWriteLock(leaderIsrUpdateLock) {
assignedReplicaMap.clear()
inSyncReplicas = Set.empty[Replica]
leaderReplicaIdOpt = None
@@ -156,7 +155,7 @@ class Partition(val topic: String,
}
def getLeaderEpoch(): Int = {
- inLock(leaderIsrUpdateLock.readLock()) {
+ inReadLock(leaderIsrUpdateLock) {
return this.leaderEpoch
}
}
@@ -168,7 +167,7 @@ class Partition(val topic: String,
def makeLeader(controllerId: Int,
partitionStateInfo: PartitionStateInfo, correlationId: Int,
offsetManager: OffsetManager): Boolean = {
- inLock(leaderIsrUpdateLock.writeLock()) {
+ inWriteLock(leaderIsrUpdateLock) {
val allReplicas = partitionStateInfo.allReplicas
val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch
val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr
@@ -201,7 +200,7 @@ class Partition(val topic: String,
def makeFollower(controllerId: Int,
partitionStateInfo: PartitionStateInfo,
correlationId: Int, offsetManager: OffsetManager): Boolean = {
- inLock(leaderIsrUpdateLock.writeLock()) {
+ inWriteLock(leaderIsrUpdateLock) {
val allReplicas = partitionStateInfo.allReplicas
val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch
val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr
@@ -235,7 +234,7 @@ class Partition(val topic: String,
}
def updateLeaderHWAndMaybeExpandIsr(replicaId: Int, offset: Long) {
- inLock(leaderIsrUpdateLock.writeLock()) {
+ inWriteLock(leaderIsrUpdateLock) {
debug("Recording follower %d position %d for partition [%s,%d].".format(replicaId, offset, topic, partitionId))
val replicaOpt = getReplica(replicaId)
if(!replicaOpt.isDefined) {
@@ -271,7 +270,7 @@ class Partition(val topic: String,
}
def checkEnoughReplicasReachOffset(requiredOffset: Long, requiredAcks: Int): (Boolean, Short) = {
- inLock(leaderIsrUpdateLock.readLock()) {
+ inReadLock(leaderIsrUpdateLock) {
leaderReplicaIfLocal() match {
case Some(_) =>
val numAcks = inSyncReplicas.count(r => {
@@ -315,7 +314,7 @@ class Partition(val topic: String,
}
def maybeShrinkIsr(replicaMaxLagTimeMs: Long, replicaMaxLagMessages: Long) {
- inLock(leaderIsrUpdateLock.writeLock()) {
+ inWriteLock(leaderIsrUpdateLock) {
leaderReplicaIfLocal() match {
case Some(leaderReplica) =>
val outOfSyncReplicas = getOutOfSyncReplicas(leaderReplica, replicaMaxLagTimeMs, replicaMaxLagMessages)
@@ -357,7 +356,7 @@ class Partition(val topic: String,
}
def appendMessagesToLeader(messages: ByteBufferMessageSet) = {
- inLock(leaderIsrUpdateLock.readLock()) {
+ inReadLock(leaderIsrUpdateLock) {
val leaderReplicaOpt = leaderReplicaIfLocal()
leaderReplicaOpt match {
case Some(leaderReplica) =>
@@ -400,7 +399,7 @@ class Partition(val topic: String,
}
override def toString(): String = {
- inLock(leaderIsrUpdateLock.readLock()) {
+ inReadLock(leaderIsrUpdateLock) {
val partitionString = new StringBuilder
partitionString.append("Topic: " + topic)
partitionString.append("; Partition: " + partitionId)
http://git-wip-us.apache.org/repos/asf/kafka/blob/7f2278fb/core/src/main/scala/kafka/server/MetadataCache.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala
index 7cd40e1..bf81a1a 100644
--- a/core/src/main/scala/kafka/server/MetadataCache.scala
+++ b/core/src/main/scala/kafka/server/MetadataCache.scala
@@ -25,7 +25,6 @@ import kafka.utils.Utils._
import kafka.common.{ErrorMapping, ReplicaNotAvailableException, LeaderNotAvailableException}
import kafka.common.TopicAndPartition
import kafka.controller.KafkaController.StateChangeLogger
-import scala.Some
/**
* A cache for the state (e.g., current leader) of each partition. This cache is updated through
@@ -34,14 +33,14 @@ import scala.Some
private[server] class MetadataCache {
private val cache: mutable.Map[String, mutable.Map[Int, PartitionStateInfo]] =
new mutable.HashMap[String, mutable.Map[Int, PartitionStateInfo]]()
- private val aliveBrokers: mutable.Map[Int, Broker] = new mutable.HashMap[Int, Broker]()
+ private var aliveBrokers: Map[Int, Broker] = Map()
private val partitionMetadataLock = new ReentrantReadWriteLock()
def getTopicMetadata(topics: Set[String]) = {
val isAllTopics = topics.isEmpty
val topicsRequested = if(isAllTopics) cache.keySet else topics
val topicResponses: mutable.ListBuffer[TopicMetadata] = new mutable.ListBuffer[TopicMetadata]
- inLock(partitionMetadataLock.readLock()) {
+ inReadLock(partitionMetadataLock) {
for (topic <- topicsRequested) {
if (isAllTopics || cache.contains(topic)) {
val partitionStateInfos = cache(topic)
@@ -82,15 +81,15 @@ private[server] class MetadataCache {
}
def getAliveBrokers = {
- inLock(partitionMetadataLock.readLock()) {
- aliveBrokers.values.toList
+ inReadLock(partitionMetadataLock) {
+ aliveBrokers.values.toSeq
}
}
def addOrUpdatePartitionInfo(topic: String,
partitionId: Int,
stateInfo: PartitionStateInfo) {
- inLock(partitionMetadataLock.writeLock()) {
+ inWriteLock(partitionMetadataLock) {
cache.get(topic) match {
case Some(infos) => infos.put(partitionId, stateInfo)
case None => {
@@ -103,7 +102,7 @@ private[server] class MetadataCache {
}
def getPartitionInfo(topic: String, partitionId: Int): Option[PartitionStateInfo] = {
- inLock(partitionMetadataLock.readLock()) {
+ inReadLock(partitionMetadataLock) {
cache.get(topic) match {
case Some(partitionInfos) => partitionInfos.get(partitionId)
case None => None
@@ -114,8 +113,8 @@ private[server] class MetadataCache {
def updateCache(updateMetadataRequest: UpdateMetadataRequest,
brokerId: Int,
stateChangeLogger: StateChangeLogger) {
- inLock(partitionMetadataLock.writeLock()) {
- updateMetadataRequest.aliveBrokers.foreach(b => aliveBrokers.put(b.id, b))
+ inWriteLock(partitionMetadataLock) {
+ aliveBrokers = updateMetadataRequest.aliveBrokers.map(b => (b.id, b)).toMap
updateMetadataRequest.partitionStateInfos.foreach { case(tp, info) =>
if (info.leaderIsrAndControllerEpoch.leaderAndIsr.leader == LeaderAndIsr.LeaderDuringDelete) {
removePartitionInfo(tp.topic, tp.partition)
http://git-wip-us.apache.org/repos/asf/kafka/blob/7f2278fb/core/src/main/scala/kafka/utils/Utils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala
index 6576adf..09bfbce 100644
--- a/core/src/main/scala/kafka/utils/Utils.scala
+++ b/core/src/main/scala/kafka/utils/Utils.scala
@@ -21,7 +21,7 @@ import java.io._
import java.nio._
import charset.Charset
import java.nio.channels._
-import java.util.concurrent.locks.Lock
+import java.util.concurrent.locks.{ReadWriteLock, Lock}
import java.lang.management._
import javax.management._
import scala.collection._
@@ -540,13 +540,18 @@ object Utils extends Logging {
def inLock[T](lock: Lock)(fun: => T): T = {
lock.lock()
try {
- return fun
+ fun
} finally {
lock.unlock()
}
}
- //JSON strings need to be escaped based on ECMA-404 standard http://json.org
+ def inReadLock[T](lock: ReadWriteLock)(fun: => T): T = inLock[T](lock.readLock)(fun)
+
+ def inWriteLock[T](lock: ReadWriteLock)(fun: => T): T = inLock[T](lock.writeLock)(fun)
+
+
+ //JSON strings need to be escaped based on ECMA-404 standard http://json.org
def JSONEscapeString (s : String) : String = {
s.map {
case '"' => "\\\""
[25/37] git commit: kafka-1192;
Enable DumpLogSegments tool to deserialize messages;
patched by Manikumar Reddy; reviewed by Guozhang Wang and Jun Rao
Posted by jj...@apache.org.
kafka-1192; Enable DumpLogSegments tool to deserialize messages; patched by Manikumar Reddy; reviewed by Guozhang Wang and Jun Rao
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/3f1a9c4c
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/3f1a9c4c
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/3f1a9c4c
Branch: refs/heads/transactional_messaging
Commit: 3f1a9c4cee778d089d3ec3167555c2b89cdc48bb
Parents: ff05e9b
Author: Manikumar Reddy <ma...@gmail.com>
Authored: Wed Jul 23 17:08:51 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Wed Jul 23 17:08:51 2014 -0700
----------------------------------------------------------------------
.../scala/kafka/tools/DumpLogSegments.scala | 27 +++++++++++++++-----
1 file changed, 21 insertions(+), 6 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/3f1a9c4c/core/src/main/scala/kafka/tools/DumpLogSegments.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala
index 6daf87b..8e9d47b 100644
--- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala
+++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala
@@ -23,7 +23,8 @@ import kafka.log._
import kafka.utils._
import collection.mutable
import joptsimple.OptionParser
-
+import kafka.serializer.Decoder
+import kafka.utils.VerifiableProperties
object DumpLogSegments {
@@ -41,7 +42,15 @@ object DumpLogSegments {
.ofType(classOf[java.lang.Integer])
.defaultsTo(5 * 1024 * 1024)
val deepIterationOpt = parser.accepts("deep-iteration", "if set, uses deep instead of shallow iteration")
-
+ val valueDecoderOpt = parser.accepts("value-decoder-class", "if set, used to deserialize the messages. This class should implement kafka.serializer.Decoder trait. Custom jar should be available in kafka/libs directory.")
+ .withOptionalArg()
+ .ofType(classOf[java.lang.String])
+ .defaultsTo("kafka.serializer.StringDecoder")
+ val keyDecoderOpt = parser.accepts("key-decoder-class", "if set, used to deserialize the keys. This class should implement kafka.serializer.Decoder trait. Custom jar should be available in kafka/libs directory.")
+ .withOptionalArg()
+ .ofType(classOf[java.lang.String])
+ .defaultsTo("kafka.serializer.StringDecoder")
+
if(args.length == 0)
CommandLineUtils.printUsageAndDie(parser, "Parse a log file and dump its contents to the console, useful for debugging a seemingly corrupt log segment.")
@@ -54,6 +63,9 @@ object DumpLogSegments {
val files = options.valueOf(filesOpt).split(",")
val maxMessageSize = options.valueOf(maxMessageSizeOpt).intValue()
val isDeepIteration = if(options.has(deepIterationOpt)) true else false
+
+ val valueDecoder: Decoder[_] = Utils.createObject[Decoder[_]](options.valueOf(valueDecoderOpt), new VerifiableProperties)
+ val keyDecoder: Decoder[_] = Utils.createObject[Decoder[_]](options.valueOf(keyDecoderOpt), new VerifiableProperties)
val misMatchesForIndexFilesMap = new mutable.HashMap[String, List[(Long, Long)]]
val nonConsecutivePairsForLogFilesMap = new mutable.HashMap[String, List[(Long, Long)]]
@@ -62,7 +74,7 @@ object DumpLogSegments {
val file = new File(arg)
if(file.getName.endsWith(Log.LogFileSuffix)) {
println("Dumping " + file)
- dumpLog(file, print, nonConsecutivePairsForLogFilesMap, isDeepIteration, maxMessageSize)
+ dumpLog(file, print, nonConsecutivePairsForLogFilesMap, isDeepIteration, maxMessageSize , valueDecoder, keyDecoder)
} else if(file.getName.endsWith(Log.IndexFileSuffix)) {
println("Dumping " + file)
dumpIndex(file, verifyOnly, misMatchesForIndexFilesMap, maxMessageSize)
@@ -118,7 +130,9 @@ object DumpLogSegments {
printContents: Boolean,
nonConsecutivePairsForLogFilesMap: mutable.HashMap[String, List[(Long, Long)]],
isDeepIteration: Boolean,
- maxMessageSize: Int) {
+ maxMessageSize: Int,
+ valueDecoder: Decoder[_],
+ keyDecoder: Decoder[_]) {
val startOffset = file.getName().split("\\.")(0).toLong
println("Starting offset: " + startOffset)
val messageSet = new FileMessageSet(file, false)
@@ -147,8 +161,8 @@ object DumpLogSegments {
print(" keysize: " + msg.keySize)
if(printContents) {
if(msg.hasKey)
- print(" key: " + Utils.readString(messageAndOffset.message.key, "UTF-8"))
- val payload = if(messageAndOffset.message.isNull) null else Utils.readString(messageAndOffset.message.payload, "UTF-8")
+ print(" key: " + keyDecoder.fromBytes(Utils.readBytes(messageAndOffset.message.key)))
+ val payload = if(messageAndOffset.message.isNull) null else valueDecoder.fromBytes(Utils.readBytes(messageAndOffset.message.payload))
print(" payload: " + payload)
}
println()
@@ -186,4 +200,5 @@ object DumpLogSegments {
}
}
}
+
}
[23/37] git commit: kafka-1462 (followup patch);
Add new request and response formats for the new consumer and
coordinator communication; patched by Jun Rao; reviewed by Jay Kreps
Posted by jj...@apache.org.
kafka-1462 (followup patch); Add new request and response formats for the new consumer and coordinator communication; patched by Jun Rao; reviewed by Jay Kreps
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/014b700f
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/014b700f
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/014b700f
Branch: refs/heads/transactional_messaging
Commit: 014b700f0323b4dc00d3aa0e1b598f7e2ed07957
Parents: 1e4b084
Author: Jun Rao <ju...@gmail.com>
Authored: Tue Jul 22 10:53:34 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Tue Jul 22 10:53:34 2014 -0700
----------------------------------------------------------------------
.../kafka/api/GenericRequestAndHeader.scala | 55 ++++++++++++++++++++
.../api/GenericRequestOrResponseAndHeader.scala | 45 ----------------
.../kafka/api/GenericResponseAndHeader.scala | 46 ++++++++++++++++
.../kafka/api/HeartbeatRequestAndHeader.scala | 20 ++++---
.../kafka/api/HeartbeatResponseAndHeader.scala | 10 ++--
.../kafka/api/JoinGroupRequestAndHeader.scala | 17 +++---
.../kafka/api/JoinGroupResponseAndHeader.scala | 10 ++--
.../api/RequestResponseSerializationTest.scala | 12 ++---
8 files changed, 139 insertions(+), 76 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/014b700f/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala b/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala
new file mode 100644
index 0000000..f40e19f
--- /dev/null
+++ b/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+
+package kafka.api
+
+import java.nio.ByteBuffer
+import org.apache.kafka.common.requests.AbstractRequestResponse
+import kafka.api.ApiUtils._
+
+private[kafka] abstract class GenericRequestAndHeader(val versionId: Short,
+ val correlationId: Int,
+ val clientId: String,
+ val body: AbstractRequestResponse,
+ val name: String,
+ override val requestId: Option[Short] = None)
+ extends RequestOrResponse(requestId) {
+
+ def writeTo(buffer: ByteBuffer) {
+ buffer.putShort(versionId)
+ buffer.putInt(correlationId)
+ writeShortString(buffer, clientId)
+ body.writeTo(buffer)
+ }
+
+ def sizeInBytes(): Int = {
+ 2 /* version id */ +
+ 4 /* correlation id */ +
+ (2 + clientId.length) /* client id */ +
+ body.sizeOf();
+ }
+
+ override def toString(): String = {
+ describe(true)
+ }
+
+ override def describe(details: Boolean): String = {
+ val strBuffer = new StringBuilder
+ strBuffer.append("Name: " + name)
+ strBuffer.append("; Version: " + versionId)
+ strBuffer.append("; CorrelationId: " + correlationId)
+ strBuffer.append("; ClientId: " + clientId)
+ strBuffer.append("; Body: " + body.toString)
+ strBuffer.toString()
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kafka/blob/014b700f/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala b/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala
deleted file mode 100644
index fb022e8..0000000
--- a/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-
-package kafka.api
-
-import java.nio.ByteBuffer
-import org.apache.kafka.common.requests.AbstractRequestResponse
-
-private[kafka] abstract class GenericRequestOrResponseAndHeader(val header: AbstractRequestResponse,
- val body: AbstractRequestResponse,
- val name: String,
- override val requestId: Option[Short] = None)
- extends RequestOrResponse(requestId) {
-
- def writeTo(buffer: ByteBuffer) {
- header.writeTo(buffer)
- body.writeTo(buffer)
- }
-
- def sizeInBytes(): Int = {
- header.sizeOf() + body.sizeOf();
- }
-
- override def toString(): String = {
- describe(true)
- }
-
- override def describe(details: Boolean): String = {
- val strBuffer = new StringBuilder
- strBuffer.append("Name: " + name)
- strBuffer.append("; header: " + header.toString)
- strBuffer.append("; body: " + body.toString)
- strBuffer.toString()
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kafka/blob/014b700f/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala b/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala
new file mode 100644
index 0000000..a4879e2
--- /dev/null
+++ b/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+
+package kafka.api
+
+import java.nio.ByteBuffer
+import org.apache.kafka.common.requests.AbstractRequestResponse
+
+private[kafka] abstract class GenericResponseAndHeader(val correlationId: Int,
+ val body: AbstractRequestResponse,
+ val name: String,
+ override val requestId: Option[Short] = None)
+ extends RequestOrResponse(requestId) {
+
+ def writeTo(buffer: ByteBuffer) {
+ buffer.putInt(correlationId)
+ body.writeTo(buffer)
+ }
+
+ def sizeInBytes(): Int = {
+ 4 /* correlation id */ +
+ body.sizeOf();
+ }
+
+ override def toString(): String = {
+ describe(true)
+ }
+
+ override def describe(details: Boolean): String = {
+ val strBuffer = new StringBuilder
+ strBuffer.append("Name: " + name)
+ strBuffer.append("; CorrelationId: " + correlationId)
+ strBuffer.append("; Body: " + body.toString)
+ strBuffer.toString()
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kafka/blob/014b700f/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala b/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala
index 932418b..f168d9f 100644
--- a/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala
+++ b/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala
@@ -16,24 +16,30 @@ package kafka.api
import java.nio.ByteBuffer
import kafka.network.{BoundedByteBufferSend, RequestChannel}
import kafka.common.ErrorMapping
+import org.apache.kafka.common.requests.{HeartbeatResponse, HeartbeatRequest}
+import kafka.api.ApiUtils._
import kafka.network.RequestChannel.Response
-import org.apache.kafka.common.requests.{HeartbeatResponse, ResponseHeader, HeartbeatRequest, RequestHeader}
+import scala.Some
object HeartbeatRequestAndHeader {
def readFrom(buffer: ByteBuffer): HeartbeatRequestAndHeader = {
- val header = RequestHeader.parse(buffer)
+ val versionId = buffer.getShort
+ val correlationId = buffer.getInt
+ val clientId = readShortString(buffer)
val body = HeartbeatRequest.parse(buffer)
- new HeartbeatRequestAndHeader(header, body)
+ new HeartbeatRequestAndHeader(versionId, correlationId, clientId, body)
}
}
-case class HeartbeatRequestAndHeader(override val header: RequestHeader, override val body: HeartbeatRequest)
- extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), Some(RequestKeys.HeartbeatKey)) {
+case class HeartbeatRequestAndHeader(override val versionId: Short,
+ override val correlationId: Int,
+ override val clientId: String,
+ override val body: HeartbeatRequest)
+ extends GenericRequestAndHeader(versionId, correlationId, clientId, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), Some(RequestKeys.HeartbeatKey)) {
override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
- val errorResponseHeader = new ResponseHeader(header.correlationId)
val errorResponseBody = new HeartbeatResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
- val errorHeartBeatResponseAndHeader = new HeartbeatResponseAndHeader(errorResponseHeader, errorResponseBody)
+ val errorHeartBeatResponseAndHeader = new HeartbeatResponseAndHeader(correlationId, errorResponseBody)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorHeartBeatResponseAndHeader)))
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/014b700f/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala b/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala
index 556f38d..9a71faa 100644
--- a/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala
+++ b/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala
@@ -12,17 +12,17 @@
*/
package kafka.api
-import org.apache.kafka.common.requests.{ResponseHeader, HeartbeatResponse}
+import org.apache.kafka.common.requests.HeartbeatResponse
import java.nio.ByteBuffer
object HeartbeatResponseAndHeader {
def readFrom(buffer: ByteBuffer): HeartbeatResponseAndHeader = {
- val header = ResponseHeader.parse(buffer)
+ val correlationId = buffer.getInt
val body = HeartbeatResponse.parse(buffer)
- new HeartbeatResponseAndHeader(header, body)
+ new HeartbeatResponseAndHeader(correlationId, body)
}
}
-case class HeartbeatResponseAndHeader(override val header: ResponseHeader, override val body: HeartbeatResponse)
- extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), None) {
+case class HeartbeatResponseAndHeader(override val correlationId: Int, override val body: HeartbeatResponse)
+ extends GenericResponseAndHeader(correlationId, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), None) {
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/014b700f/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala b/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala
index 9aea28c..3651e86 100644
--- a/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala
+++ b/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala
@@ -17,24 +17,29 @@ import java.nio.ByteBuffer
import kafka.network.{BoundedByteBufferSend, RequestChannel}
import kafka.common.ErrorMapping
import org.apache.kafka.common.requests._
+import kafka.api.ApiUtils._
import kafka.network.RequestChannel.Response
import scala.Some
object JoinGroupRequestAndHeader {
def readFrom(buffer: ByteBuffer): JoinGroupRequestAndHeader = {
- val header = RequestHeader.parse(buffer)
+ val versionId = buffer.getShort
+ val correlationId = buffer.getInt
+ val clientId = readShortString(buffer)
val body = JoinGroupRequest.parse(buffer)
- new JoinGroupRequestAndHeader(header, body)
+ new JoinGroupRequestAndHeader(versionId, correlationId, clientId, body)
}
}
-case class JoinGroupRequestAndHeader(override val header: RequestHeader, override val body: JoinGroupRequest)
- extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), Some(RequestKeys.JoinGroupKey)) {
+case class JoinGroupRequestAndHeader(override val versionId: Short,
+ override val correlationId: Int,
+ override val clientId: String,
+ override val body: JoinGroupRequest)
+ extends GenericRequestAndHeader(versionId, correlationId, clientId, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), Some(RequestKeys.JoinGroupKey)) {
override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
- val errorResponseHeader = new ResponseHeader(header.correlationId)
val errorResponseBody = new JoinGroupResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
- val errorHeartBeatResponseAndHeader = new JoinGroupResponseAndHeader(errorResponseHeader, errorResponseBody)
+ val errorHeartBeatResponseAndHeader = new JoinGroupResponseAndHeader(correlationId, errorResponseBody)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorHeartBeatResponseAndHeader)))
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/014b700f/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala b/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala
index 7389ae6..d0f07e0 100644
--- a/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala
+++ b/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala
@@ -12,17 +12,17 @@
*/
package kafka.api
-import org.apache.kafka.common.requests.{JoinGroupResponse, ResponseHeader}
+import org.apache.kafka.common.requests.JoinGroupResponse
import java.nio.ByteBuffer
object JoinGroupResponseAndHeader {
def readFrom(buffer: ByteBuffer): JoinGroupResponseAndHeader = {
- val header = ResponseHeader.parse(buffer)
+ val correlationId = buffer.getInt
val body = JoinGroupResponse.parse(buffer)
- new JoinGroupResponseAndHeader(header, body)
+ new JoinGroupResponseAndHeader(correlationId, body)
}
}
-case class JoinGroupResponseAndHeader(override val header: ResponseHeader, override val body: JoinGroupResponse)
- extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), None) {
+case class JoinGroupResponseAndHeader(override val correlationId: Int, override val body: JoinGroupResponse)
+ extends GenericResponseAndHeader(correlationId, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), None) {
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/014b700f/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
index 847a36b..cd16ced 100644
--- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
+++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
@@ -196,29 +196,25 @@ object SerializationTestUtils {
}
def createHeartbeatRequestAndHeader: HeartbeatRequestAndHeader = {
- val header = new RequestHeader(ApiKeys.HEARTBEAT.id, 0.asInstanceOf[Short], "", 1)
val body = new HeartbeatRequest("group1", 1, "consumer1")
- HeartbeatRequestAndHeader(header, body)
+ HeartbeatRequestAndHeader(0.asInstanceOf[Short], 1, "", body)
}
def createHeartbeatResponseAndHeader: HeartbeatResponseAndHeader = {
- val header = new ResponseHeader(1)
val body = new HeartbeatResponse(0.asInstanceOf[Short])
- HeartbeatResponseAndHeader(header, body)
+ HeartbeatResponseAndHeader(1, body)
}
def createJoinGroupRequestAndHeader: JoinGroupRequestAndHeader = {
import scala.collection.JavaConversions._
- val header = new RequestHeader(ApiKeys.JOIN_GROUP.id, 0.asInstanceOf[Short], "", 1)
val body = new JoinGroupRequest("group1", 30000, List("topic1"), "consumer1", "strategy1");
- JoinGroupRequestAndHeader(header, body)
+ JoinGroupRequestAndHeader(0.asInstanceOf[Short], 1, "", body)
}
def createJoinGroupResponseAndHeader: JoinGroupResponseAndHeader = {
import scala.collection.JavaConversions._
- val header = new ResponseHeader(1)
val body = new JoinGroupResponse(0.asInstanceOf[Short], 1, "consumer1", List(new TopicPartition("test11", 1)))
- JoinGroupResponseAndHeader(header, body)
+ JoinGroupResponseAndHeader(1, body)
}
}
[04/37] git commit: KAFKA-1498 Follow-up: add metric on avg record
size.
Posted by jj...@apache.org.
KAFKA-1498 Follow-up: add metric on avg record size.
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/e3dfad30
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/e3dfad30
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/e3dfad30
Branch: refs/heads/transactional_messaging
Commit: e3dfad304e6c2ffc146cdb9bc25a1d683f289ace
Parents: f1c6e97
Author: Jay Kreps <ja...@gmail.com>
Authored: Tue Jul 1 14:47:57 2014 -0700
Committer: Jay Kreps <ja...@gmail.com>
Committed: Tue Jul 1 14:47:57 2014 -0700
----------------------------------------------------------------------
.../java/org/apache/kafka/clients/producer/internals/Sender.java | 1 +
1 file changed, 1 insertion(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/e3dfad30/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
index 52d209b..37b9d1a 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
@@ -343,6 +343,7 @@ public class Sender implements Runnable {
this.maxRecordSizeSensor = metrics.sensor("record-size-max");
this.maxRecordSizeSensor.add("record-size-max", "The maximum record size", new Max());
+ this.maxRecordSizeSensor.add("record-size-avg", "The average record size", new Avg());
this.metrics.addMetric("requests-in-flight", "The current number of in-flight requests awaiting a response.", new Measurable() {
public double measure(MetricConfig config, long now) {
[10/37] git commit: kafka-1325; Fix inconsistent per topic log configs;
patched by Manikumar Reddy; reviewed by Jun Rao
Posted by jj...@apache.org.
kafka-1325; Fix inconsistent per topic log configs; patched by Manikumar Reddy; reviewed by Jun Rao
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/420628d6
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/420628d6
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/420628d6
Branch: refs/heads/transactional_messaging
Commit: 420628d695cc675711b94af5cfd14653147bf7f7
Parents: cd3ce27
Author: Manikumar Reddy <ma...@gmail.com>
Authored: Thu Jul 10 08:27:33 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Thu Jul 10 08:27:33 2014 -0700
----------------------------------------------------------------------
.../main/scala/kafka/server/KafkaConfig.scala | 23 +++++++--
.../main/scala/kafka/server/KafkaServer.scala | 2 +-
.../unit/kafka/server/KafkaConfigTest.scala | 53 ++++++++++++++++++++
3 files changed, 73 insertions(+), 5 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/420628d6/core/src/main/scala/kafka/server/KafkaConfig.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala
index ef75b67..bb2e654 100644
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -35,14 +35,29 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
private def getLogRetentionTimeMillis(): Long = {
val millisInMinute = 60L * 1000L
val millisInHour = 60L * millisInMinute
- if(props.containsKey("log.retention.minutes")){
+
+ if(props.containsKey("log.retention.ms")){
+ props.getIntInRange("log.retention.ms", (1, Int.MaxValue))
+ }
+ else if(props.containsKey("log.retention.minutes")){
millisInMinute * props.getIntInRange("log.retention.minutes", (1, Int.MaxValue))
- } else {
+ }
+ else {
millisInHour * props.getIntInRange("log.retention.hours", 24*7, (1, Int.MaxValue))
}
-
}
+ private def getLogRollTimeMillis(): Long = {
+ val millisInHour = 60L * 60L * 1000L
+
+ if(props.containsKey("log.roll.ms")){
+ props.getIntInRange("log.roll.ms", (1, Int.MaxValue))
+ }
+ else {
+ millisInHour * props.getIntInRange("log.roll.hours", 24*7, (1, Int.MaxValue))
+ }
+ }
+
/*********** General Configuration ***********/
/* the broker id for this server */
@@ -105,7 +120,7 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
val logSegmentBytes = props.getIntInRange("log.segment.bytes", 1*1024*1024*1024, (Message.MinHeaderSize, Int.MaxValue))
/* the maximum time before a new log segment is rolled out */
- val logRollHours = props.getIntInRange("log.roll.hours", 24*7, (1, Int.MaxValue))
+ val logRollTimeMillis = getLogRollTimeMillis
/* the number of hours to keep a log file before deleting it */
val logRetentionTimeMillis = getLogRetentionTimeMillis
http://git-wip-us.apache.org/repos/asf/kafka/blob/420628d6/core/src/main/scala/kafka/server/KafkaServer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala
index c22e51e..5a56f57 100644
--- a/core/src/main/scala/kafka/server/KafkaServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaServer.scala
@@ -275,7 +275,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
private def createLogManager(zkClient: ZkClient, brokerState: BrokerState): LogManager = {
val defaultLogConfig = LogConfig(segmentSize = config.logSegmentBytes,
- segmentMs = 60L * 60L * 1000L * config.logRollHours,
+ segmentMs = config.logRollTimeMillis,
flushInterval = config.logFlushIntervalMessages,
flushMs = config.logFlushIntervalMs.toLong,
retentionSize = config.logRetentionBytes,
http://git-wip-us.apache.org/repos/asf/kafka/blob/420628d6/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
index 6f4809d..2377abe 100644
--- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
@@ -45,6 +45,16 @@ class KafkaConfigTest extends JUnit3Suite {
}
@Test
+ def testLogRetentionTimeMsProvided() {
+ val props = TestUtils.createBrokerConfig(0, 8181)
+ props.put("log.retention.ms", "1800000")
+
+ val cfg = new KafkaConfig(props)
+ assertEquals(30 * 60L * 1000L, cfg.logRetentionTimeMillis)
+
+ }
+
+ @Test
def testLogRetentionTimeNoConfigProvided() {
val props = TestUtils.createBrokerConfig(0, 8181)
@@ -63,6 +73,17 @@ class KafkaConfigTest extends JUnit3Suite {
assertEquals( 30 * 60L * 1000L, cfg.logRetentionTimeMillis)
}
+
+ @Test
+ def testLogRetentionTimeBothMinutesAndMsProvided() {
+ val props = TestUtils.createBrokerConfig(0, 8181)
+ props.put("log.retention.ms", "1800000")
+ props.put("log.retention.minutes", "10")
+
+ val cfg = new KafkaConfig(props)
+ assertEquals( 30 * 60L * 1000L, cfg.logRetentionTimeMillis)
+
+ }
@Test
def testAdvertiseDefaults() {
@@ -129,4 +150,36 @@ class KafkaConfigTest extends JUnit3Suite {
new KafkaConfig(props)
}
}
+
+ @Test
+ def testLogRollTimeMsProvided() {
+ val props = TestUtils.createBrokerConfig(0, 8181)
+ props.put("log.roll.ms", "1800000")
+
+ val cfg = new KafkaConfig(props)
+ assertEquals(30 * 60L * 1000L, cfg.logRollTimeMillis)
+
+ }
+
+ @Test
+ def testLogRollTimeBothMsAndHoursProvided() {
+ val props = TestUtils.createBrokerConfig(0, 8181)
+ props.put("log.roll.ms", "1800000")
+ props.put("log.roll.hours", "1")
+
+ val cfg = new KafkaConfig(props)
+ assertEquals( 30 * 60L * 1000L, cfg.logRollTimeMillis)
+
+ }
+
+ @Test
+ def testLogRollTimeNoConfigProvided() {
+ val props = TestUtils.createBrokerConfig(0, 8181)
+
+ val cfg = new KafkaConfig(props)
+ assertEquals(24 * 7 * 60L * 60L * 1000L, cfg.logRollTimeMillis )
+
+ }
+
+
}
[18/37] kafka-1462;
Add new request and response formats for the new consumer and
coordinator communication; patched by Jun Rao;
reviewed by Guozhang Wang and Jay Kreps
Posted by jj...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/OffsetCommitRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala
index 630768a..861a6cf 100644
--- a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala
+++ b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala
@@ -26,7 +26,7 @@ import kafka.network.RequestChannel.Response
import scala.collection._
object OffsetCommitRequest extends Logging {
- val CurrentVersion: Short = 0
+ val CurrentVersion: Short = 1
val DefaultClientId = ""
def readFrom(buffer: ByteBuffer): OffsetCommitRequest = {
@@ -34,11 +34,23 @@ object OffsetCommitRequest extends Logging {
// Read values from the envelope
val versionId = buffer.getShort
+ assert(versionId == 0 || versionId == 1,
+ "Version " + versionId + " is invalid for OffsetCommitRequest. Valid versions are 0 or 1.")
+
val correlationId = buffer.getInt
val clientId = readShortString(buffer)
// Read the OffsetRequest
val consumerGroupId = readShortString(buffer)
+
+ // version 1 specific fields
+ var groupGenerationId: Int = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_GENERATION_ID
+ var consumerId: String = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_CONSUMER_ID
+ if (versionId == 1) {
+ groupGenerationId = buffer.getInt
+ consumerId = readShortString(buffer)
+ }
+
val topicCount = buffer.getInt
val pairs = (1 to topicCount).flatMap(_ => {
val topic = readShortString(buffer)
@@ -54,16 +66,20 @@ object OffsetCommitRequest extends Logging {
(TopicAndPartition(topic, partitionId), OffsetAndMetadata(offset, metadata, timestamp))
})
})
- OffsetCommitRequest(consumerGroupId, immutable.Map(pairs:_*), versionId, correlationId, clientId)
+ OffsetCommitRequest(consumerGroupId, immutable.Map(pairs:_*), versionId, correlationId, clientId, groupGenerationId, consumerId)
}
}
case class OffsetCommitRequest(groupId: String,
requestInfo: immutable.Map[TopicAndPartition, OffsetAndMetadata],
versionId: Short = OffsetCommitRequest.CurrentVersion,
- override val correlationId: Int = 0,
- clientId: String = OffsetCommitRequest.DefaultClientId)
- extends RequestOrResponse(Some(RequestKeys.OffsetCommitKey), correlationId) {
+ correlationId: Int = 0,
+ clientId: String = OffsetCommitRequest.DefaultClientId,
+ groupGenerationId: Int = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_GENERATION_ID,
+ consumerId: String = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_CONSUMER_ID)
+ extends RequestOrResponse(Some(RequestKeys.OffsetCommitKey)) {
+ assert(versionId == 0 || versionId == 1,
+ "Version " + versionId + " is invalid for OffsetCommitRequest. Valid versions are 0 or 1.")
lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic)
@@ -84,7 +100,6 @@ case class OffsetCommitRequest(groupId: String,
OffsetCommitResponse(commitStatus, correlationId)
}
-
def writeTo(buffer: ByteBuffer) {
// Write envelope
buffer.putShort(versionId)
@@ -93,6 +108,12 @@ case class OffsetCommitRequest(groupId: String,
// Write OffsetCommitRequest
writeShortString(buffer, groupId) // consumer group
+
+ // version 1 specific data
+ if (versionId == 1) {
+ buffer.putInt(groupGenerationId)
+ writeShortString(buffer, consumerId)
+ }
buffer.putInt(requestInfoGroupedByTopic.size) // number of topics
requestInfoGroupedByTopic.foreach( t1 => { // topic -> Map[TopicAndPartition, OffsetMetadataAndError]
writeShortString(buffer, t1._1) // topic
@@ -110,7 +131,8 @@ case class OffsetCommitRequest(groupId: String,
2 + /* versionId */
4 + /* correlationId */
shortStringLength(clientId) +
- shortStringLength(groupId) +
+ shortStringLength(groupId) +
+ (if (versionId == 1) 4 /* group generation id */ + shortStringLength(consumerId) else 0) +
4 + /* topic count */
requestInfoGroupedByTopic.foldLeft(0)((count, topicAndOffsets) => {
val (topic, offsets) = topicAndOffsets
@@ -139,6 +161,8 @@ case class OffsetCommitRequest(groupId: String,
offsetCommitRequest.append("; CorrelationId: " + correlationId)
offsetCommitRequest.append("; ClientId: " + clientId)
offsetCommitRequest.append("; GroupId: " + groupId)
+ offsetCommitRequest.append("; GroupGenerationId: " + groupGenerationId)
+ offsetCommitRequest.append("; ConsumerId: " + consumerId)
if(details)
offsetCommitRequest.append("; RequestInfo: " + requestInfo.mkString(","))
offsetCommitRequest.toString()
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/OffsetCommitResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/OffsetCommitResponse.scala b/core/src/main/scala/kafka/api/OffsetCommitResponse.scala
index 4946e97..624a1c1 100644
--- a/core/src/main/scala/kafka/api/OffsetCommitResponse.scala
+++ b/core/src/main/scala/kafka/api/OffsetCommitResponse.scala
@@ -42,8 +42,8 @@ object OffsetCommitResponse extends Logging {
}
case class OffsetCommitResponse(commitStatus: Map[TopicAndPartition, Short],
- override val correlationId: Int = 0)
- extends RequestOrResponse(correlationId=correlationId) {
+ correlationId: Int = 0)
+ extends RequestOrResponse() {
lazy val commitStatusGroupedByTopic = commitStatus.groupBy(_._1.topic)
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/OffsetFetchRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala
index a32f858..c7604b9 100644
--- a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala
+++ b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala
@@ -52,9 +52,9 @@ object OffsetFetchRequest extends Logging {
case class OffsetFetchRequest(groupId: String,
requestInfo: Seq[TopicAndPartition],
versionId: Short = OffsetFetchRequest.CurrentVersion,
- override val correlationId: Int = 0,
+ correlationId: Int = 0,
clientId: String = OffsetFetchRequest.DefaultClientId)
- extends RequestOrResponse(Some(RequestKeys.OffsetFetchKey), correlationId) {
+ extends RequestOrResponse(Some(RequestKeys.OffsetFetchKey)) {
lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_.topic)
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/OffsetFetchResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/OffsetFetchResponse.scala b/core/src/main/scala/kafka/api/OffsetFetchResponse.scala
index c1222f4..e3523f8 100644
--- a/core/src/main/scala/kafka/api/OffsetFetchResponse.scala
+++ b/core/src/main/scala/kafka/api/OffsetFetchResponse.scala
@@ -45,8 +45,8 @@ object OffsetFetchResponse extends Logging {
}
case class OffsetFetchResponse(requestInfo: Map[TopicAndPartition, OffsetMetadataAndError],
- override val correlationId: Int = 0)
- extends RequestOrResponse(correlationId = correlationId) {
+ correlationId: Int = 0)
+ extends RequestOrResponse() {
lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic)
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/OffsetRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/OffsetRequest.scala b/core/src/main/scala/kafka/api/OffsetRequest.scala
index 7cbc26c..3d483bc 100644
--- a/core/src/main/scala/kafka/api/OffsetRequest.scala
+++ b/core/src/main/scala/kafka/api/OffsetRequest.scala
@@ -57,10 +57,10 @@ case class PartitionOffsetRequestInfo(time: Long, maxNumOffsets: Int)
case class OffsetRequest(requestInfo: Map[TopicAndPartition, PartitionOffsetRequestInfo],
versionId: Short = OffsetRequest.CurrentVersion,
- override val correlationId: Int = 0,
+ correlationId: Int = 0,
clientId: String = OffsetRequest.DefaultClientId,
replicaId: Int = Request.OrdinaryConsumerId)
- extends RequestOrResponse(Some(RequestKeys.OffsetsKey), correlationId) {
+ extends RequestOrResponse(Some(RequestKeys.OffsetsKey)) {
def this(requestInfo: Map[TopicAndPartition, PartitionOffsetRequestInfo], correlationId: Int, replicaId: Int) = this(requestInfo, OffsetRequest.CurrentVersion, correlationId, OffsetRequest.DefaultClientId, replicaId)
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/OffsetResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/OffsetResponse.scala b/core/src/main/scala/kafka/api/OffsetResponse.scala
index 0e1d6e3..63c0899 100644
--- a/core/src/main/scala/kafka/api/OffsetResponse.scala
+++ b/core/src/main/scala/kafka/api/OffsetResponse.scala
@@ -51,9 +51,9 @@ case class PartitionOffsetsResponse(error: Short, offsets: Seq[Long]) {
}
-case class OffsetResponse(override val correlationId: Int,
+case class OffsetResponse(correlationId: Int,
partitionErrorAndOffsets: Map[TopicAndPartition, PartitionOffsetsResponse])
- extends RequestOrResponse(correlationId = correlationId) {
+ extends RequestOrResponse() {
lazy val offsetsGroupedByTopic = partitionErrorAndOffsets.groupBy(_._1.topic)
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/ProducerRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/ProducerRequest.scala b/core/src/main/scala/kafka/api/ProducerRequest.scala
index 0c295a2..b2366e7 100644
--- a/core/src/main/scala/kafka/api/ProducerRequest.scala
+++ b/core/src/main/scala/kafka/api/ProducerRequest.scala
@@ -53,12 +53,12 @@ object ProducerRequest {
}
case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion,
- override val correlationId: Int,
+ correlationId: Int,
clientId: String,
requiredAcks: Short,
ackTimeoutMs: Int,
data: collection.mutable.Map[TopicAndPartition, ByteBufferMessageSet])
- extends RequestOrResponse(Some(RequestKeys.ProduceKey), correlationId) {
+ extends RequestOrResponse(Some(RequestKeys.ProduceKey)) {
/**
* Partitions the data into a map of maps (one for each topic).
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/ProducerResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/ProducerResponse.scala b/core/src/main/scala/kafka/api/ProducerResponse.scala
index 5a1d801..a286272 100644
--- a/core/src/main/scala/kafka/api/ProducerResponse.scala
+++ b/core/src/main/scala/kafka/api/ProducerResponse.scala
@@ -43,9 +43,9 @@ object ProducerResponse {
case class ProducerResponseStatus(var error: Short, offset: Long)
-case class ProducerResponse(override val correlationId: Int,
+case class ProducerResponse(correlationId: Int,
status: Map[TopicAndPartition, ProducerResponseStatus])
- extends RequestOrResponse(correlationId = correlationId) {
+ extends RequestOrResponse() {
/**
* Partitions the status map into a map of maps (one for each topic).
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/RequestKeys.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/RequestKeys.scala b/core/src/main/scala/kafka/api/RequestKeys.scala
index fbfc9d3..c24c034 100644
--- a/core/src/main/scala/kafka/api/RequestKeys.scala
+++ b/core/src/main/scala/kafka/api/RequestKeys.scala
@@ -32,6 +32,8 @@ object RequestKeys {
val OffsetCommitKey: Short = 8
val OffsetFetchKey: Short = 9
val ConsumerMetadataKey: Short = 10
+ val JoinGroupKey: Short = 11
+ val HeartbeatKey: Short = 12
val keyToNameAndDeserializerMap: Map[Short, (String, (ByteBuffer) => RequestOrResponse)]=
Map(ProduceKey -> ("Produce", ProducerRequest.readFrom),
@@ -44,7 +46,10 @@ object RequestKeys {
ControlledShutdownKey -> ("ControlledShutdown", ControlledShutdownRequest.readFrom),
OffsetCommitKey -> ("OffsetCommit", OffsetCommitRequest.readFrom),
OffsetFetchKey -> ("OffsetFetch", OffsetFetchRequest.readFrom),
- ConsumerMetadataKey -> ("ConsumerMetadata", ConsumerMetadataRequest.readFrom))
+ ConsumerMetadataKey -> ("ConsumerMetadata", ConsumerMetadataRequest.readFrom),
+ JoinGroupKey -> ("JoinGroup", JoinGroupRequestAndHeader.readFrom),
+ HeartbeatKey -> ("Heartbeat", HeartbeatRequestAndHeader.readFrom)
+ )
def nameForKey(key: Short): String = {
keyToNameAndDeserializerMap.get(key) match {
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/RequestOrResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/RequestOrResponse.scala b/core/src/main/scala/kafka/api/RequestOrResponse.scala
index 57f87a4..0334343 100644
--- a/core/src/main/scala/kafka/api/RequestOrResponse.scala
+++ b/core/src/main/scala/kafka/api/RequestOrResponse.scala
@@ -30,7 +30,7 @@ object Request {
}
-private[kafka] abstract class RequestOrResponse(val requestId: Option[Short] = None, val correlationId: Int) extends Logging {
+private[kafka] abstract class RequestOrResponse(val requestId: Option[Short] = None) extends Logging {
def sizeInBytes: Int
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/StopReplicaRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/StopReplicaRequest.scala b/core/src/main/scala/kafka/api/StopReplicaRequest.scala
index 68fc138..5e14987 100644
--- a/core/src/main/scala/kafka/api/StopReplicaRequest.scala
+++ b/core/src/main/scala/kafka/api/StopReplicaRequest.scala
@@ -54,13 +54,13 @@ object StopReplicaRequest extends Logging {
}
case class StopReplicaRequest(versionId: Short,
- override val correlationId: Int,
+ correlationId: Int,
clientId: String,
controllerId: Int,
controllerEpoch: Int,
deletePartitions: Boolean,
partitions: Set[TopicAndPartition])
- extends RequestOrResponse(Some(RequestKeys.StopReplicaKey), correlationId) {
+ extends RequestOrResponse(Some(RequestKeys.StopReplicaKey)) {
def this(deletePartitions: Boolean, partitions: Set[TopicAndPartition], controllerId: Int, controllerEpoch: Int, correlationId: Int) = {
this(StopReplicaRequest.CurrentVersion, correlationId, StopReplicaRequest.DefaultClientId,
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/StopReplicaResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/StopReplicaResponse.scala b/core/src/main/scala/kafka/api/StopReplicaResponse.scala
index c90ddee..3431f3f 100644
--- a/core/src/main/scala/kafka/api/StopReplicaResponse.scala
+++ b/core/src/main/scala/kafka/api/StopReplicaResponse.scala
@@ -42,10 +42,10 @@ object StopReplicaResponse {
}
-case class StopReplicaResponse(override val correlationId: Int,
+case class StopReplicaResponse(val correlationId: Int,
val responseMap: Map[TopicAndPartition, Short],
val errorCode: Short = ErrorMapping.NoError)
- extends RequestOrResponse(correlationId = correlationId) {
+ extends RequestOrResponse() {
def sizeInBytes(): Int ={
var size =
4 /* correlation id */ +
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/TopicMetadataRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala
index bce004f..7dca09c 100644
--- a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala
+++ b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala
@@ -47,10 +47,10 @@ object TopicMetadataRequest extends Logging {
}
case class TopicMetadataRequest(val versionId: Short,
- override val correlationId: Int,
+ val correlationId: Int,
val clientId: String,
val topics: Seq[String])
- extends RequestOrResponse(Some(RequestKeys.MetadataKey), correlationId){
+ extends RequestOrResponse(Some(RequestKeys.MetadataKey)){
def this(topics: Seq[String], correlationId: Int) =
this(TopicMetadataRequest.CurrentVersion, correlationId, TopicMetadataRequest.DefaultClientId, topics)
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/TopicMetadataResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala
index b233d35..92ac4e6 100644
--- a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala
+++ b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala
@@ -35,8 +35,8 @@ object TopicMetadataResponse {
case class TopicMetadataResponse(brokers: Seq[Broker],
topicsMetadata: Seq[TopicMetadata],
- override val correlationId: Int)
- extends RequestOrResponse(correlationId = correlationId) {
+ correlationId: Int)
+ extends RequestOrResponse() {
val sizeInBytes: Int = {
4 + 4 + brokers.map(_.sizeInBytes).sum + 4 + topicsMetadata.map(_.sizeInBytes).sum
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala
index 543e262..530982e 100644
--- a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala
+++ b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala
@@ -55,13 +55,13 @@ object UpdateMetadataRequest {
}
case class UpdateMetadataRequest (versionId: Short,
- override val correlationId: Int,
+ correlationId: Int,
clientId: String,
controllerId: Int,
controllerEpoch: Int,
partitionStateInfos: Map[TopicAndPartition, PartitionStateInfo],
aliveBrokers: Set[Broker])
- extends RequestOrResponse(Some(RequestKeys.UpdateMetadataKey), correlationId) {
+ extends RequestOrResponse(Some(RequestKeys.UpdateMetadataKey)) {
def this(controllerId: Int, controllerEpoch: Int, correlationId: Int, clientId: String,
partitionStateInfos: Map[TopicAndPartition, PartitionStateInfo], aliveBrokers: Set[Broker]) = {
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala b/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala
index c583c1f..53f6067 100644
--- a/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala
+++ b/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala
@@ -32,9 +32,9 @@ object UpdateMetadataResponse {
}
}
-case class UpdateMetadataResponse(override val correlationId: Int,
+case class UpdateMetadataResponse(correlationId: Int,
errorCode: Short = ErrorMapping.NoError)
- extends RequestOrResponse(correlationId = correlationId) {
+ extends RequestOrResponse() {
def sizeInBytes(): Int = 4 /* correlation id */ + 2 /* error code */
def writeTo(buffer: ByteBuffer) {
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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 8763968..ecbfa0f 100644
--- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
+++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
@@ -133,9 +133,9 @@ class RequestSendThread(val controllerId: Int,
isSendSuccessful = true
} catch {
case e: Throwable => // if the send was not successful, reconnect to broker and resend the message
- error(("Controller %d epoch %d failed to send %s request with correlation id %s to broker %s. " +
+ error(("Controller %d epoch %d failed to send request %s to broker %s. " +
"Reconnecting to broker.").format(controllerId, controllerContext.epoch,
- RequestKeys.nameForKey(request.requestId.get), request.correlationId, toBroker.toString()), e)
+ request.toString, toBroker.toString()), e)
channel.disconnect()
connectToBroker(toBroker, channel)
isSendSuccessful = false
@@ -153,8 +153,8 @@ class RequestSendThread(val controllerId: Int,
case RequestKeys.UpdateMetadataKey =>
response = UpdateMetadataResponse.readFrom(receive.buffer)
}
- stateChangeLogger.trace("Controller %d epoch %d received response correlationId %d for a request sent to broker %s"
- .format(controllerId, controllerContext.epoch, response.correlationId, toBroker.toString()))
+ stateChangeLogger.trace("Controller %d epoch %d received response %s for a request sent to broker %s"
+ .format(controllerId, controllerContext.epoch, response.toString, toBroker.toString))
if(callback != null) {
callback(response)
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala b/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala
index 08dcc55..27fc1eb 100644
--- a/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala
+++ b/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala
@@ -21,7 +21,6 @@ import kafka.common.{OffsetAndMetadata, TopicAndPartition}
class OffsetCommitRequest(groupId: String,
requestInfo: java.util.Map[TopicAndPartition, OffsetAndMetadata],
- versionId: Short,
correlationId: Int,
clientId: String) {
val underlying = {
@@ -33,7 +32,6 @@ class OffsetCommitRequest(groupId: String,
kafka.api.OffsetCommitRequest(
groupId = groupId,
requestInfo = scalaMap,
- versionId = versionId,
correlationId = correlationId,
clientId = clientId
)
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala
index 7e6da16..b0b7be1 100644
--- a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala
+++ b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala
@@ -24,10 +24,10 @@ import kafka.common.ErrorMapping
import kafka.network.RequestChannel.Response
class TopicMetadataRequest(val versionId: Short,
- override val correlationId: Int,
+ val correlationId: Int,
val clientId: String,
val topics: java.util.List[String])
- extends RequestOrResponse(Some(kafka.api.RequestKeys.MetadataKey), correlationId) {
+ extends RequestOrResponse(Some(kafka.api.RequestKeys.MetadataKey)) {
val underlying: kafka.api.TopicMetadataRequest = {
import scala.collection.JavaConversions._
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
index d34ddf5..847a36b 100644
--- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
+++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
@@ -23,9 +23,14 @@ import junit.framework.Assert._
import java.nio.ByteBuffer
import kafka.message.{Message, ByteBufferMessageSet}
import kafka.cluster.Broker
-import kafka.common.{OffsetAndMetadata, TopicAndPartition, ErrorMapping, OffsetMetadataAndError}
-import kafka.controller.LeaderIsrAndControllerEpoch
+import kafka.common.{OffsetAndMetadata, ErrorMapping, OffsetMetadataAndError}
import kafka.utils.SystemTime
+import org.apache.kafka.common.requests._
+import org.apache.kafka.common.protocol.ApiKeys
+import scala.Some
+import kafka.controller.LeaderIsrAndControllerEpoch
+import kafka.common.TopicAndPartition
+import org.apache.kafka.common.TopicPartition
object SerializationTestUtils {
@@ -146,13 +151,23 @@ object SerializationTestUtils {
new TopicMetadataResponse(brokers, Seq(topicmetaData1, topicmetaData2), 1)
}
- def createTestOffsetCommitRequest: OffsetCommitRequest = {
+ def createTestOffsetCommitRequestV1: OffsetCommitRequest = {
new OffsetCommitRequest("group 1", collection.immutable.Map(
TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L, metadata="some metadata", timestamp=SystemTime.milliseconds),
TopicAndPartition(topic1, 1) -> OffsetAndMetadata(offset=100L, metadata=OffsetAndMetadata.NoMetadata, timestamp=SystemTime.milliseconds)
))
}
+ def createTestOffsetCommitRequestV0: OffsetCommitRequest = {
+ new OffsetCommitRequest(
+ versionId = 0,
+ groupId = "group 1",
+ requestInfo = collection.immutable.Map(
+ TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L, metadata="some metadata", timestamp=SystemTime.milliseconds),
+ TopicAndPartition(topic1, 1) -> OffsetAndMetadata(offset=100L, metadata=OffsetAndMetadata.NoMetadata, timestamp=SystemTime.milliseconds)
+ ))
+ }
+
def createTestOffsetCommitResponse: OffsetCommitResponse = {
new OffsetCommitResponse(collection.immutable.Map(TopicAndPartition(topic1, 0) -> ErrorMapping.NoError,
TopicAndPartition(topic1, 1) -> ErrorMapping.NoError))
@@ -180,6 +195,31 @@ object SerializationTestUtils {
ConsumerMetadataResponse(Some(brokers.head), ErrorMapping.NoError)
}
+ def createHeartbeatRequestAndHeader: HeartbeatRequestAndHeader = {
+ val header = new RequestHeader(ApiKeys.HEARTBEAT.id, 0.asInstanceOf[Short], "", 1)
+ val body = new HeartbeatRequest("group1", 1, "consumer1")
+ HeartbeatRequestAndHeader(header, body)
+ }
+
+ def createHeartbeatResponseAndHeader: HeartbeatResponseAndHeader = {
+ val header = new ResponseHeader(1)
+ val body = new HeartbeatResponse(0.asInstanceOf[Short])
+ HeartbeatResponseAndHeader(header, body)
+ }
+
+ def createJoinGroupRequestAndHeader: JoinGroupRequestAndHeader = {
+ import scala.collection.JavaConversions._
+ val header = new RequestHeader(ApiKeys.JOIN_GROUP.id, 0.asInstanceOf[Short], "", 1)
+ val body = new JoinGroupRequest("group1", 30000, List("topic1"), "consumer1", "strategy1");
+ JoinGroupRequestAndHeader(header, body)
+ }
+
+ def createJoinGroupResponseAndHeader: JoinGroupResponseAndHeader = {
+ import scala.collection.JavaConversions._
+ val header = new ResponseHeader(1)
+ val body = new JoinGroupResponse(0.asInstanceOf[Short], 1, "consumer1", List(new TopicPartition("test11", 1)))
+ JoinGroupResponseAndHeader(header, body)
+ }
}
class RequestResponseSerializationTest extends JUnitSuite {
@@ -194,27 +234,31 @@ class RequestResponseSerializationTest extends JUnitSuite {
private val offsetResponse = SerializationTestUtils.createTestOffsetResponse
private val topicMetadataRequest = SerializationTestUtils.createTestTopicMetadataRequest
private val topicMetadataResponse = SerializationTestUtils.createTestTopicMetadataResponse
- private val offsetCommitRequest = SerializationTestUtils.createTestOffsetCommitRequest
+ private val offsetCommitRequestV0 = SerializationTestUtils.createTestOffsetCommitRequestV0
+ private val offsetCommitRequestV1 = SerializationTestUtils.createTestOffsetCommitRequestV1
private val offsetCommitResponse = SerializationTestUtils.createTestOffsetCommitResponse
private val offsetFetchRequest = SerializationTestUtils.createTestOffsetFetchRequest
private val offsetFetchResponse = SerializationTestUtils.createTestOffsetFetchResponse
private val consumerMetadataRequest = SerializationTestUtils.createConsumerMetadataRequest
private val consumerMetadataResponse = SerializationTestUtils.createConsumerMetadataResponse
private val consumerMetadataResponseNoCoordinator = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode)
+ private val heartbeatRequest = SerializationTestUtils.createHeartbeatRequestAndHeader
+ private val heartbeatResponse = SerializationTestUtils.createHeartbeatResponseAndHeader
+ private val joinGroupRequest = SerializationTestUtils.createJoinGroupRequestAndHeader
+ private val joinGroupResponse = SerializationTestUtils.createJoinGroupResponseAndHeader
@Test
def testSerializationAndDeserialization() {
val requestsAndResponses =
- collection.immutable.Seq(leaderAndIsrRequest, leaderAndIsrResponse,
- stopReplicaRequest, stopReplicaResponse,
- producerRequest, producerResponse,
- fetchRequest,
- offsetRequest, offsetResponse,
- topicMetadataRequest, topicMetadataResponse,
- offsetCommitRequest, offsetCommitResponse,
- offsetFetchRequest, offsetFetchResponse,
- consumerMetadataRequest, consumerMetadataResponse, consumerMetadataResponseNoCoordinator)
+ collection.immutable.Seq(leaderAndIsrRequest, leaderAndIsrResponse, stopReplicaRequest,
+ stopReplicaResponse, producerRequest, producerResponse,
+ fetchRequest, offsetRequest, offsetResponse, topicMetadataRequest,
+ topicMetadataResponse, offsetCommitRequestV0, offsetCommitRequestV1,
+ offsetCommitResponse, offsetFetchRequest, offsetFetchResponse,
+ consumerMetadataRequest, consumerMetadataResponse,
+ consumerMetadataResponseNoCoordinator, heartbeatRequest,
+ heartbeatResponse, joinGroupRequest, joinGroupResponse)
requestsAndResponses.foreach { original =>
val buffer = ByteBuffer.allocate(original.sizeInBytes)
@@ -222,7 +266,9 @@ class RequestResponseSerializationTest extends JUnitSuite {
buffer.rewind()
val deserializer = original.getClass.getDeclaredMethod("readFrom", classOf[ByteBuffer])
val deserialized = deserializer.invoke(null, buffer)
- assertEquals("The original and deserialized request/response should be the same.", original, deserialized)
+ assertFalse("All serialized bytes in " + original.getClass.getSimpleName + " should have been consumed",
+ buffer.hasRemaining)
+ assertEquals("The original and deserialized for " + original.getClass.getSimpleName + " should be the same.", original, deserialized)
}
}
}
[05/37] git commit: KAFKA-1308 Publish jar of test utilities to Maven
Posted by jj...@apache.org.
KAFKA-1308 Publish jar of test utilities to Maven
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/b8d87d0b
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/b8d87d0b
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/b8d87d0b
Branch: refs/heads/transactional_messaging
Commit: b8d87d0b22cc4e87bac2e9fdfe761b23734851ff
Parents: e3dfad3
Author: Jakob Homan <jg...@gmail.com>
Authored: Wed Jul 2 15:01:06 2014 -0700
Committer: Jakob Homan <jg...@gmail.com>
Committed: Wed Jul 2 15:01:06 2014 -0700
----------------------------------------------------------------------
build.gradle | 7 +++++--
1 file changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/b8d87d0b/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 2f4167f..a72905d 100644
--- a/build.gradle
+++ b/build.gradle
@@ -272,7 +272,7 @@ project(':core') {
}
task testJar(type: Jar) {
- appendix = 'test'
+ classifier = 'test'
from sourceSets.test.output
}
@@ -283,6 +283,9 @@ project(':core') {
}
}
+ artifacts {
+ archives testJar
+ }
}
project(':perf') {
@@ -372,7 +375,7 @@ project(':clients') {
}
task testJar(type: Jar) {
- appendix = 'test'
+ classifier = 'test'
from sourceSets.test.output
}
[36/37] git commit: KAFKA-1550;
Patch review tool should use git format-patch for patch generation;
reviewed by Guozhang Wang and Joel Koshy
Posted by jj...@apache.org.
KAFKA-1550; Patch review tool should use git format-patch for patch generation; reviewed by Guozhang Wang and Joel Koshy
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/09690e33
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/09690e33
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/09690e33
Branch: refs/heads/transactional_messaging
Commit: 09690e339aabea6e9141f7e2ddd50449cc1c9180
Parents: 0386790
Author: Dong Lin <li...@gmail.com>
Authored: Tue Aug 5 14:28:11 2014 -0700
Committer: Joel Koshy <jj...@gmail.com>
Committed: Tue Aug 5 14:47:51 2014 -0700
----------------------------------------------------------------------
kafka-patch-review.py | 6 ++++--
1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/09690e33/kafka-patch-review.py
----------------------------------------------------------------------
diff --git a/kafka-patch-review.py b/kafka-patch-review.py
index dc45549..89afc84 100644
--- a/kafka-patch-review.py
+++ b/kafka-patch-review.py
@@ -105,11 +105,13 @@ def main():
print 'ERROR: Your reviewboard was not created/updated. Please run the script with the --debug option to troubleshoot the problem'
p.close()
sys.exit(1)
- p.close()
+ if p.close() != None:
+ print 'ERROR: reviewboard update failed. Exiting.'
+ sys.exit(1)
if opt.debug:
print 'rb url=',rb_url
- git_command="git diff " + opt.branch + " > " + patch_file
+ git_command="git format-patch " + opt.branch + " --stdout > " + patch_file
if opt.debug:
print git_command
p=os.popen(git_command)
[08/37] git commit: KAFKA-1519 Make it possible to disable the line
seperator in the console consumer. Patch from Gwen Shapira.
Posted by jj...@apache.org.
KAFKA-1519 Make it possible to disable the line seperator in the console consumer. Patch from Gwen Shapira.
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/6de56b30
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/6de56b30
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/6de56b30
Branch: refs/heads/transactional_messaging
Commit: 6de56b30b0f9aee19bf110431bcb4511d9b748f2
Parents: b1a1cae
Author: Jay Kreps <ja...@gmail.com>
Authored: Thu Jul 3 21:15:01 2014 -0700
Committer: Jay Kreps <ja...@gmail.com>
Committed: Thu Jul 3 21:15:01 2014 -0700
----------------------------------------------------------------------
.../scala/kafka/utils/CommandLineUtils.scala | 17 ++++---
.../unit/kafka/utils/CommandLineUtilsTest.scala | 49 ++++++++++++++++++++
2 files changed, 59 insertions(+), 7 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/6de56b30/core/src/main/scala/kafka/utils/CommandLineUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/utils/CommandLineUtils.scala b/core/src/main/scala/kafka/utils/CommandLineUtils.scala
index 1ba605c..086a624 100644
--- a/core/src/main/scala/kafka/utils/CommandLineUtils.scala
+++ b/core/src/main/scala/kafka/utils/CommandLineUtils.scala
@@ -60,14 +60,17 @@ object CommandLineUtils extends Logging {
* Parse key-value pairs in the form key=value
*/
def parseKeyValueArgs(args: Iterable[String]): Properties = {
- val splits = args.map(_ split "=").filterNot(_ == null).filterNot(_.length == 0)
- if(!splits.forall(_.length == 2)) {
- System.err.println("Invalid command line properties: " + args.mkString(" "))
- System.exit(1)
- }
+ val splits = args.map(_ split "=").filterNot(_.length == 0)
+
val props = new Properties
- for(a <- splits)
- props.put(a(0), a(1))
+ for(a <- splits) {
+ if (a.length == 1) props.put(a(0), "")
+ else if (a.length == 2) props.put(a(0), a(1))
+ else {
+ System.err.println("Invalid command line properties: " + args.mkString(" "))
+ System.exit(1)
+ }
+ }
props
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kafka/blob/6de56b30/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala
new file mode 100644
index 0000000..e832a01
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.utils
+
+import junit.framework.Assert._
+import org.junit.{Test, After, Before}
+import kafka.utils.CommandLineUtils;
+
+class CommandLineUtilsTest {
+
+
+ @Test
+ def testParseEmptyArg() {
+ val argArray = Array("my.empty.property=")
+ val props = CommandLineUtils.parseKeyValueArgs(argArray)
+ assertEquals("Empty value should be equal to empty string",props.getProperty("my.empty.property"),"")
+ }
+
+ @Test
+ def testParseSingleArg() {
+ val argArray = Array("my.property=value")
+ val props = CommandLineUtils.parseKeyValueArgs(argArray)
+ assertEquals("Value of a single property should be 'value' ",props.getProperty("my.property"),"value")
+ }
+
+ @Test
+ def testParseArgs() {
+ val argArray = Array("first.property=first","second.property=second")
+ val props = CommandLineUtils.parseKeyValueArgs(argArray)
+ assertEquals("Value of first property should be 'first'",props.getProperty("first.property"),"first")
+ assertEquals("Value of second property should be 'second'",props.getProperty("second.property"),"second")
+ }
+
+}
[26/37] git commit: KAFKA-1544 Log cleaner takes a long time to shut
down. Patch from Manikumar Reddy.
Posted by jj...@apache.org.
KAFKA-1544 Log cleaner takes a long time to shut down. Patch from Manikumar Reddy.
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/db41f98e
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/db41f98e
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/db41f98e
Branch: refs/heads/transactional_messaging
Commit: db41f98ea943d806362487b8f8192360a488eb39
Parents: 3f1a9c4
Author: Jay Kreps <ja...@gmail.com>
Authored: Thu Jul 24 15:53:24 2014 -0700
Committer: Jay Kreps <ja...@gmail.com>
Committed: Thu Jul 24 16:34:40 2014 -0700
----------------------------------------------------------------------
core/src/main/scala/kafka/log/LogCleaner.scala | 14 ++++++++++++--
1 file changed, 12 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/db41f98e/core/src/main/scala/kafka/log/LogCleaner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala
index 2faa196..afbeffc 100644
--- a/core/src/main/scala/kafka/log/LogCleaner.scala
+++ b/core/src/main/scala/kafka/log/LogCleaner.scala
@@ -28,6 +28,8 @@ import kafka.utils._
import kafka.metrics.KafkaMetricsGroup
import com.yammer.metrics.core.Gauge
import java.lang.IllegalStateException
+import java.util.concurrent.CountDownLatch
+import java.util.concurrent.TimeUnit
/**
* The cleaner is responsible for removing obsolete records from logs which have the dedupe retention strategy.
@@ -173,7 +175,8 @@ class LogCleaner(val config: CleanerConfig,
checkDone = checkDone)
@volatile var lastStats: CleanerStats = new CleanerStats()
-
+ private val backOffWaitLatch = new CountDownLatch(1)
+
private def checkDone(topicAndPartition: TopicAndPartition) {
if (!isRunning.get())
throw new ThreadShutdownException
@@ -187,6 +190,13 @@ class LogCleaner(val config: CleanerConfig,
cleanOrSleep()
}
+
+ override def shutdown() = {
+ initiateShutdown()
+ backOffWaitLatch.countDown()
+ awaitShutdown()
+ }
+
/**
* Clean a log if there is a dirty log available, otherwise sleep for a bit
*/
@@ -194,7 +204,7 @@ class LogCleaner(val config: CleanerConfig,
cleanerManager.grabFilthiestLog() match {
case None =>
// there are no cleanable logs, sleep a while
- time.sleep(config.backOffMs)
+ backOffWaitLatch.await(config.backOffMs, TimeUnit.MILLISECONDS)
case Some(cleanable) =>
// there's a log, clean it
var endOffset = cleanable.firstDirtyOffset
[22/37] git commit: KAFKA-1539 Fsync offset checkpoint file after
writing.
Posted by jj...@apache.org.
KAFKA-1539 Fsync offset checkpoint file after writing.
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/1e4b0841
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/1e4b0841
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/1e4b0841
Branch: refs/heads/transactional_messaging
Commit: 1e4b0841b37e9e6526d7a7a7c643b1369d9f03c5
Parents: 592678e
Author: Jay Kreps <ja...@gmail.com>
Authored: Mon Jul 21 10:22:50 2014 -0700
Committer: Jay Kreps <ja...@gmail.com>
Committed: Mon Jul 21 10:22:50 2014 -0700
----------------------------------------------------------------------
core/src/main/scala/kafka/server/OffsetCheckpoint.scala | 6 ++++--
1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/1e4b0841/core/src/main/scala/kafka/server/OffsetCheckpoint.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/OffsetCheckpoint.scala b/core/src/main/scala/kafka/server/OffsetCheckpoint.scala
index 7af2f43..8c5b054 100644
--- a/core/src/main/scala/kafka/server/OffsetCheckpoint.scala
+++ b/core/src/main/scala/kafka/server/OffsetCheckpoint.scala
@@ -34,7 +34,8 @@ class OffsetCheckpoint(val file: File) extends Logging {
// write to temp file and then swap with the existing file
val temp = new File(file.getAbsolutePath + ".tmp")
- val writer = new BufferedWriter(new FileWriter(temp))
+ val fileOutputStream = new FileOutputStream(temp)
+ val writer = new BufferedWriter(new OutputStreamWriter(fileOutputStream))
try {
// write the current version
writer.write(0.toString)
@@ -50,8 +51,9 @@ class OffsetCheckpoint(val file: File) extends Logging {
writer.newLine()
}
- // flush and overwrite old file
+ // flush the buffer and then fsync the underlying file
writer.flush()
+ fileOutputStream.getFD().sync()
} finally {
writer.close()
}
[27/37] git commit: KAFKA-1483 Split Brain about Leader Partitions;
reviewed by Guozhang, Jun and Neha
Posted by jj...@apache.org.
KAFKA-1483 Split Brain about Leader Partitions; reviewed by Guozhang, Jun and Neha
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/d9e5080d
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/d9e5080d
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/d9e5080d
Branch: refs/heads/transactional_messaging
Commit: d9e5080dfceff12ea599f8d518ad402ba7d54c9d
Parents: db41f98
Author: Sriharsha Chintalapani <sc...@hortonworks.com>
Authored: Thu Jul 24 16:48:21 2014 -0700
Committer: Neha Narkhede <ne...@gmail.com>
Committed: Thu Jul 24 16:48:41 2014 -0700
----------------------------------------------------------------------
.../scala/kafka/server/ReplicaManager.scala | 35 +++++---------------
1 file changed, 9 insertions(+), 26 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/d9e5080d/core/src/main/scala/kafka/server/ReplicaManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala
index 6a56a77..897783c 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -36,9 +36,9 @@ object ReplicaManager {
val HighWatermarkFilename = "replication-offset-checkpoint"
}
-class ReplicaManager(val config: KafkaConfig,
- time: Time,
- val zkClient: ZkClient,
+class ReplicaManager(val config: KafkaConfig,
+ time: Time,
+ val zkClient: ZkClient,
scheduler: Scheduler,
val logManager: LogManager,
val isShuttingDown: AtomicBoolean ) extends Logging with KafkaMetricsGroup {
@@ -46,8 +46,6 @@ class ReplicaManager(val config: KafkaConfig,
@volatile var controllerEpoch: Int = KafkaController.InitialControllerEpoch - 1
private val localBrokerId = config.brokerId
private val allPartitions = new Pool[(String, Int), Partition]
- private var leaderPartitions = new mutable.HashSet[Partition]()
- private val leaderPartitionsLock = new Object
private val replicaStateChangeLock = new Object
val replicaFetcherManager = new ReplicaFetcherManager(config, this)
private val highWatermarkCheckPointThreadStarted = new AtomicBoolean(false)
@@ -60,9 +58,7 @@ class ReplicaManager(val config: KafkaConfig,
"LeaderCount",
new Gauge[Int] {
def value = {
- leaderPartitionsLock synchronized {
- leaderPartitions.size
- }
+ getLeaderPartitions().size
}
}
)
@@ -82,9 +78,7 @@ class ReplicaManager(val config: KafkaConfig,
val isrShrinkRate = newMeter("IsrShrinksPerSec", "shrinks", TimeUnit.SECONDS)
def underReplicatedPartitionCount(): Int = {
- leaderPartitionsLock synchronized {
- leaderPartitions.count(_.isUnderReplicated)
- }
+ getLeaderPartitions().count(_.isUnderReplicated)
}
def startHighWaterMarksCheckPointThread() = {
@@ -117,9 +111,6 @@ class ReplicaManager(val config: KafkaConfig,
val errorCode = ErrorMapping.NoError
getPartition(topic, partitionId) match {
case Some(partition) =>
- leaderPartitionsLock synchronized {
- leaderPartitions -= partition
- }
if(deletePartition) {
val removedPartition = allPartitions.remove((topic, partitionId))
if (removedPartition != null)
@@ -331,10 +322,6 @@ class ReplicaManager(val config: KafkaConfig,
partitionState.foreach{ case (partition, partitionStateInfo) =>
partition.makeLeader(controllerId, partitionStateInfo, correlationId, offsetManager)}
- // Finally add these partitions to the list of partitions for which the leader is the current broker
- leaderPartitionsLock synchronized {
- leaderPartitions ++= partitionState.keySet
- }
} catch {
case e: Throwable =>
partitionState.foreach { state =>
@@ -383,9 +370,6 @@ class ReplicaManager(val config: KafkaConfig,
responseMap.put((partition.topic, partition.partitionId), ErrorMapping.NoError)
try {
- leaderPartitionsLock synchronized {
- leaderPartitions --= partitionState.keySet
- }
var partitionsToMakeFollower: Set[Partition] = Set()
@@ -464,11 +448,7 @@ class ReplicaManager(val config: KafkaConfig,
private def maybeShrinkIsr(): Unit = {
trace("Evaluating ISR list of partitions to see which replicas can be removed from the ISR")
- var curLeaderPartitions: List[Partition] = null
- leaderPartitionsLock synchronized {
- curLeaderPartitions = leaderPartitions.toList
- }
- curLeaderPartitions.foreach(partition => partition.maybeShrinkIsr(config.replicaLagTimeMaxMs, config.replicaLagMaxMessages))
+ allPartitions.values.foreach(partition => partition.maybeShrinkIsr(config.replicaLagTimeMaxMs, config.replicaLagMaxMessages))
}
def recordFollowerPosition(topic: String, partitionId: Int, replicaId: Int, offset: Long) = {
@@ -480,6 +460,9 @@ class ReplicaManager(val config: KafkaConfig,
}
}
+ private def getLeaderPartitions() : List[Partition] = {
+ allPartitions.values.filter(_.leaderReplicaIfLocal().isDefined).toList
+ }
/**
* Flushes the highwatermark value for all partitions to the highwatermark file
*/
[14/37] git commit: kafka-1529;
transient unit test failure in testAutoCreateAfterDeleteTopic;
patched by Jun Rao; reviewed by Guozhang Wang and Neha Narkhede
Posted by jj...@apache.org.
kafka-1529; transient unit test failure in testAutoCreateAfterDeleteTopic; patched by Jun Rao; reviewed by Guozhang Wang and Neha Narkhede
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/3e6b386b
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/3e6b386b
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/3e6b386b
Branch: refs/heads/transactional_messaging
Commit: 3e6b386b72b0d1a39621d7e166f2228c97646e1d
Parents: 4b3d03e
Author: Jun Rao <ju...@gmail.com>
Authored: Tue Jul 15 14:53:15 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Tue Jul 15 14:53:15 2014 -0700
----------------------------------------------------------------------
.../unit/kafka/admin/DeleteTopicTest.scala | 31 --------------------
1 file changed, 31 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/3e6b386b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
index 5d3c57a..29cc01b 100644
--- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
+++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
@@ -203,37 +203,6 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness {
}
@Test
- def testAutoCreateAfterDeleteTopic() {
- val topicAndPartition = TopicAndPartition("test", 0)
- val topic = topicAndPartition.topic
- val servers = createTestTopicAndCluster(topic)
- // start topic deletion
- AdminUtils.deleteTopic(zkClient, topic)
- verifyTopicDeletion(topic, servers)
- // test if first produce request after topic deletion auto creates the topic
- val props = new Properties()
- props.put("metadata.broker.list", servers.map(s => s.config.hostName + ":" + s.config.port).mkString(","))
- props.put("serializer.class", "kafka.serializer.StringEncoder")
- props.put("producer.type", "sync")
- props.put("request.required.acks", "1")
- props.put("message.send.max.retries", "1")
- val producerConfig = new ProducerConfig(props)
- val producer = new Producer[String, String](producerConfig)
- try {
- producer.send(new KeyedMessage[String, String](topic, "test", "test1"))
- } catch {
- case e: FailedToSendMessageException => fail("Topic should have been auto created")
- case oe: Throwable => fail("fails with exception", oe)
- }
- // test the topic path exists
- assertTrue("Topic not auto created", ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic)))
- // wait until leader is elected
- val leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000)
- assertTrue("New leader should be elected after re-creating topic test", leaderIdOpt.isDefined)
- servers.foreach(_.shutdown())
- }
-
- @Test
def testDeleteNonExistingTopic() {
val topicAndPartition = TopicAndPartition("test", 0)
val topic = topicAndPartition.topic
[20/37] git commit: kafka-1462;
Add new request and response formats for the new consumer and
coordinator communication; patched by Jun Rao;
reviewed by Guozhang Wang and Jay Kreps
Posted by jj...@apache.org.
kafka-1462; Add new request and response formats for the new consumer and coordinator communication; patched by Jun Rao; reviewed by Guozhang Wang and Jay Kreps
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/fc0e03f7
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/fc0e03f7
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/fc0e03f7
Branch: refs/heads/transactional_messaging
Commit: fc0e03f79131746da81c05c12e056862c08d79d4
Parents: 4ebcdfd
Author: Jun Rao <ju...@gmail.com>
Authored: Thu Jul 17 18:20:01 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Thu Jul 17 18:20:01 2014 -0700
----------------------------------------------------------------------
.../clients/producer/internals/Sender.java | 12 +-
.../java/org/apache/kafka/common/Cluster.java | 15 +-
.../apache/kafka/common/protocol/ApiKeys.java | 7 +-
.../apache/kafka/common/protocol/Protocol.java | 280 ++++++++++++++++++-
.../kafka/common/protocol/types/Struct.java | 67 +++++
.../requests/AbstractRequestResponse.java | 66 +++++
.../requests/ConsumerMetadataRequest.java | 47 ++++
.../requests/ConsumerMetadataResponse.java | 69 +++++
.../kafka/common/requests/FetchRequest.java | 132 +++++++++
.../kafka/common/requests/FetchResponse.java | 110 ++++++++
.../kafka/common/requests/HeartbeatRequest.java | 64 +++++
.../common/requests/HeartbeatResponse.java | 45 +++
.../kafka/common/requests/JoinGroupRequest.java | 87 ++++++
.../common/requests/JoinGroupResponse.java | 102 +++++++
.../common/requests/ListOffsetRequest.java | 114 ++++++++
.../common/requests/ListOffsetResponse.java | 108 +++++++
.../kafka/common/requests/MetadataRequest.java | 29 +-
.../kafka/common/requests/MetadataResponse.java | 98 +++++--
.../common/requests/OffsetCommitRequest.java | 180 ++++++++++++
.../common/requests/OffsetCommitResponse.java | 87 ++++++
.../common/requests/OffsetFetchRequest.java | 98 +++++++
.../common/requests/OffsetFetchResponse.java | 107 +++++++
.../kafka/common/requests/ProduceRequest.java | 124 +++++---
.../kafka/common/requests/ProduceResponse.java | 96 ++++---
.../kafka/common/requests/RequestHeader.java | 49 ++--
.../kafka/common/requests/ResponseHeader.java | 22 +-
.../kafka/common/utils/CollectionUtils.java | 62 ++++
.../apache/kafka/clients/NetworkClientTest.java | 4 +-
.../common/requests/RequestResponseTest.java | 173 ++++++++++++
.../kafka/api/ConsumerMetadataRequest.scala | 4 +-
.../kafka/api/ConsumerMetadataResponse.scala | 4 +-
.../kafka/api/ControlledShutdownRequest.scala | 4 +-
.../kafka/api/ControlledShutdownResponse.scala | 4 +-
.../src/main/scala/kafka/api/FetchRequest.scala | 4 +-
.../api/GenericRequestOrResponseAndHeader.scala | 45 +++
.../kafka/api/HeartbeatRequestAndHeader.scala | 39 +++
.../kafka/api/HeartbeatResponseAndHeader.scala | 28 ++
.../kafka/api/JoinGroupRequestAndHeader.scala | 40 +++
.../kafka/api/JoinGroupResponseAndHeader.scala | 28 ++
.../scala/kafka/api/LeaderAndIsrRequest.scala | 4 +-
.../scala/kafka/api/LeaderAndIsrResponse.scala | 4 +-
.../scala/kafka/api/OffsetCommitRequest.scala | 38 ++-
.../scala/kafka/api/OffsetCommitResponse.scala | 4 +-
.../scala/kafka/api/OffsetFetchRequest.scala | 4 +-
.../scala/kafka/api/OffsetFetchResponse.scala | 4 +-
.../main/scala/kafka/api/OffsetRequest.scala | 4 +-
.../main/scala/kafka/api/OffsetResponse.scala | 4 +-
.../main/scala/kafka/api/ProducerRequest.scala | 4 +-
.../main/scala/kafka/api/ProducerResponse.scala | 4 +-
core/src/main/scala/kafka/api/RequestKeys.scala | 7 +-
.../scala/kafka/api/RequestOrResponse.scala | 2 +-
.../scala/kafka/api/StopReplicaRequest.scala | 4 +-
.../scala/kafka/api/StopReplicaResponse.scala | 4 +-
.../scala/kafka/api/TopicMetadataRequest.scala | 4 +-
.../scala/kafka/api/TopicMetadataResponse.scala | 4 +-
.../scala/kafka/api/UpdateMetadataRequest.scala | 4 +-
.../kafka/api/UpdateMetadataResponse.scala | 4 +-
.../controller/ControllerChannelManager.scala | 8 +-
.../kafka/javaapi/OffsetCommitRequest.scala | 2 -
.../kafka/javaapi/TopicMetadataRequest.scala | 4 +-
.../api/RequestResponseSerializationTest.scala | 74 ++++-
61 files changed, 2610 insertions(+), 239 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
index a016269..8ebe7ed 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
@@ -12,6 +12,7 @@
*/
package org.apache.kafka.clients.producer.internals;
+import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
@@ -276,13 +277,16 @@ public class Sender implements Runnable {
* Create a produce request from the given record batches
*/
private ClientRequest produceRequest(long now, int destination, short acks, int timeout, List<RecordBatch> batches) {
- ProduceRequest request = new ProduceRequest(acks, timeout);
+ Map<TopicPartition, ByteBuffer> produceRecordsByPartition = new HashMap<TopicPartition, ByteBuffer>(batches.size());
Map<TopicPartition, RecordBatch> recordsByPartition = new HashMap<TopicPartition, RecordBatch>(batches.size());
for (RecordBatch batch : batches) {
- batch.records.buffer().flip();
- request.add(batch.topicPartition, batch.records);
- recordsByPartition.put(batch.topicPartition, batch);
+ TopicPartition tp = batch.topicPartition;
+ ByteBuffer recordsBuffer = batch.records.buffer();
+ recordsBuffer.flip();
+ produceRecordsByPartition.put(tp, recordsBuffer);
+ recordsByPartition.put(tp, batch);
}
+ ProduceRequest request = new ProduceRequest(acks, timeout, produceRecordsByPartition);
RequestSend send = new RequestSend(destination, this.client.nextRequestHeader(ApiKeys.PRODUCE), request.toStruct());
return new ClientRequest(now, acks != 0, send, recordsByPartition);
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/Cluster.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/Cluster.java b/clients/src/main/java/org/apache/kafka/common/Cluster.java
index c62707a..d3299b9 100644
--- a/clients/src/main/java/org/apache/kafka/common/Cluster.java
+++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java
@@ -15,12 +15,7 @@ package org.apache.kafka.common;
import org.apache.kafka.common.utils.Utils;
import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
/**
* A representation of a subset of the nodes, topics, and partitions in the Kafka cluster.
@@ -143,6 +138,14 @@ public final class Cluster {
return this.partitionsByNode.get(nodeId);
}
+ /**
+ * Get all topics.
+ * @return a set of all topics
+ */
+ public Set<String> topics() {
+ return this.partitionsByTopic.keySet();
+ }
+
@Override
public String toString() {
return "Cluster(nodes = " + this.nodes + ", partitions = " + this.partitionsByTopicPartition.values() + ")";
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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 6fe7573..109fc96 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
@@ -30,8 +30,11 @@ public enum ApiKeys {
METADATA(3, "metadata"),
LEADER_AND_ISR(4, "leader_and_isr"),
STOP_REPLICA(5, "stop_replica"),
- OFFSET_COMMIT(6, "offset_commit"),
- OFFSET_FETCH(7, "offset_fetch");
+ OFFSET_COMMIT(8, "offset_commit"),
+ OFFSET_FETCH(9, "offset_fetch"),
+ CONSUMER_METADATA(10, "consumer_metadata"),
+ JOIN_GROUP(11, "join_group"),
+ HEARTBEAT(12, "heartbeat");
private static ApiKeys[] codeToType;
public static int MAX_API_KEY = -1;
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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 044b030..7517b87 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
@@ -104,6 +104,264 @@ public class Protocol {
public static Schema[] PRODUCE_REQUEST = new Schema[] { PRODUCE_REQUEST_V0 };
public static Schema[] PRODUCE_RESPONSE = new Schema[] { PRODUCE_RESPONSE_V0 };
+ /* Offset commit api */
+ public static 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("timestamp",
+ INT64,
+ "Timestamp of the commit"),
+ new Field("metadata",
+ STRING,
+ "Any associated metadata the client wants to keep."));
+
+ public static 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 Schema OFFSET_COMMIT_REQUEST_V0 = new Schema(new Field("group_id",
+ STRING,
+ "The consumer group id."),
+ new Field("topics",
+ new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0),
+ "Topics to commit offsets."));
+
+ public static Schema OFFSET_COMMIT_REQUEST_V1 = new Schema(new Field("group_id",
+ STRING,
+ "The consumer group id."),
+ new Field("group_generation_id",
+ INT32,
+ "The generation of the consumer group."),
+ new Field("consumer_id",
+ STRING,
+ "The consumer id assigned by the group coordinator."),
+ new Field("topics",
+ new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0),
+ "Topics to commit offsets."));
+
+ public static Schema OFFSET_COMMIT_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
+ INT32,
+ "Topic partition id."),
+ new Field("error_code",
+ INT16));
+
+ public static 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 Schema OFFSET_COMMIT_RESPONSE_V0 = new Schema(new Field("responses",
+ new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0)));
+
+ public static Schema[] OFFSET_COMMIT_REQUEST = new Schema[] { OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1 };
+ /* The response types for both V0 and V1 of OFFSET_COMMIT_REQUEST are the same. */
+ public static Schema[] OFFSET_COMMIT_RESPONSE = new Schema[] { OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V0};
+
+ /* Offset fetch api */
+ public static Schema OFFSET_FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
+ INT32,
+ "Topic partition id."));
+
+ public static 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 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 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",
+ STRING,
+ "Any associated metadata the client wants to keep."),
+ new Field("error_code",
+ INT16));
+
+ public static 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 Schema OFFSET_FETCH_RESPONSE_V0 = new Schema(new Field("responses",
+ new ArrayOf(OFFSET_FETCH_RESPONSE_TOPIC_V0)));
+
+ public static Schema[] OFFSET_FETCH_REQUEST = new Schema[] { OFFSET_FETCH_REQUEST_V0 };
+ public static Schema[] OFFSET_FETCH_RESPONSE = new Schema[] { OFFSET_FETCH_RESPONSE_V0 };
+
+ /* List offset api */
+ public static 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 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 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 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 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 Schema LIST_OFFSET_RESPONSE_V0 = new Schema(new Field("responses",
+ new ArrayOf(LIST_OFFSET_RESPONSE_TOPIC_V0)));
+
+ public static Schema[] LIST_OFFSET_REQUEST = new Schema[] { LIST_OFFSET_REQUEST_V0 };
+ public static Schema[] LIST_OFFSET_RESPONSE = new Schema[] { LIST_OFFSET_RESPONSE_V0 };
+
+ /* Fetch api */
+ public static 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."));
+
+ public static 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 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."));
+
+ public static Schema FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
+ INT32,
+ "Topic partition id."),
+ new Field("error_code",
+ INT16),
+ new Field("high_watermark",
+ INT64,
+ "Last committed offset."),
+ new Field("record_set", BYTES));
+
+ public static Schema FETCH_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING),
+ new Field("partition_responses",
+ new ArrayOf(FETCH_RESPONSE_PARTITION_V0)));
+
+ public static Schema FETCH_RESPONSE_V0 = new Schema(new Field("responses",
+ new ArrayOf(FETCH_RESPONSE_TOPIC_V0)));
+
+ public static Schema[] FETCH_REQUEST = new Schema[] { FETCH_REQUEST_V0 };
+ public static Schema[] FETCH_RESPONSE = new Schema[] { FETCH_RESPONSE_V0 };
+
+ /* Consumer metadata api */
+ public static Schema CONSUMER_METADATA_REQUEST_V0 = new Schema(new Field("group_id",
+ STRING,
+ "The consumer group id."));
+
+ public static Schema CONSUMER_METADATA_RESPONSE_V0 = new Schema(new Field("error_code",
+ INT16),
+ new Field("coordinator",
+ BROKER,
+ "Host and port information for the coordinator for a consumer group."));
+
+ public static Schema[] CONSUMER_METADATA_REQUEST = new Schema[] { CONSUMER_METADATA_REQUEST_V0 };
+ public static Schema[] CONSUMER_METADATA_RESPONSE = new Schema[] { CONSUMER_METADATA_RESPONSE_V0 };
+
+ /* Join group api */
+ public static Schema JOIN_GROUP_REQUEST_V0 = new Schema(new Field("group_id",
+ STRING,
+ "The consumer 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("topics",
+ new ArrayOf(STRING),
+ "An array of topics to subscribe to."),
+ new Field("consumer_id",
+ STRING,
+ "The assigned consumer id or an empty string for a new consumer."),
+ new Field("partition_assignment_strategy",
+ STRING,
+ "The strategy for the coordinator to assign partitions."));
+
+ public static Schema JOIN_GROUP_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING),
+ new Field("partitions", new ArrayOf(INT32)));
+ public static Schema JOIN_GROUP_RESPONSE_V0 = new Schema(new Field("error_code",
+ INT16),
+ new Field("group_generation_id",
+ INT32,
+ "The generation of the consumer group."),
+ new Field("consumer_id",
+ STRING,
+ "The consumer id assigned by the group coordinator."),
+ new Field("assigned_partitions",
+ new ArrayOf(JOIN_GROUP_RESPONSE_TOPIC_V0)));
+
+ public static Schema[] JOIN_GROUP_REQUEST = new Schema[] { JOIN_GROUP_REQUEST_V0 };
+ public static Schema[] JOIN_GROUP_RESPONSE = new Schema[] { JOIN_GROUP_RESPONSE_V0 };
+
+ /* Heartbeat api */
+ public static Schema HEARTBEAT_REQUEST_V0 = new Schema(new Field("group_id",
+ STRING,
+ "The consumer group id."),
+ new Field("group_generation_id",
+ INT32,
+ "The generation of the consumer group."),
+ new Field("consumer_id",
+ STRING,
+ "The consumer id assigned by the group coordinator."));
+
+ public static Schema HEARTBEAT_RESPONSE_V0 = new Schema(new Field("error_code",
+ INT16));
+
+ public static Schema[] HEARTBEAT_REQUEST = new Schema[] {HEARTBEAT_REQUEST_V0};
+ public static Schema[] HEARTBEAT_RESPONSE = new Schema[] {HEARTBEAT_RESPONSE_V0};
+
/* an array of all requests and responses with all schema versions */
public static Schema[][] REQUESTS = new Schema[ApiKeys.MAX_API_KEY + 1][];
public static Schema[][] RESPONSES = new Schema[ApiKeys.MAX_API_KEY + 1][];
@@ -113,22 +371,28 @@ public class Protocol {
static {
REQUESTS[ApiKeys.PRODUCE.id] = PRODUCE_REQUEST;
- REQUESTS[ApiKeys.FETCH.id] = new Schema[] {};
- REQUESTS[ApiKeys.LIST_OFFSETS.id] = new Schema[] {};
+ REQUESTS[ApiKeys.FETCH.id] = FETCH_REQUEST;
+ REQUESTS[ApiKeys.LIST_OFFSETS.id] = LIST_OFFSET_REQUEST;
REQUESTS[ApiKeys.METADATA.id] = METADATA_REQUEST;
REQUESTS[ApiKeys.LEADER_AND_ISR.id] = new Schema[] {};
REQUESTS[ApiKeys.STOP_REPLICA.id] = new Schema[] {};
- REQUESTS[ApiKeys.OFFSET_COMMIT.id] = new Schema[] {};
- REQUESTS[ApiKeys.OFFSET_FETCH.id] = new Schema[] {};
+ REQUESTS[ApiKeys.OFFSET_COMMIT.id] = OFFSET_COMMIT_REQUEST;
+ REQUESTS[ApiKeys.OFFSET_FETCH.id] = OFFSET_FETCH_REQUEST;
+ REQUESTS[ApiKeys.CONSUMER_METADATA.id] = CONSUMER_METADATA_REQUEST;
+ REQUESTS[ApiKeys.JOIN_GROUP.id] = JOIN_GROUP_REQUEST;
+ REQUESTS[ApiKeys.HEARTBEAT.id] = HEARTBEAT_REQUEST;
RESPONSES[ApiKeys.PRODUCE.id] = PRODUCE_RESPONSE;
- RESPONSES[ApiKeys.FETCH.id] = new Schema[] {};
- RESPONSES[ApiKeys.LIST_OFFSETS.id] = new Schema[] {};
+ RESPONSES[ApiKeys.FETCH.id] = FETCH_RESPONSE;
+ RESPONSES[ApiKeys.LIST_OFFSETS.id] = LIST_OFFSET_RESPONSE;
RESPONSES[ApiKeys.METADATA.id] = METADATA_RESPONSE;
RESPONSES[ApiKeys.LEADER_AND_ISR.id] = new Schema[] {};
RESPONSES[ApiKeys.STOP_REPLICA.id] = new Schema[] {};
- RESPONSES[ApiKeys.OFFSET_COMMIT.id] = new Schema[] {};
- RESPONSES[ApiKeys.OFFSET_FETCH.id] = new Schema[] {};
+ RESPONSES[ApiKeys.OFFSET_COMMIT.id] = OFFSET_COMMIT_RESPONSE;
+ RESPONSES[ApiKeys.OFFSET_FETCH.id] = OFFSET_FETCH_RESPONSE;
+ RESPONSES[ApiKeys.CONSUMER_METADATA.id] = CONSUMER_METADATA_RESPONSE;
+ RESPONSES[ApiKeys.JOIN_GROUP.id] = JOIN_GROUP_RESPONSE;
+ RESPONSES[ApiKeys.HEARTBEAT.id] = HEARTBEAT_RESPONSE;
/* set the maximum version of each api */
for (ApiKeys api : ApiKeys.values())
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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 8cecba5..444e69e 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
@@ -83,6 +83,15 @@ public class Struct {
return getFieldOrDefault(field);
}
+ /**
+ * Check if the struct contains a field.
+ * @param name
+ * @return
+ */
+ public boolean hasField(String name) {
+ return schema.get(name) != null;
+ }
+
public Struct getStruct(Field field) {
return (Struct) get(field);
}
@@ -107,6 +116,22 @@ public class Struct {
return (Integer) get(name);
}
+ public Long getLong(Field field) {
+ return (Long) get(field);
+ }
+
+ public Long getLong(String name) {
+ return (Long) get(name);
+ }
+
+ public ByteBuffer getBytes(Field field) {
+ return (ByteBuffer) get(field);
+ }
+
+ public ByteBuffer getBytes(String name) {
+ return (ByteBuffer) get(name);
+ }
+
public Object[] getArray(Field field) {
return (Object[]) get(field);
}
@@ -253,4 +278,46 @@ public class Struct {
return b.toString();
}
+ @Override
+ public int hashCode() {
+ 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) {
+ Object[] arrayObject = (Object []) this.get(f);
+ for (Object arrayItem: arrayObject)
+ result = prime * result + arrayItem.hashCode();
+ } else {
+ result = prime * result + this.get(f).hashCode();
+ }
+ }
+ return result;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj)
+ return true;
+ if (obj == null)
+ return false;
+ if (getClass() != obj.getClass())
+ return false;
+ Struct other = (Struct) obj;
+ if (schema != other.schema)
+ return false;
+ for (int i = 0; i < this.values.length; i++) {
+ Field f = this.schema.get(i);
+ Boolean result;
+ if (f.type() instanceof ArrayOf) {
+ result = Arrays.equals((Object []) this.get(f), (Object []) other.get(f));
+ } else {
+ result = this.get(f).equals(other.get(f));
+ }
+ if (!result)
+ return false;
+ }
+ return true;
+ }
+
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java
new file mode 100644
index 0000000..37aff6c
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.protocol.types.Struct;
+
+import java.nio.ByteBuffer;
+
+public abstract class AbstractRequestResponse {
+ protected final Struct struct;
+
+
+ public AbstractRequestResponse(Struct struct) {
+ this.struct = struct;
+ }
+
+ public Struct toStruct() {
+ return struct;
+ }
+
+ /**
+ * Get the serialized size of this object
+ */
+ public int sizeOf() {
+ return struct.sizeOf();
+ }
+
+ /**
+ * Write this object to a buffer
+ */
+ public void writeTo(ByteBuffer buffer) {
+ struct.writeTo(buffer);
+ }
+
+ @Override
+ public String toString() {
+ return struct.toString();
+ }
+
+ @Override
+ public int hashCode() {
+ return struct.hashCode();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj)
+ return true;
+ if (obj == null)
+ return false;
+ if (getClass() != obj.getClass())
+ return false;
+ AbstractRequestResponse other = (AbstractRequestResponse) obj;
+ return struct.equals(other.struct);
+ }
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java
new file mode 100644
index 0000000..99b52c2
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+
+import java.nio.ByteBuffer;
+
+public class ConsumerMetadataRequest extends AbstractRequestResponse {
+ public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.CONSUMER_METADATA.id);
+ private static String GROUP_ID_KEY_NAME = "group_id";
+
+ private final String groupId;
+
+ public ConsumerMetadataRequest(String groupId) {
+ super(new Struct(curSchema));
+
+ struct.set(GROUP_ID_KEY_NAME, groupId);
+ this.groupId = groupId;
+ }
+
+ public ConsumerMetadataRequest(Struct struct) {
+ super(struct);
+ groupId = struct.getString(GROUP_ID_KEY_NAME);
+ }
+
+ public String groupId() {
+ return groupId;
+ }
+
+ public static ConsumerMetadataRequest parse(ByteBuffer buffer) {
+ return new ConsumerMetadataRequest(((Struct) curSchema.read(buffer)));
+ }
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java
new file mode 100644
index 0000000..8b8f591
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java
@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+
+import java.nio.ByteBuffer;
+
+public class ConsumerMetadataResponse extends AbstractRequestResponse {
+ private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.CONSUMER_METADATA.id);
+ private static String ERROR_CODE_KEY_NAME = "error_code";
+ private static String COORDINATOR_KEY_NAME = "coordinator";
+
+ // coordinator level field names
+ private static String NODE_ID_KEY_NAME = "node_id";
+ private static String HOST_KEY_NAME = "host";
+ private static String PORT_KEY_NAME = "port";
+
+ private final short errorCode;
+ private final Node node;
+
+ public ConsumerMetadataResponse(short errorCode, Node node) {
+ super(new Struct(curSchema));
+ struct.set(ERROR_CODE_KEY_NAME, errorCode);
+ Struct coordinator = struct.instance(COORDINATOR_KEY_NAME);
+ coordinator.set(NODE_ID_KEY_NAME, node.id());
+ coordinator.set(HOST_KEY_NAME, node.host());
+ coordinator.set(PORT_KEY_NAME, node.port());
+ struct.set(COORDINATOR_KEY_NAME, coordinator);
+ this.errorCode = errorCode;
+ this.node = node;
+ }
+
+ public ConsumerMetadataResponse(Struct struct) {
+ super(struct);
+ errorCode = struct.getShort(ERROR_CODE_KEY_NAME);
+ Struct broker = (Struct) struct.get(COORDINATOR_KEY_NAME);
+ int nodeId = broker.getInt(NODE_ID_KEY_NAME);
+ String host = broker.getString(HOST_KEY_NAME);
+ int port = broker.getInt(PORT_KEY_NAME);
+ node = new Node(nodeId, host, port);
+ }
+
+ public short errorCode() {
+ return errorCode;
+ }
+
+ public Node node() {
+ return node;
+ }
+
+ public static ConsumerMetadataResponse parse(ByteBuffer buffer) {
+ return new ConsumerMetadataResponse(((Struct) curSchema.read(buffer)));
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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
new file mode 100644
index 0000000..2fc471f
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
@@ -0,0 +1,132 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+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;
+
+public class FetchRequest extends AbstractRequestResponse {
+ public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.FETCH.id);
+ private static String REPLICA_ID_KEY_NAME = "replica_id";
+ private static String MAX_WAIT_KEY_NAME = "max_wait_time";
+ private static String MIN_BYTES_KEY_NAME = "min_bytes";
+ private static String TOPICS_KEY_NAME = "topics";
+
+ // topic level field names
+ private static String TOPIC_KEY_NAME = "topic";
+ private static String PARTITIONS_KEY_NAME = "partitions";
+
+ // partition level field names
+ private static String PARTITION_KEY_NAME = "partition";
+ private static String FETCH_OFFSET_KEY_NAME = "fetch_offset";
+ private static String MAX_BYTES_KEY_NAME = "max_bytes";
+
+ private final int replicaId;
+ private final int maxWait;
+ private final int minBytes;
+ private final Map<TopicPartition, PartitionData> fetchData;
+
+ public static final class PartitionData {
+ public final long offset;
+ public final int maxBytes;
+
+ public PartitionData(long offset, int maxBytes) {
+ this.offset = offset;
+ this.maxBytes = maxBytes;
+ }
+ }
+
+ public FetchRequest(int replicaId, int maxWait, int minBytes, Map<TopicPartition, PartitionData> fetchData) {
+ super(new Struct(curSchema));
+ Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(fetchData);
+
+ struct.set(REPLICA_ID_KEY_NAME, replicaId);
+ struct.set(MAX_WAIT_KEY_NAME, maxWait);
+ struct.set(MIN_BYTES_KEY_NAME, minBytes);
+ List<Struct> topicArray = new ArrayList<Struct>();
+ for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
+ Struct topicData = struct.instance(TOPICS_KEY_NAME);
+ topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
+ List<Struct> partitionArray = new ArrayList<Struct>();
+ 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(FETCH_OFFSET_KEY_NAME, fetchPartitionData.offset);
+ partitionData.set(MAX_BYTES_KEY_NAME, fetchPartitionData.maxBytes);
+ partitionArray.add(partitionData);
+ }
+ topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+ topicArray.add(topicData);
+ }
+ struct.set(TOPICS_KEY_NAME, topicArray.toArray());
+ this.replicaId = replicaId;
+ this.maxWait = maxWait;
+ this.minBytes = minBytes;
+ this.fetchData = fetchData;
+ }
+
+ public FetchRequest(Struct struct) {
+ super(struct);
+ replicaId = struct.getInt(REPLICA_ID_KEY_NAME);
+ maxWait = struct.getInt(MAX_WAIT_KEY_NAME);
+ minBytes = struct.getInt(MIN_BYTES_KEY_NAME);
+ fetchData = new HashMap<TopicPartition, PartitionData>();
+ for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
+ Struct topicResponse = (Struct) topicResponseObj;
+ String topic = topicResponse.getString(TOPIC_KEY_NAME);
+ for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+ Struct partitionResponse = (Struct) partitionResponseObj;
+ int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+ long offset = partitionResponse.getLong(FETCH_OFFSET_KEY_NAME);
+ int maxBytes = partitionResponse.getInt(MAX_BYTES_KEY_NAME);
+ PartitionData partitionData = new PartitionData(offset, maxBytes);
+ fetchData.put(new TopicPartition(topic, partition), partitionData);
+ }
+ }
+ }
+
+ public int replicaId() {
+ return replicaId;
+ }
+
+ public int maxWait() {
+ return maxWait;
+ }
+
+ public int minBytes() {
+ return minBytes;
+ }
+
+ public Map<TopicPartition, PartitionData> fetchData() {
+ return fetchData;
+ }
+
+ public static FetchRequest parse(ByteBuffer buffer) {
+ return new FetchRequest(((Struct) curSchema.read(buffer)));
+ }
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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
new file mode 100644
index 0000000..f719010
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java
@@ -0,0 +1,110 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+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;
+
+public class FetchResponse extends AbstractRequestResponse {
+ public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.FETCH.id);
+ private static String RESPONSES_KEY_NAME = "responses";
+
+ // topic level field names
+ private static String TOPIC_KEY_NAME = "topic";
+ private static String PARTITIONS_KEY_NAME = "partition_responses";
+
+ // partition level field names
+ private static String PARTITION_KEY_NAME = "partition";
+ private static String ERROR_CODE_KEY_NAME = "error_code";
+ private static String HIGH_WATERMARK_KEY_NAME = "high_watermark";
+ private static String RECORD_SET_KEY_NAME = "record_set";
+
+ private final Map<TopicPartition, PartitionData> responseData;
+
+ public static final class PartitionData {
+ public final short errorCode;
+ public final long highWatermark;
+ public final ByteBuffer recordSet;
+
+ public PartitionData(short errorCode, long highWatermark, ByteBuffer recordSet) {
+ this.errorCode = errorCode;
+ this.highWatermark = highWatermark;
+ this.recordSet = recordSet;
+ }
+ }
+
+ public FetchResponse(Map<TopicPartition, PartitionData> responseData) {
+ super(new Struct(curSchema));
+ Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(responseData);
+
+ List<Struct> topicArray = new ArrayList<Struct>();
+ for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
+ Struct topicData = struct.instance(RESPONSES_KEY_NAME);
+ topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
+ List<Struct> partitionArray = new ArrayList<Struct>();
+ 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(ERROR_CODE_KEY_NAME, fetchPartitionData.errorCode);
+ partitionData.set(HIGH_WATERMARK_KEY_NAME, fetchPartitionData.highWatermark);
+ partitionData.set(RECORD_SET_KEY_NAME, fetchPartitionData.recordSet);
+ partitionArray.add(partitionData);
+ }
+ topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+ topicArray.add(topicData);
+ }
+ struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
+ this.responseData = responseData;
+ }
+
+ public FetchResponse(Struct struct) {
+ super(struct);
+ responseData = new HashMap<TopicPartition, PartitionData>();
+ for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
+ Struct topicResponse = (Struct) topicResponseObj;
+ String topic = topicResponse.getString(TOPIC_KEY_NAME);
+ for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+ Struct partitionResponse = (Struct) partitionResponseObj;
+ int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+ short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME);
+ long highWatermark = partitionResponse.getLong(HIGH_WATERMARK_KEY_NAME);
+ ByteBuffer recordSet = partitionResponse.getBytes(RECORD_SET_KEY_NAME);
+ PartitionData partitionData = new PartitionData(errorCode, highWatermark, recordSet);
+ responseData.put(new TopicPartition(topic, partition), partitionData);
+ }
+ }
+ }
+
+ public Map<TopicPartition, PartitionData> responseData() {
+ return responseData;
+ }
+
+ public static FetchResponse parse(ByteBuffer buffer) {
+ return new FetchResponse(((Struct) curSchema.read(buffer)));
+ }
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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
new file mode 100644
index 0000000..9512db2
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+
+import java.nio.ByteBuffer;
+
+public class HeartbeatRequest extends AbstractRequestResponse {
+ public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.HEARTBEAT.id);
+ private static String GROUP_ID_KEY_NAME = "group_id";
+ private static String GROUP_GENERATION_ID_KEY_NAME = "group_generation_id";
+ private static String CONSUMER_ID_KEY_NAME = "consumer_id";
+
+ private final String groupId;
+ private final int groupGenerationId;
+ private final String consumerId;
+
+ public HeartbeatRequest(String groupId, int groupGenerationId, String consumerId) {
+ super(new Struct(curSchema));
+ struct.set(GROUP_ID_KEY_NAME, groupId);
+ struct.set(GROUP_GENERATION_ID_KEY_NAME, groupGenerationId);
+ struct.set(CONSUMER_ID_KEY_NAME, consumerId);
+ this.groupId = groupId;
+ this.groupGenerationId = groupGenerationId;
+ this.consumerId = consumerId;
+ }
+
+ public HeartbeatRequest(Struct struct) {
+ super(struct);
+ groupId = struct.getString(GROUP_ID_KEY_NAME);
+ groupGenerationId = struct.getInt(GROUP_GENERATION_ID_KEY_NAME);
+ consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
+ }
+
+ public String groupId() {
+ return groupId;
+ }
+
+ public int groupGenerationId() {
+ return groupGenerationId;
+ }
+
+ public String consumerId() {
+ return consumerId;
+ }
+
+ public static HeartbeatRequest parse(ByteBuffer buffer) {
+ return new HeartbeatRequest(((Struct) curSchema.read(buffer)));
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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
new file mode 100644
index 0000000..8997ffc
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+
+import java.nio.ByteBuffer;
+
+public class HeartbeatResponse extends AbstractRequestResponse {
+ private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.HEARTBEAT.id);
+ private static String ERROR_CODE_KEY_NAME = "error_code";
+
+ private final short errorCode;
+ public HeartbeatResponse(short errorCode) {
+ super(new Struct(curSchema));
+ struct.set(ERROR_CODE_KEY_NAME, errorCode);
+ this.errorCode = errorCode;
+ }
+
+ public HeartbeatResponse(Struct struct) {
+ super(struct);
+ errorCode = struct.getShort(ERROR_CODE_KEY_NAME);
+ }
+
+ public short errorCode() {
+ return errorCode;
+ }
+
+ public static HeartbeatResponse parse(ByteBuffer buffer) {
+ return new HeartbeatResponse(((Struct) curSchema.read(buffer)));
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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
new file mode 100644
index 0000000..d6e91f3
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+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;
+
+public class JoinGroupRequest extends AbstractRequestResponse {
+ public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.JOIN_GROUP.id);
+ private static String GROUP_ID_KEY_NAME = "group_id";
+ private static String SESSION_TIMEOUT_KEY_NAME = "session_timeout";
+ private static String TOPICS_KEY_NAME = "topics";
+ private static String CONSUMER_ID_KEY_NAME = "consumer_id";
+ private static String STRATEGY_KEY_NAME = "partition_assignment_strategy";
+
+ private final String groupId;
+ private final int sessionTimeout;
+ private final List<String> topics;
+ private final String consumerId;
+ private final String strategy;
+
+ public JoinGroupRequest(String groupId, int sessionTimeout, List<String> topics, String consumerId, String strategy) {
+ super(new Struct(curSchema));
+ struct.set(GROUP_ID_KEY_NAME, groupId);
+ struct.set(SESSION_TIMEOUT_KEY_NAME, sessionTimeout);
+ struct.set(TOPICS_KEY_NAME, topics.toArray());
+ struct.set(CONSUMER_ID_KEY_NAME, consumerId);
+ struct.set(STRATEGY_KEY_NAME, strategy);
+ this.groupId = groupId;
+ this.sessionTimeout = sessionTimeout;
+ this.topics = topics;
+ this.consumerId = consumerId;
+ this.strategy = strategy;
+ }
+
+ public JoinGroupRequest(Struct struct) {
+ super(struct);
+ groupId = struct.getString(GROUP_ID_KEY_NAME);
+ sessionTimeout = struct.getInt(SESSION_TIMEOUT_KEY_NAME);
+ Object[] topicsArray = struct.getArray(TOPICS_KEY_NAME);
+ topics = new ArrayList<String>();
+ for (Object topic: topicsArray)
+ topics.add((String) topic);
+ consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
+ strategy = struct.getString(STRATEGY_KEY_NAME);
+ }
+
+ public String groupId() {
+ return groupId;
+ }
+
+ public int sessionTimeout() {
+ return sessionTimeout;
+ }
+
+ public List<String> topics() {
+ return topics;
+ }
+
+ public String consumerId() {
+ return consumerId;
+ }
+
+ public String strategy() {
+ return strategy;
+ }
+
+ public static JoinGroupRequest parse(ByteBuffer buffer) {
+ return new JoinGroupRequest(((Struct) curSchema.read(buffer)));
+ }
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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
new file mode 100644
index 0000000..efe8979
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java
@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+public class JoinGroupResponse extends AbstractRequestResponse {
+ public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.JOIN_GROUP.id);
+ private static String ERROR_CODE_KEY_NAME = "error_code";
+ private static String GENERATION_ID_KEY_NAME = "group_generation_id";
+ private static String CONSUMER_ID_KEY_NAME = "consumer_id";
+ private static String ASSIGNED_PARTITIONS_KEY_NAME = "assigned_partitions";
+ private static String TOPIC_KEY_NAME = "topic";
+ private static String PARTITIONS_KEY_NAME = "partitions";
+
+ public static int UNKNOWN_GENERATION_ID = -1;
+ public static String UNKNOWN_CONSUMER_ID = "";
+
+ private final short errorCode;
+ private final int generationId;
+ private final String consumerId;
+ private final List<TopicPartition> assignedPartitions;
+
+ public JoinGroupResponse(short errorCode, int generationId, String consumerId, List<TopicPartition> assignedPartitions) {
+ super(new Struct(curSchema));
+
+ Map<String, List<Integer>> partitionsByTopic = CollectionUtils.groupDataByTopic(assignedPartitions);
+
+ struct.set(ERROR_CODE_KEY_NAME, errorCode);
+ struct.set(GENERATION_ID_KEY_NAME, generationId);
+ struct.set(CONSUMER_ID_KEY_NAME, consumerId);
+ List<Struct> topicArray = new ArrayList<Struct>();
+ for (Map.Entry<String, List<Integer>> entries: partitionsByTopic.entrySet()) {
+ Struct topicData = struct.instance(ASSIGNED_PARTITIONS_KEY_NAME);
+ topicData.set(TOPIC_KEY_NAME, entries.getKey());
+ topicData.set(PARTITIONS_KEY_NAME, entries.getValue().toArray());
+ topicArray.add(topicData);
+ }
+ struct.set(ASSIGNED_PARTITIONS_KEY_NAME, topicArray.toArray());
+
+ this.errorCode = errorCode;
+ this.generationId = generationId;
+ this.consumerId = consumerId;
+ this.assignedPartitions = assignedPartitions;
+ }
+
+ public JoinGroupResponse(short errorCode) {
+ this(errorCode, UNKNOWN_GENERATION_ID, UNKNOWN_CONSUMER_ID, Collections.<TopicPartition>emptyList());
+ }
+
+ public JoinGroupResponse(Struct struct) {
+ super(struct);
+ assignedPartitions = new ArrayList<TopicPartition>();
+ for (Object topicDataObj : struct.getArray(ASSIGNED_PARTITIONS_KEY_NAME)) {
+ Struct topicData = (Struct) topicDataObj;
+ String topic = topicData.getString(TOPIC_KEY_NAME);
+ for (Object partitionObj : topicData.getArray(PARTITIONS_KEY_NAME))
+ assignedPartitions.add(new TopicPartition(topic, (Integer) partitionObj));
+ }
+ errorCode = struct.getShort(ERROR_CODE_KEY_NAME);
+ generationId = struct.getInt(GENERATION_ID_KEY_NAME);
+ consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
+ }
+
+ public short errorCode() {
+ return errorCode;
+ }
+
+ public int generationId() {
+ return generationId;
+ }
+
+ public String consumerId() {
+ return consumerId;
+ }
+
+ public List<TopicPartition> assignedPartitions() {
+ return assignedPartitions;
+ }
+
+ public static JoinGroupResponse parse(ByteBuffer buffer) {
+ return new JoinGroupResponse(((Struct) curSchema.read(buffer)));
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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
new file mode 100644
index 0000000..99364c1
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java
@@ -0,0 +1,114 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+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;
+
+public class ListOffsetRequest extends AbstractRequestResponse {
+ public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.LIST_OFFSETS.id);
+ private static String REPLICA_ID_KEY_NAME = "replica_id";
+ private static String TOPICS_KEY_NAME = "topics";
+
+ // topic level field names
+ private static String TOPIC_KEY_NAME = "topic";
+ private static String PARTITIONS_KEY_NAME = "partitions";
+
+ // partition level field names
+ private static String PARTITION_KEY_NAME = "partition";
+ private static String TIMESTAMP_KEY_NAME = "timestamp";
+ private static String MAX_NUM_OFFSETS_KEY_NAME = "max_num_offsets";
+
+ private final int replicaId;
+ private final Map<TopicPartition, PartitionData> offsetData;
+
+ public static final class PartitionData {
+ public final long timestamp;
+ public final int maxNumOffsets;
+
+ public PartitionData(long timestamp, int maxNumOffsets) {
+ this.timestamp = timestamp;
+ this.maxNumOffsets = maxNumOffsets;
+ }
+ }
+
+ public ListOffsetRequest(int replicaId, Map<TopicPartition, PartitionData> offsetData) {
+ super(new Struct(curSchema));
+ Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(offsetData);
+
+ struct.set(REPLICA_ID_KEY_NAME, replicaId);
+ List<Struct> topicArray = new ArrayList<Struct>();
+ for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
+ Struct topicData = struct.instance(TOPICS_KEY_NAME);
+ topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
+ List<Struct> partitionArray = new ArrayList<Struct>();
+ 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(TIMESTAMP_KEY_NAME, offsetPartitionData.timestamp);
+ partitionData.set(MAX_NUM_OFFSETS_KEY_NAME, offsetPartitionData.maxNumOffsets);
+ partitionArray.add(partitionData);
+ }
+ topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+ topicArray.add(topicData);
+ }
+ struct.set(TOPICS_KEY_NAME, topicArray.toArray());
+ this.replicaId = replicaId;
+ this.offsetData = offsetData;
+ }
+
+ public ListOffsetRequest(Struct struct) {
+ super(struct);
+ replicaId = struct.getInt(REPLICA_ID_KEY_NAME);
+ offsetData = new HashMap<TopicPartition, PartitionData>();
+ for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
+ Struct topicResponse = (Struct) topicResponseObj;
+ String topic = topicResponse.getString(TOPIC_KEY_NAME);
+ for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+ Struct partitionResponse = (Struct) partitionResponseObj;
+ int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+ long timestamp = partitionResponse.getLong(TIMESTAMP_KEY_NAME);
+ int maxNumOffsets = partitionResponse.getInt(MAX_NUM_OFFSETS_KEY_NAME);
+ PartitionData partitionData = new PartitionData(timestamp, maxNumOffsets);
+ offsetData.put(new TopicPartition(topic, partition), partitionData);
+ }
+ }
+ }
+
+ public int replicaId() {
+ return replicaId;
+ }
+
+ public Map<TopicPartition, PartitionData> offsetData() {
+ return offsetData;
+ }
+
+ public static ListOffsetRequest parse(ByteBuffer buffer) {
+ return new ListOffsetRequest(((Struct) curSchema.read(buffer)));
+ }
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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
new file mode 100644
index 0000000..ac23971
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+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;
+
+public class ListOffsetResponse extends AbstractRequestResponse {
+ public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.LIST_OFFSETS.id);
+ private static String RESPONSES_KEY_NAME = "responses";
+
+ // topic level field names
+ private static String TOPIC_KEY_NAME = "topic";
+ private static String PARTITIONS_KEY_NAME = "partition_responses";
+
+ // partition level field names
+ private static String PARTITION_KEY_NAME = "partition";
+ private static String ERROR_CODE_KEY_NAME = "error_code";
+ private static String OFFSETS_KEY_NAME = "offsets";
+
+ private final Map<TopicPartition, PartitionData> responseData;
+
+ public static final class PartitionData {
+ public final short errorCode;
+ public final List<Long> offsets;
+
+ public PartitionData(short errorCode, List<Long> offsets) {
+ this.errorCode = errorCode;
+ this.offsets = offsets;
+ }
+ }
+
+ public ListOffsetResponse(Map<TopicPartition, PartitionData> responseData) {
+ super(new Struct(curSchema));
+ Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(responseData);
+
+ List<Struct> topicArray = new ArrayList<Struct>();
+ for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
+ Struct topicData = struct.instance(RESPONSES_KEY_NAME);
+ topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
+ List<Struct> partitionArray = new ArrayList<Struct>();
+ 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.errorCode);
+ partitionData.set(OFFSETS_KEY_NAME, offsetPartitionData.offsets.toArray());
+ partitionArray.add(partitionData);
+ }
+ topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+ topicArray.add(topicData);
+ }
+ struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
+ this.responseData = responseData;
+ }
+
+ public ListOffsetResponse(Struct struct) {
+ super(struct);
+ responseData = new HashMap<TopicPartition, PartitionData>();
+ for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
+ Struct topicResponse = (Struct) topicResponseObj;
+ String topic = topicResponse.getString(TOPIC_KEY_NAME);
+ for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+ Struct partitionResponse = (Struct) partitionResponseObj;
+ int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+ short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME);
+ Object[] offsets = partitionResponse.getArray(OFFSETS_KEY_NAME);
+ List<Long> offsetsList = new ArrayList<Long>();
+ for (Object offset: offsets)
+ offsetsList.add((Long) offset);
+ PartitionData partitionData = new PartitionData(errorCode, offsetsList);
+ responseData.put(new TopicPartition(topic, partition), partitionData);
+ }
+ }
+ }
+
+ public Map<TopicPartition, PartitionData> responseData() {
+ return responseData;
+ }
+
+ public static ListOffsetResponse parse(ByteBuffer buffer) {
+ return new ListOffsetResponse(((Struct) curSchema.read(buffer)));
+ }
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/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 f35bd87..b22ca1d 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
@@ -12,26 +12,41 @@
*/
package org.apache.kafka.common.requests;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
import java.util.List;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
import org.apache.kafka.common.protocol.types.Struct;
-public class MetadataRequest {
+public class MetadataRequest extends AbstractRequestResponse {
+ public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id);
+ private static String TOPICS_KEY_NAME = "topics";
private final List<String> topics;
public MetadataRequest(List<String> topics) {
+ super(new Struct(curSchema));
+ struct.set(TOPICS_KEY_NAME, topics.toArray());
this.topics = topics;
}
- public Struct toStruct() {
- String[] ts = new String[topics.size()];
- topics.toArray(ts);
- Struct body = new Struct(ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id));
- body.set("topics", topics.toArray());
- return body;
+ public MetadataRequest(Struct struct) {
+ super(struct);
+ Object[] topicArray = struct.getArray(TOPICS_KEY_NAME);
+ topics = new ArrayList<String>();
+ for (Object topicObj: topicArray) {
+ topics.add((String) topicObj);
+ }
}
+ public List<String> topics() {
+ return topics;
+ }
+
+ public static MetadataRequest parse(ByteBuffer buffer) {
+ return new MetadataRequest(((Struct) curSchema.read(buffer)));
+ }
}
[31/37] git commit: kafka-1542 (trivail followup patch to fix
NullPointerException);
normal IOException in the new producer is logged as ERROR; patched by Jun Rao
Posted by jj...@apache.org.
kafka-1542 (trivail followup patch to fix NullPointerException); normal IOException in the new producer is logged as ERROR; patched by Jun Rao
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/50f2b245
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/50f2b245
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/50f2b245
Branch: refs/heads/transactional_messaging
Commit: 50f2b245c1a666400fab7ddfdebf7c1c2bdf3084
Parents: f489493
Author: Jun Rao <ju...@gmail.com>
Authored: Mon Jul 28 10:58:58 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Mon Jul 28 10:58:58 2014 -0700
----------------------------------------------------------------------
.../main/java/org/apache/kafka/common/network/Selector.java | 7 ++++++-
1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/50f2b245/clients/src/main/java/org/apache/kafka/common/network/Selector.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java
index 45e57b4..4dd2cdf 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java
@@ -13,6 +13,7 @@
package org.apache.kafka.common.network;
import java.io.IOException;
+import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.nio.channels.CancelledKeyException;
@@ -262,7 +263,11 @@ public class Selector implements Selectable {
if (!key.isValid())
close(key);
} catch (IOException e) {
- log.warn("Error in I/O with {}",channel.socket().getInetAddress().getHostAddress() , e);
+ InetAddress remoteAddress = null;
+ Socket socket = channel.socket();
+ if (socket != null)
+ remoteAddress = socket.getInetAddress();
+ log.warn("Error in I/O with {}", remoteAddress , e);
close(key);
}
}
[35/37] git commit: trivial change to remove a compilation warning;
patched by Jun Rao
Posted by jj...@apache.org.
trivial change to remove a compilation warning; patched by Jun Rao
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/0386790e
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/0386790e
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/0386790e
Branch: refs/heads/transactional_messaging
Commit: 0386790e757fe4eeae4a179e573f71ef8c7de43c
Parents: 1d2e776
Author: Jun Rao <ju...@gmail.com>
Authored: Mon Aug 4 21:24:05 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Mon Aug 4 21:24:05 2014 -0700
----------------------------------------------------------------------
.../main/java/org/apache/kafka/common/protocol/types/Struct.java | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/0386790e/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 444e69e..121e880 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
@@ -86,7 +86,7 @@ public class Struct {
/**
* Check if the struct contains a field.
* @param name
- * @return
+ * @return Whether a field exists.
*/
public boolean hasField(String name) {
return schema.get(name) != null;
[17/37] git commit: KAFKA-1535 Have the metadata response contain all
alive brokers rather than just the ones needed for the given topics.
Posted by jj...@apache.org.
KAFKA-1535 Have the metadata response contain all alive brokers rather than just the ones needed for the given topics.
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/4ebcdfd5
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/4ebcdfd5
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/4ebcdfd5
Branch: refs/heads/transactional_messaging
Commit: 4ebcdfd51f1e9e4c3d684204e6a785fae6c0e549
Parents: 8e444a3
Author: Jay Kreps <ja...@gmail.com>
Authored: Thu Jul 17 15:53:52 2014 -0700
Committer: Jay Kreps <ja...@gmail.com>
Committed: Thu Jul 17 15:53:52 2014 -0700
----------------------------------------------------------------------
.../main/scala/kafka/api/TopicMetadataRequest.scala | 2 +-
.../main/scala/kafka/api/TopicMetadataResponse.scala | 13 +++----------
core/src/main/scala/kafka/server/KafkaApis.scala | 5 +++--
core/src/main/scala/kafka/server/MetadataCache.scala | 6 ++++++
.../scala/kafka/tools/ReplicaVerificationTool.scala | 2 +-
.../kafka/api/RequestResponseSerializationTest.scala | 2 +-
6 files changed, 15 insertions(+), 15 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/4ebcdfd5/core/src/main/scala/kafka/api/TopicMetadataRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala
index a319f2f..bce004f 100644
--- a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala
+++ b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala
@@ -79,7 +79,7 @@ case class TopicMetadataRequest(val versionId: Short,
val topicMetadata = topics.map {
topic => TopicMetadata(topic, Nil, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
}
- val errorResponse = TopicMetadataResponse(topicMetadata, correlationId)
+ val errorResponse = TopicMetadataResponse(Seq(), topicMetadata, correlationId)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse)))
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/4ebcdfd5/core/src/main/scala/kafka/api/TopicMetadataResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala
index f6b7429..b233d35 100644
--- a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala
+++ b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala
@@ -29,34 +29,27 @@ object TopicMetadataResponse {
val brokerMap = brokers.map(b => (b.id, b)).toMap
val topicCount = buffer.getInt
val topicsMetadata = (0 until topicCount).map(_ => TopicMetadata.readFrom(buffer, brokerMap))
- new TopicMetadataResponse(topicsMetadata, correlationId)
+ new TopicMetadataResponse(brokers, topicsMetadata, correlationId)
}
}
-case class TopicMetadataResponse(topicsMetadata: Seq[TopicMetadata],
+case class TopicMetadataResponse(brokers: Seq[Broker],
+ topicsMetadata: Seq[TopicMetadata],
override val correlationId: Int)
extends RequestOrResponse(correlationId = correlationId) {
val sizeInBytes: Int = {
- val brokers = extractBrokers(topicsMetadata).values
4 + 4 + brokers.map(_.sizeInBytes).sum + 4 + topicsMetadata.map(_.sizeInBytes).sum
}
def writeTo(buffer: ByteBuffer) {
buffer.putInt(correlationId)
/* brokers */
- val brokers = extractBrokers(topicsMetadata).values
buffer.putInt(brokers.size)
brokers.foreach(_.writeTo(buffer))
/* topic metadata */
buffer.putInt(topicsMetadata.length)
topicsMetadata.foreach(_.writeTo(buffer))
}
-
- def extractBrokers(topicMetadatas: Seq[TopicMetadata]): Map[Int, Broker] = {
- val parts = topicsMetadata.flatMap(_.partitionsMetadata)
- val brokers = (parts.flatMap(_.replicas)) ++ (parts.map(_.leader).collect{case Some(l) => l})
- brokers.map(b => (b.id, b)).toMap
- }
override def describe(details: Boolean):String = { toString }
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/4ebcdfd5/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 0b668f2..fd5f12e 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -591,8 +591,9 @@ class KafkaApis(val requestChannel: RequestChannel,
def handleTopicMetadataRequest(request: RequestChannel.Request) {
val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest]
val topicMetadata = getTopicMetadata(metadataRequest.topics.toSet)
- trace("Sending topic metadata %s for correlation id %d to client %s".format(topicMetadata.mkString(","), metadataRequest.correlationId, metadataRequest.clientId))
- val response = new TopicMetadataResponse(topicMetadata, metadataRequest.correlationId)
+ val brokers = metadataCache.getAliveBrokers
+ trace("Sending topic metadata %s and brokers %s for correlation id %d to client %s".format(topicMetadata.mkString(","), brokers.mkString(","), metadataRequest.correlationId, metadataRequest.clientId))
+ val response = new TopicMetadataResponse(brokers, topicMetadata, metadataRequest.correlationId)
requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response)))
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/4ebcdfd5/core/src/main/scala/kafka/server/MetadataCache.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala
index 3198cdf..7cd40e1 100644
--- a/core/src/main/scala/kafka/server/MetadataCache.scala
+++ b/core/src/main/scala/kafka/server/MetadataCache.scala
@@ -81,6 +81,12 @@ private[server] class MetadataCache {
topicResponses
}
+ def getAliveBrokers = {
+ inLock(partitionMetadataLock.readLock()) {
+ aliveBrokers.values.toList
+ }
+ }
+
def addOrUpdatePartitionInfo(topic: String,
partitionId: Int,
stateInfo: PartitionStateInfo) {
http://git-wip-us.apache.org/repos/asf/kafka/blob/4ebcdfd5/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
index c040f49..af47836 100644
--- a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
+++ b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
@@ -118,7 +118,7 @@ object ReplicaVerificationTool extends Logging {
info("Getting topic metatdata...")
val metadataTargetBrokers = ClientUtils.parseBrokerList(options.valueOf(brokerListOpt))
val topicsMetadataResponse = ClientUtils.fetchTopicMetadata(Set[String](), metadataTargetBrokers, clientId, maxWaitMs)
- val brokerMap = topicsMetadataResponse.extractBrokers(topicsMetadataResponse.topicsMetadata)
+ val brokerMap = topicsMetadataResponse.brokers.map(b => (b.id, b)).toMap
val filteredTopicMetadata = topicsMetadataResponse.topicsMetadata.filter(
topicMetadata => if (topicWhiteListFiler.isTopicAllowed(topicMetadata.topic, excludeInternalTopics = false))
true
http://git-wip-us.apache.org/repos/asf/kafka/blob/4ebcdfd5/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
index a2117b3..d34ddf5 100644
--- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
+++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
@@ -143,7 +143,7 @@ object SerializationTestUtils {
}
def createTestTopicMetadataResponse: TopicMetadataResponse = {
- new TopicMetadataResponse(Seq(topicmetaData1, topicmetaData2), 1)
+ new TopicMetadataResponse(brokers, Seq(topicmetaData1, topicmetaData2), 1)
}
def createTestOffsetCommitRequest: OffsetCommitRequest = {
[09/37] git commit: KAFKA-1515 Fix a bug that could result in
blocking for a long period of time in the producer. Patch from Guozhang Wang.
Posted by jj...@apache.org.
KAFKA-1515 Fix a bug that could result in blocking for a long period of time in the producer. Patch from Guozhang Wang.
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/cd3ce27d
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/cd3ce27d
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/cd3ce27d
Branch: refs/heads/transactional_messaging
Commit: cd3ce27d4baf5434676ec040d64663ad3ce09817
Parents: 6de56b3
Author: Jay Kreps <ja...@gmail.com>
Authored: Tue Jul 8 13:16:56 2014 -0700
Committer: Jay Kreps <ja...@gmail.com>
Committed: Tue Jul 8 13:16:56 2014 -0700
----------------------------------------------------------------------
.../org/apache/kafka/clients/NetworkClient.java | 4 +-
.../kafka/clients/producer/KafkaProducer.java | 40 ++++++++++--
.../clients/producer/internals/Metadata.java | 68 +++++++++-----------
.../clients/producer/internals/Sender.java | 4 +-
.../kafka/clients/producer/MetadataTest.java | 9 ++-
5 files changed, 77 insertions(+), 48 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/cd3ce27d/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 d21f922..f739279 100644
--- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
@@ -309,7 +309,7 @@ public class NetworkClient implements KafkaClient {
}
// we got a disconnect so we should probably refresh our metadata and see if that broker is dead
if (this.selector.disconnected().size() > 0)
- this.metadata.forceUpdate();
+ this.metadata.requestUpdate();
}
/**
@@ -375,7 +375,7 @@ public class NetworkClient implements KafkaClient {
/* attempt failed, we'll try again after the backoff */
connectionStates.disconnected(node.id());
/* maybe the problem is our metadata, update it */
- metadata.forceUpdate();
+ metadata.requestUpdate();
log.debug("Error connecting to node {} at {}:{}:", node.id(), node.host(), node.port(), e);
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/cd3ce27d/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
index d85ca30..f58b850 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
@@ -34,6 +34,7 @@ import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.errors.ApiException;
import org.apache.kafka.common.errors.RecordTooLargeException;
+import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.metrics.JmxReporter;
import org.apache.kafka.common.metrics.MetricConfig;
import org.apache.kafka.common.metrics.Metrics;
@@ -73,6 +74,7 @@ public class KafkaProducer implements Producer {
private final Thread ioThread;
private final CompressionType compressionType;
private final Sensor errors;
+ private final Time time;
/**
* A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings
@@ -94,7 +96,7 @@ public class KafkaProducer implements Producer {
private KafkaProducer(ProducerConfig config) {
log.trace("Starting the Kafka producer");
- Time time = new SystemTime();
+ this.time = new SystemTime();
MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ProducerConfig.METRICS_NUM_SAMPLES_CONFIG))
.timeWindow(config.getLong(ProducerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG),
TimeUnit.MILLISECONDS);
@@ -119,7 +121,7 @@ public class KafkaProducer implements Producer {
metrics,
time);
List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG));
- this.metadata.update(Cluster.bootstrap(addresses), 0);
+ this.metadata.update(Cluster.bootstrap(addresses), time.milliseconds());
NetworkClient client = new NetworkClient(new Selector(this.metrics, time),
this.metadata,
@@ -225,8 +227,9 @@ public class KafkaProducer implements Producer {
@Override
public Future<RecordMetadata> send(ProducerRecord record, Callback callback) {
try {
- Cluster cluster = metadata.fetch(record.topic(), this.metadataFetchTimeoutMs);
- int partition = partitioner.partition(record, cluster);
+ // first make sure the metadata for the topic is available
+ waitOnMetadata(record.topic(), this.metadataFetchTimeoutMs);
+ int partition = partitioner.partition(record, metadata.fetch());
int serializedSize = Records.LOG_OVERHEAD + Record.recordSize(record.key(), record.value());
ensureValidRecordSize(serializedSize);
TopicPartition tp = new TopicPartition(record.topic(), partition);
@@ -256,6 +259,31 @@ public class KafkaProducer implements Producer {
}
/**
+ * Wait for cluster metadata including partitions for the given topic to be available.
+ * @param topic The topic we want metadata for
+ * @param maxWaitMs The maximum time in ms for waiting on the metadata
+ */
+ private void waitOnMetadata(String topic, long maxWaitMs) {
+ if (metadata.fetch().partitionsForTopic(topic) != null) {
+ return;
+ } else {
+ long begin = time.milliseconds();
+ long remainingWaitMs = maxWaitMs;
+ while (metadata.fetch().partitionsForTopic(topic) == null) {
+ log.trace("Requesting metadata update for topic {}.", topic);
+ int version = metadata.requestUpdate();
+ metadata.add(topic);
+ sender.wakeup();
+ metadata.awaitUpdate(version, remainingWaitMs);
+ long elapsed = time.milliseconds() - begin;
+ if (elapsed >= maxWaitMs)
+ throw new TimeoutException("Failed to update metadata after " + maxWaitMs + " ms.");
+ remainingWaitMs = maxWaitMs - elapsed;
+ }
+ }
+ }
+
+ /**
* Validate that the record size isn't too large
*/
private void ensureValidRecordSize(int size) {
@@ -271,8 +299,10 @@ public class KafkaProducer implements Producer {
" configuration.");
}
+ @Override
public List<PartitionInfo> partitionsFor(String topic) {
- return this.metadata.fetch(topic, this.metadataFetchTimeoutMs).partitionsForTopic(topic);
+ waitOnMetadata(topic, this.metadataFetchTimeoutMs);
+ return this.metadata.fetch().partitionsForTopic(topic);
}
@Override
http://git-wip-us.apache.org/repos/asf/kafka/blob/cd3ce27d/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
index 8890aa2..140237f 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
@@ -13,11 +13,9 @@
package org.apache.kafka.clients.producer.internals;
import java.util.HashSet;
-import java.util.List;
import java.util.Set;
import org.apache.kafka.common.Cluster;
-import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.errors.TimeoutException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -36,9 +34,10 @@ public final class Metadata {
private final long refreshBackoffMs;
private final long metadataExpireMs;
+ private int version;
private long lastRefreshMs;
private Cluster cluster;
- private boolean forceUpdate;
+ private boolean needUpdate;
private final Set<String> topics;
/**
@@ -58,8 +57,9 @@ public final class Metadata {
this.refreshBackoffMs = refreshBackoffMs;
this.metadataExpireMs = metadataExpireMs;
this.lastRefreshMs = 0L;
+ this.version = 0;
this.cluster = Cluster.empty();
- this.forceUpdate = false;
+ this.needUpdate = false;
this.topics = new HashSet<String>();
}
@@ -71,33 +71,10 @@ public final class Metadata {
}
/**
- * Fetch cluster metadata including partitions for the given topic. If there is no metadata for the given topic,
- * block waiting for an update.
- * @param topic The topic we want metadata for
- * @param maxWaitMs The maximum amount of time to block waiting for metadata
+ * Add the topic to maintain in the metadata
*/
- public synchronized Cluster fetch(String topic, long maxWaitMs) {
- List<PartitionInfo> partitions = null;
- long begin = System.currentTimeMillis();
- long remainingWaitMs = maxWaitMs;
- do {
- partitions = cluster.partitionsForTopic(topic);
- if (partitions == null) {
- topics.add(topic);
- forceUpdate = true;
- try {
- log.trace("Requesting metadata update for topic {}.", topic);
- wait(remainingWaitMs);
- } catch (InterruptedException e) { /* this is fine, just try again */
- }
- long elapsed = System.currentTimeMillis() - begin;
- if (elapsed >= maxWaitMs)
- throw new TimeoutException("Failed to update metadata after " + maxWaitMs + " ms.");
- remainingWaitMs = maxWaitMs - elapsed;
- } else {
- return cluster;
- }
- } while (true);
+ public synchronized void add(String topic) {
+ topics.add(topic);
}
/**
@@ -106,16 +83,35 @@ public final class Metadata {
* been request then the expiry time is now
*/
public synchronized long timeToNextUpdate(long nowMs) {
- long timeToExpire = forceUpdate ? 0 : Math.max(this.lastRefreshMs + this.metadataExpireMs - nowMs, 0);
+ long timeToExpire = needUpdate ? 0 : Math.max(this.lastRefreshMs + this.metadataExpireMs - nowMs, 0);
long timeToAllowUpdate = this.lastRefreshMs + this.refreshBackoffMs - nowMs;
return Math.max(timeToExpire, timeToAllowUpdate);
}
/**
- * Force an update of the current cluster info
+ * Request an update of the current cluster metadata info, return the current version before the update
*/
- public synchronized void forceUpdate() {
- this.forceUpdate = true;
+ public synchronized int requestUpdate() {
+ this.needUpdate = true;
+ return this.version;
+ }
+
+ /**
+ * Wait for metadata update until the current version is larger than the last version we know of
+ */
+ public synchronized void awaitUpdate(int lastVerison, long maxWaitMs) {
+ long begin = System.currentTimeMillis();
+ long remainingWaitMs = maxWaitMs;
+ while (this.version <= lastVerison) {
+ try {
+ wait(remainingWaitMs);
+ } catch (InterruptedException e) { /* this is fine */
+ }
+ long elapsed = System.currentTimeMillis() - begin;
+ if (elapsed >= maxWaitMs)
+ throw new TimeoutException("Failed to update metadata after " + maxWaitMs + " ms.");
+ remainingWaitMs = maxWaitMs - elapsed;
+ }
}
/**
@@ -129,8 +125,9 @@ public final class Metadata {
* Update the cluster metadata
*/
public synchronized void update(Cluster cluster, long now) {
- this.forceUpdate = false;
+ this.needUpdate = false;
this.lastRefreshMs = now;
+ this.version += 1;
this.cluster = cluster;
notifyAll();
log.debug("Updated cluster metadata to {}", cluster);
@@ -142,5 +139,4 @@ public final class Metadata {
public synchronized long lastUpdate() {
return this.lastRefreshMs;
}
-
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/cd3ce27d/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
index 37b9d1a..a016269 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
@@ -147,7 +147,7 @@ public class Sender implements Runnable {
// if there are any partitions whose leaders are not known yet, force metadata update
if (result.unknownLeadersExist)
- this.metadata.forceUpdate();
+ this.metadata.requestUpdate();
// remove any nodes we aren't ready to send to
Iterator<Node> iter = result.readyNodes.iterator();
@@ -252,7 +252,7 @@ public class Sender implements Runnable {
this.sensors.recordErrors(batch.topicPartition.topic(), batch.recordCount);
}
if (error.exception() instanceof InvalidMetadataException)
- metadata.forceUpdate();
+ metadata.requestUpdate();
}
/**
http://git-wip-us.apache.org/repos/asf/kafka/blob/cd3ce27d/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
index 0d7d04c..543304c 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
@@ -31,7 +31,7 @@ public class MetadataTest {
long time = 0;
metadata.update(Cluster.empty(), time);
assertFalse("No update needed.", metadata.timeToNextUpdate(time) == 0);
- metadata.forceUpdate();
+ metadata.requestUpdate();
assertFalse("Still no updated needed due to backoff", metadata.timeToNextUpdate(time) == 0);
time += refreshBackoffMs;
assertTrue("Update needed now that backoff time expired", metadata.timeToNextUpdate(time) == 0);
@@ -40,7 +40,9 @@ public class MetadataTest {
Thread t2 = asyncFetch(topic);
assertTrue("Awaiting update", t1.isAlive());
assertTrue("Awaiting update", t2.isAlive());
- metadata.update(TestUtils.singletonCluster(topic, 1), time);
+ // keep updating the metadata until no need to
+ while (metadata.timeToNextUpdate(time) == 0)
+ metadata.update(TestUtils.singletonCluster(topic, 1), time);
t1.join();
t2.join();
assertFalse("No update needed.", metadata.timeToNextUpdate(time) == 0);
@@ -51,7 +53,8 @@ public class MetadataTest {
private Thread asyncFetch(final String topic) {
Thread thread = new Thread() {
public void run() {
- metadata.fetch(topic, Integer.MAX_VALUE);
+ while (metadata.fetch().partitionsForTopic(topic) == null)
+ metadata.awaitUpdate(metadata.requestUpdate(), Long.MAX_VALUE);
}
};
thread.start();
[29/37] git commit: kafka-1542;
normal IOException in the new producer is logged as ERROR;
patched by David Corley; reviewed by Jun Rao
Posted by jj...@apache.org.
kafka-1542; normal IOException in the new producer is logged as ERROR; patched by David Corley; reviewed by Jun Rao
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/fa34841d
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/fa34841d
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/fa34841d
Branch: refs/heads/transactional_messaging
Commit: fa34841d9889fc45b4824b729b71bfb2215a082d
Parents: 7f2278f
Author: David Corley <da...@gmail.com>
Authored: Sun Jul 27 10:55:30 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Sun Jul 27 10:55:30 2014 -0700
----------------------------------------------------------------------
.../src/main/java/org/apache/kafka/common/network/Selector.java | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/fa34841d/clients/src/main/java/org/apache/kafka/common/network/Selector.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java
index 93f2f1c..45e57b4 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java
@@ -262,7 +262,7 @@ public class Selector implements Selectable {
if (!key.isValid())
close(key);
} catch (IOException e) {
- log.error("Error in I/O: ", e);
+ log.warn("Error in I/O with {}",channel.socket().getInetAddress().getHostAddress() , e);
close(key);
}
}
[32/37] git commit: kafka-1451;
Broker stuck due to leader election race; patched by Manikumar Reddy;
reviewed by Jun Rao
Posted by jj...@apache.org.
kafka-1451; Broker stuck due to leader election race; patched by Manikumar Reddy; reviewed by Jun Rao
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/a01a101e
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/a01a101e
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/a01a101e
Branch: refs/heads/transactional_messaging
Commit: a01a101e82d5b06e89857e79c4b8268589d81fca
Parents: 50f2b24
Author: Manikumar Reddy <ma...@gmail.com>
Authored: Wed Jul 30 08:14:41 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Wed Jul 30 08:14:41 2014 -0700
----------------------------------------------------------------------
.../kafka/server/ZookeeperLeaderElector.scala | 30 +++++++++++++++-----
1 file changed, 23 insertions(+), 7 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/a01a101e/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala b/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala
index e5b6ff1..a75818a 100644
--- a/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala
+++ b/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala
@@ -50,9 +50,27 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext,
}
}
+ private def getControllerID(): Int = {
+ readDataMaybeNull(controllerContext.zkClient, electionPath)._1 match {
+ case Some(controller) => KafkaController.parseControllerId(controller)
+ case None => -1
+ }
+ }
+
def elect: Boolean = {
val timestamp = SystemTime.milliseconds.toString
val electString = Json.encode(Map("version" -> 1, "brokerid" -> brokerId, "timestamp" -> timestamp))
+
+ leaderId = getControllerID
+ /*
+ * We can get here during the initial startup and the handleDeleted ZK callback. Because of the potential race condition,
+ * it's possible that the controller has already been elected when we get here. This check will prevent the following
+ * createEphemeralPath method from getting into an infinite loop if this broker is already the controller.
+ */
+ if(leaderId != -1) {
+ debug("Broker %d has been elected as leader, so stopping the election process.".format(leaderId))
+ return amILeader
+ }
try {
createEphemeralPathExpectConflictHandleZKBug(controllerContext.zkClient, electionPath, electString, brokerId,
@@ -64,15 +82,13 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext,
} catch {
case e: ZkNodeExistsException =>
// If someone else has written the path, then
- leaderId = readDataMaybeNull(controllerContext.zkClient, electionPath)._1 match {
- case Some(controller) => KafkaController.parseControllerId(controller)
- case None => {
- warn("A leader has been elected but just resigned, this will result in another round of election")
- -1
- }
- }
+ leaderId = getControllerID
+
if (leaderId != -1)
debug("Broker %d was elected as leader instead of broker %d".format(leaderId, brokerId))
+ else
+ warn("A leader has been elected but just resigned, this will result in another round of election")
+
case e2: Throwable =>
error("Error while electing or becoming leader on broker %d".format(brokerId), e2)
resign()
[11/37] git commit: kafka-1406; Fix scaladoc/javadoc warnings;
patched by Alan Lee; reviewed by Jun Rao
Posted by jj...@apache.org.
kafka-1406; Fix scaladoc/javadoc warnings; patched by Alan Lee; reviewed by Jun Rao
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/8034390e
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/8034390e
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/8034390e
Branch: refs/heads/transactional_messaging
Commit: 8034390ef0f49a1ed180bc1e883dbbd1f6d864a0
Parents: 420628d
Author: Alan Lee <co...@gmail.com>
Authored: Thu Jul 10 09:45:05 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Thu Jul 10 09:45:05 2014 -0700
----------------------------------------------------------------------
.../consumer/ConsumerRebalanceCallback.java | 4 +-
.../apache/kafka/common/network/Selectable.java | 8 +--
.../src/main/java/kafka/etl/Props.java | 73 +++++++++-----------
.../javaapi/consumer/ConsumerConnector.java | 3 +-
core/src/main/scala/kafka/log/OffsetIndex.scala | 2 +-
core/src/main/scala/kafka/utils/Utils.scala | 2 +-
6 files changed, 44 insertions(+), 48 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/8034390e/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java
index 05eb6ce..f026ae4 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java
@@ -21,8 +21,8 @@ import org.apache.kafka.common.TopicPartition;
* every rebalance operation. This callback will execute in the user thread as part of the
* {@link Consumer#poll(long) poll(long)} API on every rebalance attempt.
* Default implementation of the callback will {@link Consumer#seek(java.util.Map) seek(offsets)} to the last committed offsets in the
- * {@link #onPartitionsAssigned(Consumer, TopicPartition...) onPartitionsAssigned()} callback. And will commit offsets synchronously
- * for the specified list of partitions to Kafka in the {@link #onPartitionsRevoked(Consumer, TopicPartition...) onPartitionsRevoked()}
+ * {@link #onPartitionsAssigned(Consumer, Collection) onPartitionsAssigned()} callback. And will commit offsets synchronously
+ * for the specified list of partitions to Kafka in the {@link #onPartitionsRevoked(Consumer, Collection) onPartitionsRevoked()}
* callback.
*/
public interface ConsumerRebalanceCallback {
http://git-wip-us.apache.org/repos/asf/kafka/blob/8034390e/clients/src/main/java/org/apache/kafka/common/network/Selectable.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selectable.java b/clients/src/main/java/org/apache/kafka/common/network/Selectable.java
index cef75d8..b68bbf0 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/Selectable.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/Selectable.java
@@ -60,23 +60,23 @@ public interface Selectable {
public void poll(long timeout, List<NetworkSend> sends) throws IOException;
/**
- * The list of sends that completed on the last {@link #poll(long, List<NetworkSend>) poll()} call.
+ * The list of sends that completed on the last {@link #poll(long, List) poll()} call.
*/
public List<NetworkSend> completedSends();
/**
- * The list of receives that completed on the last {@link #poll(long, List<NetworkSend>) poll()} call.
+ * The list of receives that completed on the last {@link #poll(long, List) poll()} call.
*/
public List<NetworkReceive> completedReceives();
/**
- * The list of connections that finished disconnecting on the last {@link #poll(long, List<NetworkSend>) poll()}
+ * The list of connections that finished disconnecting on the last {@link #poll(long, List) poll()}
* call.
*/
public List<Integer> disconnected();
/**
- * The list of connections that completed their connection on the last {@link #poll(long, List<NetworkSend>) poll()}
+ * The list of connections that completed their connection on the last {@link #poll(long, List) poll()}
* call.
*/
public List<Integer> connected();
http://git-wip-us.apache.org/repos/asf/kafka/blob/8034390e/contrib/hadoop-consumer/src/main/java/kafka/etl/Props.java
----------------------------------------------------------------------
diff --git a/contrib/hadoop-consumer/src/main/java/kafka/etl/Props.java b/contrib/hadoop-consumer/src/main/java/kafka/etl/Props.java
index 3514ec7..71eb80f 100644
--- a/contrib/hadoop-consumer/src/main/java/kafka/etl/Props.java
+++ b/contrib/hadoop-consumer/src/main/java/kafka/etl/Props.java
@@ -115,11 +115,11 @@ public class Props extends Properties {
}
/**
- * build props from a list of strings and interprate them as
+ * build props from a list of strings and interpret them as
* key, value, key, value,....
*
* @param args
- * @return
+ * @return props
*/
@SuppressWarnings("unchecked")
public static Props of(String... args) {
@@ -148,7 +148,7 @@ public class Props extends Properties {
/**
* get property of "key" and split the value by " ,"
* @param key
- * @return
+ * @return list of values
*/
public List<String> getStringList(String key) {
return getStringList(key, "\\s*,\\s*");
@@ -158,7 +158,7 @@ public class Props extends Properties {
* get property of "key" and split the value by "sep"
* @param key
* @param sep
- * @return
+ * @return string list of values
*/
public List<String> getStringList(String key, String sep) {
String val = super.getProperty(key);
@@ -176,7 +176,7 @@ public class Props extends Properties {
* get string list with default value. default delimiter is ","
* @param key
* @param defaultValue
- * @return
+ * @return string list of values
*/
public List<String> getStringList(String key, List<String> defaultValue) {
if (containsKey(key))
@@ -189,7 +189,7 @@ public class Props extends Properties {
* get string list with default value
* @param key
* @param defaultValue
- * @return
+ * @return string list of values
*/
public List<String> getStringList(String key, List<String> defaultValue,
String sep) {
@@ -251,10 +251,10 @@ public class Props extends Properties {
}
/**
- * get boolean value
+ * get boolean value with default value
* @param key
* @param defaultValue
- * @return
+ * @return boolean value
* @throws Exception if value is not of type boolean or string
*/
public Boolean getBoolean(String key, Boolean defaultValue)
@@ -265,8 +265,7 @@ public class Props extends Properties {
/**
* get boolean value
* @param key
- * @param defaultValue
- * @return
+ * @return boolean value
* @throws Exception if value is not of type boolean or string or
* if value doesn't exist
*/
@@ -275,10 +274,10 @@ public class Props extends Properties {
}
/**
- * get long value
- * @param key
+ * get long value with default value
+ * @param name
* @param defaultValue
- * @return
+ * @return long value
* @throws Exception if value is not of type long or string
*/
public Long getLong(String name, Long defaultValue)
@@ -288,9 +287,8 @@ public class Props extends Properties {
/**
* get long value
- * @param key
- * @param defaultValue
- * @return
+ * @param name
+ * @return long value
* @throws Exception if value is not of type long or string or
* if value doesn't exist
*/
@@ -299,10 +297,10 @@ public class Props extends Properties {
}
/**
- * get integer value
- * @param key
+ * get integer value with default value
+ * @param name
* @param defaultValue
- * @return
+ * @return integer value
* @throws Exception if value is not of type integer or string
*/
public Integer getInt(String name, Integer defaultValue)
@@ -312,9 +310,8 @@ public class Props extends Properties {
/**
* get integer value
- * @param key
- * @param defaultValue
- * @return
+ * @param name
+ * @return integer value
* @throws Exception if value is not of type integer or string or
* if value doesn't exist
*/
@@ -323,10 +320,10 @@ public class Props extends Properties {
}
/**
- * get double value
- * @param key
+ * get double value with default value
+ * @param name
* @param defaultValue
- * @return
+ * @return double value
* @throws Exception if value is not of type double or string
*/
public Double getDouble(String name, double defaultValue)
@@ -336,9 +333,8 @@ public class Props extends Properties {
/**
* get double value
- * @param key
- * @param defaultValue
- * @return
+ * @param name
+ * @return double value
* @throws Exception if value is not of type double or string or
* if value doesn't exist
*/
@@ -347,10 +343,10 @@ public class Props extends Properties {
}
/**
- * get URI value
- * @param key
+ * get URI value with default value
+ * @param name
* @param defaultValue
- * @return
+ * @return URI value
* @throws Exception if value is not of type URI or string
*/
public URI getUri(String name, URI defaultValue) throws Exception {
@@ -359,9 +355,9 @@ public class Props extends Properties {
/**
* get URI value
- * @param key
+ * @param name
* @param defaultValue
- * @return
+ * @return URI value
* @throws Exception if value is not of type URI or string
*/
public URI getUri(String name, String defaultValue)
@@ -372,9 +368,8 @@ public class Props extends Properties {
/**
* get URI value
- * @param key
- * @param defaultValue
- * @return
+ * @param name
+ * @return URI value
* @throws Exception if value is not of type URI or string or
* if value doesn't exist
*/
@@ -385,7 +380,7 @@ public class Props extends Properties {
/**
* compare two props
* @param p
- * @return
+ * @return true or false
*/
public boolean equalsProps(Props p) {
if (p == null) {
@@ -432,7 +427,7 @@ public class Props extends Properties {
/**
* get all property names
- * @return
+ * @return set of property names
*/
public Set<String> getKeySet() {
return super.stringPropertyNames();
@@ -453,7 +448,7 @@ public class Props extends Properties {
/**
* clone a Props
* @param p
- * @return
+ * @return props
*/
public static Props clone(Props p) {
return new Props(p);
http://git-wip-us.apache.org/repos/asf/kafka/blob/8034390e/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java
index 44d3d35..cc3400f 100644
--- a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java
+++ b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java
@@ -29,7 +29,8 @@ public interface ConsumerConnector {
* Create a list of MessageStreams of type T for each topic.
*
* @param topicCountMap a map of (topic, #streams) pair
- * @param decoder a decoder that converts from Message to T
+ * @param keyDecoder a decoder that decodes the message key
+ * @param valueDecoder a decoder that decodes the message itself
* @return a map of (topic, list of KafkaStream) pairs.
* The number of items in the list is #streams. Each stream supports
* an iterator over message/metadata pairs.
http://git-wip-us.apache.org/repos/asf/kafka/blob/8034390e/core/src/main/scala/kafka/log/OffsetIndex.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/OffsetIndex.scala b/core/src/main/scala/kafka/log/OffsetIndex.scala
index 8a62dfa..1c4c7bd 100644
--- a/core/src/main/scala/kafka/log/OffsetIndex.scala
+++ b/core/src/main/scala/kafka/log/OffsetIndex.scala
@@ -346,7 +346,7 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi
/**
* Do a basic sanity check on this index to detect obvious problems
- * @throw IllegalArgumentException if any problems are found
+ * @throws IllegalArgumentException if any problems are found
*/
def sanityCheck() {
require(entries == 0 || lastOffset > baseOffset,
http://git-wip-us.apache.org/repos/asf/kafka/blob/8034390e/core/src/main/scala/kafka/utils/Utils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala
index b29981b..63d3dda 100644
--- a/core/src/main/scala/kafka/utils/Utils.scala
+++ b/core/src/main/scala/kafka/utils/Utils.scala
@@ -494,7 +494,7 @@ object Utils extends Logging {
/**
* Create a file with the given path
* @param path The path to create
- * @throw KafkaStorageException If the file create fails
+ * @throws KafkaStorageException If the file create fails
* @return The created file
*/
def createFile(path: String): File = {
[15/37] git commit: kafka-1531;
zookeeper.connection.timeout.ms is set to 10000000 in configuration
file in Kafka tarball; patched by Manikumar Reddy; reviewed by Jun Rao
Posted by jj...@apache.org.
kafka-1531; zookeeper.connection.timeout.ms is set to 10000000 in configuration file in Kafka tarball; patched by Manikumar Reddy; reviewed by Jun Rao
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/b428d8cc
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/b428d8cc
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/b428d8cc
Branch: refs/heads/transactional_messaging
Commit: b428d8cc48237099af648de12d18be78d54446eb
Parents: 3e6b386
Author: Manikumar Reddy <ma...@gmail.com>
Authored: Wed Jul 16 08:13:06 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Wed Jul 16 08:13:06 2014 -0700
----------------------------------------------------------------------
config/consumer.properties | 2 +-
config/server.properties | 14 +++++++-------
2 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/b428d8cc/config/consumer.properties
----------------------------------------------------------------------
diff --git a/config/consumer.properties b/config/consumer.properties
index 7343cbc..83847de 100644
--- a/config/consumer.properties
+++ b/config/consumer.properties
@@ -20,7 +20,7 @@
zookeeper.connect=127.0.0.1:2181
# timeout in ms for connecting to zookeeper
-zookeeper.connection.timeout.ms=1000000
+zookeeper.connection.timeout.ms=6000
#consumer group id
group.id=test-consumer-group
http://git-wip-us.apache.org/repos/asf/kafka/blob/b428d8cc/config/server.properties
----------------------------------------------------------------------
diff --git a/config/server.properties b/config/server.properties
index c9e923a..f16c84c 100644
--- a/config/server.properties
+++ b/config/server.properties
@@ -37,16 +37,16 @@ port=9092
#advertised.port=<port accessible by clients>
# The number of threads handling network requests
-num.network.threads=2
+num.network.threads=3
# The number of threads doing disk I/O
num.io.threads=8
# The send buffer (SO_SNDBUF) used by the socket server
-socket.send.buffer.bytes=1048576
+socket.send.buffer.bytes=102400
# The receive buffer (SO_RCVBUF) used by the socket server
-socket.receive.buffer.bytes=1048576
+socket.receive.buffer.bytes=65536
# The maximum size of a request that the socket server will accept (protection against OOM)
socket.request.max.bytes=104857600
@@ -60,7 +60,7 @@ log.dirs=/tmp/kafka-logs
# The default number of log partitions per topic. More partitions allow greater
# parallelism for consumption, but this will also result in more files across
# the brokers.
-num.partitions=2
+num.partitions=1
############################# Log Flush Policy #############################
@@ -94,11 +94,11 @@ log.retention.hours=168
#log.retention.bytes=1073741824
# The maximum size of a log segment file. When this size is reached a new log segment will be created.
-log.segment.bytes=536870912
+log.segment.bytes=1073741824
# The interval at which log segments are checked to see if they can be deleted according
# to the retention policies
-log.retention.check.interval.ms=60000
+log.retention.check.interval.ms=300000
# By default the log cleaner is disabled and the log retention policy will default to just delete segments after their retention expires.
# If log.cleaner.enable=true is set the cleaner will be enabled and individual logs can then be marked for log compaction.
@@ -114,4 +114,4 @@ log.cleaner.enable=false
zookeeper.connect=localhost:2181
# Timeout in ms for connecting to zookeeper
-zookeeper.connection.timeout.ms=1000000
+zookeeper.connection.timeout.ms=6000
[30/37] git commit: kafka-1414;
Speedup broker startup after hard reset; patched by Anton Karamanov;
reviewed by Jay Kreps and Jun Rao
Posted by jj...@apache.org.
kafka-1414; Speedup broker startup after hard reset; patched by Anton Karamanov; reviewed by Jay Kreps and Jun Rao
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/f489493c
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/f489493c
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/f489493c
Branch: refs/heads/transactional_messaging
Commit: f489493c385e3266c3bc17db2e8ebc215a6e54e2
Parents: fa34841
Author: Anton Karamanov <at...@gmail.com>
Authored: Sun Jul 27 21:13:20 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Sun Jul 27 21:13:20 2014 -0700
----------------------------------------------------------------------
config/server.properties | 4 +
core/src/main/scala/kafka/log/LogManager.scala | 179 ++++++++++++++-----
.../main/scala/kafka/server/KafkaConfig.scala | 3 +
.../main/scala/kafka/server/KafkaServer.scala | 1 +
.../main/scala/kafka/utils/KafkaScheduler.scala | 18 +-
core/src/main/scala/kafka/utils/Utils.scala | 6 +-
.../scala/unit/kafka/log/LogManagerTest.scala | 97 ++--------
.../server/HighwatermarkPersistenceTest.scala | 17 +-
.../unit/kafka/server/ReplicaManagerTest.scala | 20 +--
.../test/scala/unit/kafka/utils/TestUtils.scala | 25 +++
10 files changed, 203 insertions(+), 167 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/f489493c/config/server.properties
----------------------------------------------------------------------
diff --git a/config/server.properties b/config/server.properties
index f16c84c..5c0905a 100644
--- a/config/server.properties
+++ b/config/server.properties
@@ -62,6 +62,10 @@ log.dirs=/tmp/kafka-logs
# the brokers.
num.partitions=1
+# The number of threads per data directory to be used for log recovery at startup and flushing at shutdown.
+# This value is recommended to be increased for installations with data dirs located in RAID array.
+num.recovery.threads.per.data.dir=1
+
############################# Log Flush Policy #############################
# Messages are immediately written to the filesystem but by default we only fsync() to sync
http://git-wip-us.apache.org/repos/asf/kafka/blob/f489493c/core/src/main/scala/kafka/log/LogManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala
index 1946c94..4d2924d 100644
--- a/core/src/main/scala/kafka/log/LogManager.scala
+++ b/core/src/main/scala/kafka/log/LogManager.scala
@@ -23,6 +23,7 @@ import kafka.utils._
import scala.collection._
import kafka.common.{TopicAndPartition, KafkaException}
import kafka.server.{RecoveringFromUncleanShutdown, BrokerState, OffsetCheckpoint}
+import java.util.concurrent.{Executors, ExecutorService, ExecutionException, Future}
/**
* The entry point to the kafka log management subsystem. The log manager is responsible for log creation, retrieval, and cleaning.
@@ -39,6 +40,7 @@ class LogManager(val logDirs: Array[File],
val topicConfigs: Map[String, LogConfig],
val defaultConfig: LogConfig,
val cleanerConfig: CleanerConfig,
+ ioThreads: Int,
val flushCheckMs: Long,
val flushCheckpointMs: Long,
val retentionCheckMs: Long,
@@ -54,7 +56,7 @@ class LogManager(val logDirs: Array[File],
createAndValidateLogDirs(logDirs)
private val dirLocks = lockLogDirs(logDirs)
private val recoveryPointCheckpoints = logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, RecoveryPointCheckpointFile)))).toMap
- loadLogs(logDirs)
+ loadLogs()
private val cleaner: LogCleaner =
if(cleanerConfig.enableCleaner)
@@ -101,36 +103,71 @@ class LogManager(val logDirs: Array[File],
/**
* Recover and load all logs in the given data directories
*/
- private def loadLogs(dirs: Seq[File]) {
- for(dir <- dirs) {
+ private def loadLogs(): Unit = {
+ info("Loading logs.")
+
+ val threadPools = mutable.ArrayBuffer.empty[ExecutorService]
+ val jobs = mutable.Map.empty[File, Seq[Future[_]]]
+
+ for (dir <- this.logDirs) {
+ val pool = Executors.newFixedThreadPool(ioThreads)
+ threadPools.append(pool)
+
+ val cleanShutdownFile = new File(dir, Log.CleanShutdownFile)
+
+ if (cleanShutdownFile.exists) {
+ debug(
+ "Found clean shutdown file. " +
+ "Skipping recovery for all logs in data directory: " +
+ dir.getAbsolutePath)
+ } else {
+ // log recovery itself is being performed by `Log` class during initialization
+ brokerState.newState(RecoveringFromUncleanShutdown)
+ }
+
val recoveryPoints = this.recoveryPointCheckpoints(dir).read
- /* load the logs */
- val subDirs = dir.listFiles()
- if(subDirs != null) {
- val cleanShutDownFile = new File(dir, Log.CleanShutdownFile)
- if(cleanShutDownFile.exists())
- info("Found clean shutdown file. Skipping recovery for all logs in data directory '%s'".format(dir.getAbsolutePath))
- else
- brokerState.newState(RecoveringFromUncleanShutdown)
-
- for(dir <- subDirs) {
- if(dir.isDirectory) {
- info("Loading log '" + dir.getName + "'")
- val topicPartition = Log.parseTopicPartitionName(dir.getName)
- val config = topicConfigs.getOrElse(topicPartition.topic, defaultConfig)
- val log = new Log(dir,
- config,
- recoveryPoints.getOrElse(topicPartition, 0L),
- scheduler,
- time)
- val previous = this.logs.put(topicPartition, log)
- if(previous != null)
- throw new IllegalArgumentException("Duplicate log directories found: %s, %s!".format(log.dir.getAbsolutePath, previous.dir.getAbsolutePath))
+
+ val jobsForDir = for {
+ dirContent <- Option(dir.listFiles).toList
+ logDir <- dirContent if logDir.isDirectory
+ } yield {
+ Utils.runnable {
+ debug("Loading log '" + logDir.getName + "'")
+
+ val topicPartition = Log.parseTopicPartitionName(logDir.getName)
+ val config = topicConfigs.getOrElse(topicPartition.topic, defaultConfig)
+ val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L)
+
+ val current = new Log(logDir, config, logRecoveryPoint, scheduler, time)
+ val previous = this.logs.put(topicPartition, current)
+
+ if (previous != null) {
+ throw new IllegalArgumentException(
+ "Duplicate log directories found: %s, %s!".format(
+ current.dir.getAbsolutePath, previous.dir.getAbsolutePath))
}
}
- cleanShutDownFile.delete()
}
+
+ jobs(cleanShutdownFile) = jobsForDir.map(pool.submit).toSeq
+ }
+
+
+ try {
+ for ((cleanShutdownFile, dirJobs) <- jobs) {
+ dirJobs.foreach(_.get)
+ cleanShutdownFile.delete()
+ }
+ } catch {
+ case e: ExecutionException => {
+ error("There was an error in one of the threads during logs loading: " + e.getCause)
+ throw e.getCause
+ }
+ } finally {
+ threadPools.foreach(_.shutdown())
}
+
+ info("Logs loading complete.")
}
/**
@@ -160,31 +197,69 @@ class LogManager(val logDirs: Array[File],
if(cleanerConfig.enableCleaner)
cleaner.startup()
}
-
+
/**
* Close all the logs
*/
def shutdown() {
info("Shutting down.")
+
+ val threadPools = mutable.ArrayBuffer.empty[ExecutorService]
+ val jobs = mutable.Map.empty[File, Seq[Future[_]]]
+
+ // stop the cleaner first
+ if (cleaner != null) {
+ Utils.swallow(cleaner.shutdown())
+ }
+
+ // close logs in each dir
+ for (dir <- this.logDirs) {
+ debug("Flushing and closing logs at " + dir)
+
+ val pool = Executors.newFixedThreadPool(ioThreads)
+ threadPools.append(pool)
+
+ val logsInDir = logsByDir.getOrElse(dir.toString, Map()).values
+
+ val jobsForDir = logsInDir map { log =>
+ Utils.runnable {
+ // flush the log to ensure latest possible recovery point
+ log.flush()
+ log.close()
+ }
+ }
+
+ jobs(dir) = jobsForDir.map(pool.submit).toSeq
+ }
+
+
try {
- // stop the cleaner first
- if(cleaner != null)
- Utils.swallow(cleaner.shutdown())
- // flush the logs to ensure latest possible recovery point
- allLogs.foreach(_.flush())
- // close the logs
- allLogs.foreach(_.close())
- // update the last flush point
- checkpointRecoveryPointOffsets()
- // mark that the shutdown was clean by creating the clean shutdown marker file
- logDirs.foreach(dir => Utils.swallow(new File(dir, Log.CleanShutdownFile).createNewFile()))
+ for ((dir, dirJobs) <- jobs) {
+ dirJobs.foreach(_.get)
+
+ // update the last flush point
+ debug("Updating recovery points at " + dir)
+ checkpointLogsInDir(dir)
+
+ // mark that the shutdown was clean by creating marker file
+ debug("Writing clean shutdown marker at " + dir)
+ Utils.swallow(new File(dir, Log.CleanShutdownFile).createNewFile())
+ }
+ } catch {
+ case e: ExecutionException => {
+ error("There was an error in one of the threads during LogManager shutdown: " + e.getCause)
+ throw e.getCause
+ }
} finally {
+ threadPools.foreach(_.shutdown())
// regardless of whether the close succeeded, we need to unlock the data directories
dirLocks.foreach(_.destroy())
}
+
info("Shutdown complete.")
}
+
/**
* Truncate the partition logs to the specified offsets and checkpoint the recovery point to this offset
*
@@ -230,14 +305,19 @@ class LogManager(val logDirs: Array[File],
* to avoid recovering the whole log on startup.
*/
def checkpointRecoveryPointOffsets() {
- val recoveryPointsByDir = this.logsByTopicPartition.groupBy(_._2.dir.getParent.toString)
- for(dir <- logDirs) {
- val recoveryPoints = recoveryPointsByDir.get(dir.toString)
- if(recoveryPoints.isDefined)
- this.recoveryPointCheckpoints(dir).write(recoveryPoints.get.mapValues(_.recoveryPoint))
+ this.logDirs.foreach(checkpointLogsInDir)
+ }
+
+ /**
+ * Make a checkpoint for all logs in provided directory.
+ */
+ private def checkpointLogsInDir(dir: File): Unit = {
+ val recoveryPoints = this.logsByDir.get(dir.toString)
+ if (recoveryPoints.isDefined) {
+ this.recoveryPointCheckpoints(dir).write(recoveryPoints.get.mapValues(_.recoveryPoint))
}
}
-
+
/**
* Get the log if it exists, otherwise return None
*/
@@ -366,13 +446,22 @@ class LogManager(val logDirs: Array[File],
* Get all the partition logs
*/
def allLogs(): Iterable[Log] = logs.values
-
+
/**
* Get a map of TopicAndPartition => Log
*/
def logsByTopicPartition = logs.toMap
/**
+ * Map of log dir to logs by topic and partitions in that dir
+ */
+ private def logsByDir = {
+ this.logsByTopicPartition.groupBy {
+ case (_, log) => log.dir.getParent
+ }
+ }
+
+ /**
* Flush any log which has exceeded its flush interval and has unwritten messages.
*/
private def flushDirtyLogs() = {
http://git-wip-us.apache.org/repos/asf/kafka/blob/f489493c/core/src/main/scala/kafka/server/KafkaConfig.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala
index 50b09ed..1a45f87 100644
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -190,6 +190,9 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
/* the frequency with which we update the persistent record of the last flush which acts as the log recovery point */
val logFlushOffsetCheckpointIntervalMs = props.getIntInRange("log.flush.offset.checkpoint.interval.ms", 60000, (0, Int.MaxValue))
+ /* the number of threads per data directory to be used for log recovery at startup and flushing at shutdown */
+ val numRecoveryThreadsPerDataDir = props.getIntInRange("num.recovery.threads.per.data.dir", 1, (1, Int.MaxValue))
+
/* enable auto creation of topic on the server */
val autoCreateTopicsEnable = props.getBoolean("auto.create.topics.enable", true)
http://git-wip-us.apache.org/repos/asf/kafka/blob/f489493c/core/src/main/scala/kafka/server/KafkaServer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala
index def1dc2..2871118 100644
--- a/core/src/main/scala/kafka/server/KafkaServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaServer.scala
@@ -303,6 +303,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
topicConfigs = configs,
defaultConfig = defaultLogConfig,
cleanerConfig = cleanerConfig,
+ ioThreads = config.numRecoveryThreadsPerDataDir,
flushCheckMs = config.logFlushSchedulerIntervalMs,
flushCheckpointMs = config.logFlushOffsetCheckpointIntervalMs,
retentionCheckMs = config.logCleanupIntervalMs,
http://git-wip-us.apache.org/repos/asf/kafka/blob/f489493c/core/src/main/scala/kafka/utils/KafkaScheduler.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/utils/KafkaScheduler.scala b/core/src/main/scala/kafka/utils/KafkaScheduler.scala
index 8e37505..9a16343 100644
--- a/core/src/main/scala/kafka/utils/KafkaScheduler.scala
+++ b/core/src/main/scala/kafka/utils/KafkaScheduler.scala
@@ -93,16 +93,14 @@ class KafkaScheduler(val threads: Int,
debug("Scheduling task %s with initial delay %d ms and period %d ms."
.format(name, TimeUnit.MILLISECONDS.convert(delay, unit), TimeUnit.MILLISECONDS.convert(period, unit)))
ensureStarted
- val runnable = new Runnable {
- def run() = {
- try {
- trace("Begining execution of scheduled task '%s'.".format(name))
- fun()
- } catch {
- case t: Throwable => error("Uncaught exception in scheduled task '" + name +"'", t)
- } finally {
- trace("Completed execution of scheduled task '%s'.".format(name))
- }
+ val runnable = Utils.runnable {
+ try {
+ trace("Begining execution of scheduled task '%s'.".format(name))
+ fun()
+ } catch {
+ case t: Throwable => error("Uncaught exception in scheduled task '" + name +"'", t)
+ } finally {
+ trace("Completed execution of scheduled task '%s'.".format(name))
}
}
if(period >= 0)
http://git-wip-us.apache.org/repos/asf/kafka/blob/f489493c/core/src/main/scala/kafka/utils/Utils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala
index 09bfbce..da52b42 100644
--- a/core/src/main/scala/kafka/utils/Utils.scala
+++ b/core/src/main/scala/kafka/utils/Utils.scala
@@ -49,9 +49,9 @@ object Utils extends Logging {
* @param fun A function
* @return A Runnable that just executes the function
*/
- def runnable(fun: () => Unit): Runnable =
- new Runnable() {
- def run() = fun()
+ def runnable(fun: => Unit): Runnable =
+ new Runnable {
+ def run() = fun
}
/**
http://git-wip-us.apache.org/repos/asf/kafka/blob/f489493c/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
index d03d4c4..7d4c70c 100644
--- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
@@ -35,21 +35,11 @@ class LogManagerTest extends JUnit3Suite {
var logManager: LogManager = null
val name = "kafka"
val veryLargeLogFlushInterval = 10000000L
- val cleanerConfig = CleanerConfig(enableCleaner = false)
override def setUp() {
super.setUp()
logDir = TestUtils.tempDir()
- logManager = new LogManager(logDirs = Array(logDir),
- topicConfigs = Map(),
- defaultConfig = logConfig,
- cleanerConfig = cleanerConfig,
- flushCheckMs = 1000L,
- flushCheckpointMs = 100000L,
- retentionCheckMs = 1000L,
- scheduler = time.scheduler,
- time = time,
- brokerState = new BrokerState())
+ logManager = createLogManager()
logManager.startup
logDir = logManager.logDirs(0)
}
@@ -125,18 +115,7 @@ class LogManagerTest extends JUnit3Suite {
logManager.shutdown()
val config = logConfig.copy(segmentSize = 10 * (setSize - 1), retentionSize = 5L * 10L * setSize + 10L)
- logManager = new LogManager(
- logDirs = Array(logDir),
- topicConfigs = Map(),
- defaultConfig = config,
- cleanerConfig = cleanerConfig,
- flushCheckMs = 1000L,
- flushCheckpointMs = 100000L,
- retentionCheckMs = 1000L,
- scheduler = time.scheduler,
- brokerState = new BrokerState(),
- time = time
- )
+ logManager = createLogManager()
logManager.startup
// create a log
@@ -176,18 +155,7 @@ class LogManagerTest extends JUnit3Suite {
def testTimeBasedFlush() {
logManager.shutdown()
val config = logConfig.copy(flushMs = 1000)
- logManager = new LogManager(
- logDirs = Array(logDir),
- topicConfigs = Map(),
- defaultConfig = config,
- cleanerConfig = cleanerConfig,
- flushCheckMs = 1000L,
- flushCheckpointMs = 10000L,
- retentionCheckMs = 1000L,
- scheduler = time.scheduler,
- brokerState = new BrokerState(),
- time = time
- )
+ logManager = createLogManager()
logManager.startup
val log = logManager.createLog(TopicAndPartition(name, 0), config)
val lastFlush = log.lastFlushTime
@@ -209,19 +177,8 @@ class LogManagerTest extends JUnit3Suite {
TestUtils.tempDir(),
TestUtils.tempDir())
logManager.shutdown()
- logManager = new LogManager(
- logDirs = dirs,
- topicConfigs = Map(),
- defaultConfig = logConfig,
- cleanerConfig = cleanerConfig,
- flushCheckMs = 1000L,
- flushCheckpointMs = 10000L,
- retentionCheckMs = 1000L,
- scheduler = time.scheduler,
- brokerState = new BrokerState(),
- time = time
- )
-
+ logManager = createLogManager()
+
// verify that logs are always assigned to the least loaded partition
for(partition <- 0 until 20) {
logManager.createLog(TopicAndPartition("test", partition), logConfig)
@@ -237,18 +194,7 @@ class LogManagerTest extends JUnit3Suite {
@Test
def testTwoLogManagersUsingSameDirFails() {
try {
- new LogManager(
- logDirs = Array(logDir),
- topicConfigs = Map(),
- defaultConfig = logConfig,
- cleanerConfig = cleanerConfig,
- flushCheckMs = 1000L,
- flushCheckpointMs = 10000L,
- retentionCheckMs = 1000L,
- scheduler = time.scheduler,
- brokerState = new BrokerState(),
- time = time
- )
+ createLogManager()
fail("Should not be able to create a second log manager instance with the same data directory")
} catch {
case e: KafkaException => // this is good
@@ -270,16 +216,8 @@ class LogManagerTest extends JUnit3Suite {
def testRecoveryDirectoryMappingWithTrailingSlash() {
logManager.shutdown()
logDir = TestUtils.tempDir()
- logManager = new LogManager(logDirs = Array(new File(logDir.getAbsolutePath + File.separator)),
- topicConfigs = Map(),
- defaultConfig = logConfig,
- cleanerConfig = cleanerConfig,
- flushCheckMs = 1000L,
- flushCheckpointMs = 100000L,
- retentionCheckMs = 1000L,
- scheduler = time.scheduler,
- time = time,
- brokerState = new BrokerState())
+ logManager = TestUtils.createLogManager(
+ logDirs = Array(new File(logDir.getAbsolutePath + File.separator)))
logManager.startup
verifyCheckpointRecovery(Seq(TopicAndPartition("test-a", 1)), logManager)
}
@@ -293,16 +231,7 @@ class LogManagerTest extends JUnit3Suite {
logDir = new File("data" + File.separator + logDir.getName)
logDir.mkdirs()
logDir.deleteOnExit()
- logManager = new LogManager(logDirs = Array(logDir),
- topicConfigs = Map(),
- defaultConfig = logConfig,
- cleanerConfig = cleanerConfig,
- flushCheckMs = 1000L,
- flushCheckpointMs = 100000L,
- retentionCheckMs = 1000L,
- scheduler = time.scheduler,
- time = time,
- brokerState = new BrokerState())
+ logManager = createLogManager()
logManager.startup
verifyCheckpointRecovery(Seq(TopicAndPartition("test-a", 1)), logManager)
}
@@ -327,4 +256,12 @@ class LogManagerTest extends JUnit3Suite {
}
}
}
+
+
+ private def createLogManager(logDirs: Array[File] = Array(this.logDir)): LogManager = {
+ TestUtils.createLogManager(
+ defaultConfig = logConfig,
+ logDirs = logDirs,
+ time = this.time)
+ }
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/f489493c/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
index 558a5d6..e532c28 100644
--- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
+++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
@@ -32,16 +32,11 @@ class HighwatermarkPersistenceTest extends JUnit3Suite {
val configs = TestUtils.createBrokerConfigs(2).map(new KafkaConfig(_))
val topic = "foo"
- val logManagers = configs.map(config => new LogManager(logDirs = config.logDirs.map(new File(_)).toArray,
- topicConfigs = Map(),
- defaultConfig = LogConfig(),
- cleanerConfig = CleanerConfig(),
- flushCheckMs = 30000,
- flushCheckpointMs = 10000L,
- retentionCheckMs = 30000,
- scheduler = new KafkaScheduler(1),
- brokerState = new BrokerState(),
- time = new MockTime))
+ val logManagers = configs map { config =>
+ TestUtils.createLogManager(
+ logDirs = config.logDirs.map(new File(_)).toArray,
+ cleanerConfig = CleanerConfig())
+ }
@After
def teardown() {
@@ -147,4 +142,4 @@ class HighwatermarkPersistenceTest extends JUnit3Suite {
replicaManager.highWatermarkCheckpoints(new File(replicaManager.config.logDirs(0)).getAbsolutePath).read.getOrElse(TopicAndPartition(topic, partition), 0L)
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/f489493c/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index 518d416..9abf219 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -18,7 +18,6 @@
package kafka.server
import kafka.utils.{MockScheduler, MockTime, TestUtils}
-import kafka.log.{CleanerConfig, LogManager, LogConfig}
import java.util.concurrent.atomic.AtomicBoolean
import java.io.File
@@ -37,7 +36,7 @@ class ReplicaManagerTest extends JUnit3Suite {
val props = TestUtils.createBrokerConfig(1)
val config = new KafkaConfig(props)
val zkClient = EasyMock.createMock(classOf[ZkClient])
- val mockLogMgr = createLogManager(config.logDirs.map(new File(_)).toArray)
+ val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray)
val time: MockTime = new MockTime()
val rm = new ReplicaManager(config, time, zkClient, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false))
val partition = rm.getOrCreatePartition(topic, 1, 1)
@@ -51,26 +50,11 @@ class ReplicaManagerTest extends JUnit3Suite {
props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath)
val config = new KafkaConfig(props)
val zkClient = EasyMock.createMock(classOf[ZkClient])
- val mockLogMgr = createLogManager(config.logDirs.map(new File(_)).toArray)
+ val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray)
val time: MockTime = new MockTime()
val rm = new ReplicaManager(config, time, zkClient, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false))
val partition = rm.getOrCreatePartition(topic, 1, 1)
partition.getOrCreateReplica(1)
rm.checkpointHighWatermarks()
}
-
- private def createLogManager(logDirs: Array[File]): LogManager = {
- val time = new MockTime()
- return new LogManager(logDirs,
- topicConfigs = Map(),
- defaultConfig = new LogConfig(),
- cleanerConfig = CleanerConfig(enableCleaner = false),
- flushCheckMs = 1000L,
- flushCheckpointMs = 100000L,
- retentionCheckMs = 1000L,
- scheduler = time.scheduler,
- brokerState = new BrokerState(),
- time = time)
- }
-
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/f489493c/core/src/test/scala/unit/kafka/utils/TestUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index 4d01d25..c4e13c5 100644
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -39,6 +39,7 @@ import kafka.serializer.{StringEncoder, DefaultEncoder, Encoder}
import kafka.common.TopicAndPartition
import kafka.admin.AdminUtils
import kafka.producer.ProducerConfig
+import kafka.log._
import junit.framework.AssertionFailedError
import junit.framework.Assert._
@@ -689,6 +690,30 @@ object TestUtils extends Logging {
def checkIfReassignPartitionPathExists(zkClient: ZkClient): Boolean = {
ZkUtils.pathExists(zkClient, ZkUtils.ReassignPartitionsPath)
}
+
+
+ /**
+ * Create new LogManager instance with default configuration for testing
+ */
+ def createLogManager(
+ logDirs: Array[File] = Array.empty[File],
+ defaultConfig: LogConfig = LogConfig(),
+ cleanerConfig: CleanerConfig = CleanerConfig(enableCleaner = false),
+ time: MockTime = new MockTime()) =
+ {
+ new LogManager(
+ logDirs = logDirs,
+ topicConfigs = Map(),
+ defaultConfig = defaultConfig,
+ cleanerConfig = cleanerConfig,
+ ioThreads = 4,
+ flushCheckMs = 1000L,
+ flushCheckpointMs = 10000L,
+ retentionCheckMs = 1000L,
+ scheduler = time.scheduler,
+ time = time,
+ brokerState = new BrokerState())
+ }
}
object TestZKUtils {
[16/37] git commit: KAFKA-1512 Add per-ip connection limits.
Posted by jj...@apache.org.
KAFKA-1512 Add per-ip connection limits.
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/8e444a35
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/8e444a35
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/8e444a35
Branch: refs/heads/transactional_messaging
Commit: 8e444a3562d6723b9f33cbdaa6a461409c84c98b
Parents: b428d8c
Author: Jay Kreps <ja...@gmail.com>
Authored: Mon Jun 30 21:28:04 2014 -0700
Committer: Jay Kreps <ja...@gmail.com>
Committed: Wed Jul 16 09:53:15 2014 -0700
----------------------------------------------------------------------
.../main/scala/kafka/network/SocketServer.scala | 146 +++++++++++++------
.../main/scala/kafka/server/KafkaConfig.scala | 6 +
.../main/scala/kafka/server/KafkaServer.scala | 3 +-
.../unit/kafka/network/SocketServerTest.scala | 20 ++-
4 files changed, 129 insertions(+), 46 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/8e444a35/core/src/main/scala/kafka/network/SocketServer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala
index 4976d9c..8e99de0 100644
--- a/core/src/main/scala/kafka/network/SocketServer.scala
+++ b/core/src/main/scala/kafka/network/SocketServer.scala
@@ -23,6 +23,8 @@ import java.net._
import java.io._
import java.nio.channels._
+import scala.collection._
+
import kafka.common.KafkaException
import kafka.metrics.KafkaMetricsGroup
import kafka.utils._
@@ -41,7 +43,9 @@ class SocketServer(val brokerId: Int,
val maxQueuedRequests: Int,
val sendBufferSize: Int,
val recvBufferSize: Int,
- val maxRequestSize: Int = Int.MaxValue) extends Logging with KafkaMetricsGroup {
+ val maxRequestSize: Int = Int.MaxValue,
+ val maxConnectionsPerIp: Int = Int.MaxValue,
+ val maxConnectionsPerIpOverrides: Map[String, Int] = Map[String, Int]()) extends Logging with KafkaMetricsGroup {
this.logIdent = "[Socket Server on Broker " + brokerId + "], "
private val time = SystemTime
private val processors = new Array[Processor](numProcessorThreads)
@@ -55,17 +59,23 @@ class SocketServer(val brokerId: Int,
* Start the socket server
*/
def startup() {
+ val quotas = new ConnectionQuotas(maxConnectionsPerIp, maxConnectionsPerIpOverrides)
for(i <- 0 until numProcessorThreads) {
- processors(i) = new Processor(i, time, maxRequestSize, aggregateIdleMeter,
- newMeter("NetworkProcessor-" + i + "-IdlePercent", "percent", TimeUnit.NANOSECONDS),
- numProcessorThreads, requestChannel)
+ processors(i) = new Processor(i,
+ time,
+ maxRequestSize,
+ aggregateIdleMeter,
+ newMeter("NetworkProcessor-" + i + "-IdlePercent", "percent", TimeUnit.NANOSECONDS),
+ numProcessorThreads,
+ requestChannel,
+ quotas)
Utils.newThread("kafka-network-thread-%d-%d".format(port, i), processors(i), false).start()
}
// register the processor threads for notification of responses
requestChannel.addResponseListener((id:Int) => processors(id).wakeup())
// start accepting connections
- this.acceptor = new Acceptor(host, port, processors, sendBufferSize, recvBufferSize)
+ this.acceptor = new Acceptor(host, port, processors, sendBufferSize, recvBufferSize, quotas)
Utils.newThread("kafka-socket-acceptor", acceptor, false).start()
acceptor.awaitStartup
info("Started")
@@ -87,7 +97,7 @@ class SocketServer(val brokerId: Int,
/**
* A base class with some helper variables and methods
*/
-private[kafka] abstract class AbstractServerThread extends Runnable with Logging {
+private[kafka] abstract class AbstractServerThread(connectionQuotas: ConnectionQuotas) extends Runnable with Logging {
protected val selector = Selector.open();
private val startupLatch = new CountDownLatch(1)
@@ -131,13 +141,48 @@ private[kafka] abstract class AbstractServerThread extends Runnable with Logging
*/
def wakeup() = selector.wakeup()
+ /**
+ * Close the given key and associated socket
+ */
+ def close(key: SelectionKey) {
+ if(key != null) {
+ key.attach(null)
+ close(key.channel.asInstanceOf[SocketChannel])
+ swallowError(key.cancel())
+ }
+ }
+
+ def close(channel: SocketChannel) {
+ if(channel != null) {
+ debug("Closing connection from " + channel.socket.getRemoteSocketAddress())
+ connectionQuotas.dec(channel.socket.getInetAddress)
+ swallowError(channel.socket().close())
+ swallowError(channel.close())
+ }
+ }
+
+ /**
+ * Close all open connections
+ */
+ def closeAll() {
+ val iter = this.selector.keys().iterator()
+ while (iter.hasNext) {
+ val key = iter.next()
+ close(key)
+ }
+ }
+
}
/**
* Thread that accepts and configures new connections. There is only need for one of these
*/
-private[kafka] class Acceptor(val host: String, val port: Int, private val processors: Array[Processor],
- val sendBufferSize: Int, val recvBufferSize: Int) extends AbstractServerThread {
+private[kafka] class Acceptor(val host: String,
+ val port: Int,
+ private val processors: Array[Processor],
+ val sendBufferSize: Int,
+ val recvBufferSize: Int,
+ connectionQuotas: ConnectionQuotas) extends AbstractServerThread(connectionQuotas) {
val serverChannel = openServerSocket(host, port)
/**
@@ -158,14 +203,14 @@ private[kafka] class Acceptor(val host: String, val port: Int, private val proce
key = iter.next
iter.remove()
if(key.isAcceptable)
- accept(key, processors(currentProcessor))
- else
- throw new IllegalStateException("Unrecognized key state for acceptor thread.")
+ accept(key, processors(currentProcessor))
+ else
+ throw new IllegalStateException("Unrecognized key state for acceptor thread.")
- // round robin to the next processor thread
- currentProcessor = (currentProcessor + 1) % processors.length
+ // round robin to the next processor thread
+ currentProcessor = (currentProcessor + 1) % processors.length
} catch {
- case e: Throwable => error("Error in acceptor", e)
+ case e: Throwable => error("Error while accepting connection", e)
}
}
}
@@ -187,6 +232,7 @@ private[kafka] class Acceptor(val host: String, val port: Int, private val proce
new InetSocketAddress(host, port)
val serverChannel = ServerSocketChannel.open()
serverChannel.configureBlocking(false)
+ serverChannel.socket().setReceiveBufferSize(recvBufferSize)
try {
serverChannel.socket.bind(socketAddress)
info("Awaiting socket connections on %s:%d.".format(socketAddress.getHostName, port))
@@ -202,19 +248,24 @@ private[kafka] class Acceptor(val host: String, val port: Int, private val proce
*/
def accept(key: SelectionKey, processor: Processor) {
val serverSocketChannel = key.channel().asInstanceOf[ServerSocketChannel]
- serverSocketChannel.socket().setReceiveBufferSize(recvBufferSize)
-
val socketChannel = serverSocketChannel.accept()
- socketChannel.configureBlocking(false)
- socketChannel.socket().setTcpNoDelay(true)
- socketChannel.socket().setSendBufferSize(sendBufferSize)
+ try {
+ connectionQuotas.inc(socketChannel.socket().getInetAddress)
+ socketChannel.configureBlocking(false)
+ socketChannel.socket().setTcpNoDelay(true)
+ socketChannel.socket().setSendBufferSize(sendBufferSize)
- debug("Accepted connection from %s on %s. sendBufferSize [actual|requested]: [%d|%d] recvBufferSize [actual|requested]: [%d|%d]"
- .format(socketChannel.socket.getInetAddress, socketChannel.socket.getLocalSocketAddress,
+ debug("Accepted connection from %s on %s. sendBufferSize [actual|requested]: [%d|%d] recvBufferSize [actual|requested]: [%d|%d]"
+ .format(socketChannel.socket.getInetAddress, socketChannel.socket.getLocalSocketAddress,
socketChannel.socket.getSendBufferSize, sendBufferSize,
socketChannel.socket.getReceiveBufferSize, recvBufferSize))
- processor.accept(socketChannel)
+ processor.accept(socketChannel)
+ } catch {
+ case e: TooManyConnectionsException =>
+ info("Rejected connection from %s, address already has the configured maximum of %d connections.".format(e.ip, e.count))
+ close(socketChannel)
+ }
}
}
@@ -229,7 +280,8 @@ private[kafka] class Processor(val id: Int,
val aggregateIdleMeter: Meter,
val idleMeter: Meter,
val totalProcessorThreads: Int,
- val requestChannel: RequestChannel) extends AbstractServerThread {
+ val requestChannel: RequestChannel,
+ connectionQuotas: ConnectionQuotas) extends AbstractServerThread(connectionQuotas) {
private val newConnections = new ConcurrentLinkedQueue[SocketChannel]()
@@ -324,26 +376,6 @@ private[kafka] class Processor(val id: Int,
}
}
}
-
- private def close(key: SelectionKey) {
- val channel = key.channel.asInstanceOf[SocketChannel]
- debug("Closing connection from " + channel.socket.getRemoteSocketAddress())
- swallowError(channel.socket().close())
- swallowError(channel.close())
- key.attach(null)
- swallowError(key.cancel())
- }
-
- /*
- * Close all open connections
- */
- private def closeAll() {
- val iter = this.selector.keys().iterator()
- while (iter.hasNext) {
- val key = iter.next()
- close(key)
- }
- }
/**
* Queue up a new connection for reading
@@ -419,3 +451,31 @@ private[kafka] class Processor(val id: Int,
private def channelFor(key: SelectionKey) = key.channel().asInstanceOf[SocketChannel]
}
+
+class ConnectionQuotas(val defaultMax: Int, overrideQuotas: Map[String, Int]) {
+ private val overrides = overrideQuotas.map(entry => (InetAddress.getByName(entry._1), entry._2))
+ private val counts = mutable.Map[InetAddress, Int]()
+
+ def inc(addr: InetAddress) {
+ counts synchronized {
+ val count = counts.getOrElse(addr, 0)
+ counts.put(addr, count + 1)
+ val max = overrides.getOrElse(addr, defaultMax)
+ if(count >= max)
+ throw new TooManyConnectionsException(addr, max)
+ }
+ }
+
+ def dec(addr: InetAddress) {
+ counts synchronized {
+ val count = counts.get(addr).get
+ if(count == 1)
+ counts.remove(addr)
+ else
+ counts.put(addr, count - 1)
+ }
+ }
+
+}
+
+class TooManyConnectionsException(val ip: InetAddress, val count: Int) extends KafkaException("Too many connections from %s (maximum = %d)".format(ip, count))
http://git-wip-us.apache.org/repos/asf/kafka/blob/8e444a35/core/src/main/scala/kafka/server/KafkaConfig.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala
index bb2e654..50b09ed 100644
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -106,6 +106,12 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
/* the maximum number of bytes in a socket request */
val socketRequestMaxBytes: Int = props.getIntInRange("socket.request.max.bytes", 100*1024*1024, (1, Int.MaxValue))
+
+ /* the maximum number of connections we allow from each ip address */
+ val maxConnectionsPerIp: Int = props.getIntInRange("max.connections.per.ip", Int.MaxValue, (1, Int.MaxValue))
+
+ /* per-ip or hostname overrides to the default maximum number of connections */
+ val maxConnectionsPerIpOverrides = props.getMap("max.connections.per.ip.overrides").map(entry => (entry._1, entry._2.toInt))
/*********** Log Configuration ***********/
http://git-wip-us.apache.org/repos/asf/kafka/blob/8e444a35/core/src/main/scala/kafka/server/KafkaServer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala
index 5a56f57..def1dc2 100644
--- a/core/src/main/scala/kafka/server/KafkaServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaServer.scala
@@ -91,7 +91,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
config.queuedMaxRequests,
config.socketSendBufferBytes,
config.socketReceiveBufferBytes,
- config.socketRequestMaxBytes)
+ config.socketRequestMaxBytes,
+ config.maxConnectionsPerIp)
socketServer.startup()
replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler, logManager, isShuttingDown)
http://git-wip-us.apache.org/repos/asf/kafka/blob/8e444a35/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
index 1c492de..3b83a86 100644
--- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
+++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
@@ -40,7 +40,8 @@ class SocketServerTest extends JUnitSuite {
maxQueuedRequests = 50,
sendBufferSize = 300000,
recvBufferSize = 300000,
- maxRequestSize = 50)
+ maxRequestSize = 50,
+ maxConnectionsPerIp = 5)
server.startup()
def sendRequest(socket: Socket, id: Short, request: Array[Byte]) {
@@ -75,7 +76,7 @@ class SocketServerTest extends JUnitSuite {
def cleanup() {
server.shutdown()
}
-
+
@Test
def simpleRequest() {
val socket = connect()
@@ -139,4 +140,19 @@ class SocketServerTest extends JUnitSuite {
// doing a subsequent send should throw an exception as the connection should be closed.
sendRequest(socket, 0, bytes)
}
+
+ @Test
+ def testMaxConnectionsPerIp() {
+ // make the maximum allowable number of connections and then leak them
+ val conns = (0 until server.maxConnectionsPerIp).map(i => connect())
+
+ // now try one more (should fail)
+ try {
+ val conn = connect()
+ sendRequest(conn, 100, "hello".getBytes)
+ assertEquals(-1, conn.getInputStream().read())
+ } catch {
+ case e: IOException => // this is good
+ }
+ }
}
[03/37] git commit: KAFKA-1498 Misc. producer performance
enhancements. Patch from Guozhang.
Posted by jj...@apache.org.
KAFKA-1498 Misc. producer performance enhancements. Patch from Guozhang.
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/f1c6e97d
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/f1c6e97d
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/f1c6e97d
Branch: refs/heads/transactional_messaging
Commit: f1c6e97d718581566d037a48640ac3d869d1f15a
Parents: c4b9564
Author: Jay Kreps <ja...@gmail.com>
Authored: Tue Jul 1 13:21:36 2014 -0700
Committer: Jay Kreps <ja...@gmail.com>
Committed: Tue Jul 1 13:21:36 2014 -0700
----------------------------------------------------------------------
.../org/apache/kafka/clients/NetworkClient.java | 20 +++---
.../kafka/clients/producer/KafkaProducer.java | 16 +++--
.../clients/producer/internals/Metadata.java | 15 ++--
.../producer/internals/RecordAccumulator.java | 76 ++++++++++++++------
.../clients/producer/internals/Sender.java | 26 ++++---
.../kafka/common/record/MemoryRecords.java | 26 +++++--
.../apache/kafka/clients/NetworkClientTest.java | 2 +-
.../kafka/clients/producer/MetadataTest.java | 10 +--
.../clients/producer/RecordAccumulatorTest.java | 14 ++--
.../kafka/clients/producer/SenderTest.java | 6 +-
.../main/scala/kafka/tools/MirrorMaker.scala | 63 ++++++++--------
.../kafka/api/ProducerFailureHandlingTest.scala | 25 ++++---
.../unit/kafka/network/SocketServerTest.scala | 8 ++-
13 files changed, 187 insertions(+), 120 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/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 522881c..d21f922 100644
--- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
@@ -116,7 +116,7 @@ public class NetworkClient implements KafkaClient {
/**
* Check if the node with the given id is ready to send more requests.
- * @param nodeId The node id
+ * @param node The given node id
* @param now The current time in ms
* @return true if the node is ready
*/
@@ -126,8 +126,8 @@ public class NetworkClient implements KafkaClient {
}
private boolean isReady(int node, long now) {
- if (this.metadata.needsUpdate(now))
- // if we need to update our metadata declare all requests unready to metadata requests first priority
+ if (!this.metadataFetchInProgress && this.metadata.timeToNextUpdate(now) == 0)
+ // if we need to update our metadata now declare all requests unready to make metadata requests first priority
return false;
else
// otherwise we are ready if we are connected and can send more requests
@@ -144,9 +144,12 @@ public class NetworkClient implements KafkaClient {
*/
@Override
public List<ClientResponse> poll(List<ClientRequest> requests, long timeout, long now) {
- // should we update our metadata?
List<NetworkSend> sends = new ArrayList<NetworkSend>();
- maybeUpdateMetadata(sends, now);
+
+ // should we update our metadata?
+ long metadataTimeout = metadata.timeToNextUpdate(now);
+ if (!this.metadataFetchInProgress && metadataTimeout == 0)
+ maybeUpdateMetadata(sends, now);
for (int i = 0; i < requests.size(); i++) {
ClientRequest request = requests.get(i);
@@ -160,7 +163,7 @@ public class NetworkClient implements KafkaClient {
// do the I/O
try {
- this.selector.poll(timeout, sends);
+ this.selector.poll(Math.min(timeout, metadataTimeout), sends);
} catch (IOException e) {
log.error("Unexpected error during I/O in producer network thread", e);
}
@@ -340,12 +343,9 @@ public class NetworkClient implements KafkaClient {
}
/**
- * Add a metadata request to the list of sends if we need to make one
+ * Add a metadata request to the list of sends if we can make one
*/
private void maybeUpdateMetadata(List<NetworkSend> sends, long now) {
- if (this.metadataFetchInProgress || !metadata.needsUpdate(now))
- return;
-
Node node = this.leastLoadedNode(now);
if (node == null)
return;
http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
index 00775ab..d85ca30 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
@@ -22,7 +22,6 @@ import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import org.apache.kafka.clients.NetworkClient;
-import org.apache.kafka.clients.producer.internals.FutureRecordMetadata;
import org.apache.kafka.clients.producer.internals.Metadata;
import org.apache.kafka.clients.producer.internals.Partitioner;
import org.apache.kafka.clients.producer.internals.RecordAccumulator;
@@ -232,10 +231,14 @@ public class KafkaProducer implements Producer {
ensureValidRecordSize(serializedSize);
TopicPartition tp = new TopicPartition(record.topic(), partition);
log.trace("Sending record {} with callback {} to topic {} partition {}", record, callback, record.topic(), partition);
- FutureRecordMetadata future = accumulator.append(tp, record.key(), record.value(), compressionType, callback);
- this.sender.wakeup();
- return future;
- // For API exceptions return them in the future;
+ RecordAccumulator.RecordAppendResult result = accumulator.append(tp, record.key(), record.value(), compressionType, callback);
+ if (result.batchIsFull || result.newBatchCreated) {
+ log.trace("Waking up the sender since topic {} partition {} is either full or getting a new batch", record.topic(), partition);
+ this.sender.wakeup();
+ }
+ return result.future;
+ // Handling exceptions and record the errors;
+ // For API exceptions return them in the future,
// for other exceptions throw directly
} catch (ApiException e) {
log.debug("Exception occurred during message send:", e);
@@ -246,6 +249,9 @@ public class KafkaProducer implements Producer {
} catch (InterruptedException e) {
this.errors.record();
throw new KafkaException(e);
+ } catch (KafkaException e) {
+ this.errors.record();
+ throw e;
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
index 57bc285..8890aa2 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
@@ -101,15 +101,14 @@ public final class Metadata {
}
/**
- * Does the current cluster info need to be updated? An update is needed if it has been at least refreshBackoffMs
- * since our last update and either (1) an update has been requested or (2) the current metadata has expired (more
- * than metadataExpireMs has passed since the last refresh)
+ * The next time to update the cluster info is the maximum of the time the current info will expire
+ * and the time the current info can be updated (i.e. backoff time has elapsed); If an update has
+ * been request then the expiry time is now
*/
- public synchronized boolean needsUpdate(long now) {
- long msSinceLastUpdate = now - this.lastRefreshMs;
- boolean updateAllowed = msSinceLastUpdate >= this.refreshBackoffMs;
- boolean updateNeeded = this.forceUpdate || msSinceLastUpdate >= this.metadataExpireMs;
- return updateAllowed && updateNeeded;
+ public synchronized long timeToNextUpdate(long nowMs) {
+ long timeToExpire = forceUpdate ? 0 : Math.max(this.lastRefreshMs + this.metadataExpireMs - nowMs, 0);
+ long timeToAllowUpdate = this.lastRefreshMs + this.refreshBackoffMs - nowMs;
+ return Math.max(timeToExpire, timeToAllowUpdate);
}
/**
http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
index 1ed3c28..c5d4700 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
@@ -120,27 +120,29 @@ public final class RecordAccumulator {
}
/**
- * Add a record to the accumulator.
+ * Add a record to the accumulator, return the append result
* <p>
- * This method will block if sufficient memory isn't available for the record unless blocking has been disabled.
- *
+ * The append result will contain the future metadata, and flag for whether the appended batch is full or a new batch is created
+ * <p>
+ *
* @param tp The topic/partition to which this record is being sent
* @param key The key for the record
* @param value The value for the record
* @param compression The compression codec for the record
* @param callback The user-supplied callback to execute when the request is complete
*/
- public FutureRecordMetadata append(TopicPartition tp, byte[] key, byte[] value, CompressionType compression, Callback callback) throws InterruptedException {
+ public RecordAppendResult append(TopicPartition tp, byte[] key, byte[] value, CompressionType compression, Callback callback) throws InterruptedException {
if (closed)
throw new IllegalStateException("Cannot send after the producer is closed.");
// check if we have an in-progress batch
Deque<RecordBatch> dq = dequeFor(tp);
synchronized (dq) {
- RecordBatch batch = dq.peekLast();
- if (batch != null) {
- FutureRecordMetadata future = batch.tryAppend(key, value, callback);
- if (future != null)
- return future;
+ RecordBatch last = dq.peekLast();
+ if (last != null) {
+ FutureRecordMetadata future = last.tryAppend(key, value, callback);
+ if (future != null) {
+ return new RecordAppendResult(future, dq.size() > 1 || last.records.isFull(), false);
+ }
}
}
@@ -156,15 +158,15 @@ public final class RecordAccumulator {
// Somebody else found us a batch, return the one we waited for! Hopefully this doesn't happen
// often...
free.deallocate(buffer);
- return future;
+ return new RecordAppendResult(future, dq.size() > 1 || last.records.isFull(), false);
}
}
- MemoryRecords records = MemoryRecords.emptyRecords(buffer, compression);
+ MemoryRecords records = MemoryRecords.emptyRecords(buffer, compression, this.batchSize);
RecordBatch batch = new RecordBatch(tp, records, time.milliseconds());
FutureRecordMetadata future = Utils.notNull(batch.tryAppend(key, value, callback));
dq.addLast(batch);
- return future;
+ return new RecordAppendResult(future, dq.size() > 1 || batch.records.isFull(), true);
}
}
@@ -181,7 +183,9 @@ public final class RecordAccumulator {
}
/**
- * Get a list of nodes whose partitions are ready to be sent.
+ * Get a list of nodes whose partitions are ready to be sent, and the time to when any partition will be ready if no
+ * partitions are ready yet; If the ready nodes list is non-empty, the timeout value will be 0. Also return the flag
+ * for whether there are any unknown leaders for the accumulated partition batches.
* <p>
* A destination node is ready to send data if ANY one of its partition is not backing off the send and ANY of the
* following are true :
@@ -193,31 +197,39 @@ public final class RecordAccumulator {
* <li>The accumulator has been closed
* </ol>
*/
- public Set<Node> ready(Cluster cluster, long now) {
+ public ReadyCheckResult ready(Cluster cluster, long nowMs) {
Set<Node> readyNodes = new HashSet<Node>();
- boolean exhausted = this.free.queued() > 0;
+ long nextReadyCheckDelayMs = Long.MAX_VALUE;
+ boolean unknownLeadersExist = false;
+ boolean exhausted = this.free.queued() > 0;
for (Map.Entry<TopicPartition, Deque<RecordBatch>> entry : this.batches.entrySet()) {
TopicPartition part = entry.getKey();
Deque<RecordBatch> deque = entry.getValue();
- // if the leader is unknown use an Unknown node placeholder
+
Node leader = cluster.leaderFor(part);
- if (!readyNodes.contains(leader)) {
+ if (leader == null) {
+ unknownLeadersExist = true;
+ } else if (!readyNodes.contains(leader)) {
synchronized (deque) {
RecordBatch batch = deque.peekFirst();
if (batch != null) {
- boolean backingOff = batch.attempts > 0 && batch.lastAttemptMs + retryBackoffMs > now;
+ boolean backingOff = batch.attempts > 0 && batch.lastAttemptMs + retryBackoffMs > nowMs;
+ long waitedTimeMs = nowMs - batch.lastAttemptMs;
+ long timeToWaitMs = backingOff ? retryBackoffMs : lingerMs;
+ long timeLeftMs = Math.max(timeToWaitMs - waitedTimeMs, 0);
boolean full = deque.size() > 1 || batch.records.isFull();
- boolean expired = now - batch.createdMs >= lingerMs;
+ boolean expired = waitedTimeMs >= lingerMs;
boolean sendable = full || expired || exhausted || closed;
if (sendable && !backingOff)
readyNodes.add(leader);
+ nextReadyCheckDelayMs = Math.min(timeLeftMs, nextReadyCheckDelayMs);
}
}
}
}
- return readyNodes;
+ return new ReadyCheckResult(readyNodes, nextReadyCheckDelayMs, unknownLeadersExist);
}
/**
@@ -311,4 +323,28 @@ public final class RecordAccumulator {
this.closed = true;
}
+
+ public final static class RecordAppendResult {
+ public final FutureRecordMetadata future;
+ public final boolean batchIsFull;
+ public final boolean newBatchCreated;
+
+ public RecordAppendResult(FutureRecordMetadata future, boolean batchIsFull, boolean newBatchCreated) {
+ this.future = future;
+ this.batchIsFull = batchIsFull;
+ this.newBatchCreated = newBatchCreated;
+ }
+ }
+
+ public final static class ReadyCheckResult {
+ public final Set<Node> readyNodes;
+ public final long nextReadyCheckDelayMs;
+ public final boolean unknownLeadersExist;
+
+ public ReadyCheckResult(Set<Node> readyNodes, long nextReadyCheckDelayMs, boolean unknownLeadersExist) {
+ this.readyNodes = readyNodes;
+ this.nextReadyCheckDelayMs = nextReadyCheckDelayMs;
+ this.unknownLeadersExist = unknownLeadersExist;
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
index 6fb5b82..52d209b 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
@@ -17,7 +17,6 @@ import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
-import java.util.Set;
import org.apache.kafka.clients.ClientRequest;
import org.apache.kafka.clients.ClientResponse;
@@ -123,13 +122,13 @@ public class Sender implements Runnable {
// okay we stopped accepting requests but there may still be
// requests in the accumulator or waiting for acknowledgment,
// wait until these are completed.
- do {
+ while (this.accumulator.hasUnsent() || this.client.inFlightRequestCount() > 0) {
try {
run(time.milliseconds());
} catch (Exception e) {
log.error("Uncaught error in kafka producer I/O thread: ", e);
}
- } while (this.accumulator.hasUnsent() || this.client.inFlightRequestCount() > 0);
+ }
this.client.close();
@@ -144,10 +143,14 @@ public class Sender implements Runnable {
public void run(long now) {
Cluster cluster = metadata.fetch();
// get the list of partitions with data ready to send
- Set<Node> ready = this.accumulator.ready(cluster, now);
+ RecordAccumulator.ReadyCheckResult result = this.accumulator.ready(cluster, now);
+
+ // if there are any partitions whose leaders are not known yet, force metadata update
+ if (result.unknownLeadersExist)
+ this.metadata.forceUpdate();
// remove any nodes we aren't ready to send to
- Iterator<Node> iter = ready.iterator();
+ Iterator<Node> iter = result.readyNodes.iterator();
while (iter.hasNext()) {
Node node = iter.next();
if (!this.client.ready(node, now))
@@ -155,16 +158,20 @@ public class Sender implements Runnable {
}
// create produce requests
- Map<Integer, List<RecordBatch>> batches = this.accumulator.drain(cluster, ready, this.maxRequestSize, now);
+ Map<Integer, List<RecordBatch>> batches = this.accumulator.drain(cluster, result.readyNodes, this.maxRequestSize, now);
List<ClientRequest> requests = createProduceRequests(batches, now);
sensors.updateProduceRequestMetrics(requests);
- if (ready.size() > 0) {
- log.trace("Nodes with data ready to send: {}", ready);
+ if (result.readyNodes.size() > 0) {
+ log.trace("Nodes with data ready to send: {}", result.readyNodes);
log.trace("Created {} produce requests: {}", requests.size(), requests);
}
- List<ClientResponse> responses = this.client.poll(requests, 100L, now);
+ // if some partitions are already ready to be sent, the select time would be 0;
+ // otherwise if some partition already has some data accumulated but not ready yet,
+ // the select time will be the time difference between now and its linger expiry time;
+ // otherwise the select time will be the time difference between now and the metadata expiry time;
+ List<ClientResponse> responses = this.client.poll(requests, result.nextReadyCheckDelayMs, now);
for (ClientResponse response : responses) {
if (response.wasDisconnected())
handleDisconnect(response, now);
@@ -307,6 +314,7 @@ public class Sender implements Runnable {
this.batchSizeSensor = metrics.sensor("batch-size");
this.batchSizeSensor.add("batch-size-avg", "The average number of bytes sent per partition per-request.", new Avg());
+ this.batchSizeSensor.add("batch-size-max", "The max number of bytes sent per partition per-request.", new Max());
this.compressionRateSensor = metrics.sensor("compression-rate");
this.compressionRateSensor.add("compression-rate-avg", "The average compression rate of record batches.", new Avg());
http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
index 759f577..040e5b9 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
@@ -29,13 +29,15 @@ public class MemoryRecords implements Records {
private final Compressor compressor;
private final int capacity;
+ private final int sizeLimit;
private ByteBuffer buffer;
private boolean writable;
// Construct a writable memory records
- private MemoryRecords(ByteBuffer buffer, CompressionType type, boolean writable) {
+ private MemoryRecords(ByteBuffer buffer, CompressionType type, boolean writable, int sizeLimit) {
this.writable = writable;
this.capacity = buffer.capacity();
+ this.sizeLimit = sizeLimit;
if (this.writable) {
this.buffer = null;
this.compressor = new Compressor(buffer, type);
@@ -45,12 +47,16 @@ public class MemoryRecords implements Records {
}
}
+ public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type, int capacity) {
+ return new MemoryRecords(buffer, type, true, capacity);
+ }
+
public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type) {
- return new MemoryRecords(buffer, type, true);
+ return emptyRecords(buffer, type, buffer.capacity());
}
public static MemoryRecords iterableRecords(ByteBuffer buffer) {
- return new MemoryRecords(buffer, CompressionType.NONE, false);
+ return new MemoryRecords(buffer, CompressionType.NONE, false, buffer.capacity());
}
/**
@@ -88,14 +94,22 @@ public class MemoryRecords implements Records {
* Note that the return value is based on the estimate of the bytes written to the compressor, which may not be
* accurate if compression is really used. When this happens, the following append may cause dynamic buffer
* re-allocation in the underlying byte buffer stream.
+ *
+ * Also note that besides the records' capacity, there is also a size limit for the batch. This size limit may be
+ * smaller than the capacity (e.g. when appending a single message whose size is larger than the batch size, the
+ * capacity will be the message size, but the size limit will still be the batch size), and when the records' size has
+ * exceed this limit we also mark this record as full.
*/
public boolean hasRoomFor(byte[] key, byte[] value) {
- return this.writable && this.capacity >= this.compressor.estimatedBytesWritten() + Records.LOG_OVERHEAD +
- Record.recordSize(key, value);
+ return this.writable &&
+ this.capacity >= this.compressor.estimatedBytesWritten() + Records.LOG_OVERHEAD + Record.recordSize(key, value) &&
+ this.sizeLimit >= this.compressor.estimatedBytesWritten();
}
public boolean isFull() {
- return !this.writable || this.capacity <= this.compressor.estimatedBytesWritten();
+ return !this.writable ||
+ this.capacity <= this.compressor.estimatedBytesWritten() ||
+ this.sizeLimit <= this.compressor.estimatedBytesWritten();
}
/**
http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/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 6a3cdcc..2f98192 100644
--- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
@@ -54,7 +54,7 @@ public class NetworkClientTest {
client.poll(reqs, 1, time.milliseconds());
selector.clear();
assertFalse("After we forced the disconnection the client is no longer ready.", client.ready(node, time.milliseconds()));
- assertTrue("Metadata should get updated.", metadata.needsUpdate(time.milliseconds()));
+ assertTrue("Metadata should get updated.", metadata.timeToNextUpdate(time.milliseconds()) == 0);
}
@Test(expected = IllegalStateException.class)
http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
index 8b4ac0f..0d7d04c 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
@@ -30,11 +30,11 @@ public class MetadataTest {
public void testMetadata() throws Exception {
long time = 0;
metadata.update(Cluster.empty(), time);
- assertFalse("No update needed.", metadata.needsUpdate(time));
+ assertFalse("No update needed.", metadata.timeToNextUpdate(time) == 0);
metadata.forceUpdate();
- assertFalse("Still no updated needed due to backoff", metadata.needsUpdate(time));
+ assertFalse("Still no updated needed due to backoff", metadata.timeToNextUpdate(time) == 0);
time += refreshBackoffMs;
- assertTrue("Update needed now that backoff time expired", metadata.needsUpdate(time));
+ assertTrue("Update needed now that backoff time expired", metadata.timeToNextUpdate(time) == 0);
String topic = "my-topic";
Thread t1 = asyncFetch(topic);
Thread t2 = asyncFetch(topic);
@@ -43,9 +43,9 @@ public class MetadataTest {
metadata.update(TestUtils.singletonCluster(topic, 1), time);
t1.join();
t2.join();
- assertFalse("No update needed.", metadata.needsUpdate(time));
+ assertFalse("No update needed.", metadata.timeToNextUpdate(time) == 0);
time += metadataExpireMs;
- assertTrue("Update needed due to stale metadata.", metadata.needsUpdate(time));
+ assertTrue("Update needed due to stale metadata.", metadata.timeToNextUpdate(time) == 0);
}
private Thread asyncFetch(final String topic) {
http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java
index 93b58d0..0762b35 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java
@@ -62,10 +62,10 @@ public class RecordAccumulatorTest {
int appends = 1024 / msgSize;
for (int i = 0; i < appends; i++) {
accum.append(tp1, key, value, CompressionType.NONE, null);
- assertEquals("No partitions should be ready.", 0, accum.ready(cluster, now).size());
+ assertEquals("No partitions should be ready.", 0, accum.ready(cluster, now).readyNodes.size());
}
accum.append(tp1, key, value, CompressionType.NONE, null);
- assertEquals("Our partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()));
+ assertEquals("Our partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()).readyNodes);
List<RecordBatch> batches = accum.drain(cluster, Collections.singleton(node), Integer.MAX_VALUE, 0).get(node.id());
assertEquals(1, batches.size());
RecordBatch batch = batches.get(0);
@@ -83,7 +83,7 @@ public class RecordAccumulatorTest {
int batchSize = 512;
RecordAccumulator accum = new RecordAccumulator(batchSize, 10 * 1024, 0L, 100L, false, metrics, time);
accum.append(tp1, key, new byte[2 * batchSize], CompressionType.NONE, null);
- assertEquals("Our partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()));
+ assertEquals("Our partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()).readyNodes);
}
@Test
@@ -91,9 +91,9 @@ public class RecordAccumulatorTest {
long lingerMs = 10L;
RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, lingerMs, 100L, false, metrics, time);
accum.append(tp1, key, value, CompressionType.NONE, null);
- assertEquals("No partitions should be ready", 0, accum.ready(cluster, time.milliseconds()).size());
+ assertEquals("No partitions should be ready", 0, accum.ready(cluster, time.milliseconds()).readyNodes.size());
time.sleep(10);
- assertEquals("Our partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()));
+ assertEquals("Our partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()).readyNodes);
List<RecordBatch> batches = accum.drain(cluster, Collections.singleton(node), Integer.MAX_VALUE, 0).get(node.id());
assertEquals(1, batches.size());
RecordBatch batch = batches.get(0);
@@ -113,7 +113,7 @@ public class RecordAccumulatorTest {
for (int i = 0; i < appends; i++)
accum.append(tp, key, value, CompressionType.NONE, null);
}
- assertEquals("Partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()));
+ assertEquals("Partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()).readyNodes);
List<RecordBatch> batches = accum.drain(cluster, Collections.singleton(node), 1024, 0).get(node.id());
assertEquals("But due to size bound only one partition should have been retrieved", 1, batches.size());
@@ -144,7 +144,7 @@ public class RecordAccumulatorTest {
int read = 0;
long now = time.milliseconds();
while (read < numThreads * msgs) {
- Set<Node> nodes = accum.ready(cluster, now);
+ Set<Node> nodes = accum.ready(cluster, now).readyNodes;
List<RecordBatch> batches = accum.drain(cluster, nodes, 5 * 1024, 0).get(node.id());
if (batches != null) {
for (RecordBatch batch : batches) {
http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java
index 5489aca..ef2ca65 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java
@@ -69,7 +69,7 @@ public class SenderTest {
@Test
public void testSimple() throws Exception {
int offset = 0;
- Future<RecordMetadata> future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null);
+ Future<RecordMetadata> future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null).future;
sender.run(time.milliseconds()); // connect
sender.run(time.milliseconds()); // send produce request
assertEquals("We should have a single produce request in flight.", 1, client.inFlightRequestCount());
@@ -95,7 +95,7 @@ public class SenderTest {
new Metrics(),
time);
// do a successful retry
- Future<RecordMetadata> future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null);
+ Future<RecordMetadata> future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null).future;
sender.run(time.milliseconds()); // connect
sender.run(time.milliseconds()); // send produce request
assertEquals(1, client.inFlightRequestCount());
@@ -112,7 +112,7 @@ public class SenderTest {
assertEquals(offset, future.get().offset());
// do an unsuccessful retry
- future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null);
+ future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null).future;
sender.run(time.milliseconds()); // send produce request
for (int i = 0; i < maxRetries + 1; i++) {
client.disconnect(client.requests().peek().request().destination());
http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/core/src/main/scala/kafka/tools/MirrorMaker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala
index 7638391..4f06e34 100644
--- a/core/src/main/scala/kafka/tools/MirrorMaker.scala
+++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala
@@ -21,22 +21,22 @@ import kafka.utils.{SystemTime, Utils, CommandLineUtils, Logging}
import kafka.consumer._
import kafka.serializer._
import kafka.producer.{OldProducer, NewShinyProducer, BaseProducer}
+import kafka.metrics.KafkaMetricsGroup
+
import org.apache.kafka.clients.producer.ProducerRecord
-import scala.collection.mutable.ListBuffer
import scala.collection.JavaConversions._
-import java.util.concurrent.{TimeUnit, BlockingQueue, ArrayBlockingQueue, CountDownLatch}
-
import joptsimple.OptionParser
-import kafka.metrics.KafkaMetricsGroup
-import com.yammer.metrics.core.Gauge
+import java.util.Random
+import java.util.concurrent.atomic.AtomicInteger
+import java.util.concurrent.{TimeUnit, BlockingQueue, ArrayBlockingQueue, CountDownLatch}
object MirrorMaker extends Logging {
private var connectors: Seq[ZookeeperConsumerConnector] = null
private var consumerThreads: Seq[ConsumerThread] = null
- private var producerThreads: ListBuffer[ProducerThread] = null
+ private var producerThreads: Seq[ProducerThread] = null
private val shutdownMessage : ProducerRecord = new ProducerRecord("shutdown", "shutdown".getBytes)
@@ -138,13 +138,7 @@ object MirrorMaker extends Logging {
// create a data channel btw the consumers and the producers
val mirrorDataChannel = new DataChannel(bufferSize, numConsumers, numProducers)
- producerThreads = new ListBuffer[ProducerThread]()
- var producerIndex: Int = 1
- for(producer <- producers) {
- val producerThread = new ProducerThread(mirrorDataChannel, producer, producerIndex)
- producerThreads += producerThread
- producerIndex += 1
- }
+ producerThreads = producers.zipWithIndex.map(producerAndIndex => new ProducerThread(mirrorDataChannel, producerAndIndex._1, producerAndIndex._2))
val filterSpec = if (options.has(whitelistOpt))
new Whitelist(options.valueOf(whitelistOpt))
@@ -190,14 +184,11 @@ object MirrorMaker extends Logging {
class DataChannel(capacity: Int, numProducers: Int, numConsumers: Int) extends KafkaMetricsGroup {
- val queue = new ArrayBlockingQueue[ProducerRecord](capacity)
+ val queues = new Array[BlockingQueue[ProducerRecord]](numConsumers)
+ for (i <- 0 until numConsumers)
+ queues(i) = new ArrayBlockingQueue[ProducerRecord](capacity)
- newGauge(
- "MirrorMaker-DataChannel-Size",
- new Gauge[Int] {
- def value = queue.size
- }
- )
+ private val counter = new AtomicInteger(new Random().nextInt())
// We use a single meter for aggregated wait percentage for the data channel.
// Since meter is calculated as total_recorded_value / time_window and
@@ -205,23 +196,37 @@ object MirrorMaker extends Logging {
// time should be discounted by # threads.
private val waitPut = newMeter("MirrorMaker-DataChannel-WaitOnPut", "percent", TimeUnit.NANOSECONDS)
private val waitTake = newMeter("MirrorMaker-DataChannel-WaitOnTake", "percent", TimeUnit.NANOSECONDS)
+ private val channelSizeHist = newHistogram("MirrorMaker-DataChannel-Size")
def put(record: ProducerRecord) {
+ // If the key of the message is empty, use round-robin to select the queue
+ // Otherwise use the queue based on the key value so that same key-ed messages go to the same queue
+ val queueId =
+ if(record.key() != null) {
+ Utils.abs(java.util.Arrays.hashCode(record.key())) % numConsumers
+ } else {
+ Utils.abs(counter.getAndIncrement()) % numConsumers
+ }
+ val queue = queues(queueId)
+
var putSucceed = false
while (!putSucceed) {
val startPutTime = SystemTime.nanoseconds
putSucceed = queue.offer(record, 500, TimeUnit.MILLISECONDS)
waitPut.mark((SystemTime.nanoseconds - startPutTime) / numProducers)
}
+ channelSizeHist.update(queue.size)
}
- def take(): ProducerRecord = {
+ def take(queueId: Int): ProducerRecord = {
+ val queue = queues(queueId)
var data: ProducerRecord = null
while (data == null) {
val startTakeTime = SystemTime.nanoseconds
data = queue.poll(500, TimeUnit.MILLISECONDS)
waitTake.mark((SystemTime.nanoseconds - startTakeTime) / numConsumers)
}
+ channelSizeHist.update(queue.size)
data
}
}
@@ -242,18 +247,8 @@ object MirrorMaker extends Logging {
info("Starting mirror maker consumer thread " + threadName)
try {
for (msgAndMetadata <- stream) {
- // If the key of the message is empty, put it into the universal channel
- // Otherwise use a pre-assigned producer to send the message
- if (msgAndMetadata.key == null) {
- trace("Send the non-keyed message the producer channel.")
- val data = new ProducerRecord(msgAndMetadata.topic, msgAndMetadata.message)
- mirrorDataChannel.put(data)
- } else {
- val producerId = Utils.abs(java.util.Arrays.hashCode(msgAndMetadata.key)) % producers.size()
- trace("Send message with key %s to producer %d.".format(java.util.Arrays.toString(msgAndMetadata.key), producerId))
- val producer = producers(producerId)
- producer.send(msgAndMetadata.topic, msgAndMetadata.key, msgAndMetadata.message)
- }
+ val data = new ProducerRecord(msgAndMetadata.topic, msgAndMetadata.message)
+ mirrorDataChannel.put(data)
}
} catch {
case e: Throwable =>
@@ -287,7 +282,7 @@ object MirrorMaker extends Logging {
info("Starting mirror maker producer thread " + threadName)
try {
while (true) {
- val data: ProducerRecord = dataChannel.take
+ val data: ProducerRecord = dataChannel.take(threadId)
trace("Sending message with value size %d".format(data.value().size))
if(data eq shutdownMessage) {
info("Received shutdown message")
http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
index d146444..15fd5bc 100644
--- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
@@ -21,7 +21,7 @@ import org.scalatest.junit.JUnit3Suite
import org.junit.Test
import org.junit.Assert._
-import java.util.Properties
+import java.util.{Random, Properties}
import java.lang.Integer
import java.util.concurrent.{TimeoutException, TimeUnit, ExecutionException}
@@ -76,8 +76,6 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
producer1 = TestUtils.createNewProducer(brokerList, acks = 0, blockOnBufferFull = false, bufferSize = bufferSize);
producer2 = TestUtils.createNewProducer(brokerList, acks = 1, blockOnBufferFull = false, bufferSize = bufferSize)
producer3 = TestUtils.createNewProducer(brokerList, acks = -1, blockOnBufferFull = false, bufferSize = bufferSize)
- // producer with incorrect broker list
- producer4 = TestUtils.createNewProducer("localhost:8686,localhost:4242", acks = 1, blockOnBufferFull = false, bufferSize = bufferSize)
}
override def tearDown() {
@@ -150,6 +148,9 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
// create topic
TestUtils.createTopic(zkClient, topic1, 1, 2, servers)
+ // producer with incorrect broker list
+ producer4 = TestUtils.createNewProducer("localhost:8686,localhost:4242", acks = 1, blockOnBufferFull = false, bufferSize = bufferSize)
+
// send a record with incorrect broker list
val record = new ProducerRecord(topic1, null, "key".getBytes, "value".getBytes)
intercept[ExecutionException] {
@@ -168,28 +169,32 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
TestUtils.createTopic(zkClient, topic1, 1, 2, servers)
// first send a message to make sure the metadata is refreshed
- val record = new ProducerRecord(topic1, null, "key".getBytes, "value".getBytes)
- producer1.send(record).get
- producer2.send(record).get
+ val record1 = new ProducerRecord(topic1, null, "key".getBytes, "value".getBytes)
+ producer1.send(record1).get
+ producer2.send(record1).get
// stop IO threads and request handling, but leave networking operational
// any requests should be accepted and queue up, but not handled
server1.requestHandlerPool.shutdown()
server2.requestHandlerPool.shutdown()
- producer1.send(record).get(5000, TimeUnit.MILLISECONDS)
+ producer1.send(record1).get(5000, TimeUnit.MILLISECONDS)
intercept[TimeoutException] {
- producer2.send(record).get(5000, TimeUnit.MILLISECONDS)
+ producer2.send(record1).get(5000, TimeUnit.MILLISECONDS)
}
// TODO: expose producer configs after creating them
// send enough messages to get buffer full
- val tooManyRecords = bufferSize / ("key".getBytes.length + "value".getBytes.length)
+ val msgSize = 10000
+ val value = new Array[Byte](msgSize)
+ new Random().nextBytes(value)
+ val record2 = new ProducerRecord(topic1, null, "key".getBytes, value)
+ val tooManyRecords = bufferSize / ("key".getBytes.length + value.length)
intercept[KafkaException] {
for (i <- 1 to tooManyRecords)
- producer2.send(record)
+ producer2.send(record2)
}
// do not close produce2 since it will block
http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
index 62fb02c..1c492de 100644
--- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
+++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
@@ -128,11 +128,15 @@ class SocketServerTest extends JUnitSuite {
@Test(expected = classOf[IOException])
def testSocketsCloseOnShutdown() {
- // open a connection and then shutdown the server
+ // open a connection
val socket = connect()
+ val bytes = new Array[Byte](40)
+ // send a request first to make sure the connection has been picked up by the socket server
+ sendRequest(socket, 0, bytes)
+ processRequest(server.requestChannel)
+ // then shutdown the server
server.shutdown()
// doing a subsequent send should throw an exception as the connection should be closed.
- val bytes = new Array[Byte](10)
sendRequest(socket, 0, bytes)
}
}