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/03/14 23:14:47 UTC

[1/5] KAFKA-1012; Consumer offset management in Kafka; patched by Tejas Patil and Joel Koshy; feedback and reviews from Neha Narkhede, Jun Rao, Guozhang Wang, Sriram Subramanian, Joe Stein, Chris Riccomini

Repository: kafka
Updated Branches:
  refs/heads/trunk 84a3a9a3d -> a670537aa


http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/utils/kafka_system_test_utils.py
----------------------------------------------------------------------
diff --git a/system_test/utils/kafka_system_test_utils.py b/system_test/utils/kafka_system_test_utils.py
index 29ab2ba..35f2d1b 100644
--- a/system_test/utils/kafka_system_test_utils.py
+++ b/system_test/utils/kafka_system_test_utils.py
@@ -539,6 +539,17 @@ def start_brokers(systemTestEnv, testcaseEnv):
     for brokerEntityId in brokerEntityIdList:
         start_entity_in_background(systemTestEnv, testcaseEnv, brokerEntityId)
 
+def start_console_consumers(systemTestEnv, testcaseEnv, onlyThisEntityId=None):
+
+    if onlyThisEntityId is not None:
+        start_entity_in_background(systemTestEnv, testcaseEnv, onlyThisEntityId)
+    else:
+        clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList
+        consoleConsumerEntityIdList = system_test_utils.get_data_from_list_of_dicts(
+            clusterEntityConfigDictList, "role", "console_consumer", "entity_id")
+        for entityId in consoleConsumerEntityIdList:
+            start_entity_in_background(systemTestEnv, testcaseEnv, entityId)
+
 
 def start_mirror_makers(systemTestEnv, testcaseEnv, onlyThisEntityId=None):
 
@@ -751,10 +762,72 @@ def start_entity_in_background(systemTestEnv, testcaseEnv, entityId):
                       logPathName + "/" + logFile + " & echo pid:$! > ",
                       logPathName + "/entity_" + entityId + "_pid'"]
 
+    elif role == "console_consumer":
+        clusterToConsumeFrom = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "cluster_name")
+        numTopicsForAutoGenString = -1
+        try:
+            numTopicsForAutoGenString = int(testcaseEnv.testcaseArgumentsDict["num_topics_for_auto_generated_string"])
+        except:
+            pass
+
+        topic = ""
+        if numTopicsForAutoGenString < 0:
+            topic = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "topic")
+        else:
+            topic = generate_topics_string("topic", numTopicsForAutoGenString)
+
+        # update this variable and will be used by data validation functions
+        testcaseEnv.consumerTopicsString = topic
+
+        # 2. consumer timeout
+        timeoutMs = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "consumer-timeout-ms")
+
+        # 3. consumer formatter
+        formatterOption = ""
+        try:
+            formatterOption = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "formatter")
+        except:
+            pass
+
+        # 4. group
+        groupOption = ""
+        try:
+            groupOption = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "group.id")
+            groupOption = "--group " + groupOption
+        except:
+            pass
+
+        if len(formatterOption) > 0:
+            formatterOption = " --formatter " + formatterOption + " "
+
+        # get zookeeper connect string
+        zkConnectStr = ""
+        if clusterName == "source":
+            zkConnectStr = testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"]
+        elif clusterName == "target":
+            zkConnectStr = testcaseEnv.userDefinedEnvVarDict["targetZkConnectStr"]
+        else:
+            logger.error("Invalid cluster name : " + clusterName, extra=d)
+            sys.exit(1)
+        cmdList = ["ssh " + hostname,
+                   "'JAVA_HOME=" + javaHome,
+                   "JMX_PORT=" + jmxPort,
+                   kafkaHome + "/bin/kafka-run-class.sh kafka.consumer.ConsoleConsumer",
+                   "--zookeeper " + zkConnectStr,
+                   "--topic " + topic,
+                   "--consumer-timeout-ms " + timeoutMs,
+                   "--csv-reporter-enabled",
+                   groupOption,
+                   formatterOption,
+                   "--from-beginning",
+                   " >> " + logPathName + "/" + logFile + " & echo pid:$! > ",
+                   logPathName + "/entity_" + entityId + "_pid'"]
+
     cmdStr = " ".join(cmdList)
 
     logger.debug("executing command: [" + cmdStr + "]", extra=d)
     system_test_utils.async_sys_call(cmdStr)
+    logger.info("sleeping for 5 seconds.", extra=d)
     time.sleep(5)
 
     pidCmdStr = "ssh " + hostname + " 'cat " + logPathName + "/entity_" + entityId + "_pid' 2> /dev/null"
@@ -773,6 +846,8 @@ def start_entity_in_background(systemTestEnv, testcaseEnv, entityId):
                 testcaseEnv.entityBrokerParentPidDict[entityId] = tokens[1]
             elif role == "mirror_maker":
                 testcaseEnv.entityMirrorMakerParentPidDict[entityId] = tokens[1]
+            elif role == "console_consumer":
+                testcaseEnv.entityConsoleConsumerParentPidDict[entityId] = tokens[1]
 
 
 def start_console_consumer(systemTestEnv, testcaseEnv):
@@ -1117,7 +1192,7 @@ def stop_remote_entity(systemTestEnv, entityId, parentPid, signalType="SIGTERM")
     hostname  = system_test_utils.get_data_by_lookup_keyval(clusterEntityConfigDictList, "entity_id", entityId, "hostname")
     pidStack  = system_test_utils.get_remote_child_processes(hostname, parentPid)
 
-    logger.debug("terminating (" + signalType + ") process id: " + parentPid + " in host: " + hostname, extra=d)
+    logger.info("terminating (" + signalType + ") process id: " + parentPid + " in host: " + hostname, extra=d)
 
     if signalType.lower() == "sigterm":
         system_test_utils.sigterm_remote_process(hostname, pidStack)
@@ -1138,7 +1213,7 @@ def force_stop_remote_entity(systemTestEnv, entityId, parentPid):
     system_test_utils.sigkill_remote_process(hostname, pidStack)
 
 
-def create_topic(systemTestEnv, testcaseEnv):
+def create_topic_for_producer_performance(systemTestEnv, testcaseEnv):
     clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList
 
     prodPerfCfgList = system_test_utils.get_dict_from_list_of_dicts(clusterEntityConfigDictList, "role", "producer_performance")
@@ -1184,6 +1259,40 @@ def create_topic(systemTestEnv, testcaseEnv):
             logger.debug("executing command: [" + cmdStr + "]", extra=d)
             subproc = system_test_utils.sys_call_return_subproc(cmdStr)
 
+def create_topic(systemTestEnv, testcaseEnv, topic, replication_factor, num_partitions):
+    clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList
+    zkEntityId      = system_test_utils.get_data_by_lookup_keyval(clusterEntityConfigDictList, "role", "zookeeper", "entity_id")
+    kafkaHome       = system_test_utils.get_data_by_lookup_keyval(clusterEntityConfigDictList, "entity_id", zkEntityId, "kafka_home")
+    javaHome        = system_test_utils.get_data_by_lookup_keyval(clusterEntityConfigDictList, "entity_id", zkEntityId, "java_home")
+    createTopicBin  = kafkaHome + "/bin/kafka-topics.sh --create"
+    zkConnectStr = ""
+    zkHost = system_test_utils.get_data_by_lookup_keyval(clusterEntityConfigDictList, "role", "zookeeper", "hostname")
+    if len(testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"]) > 0:
+        zkConnectStr = testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"]
+    elif len(testcaseEnv.userDefinedEnvVarDict["targetZkConnectStr"]) > 0:
+        zkConnectStr = testcaseEnv.userDefinedEnvVarDict["targetZkConnectStr"]
+    else:
+        raise Exception("Empty zkConnectStr found")
+
+    testcaseBaseDir = testcaseEnv.testCaseBaseDir
+
+    testcaseBaseDir = replace_kafka_home(testcaseBaseDir, kafkaHome)
+
+    logger.debug("creating topic: [" + topic + "] at: [" + zkConnectStr + "]", extra=d)
+    cmdList = ["ssh " + zkHost,
+               "'JAVA_HOME=" + javaHome,
+               createTopicBin,
+               " --topic "     + topic,
+               " --zookeeper " + zkConnectStr,
+               " --replication-factor "   + str(replication_factor),
+               " --partitions " + str(num_partitions) + " >> ",
+               testcaseBaseDir + "/logs/create_source_cluster_topic.log'"]
+
+    cmdStr = " ".join(cmdList)
+    logger.info("executing command: [" + cmdStr + "]", extra=d)
+    subproc = system_test_utils.sys_call_return_subproc(cmdStr)
+
+
 
 def get_message_id(logPathName, topic=""):
     logLines      = open(logPathName, "r").readlines()
@@ -1221,7 +1330,7 @@ def get_message_checksum(logPathName):
 
 
 def validate_data_matched(systemTestEnv, testcaseEnv, replicationUtils):
-    logger.debug("#### Inside validate_data_matched", extra=d)
+    logger.info("#### Inside validate_data_matched", extra=d)
 
     validationStatusDict        = testcaseEnv.validationStatusDict
     clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList
@@ -1245,11 +1354,11 @@ def validate_data_matched(systemTestEnv, testcaseEnv, replicationUtils):
             consumerTopic = system_test_utils.get_data_by_lookup_keyval(testcaseEnv.testcaseConfigsList, "entity_id", consumerEntityId, "topic")
             if consumerTopic in topic:
                 matchingConsumerEntityId = consumerEntityId
-                logger.debug("matching consumer entity id found", extra=d)
+                logger.info("matching consumer entity id found", extra=d)
                 break
 
         if matchingConsumerEntityId is None:
-            logger.debug("matching consumer entity id NOT found", extra=d)
+            logger.info("matching consumer entity id NOT found", extra=d)
             break
 
         msgIdMissingInConsumerLogPathName = get_testcase_config_log_dir_pathname( \
@@ -1337,7 +1446,7 @@ def cleanup_data_at_remote_hosts(systemTestEnv, testcaseEnv):
     logger.info("cleaning up test case dir: [" + testCaseBaseDir + "]", extra=d)
 
     if "system_test" not in testCaseBaseDir:
-        logger.warn("possible destructive command [" + cmdStr + "]", extra=d)
+        # logger.warn("possible destructive command [" + cmdStr + "]", extra=d)
         logger.warn("check config file: system_test/cluster_config.properties", extra=d)
         logger.warn("aborting test...", extra=d)
         sys.exit(1)
@@ -1539,6 +1648,9 @@ def stop_all_remote_running_processes(systemTestEnv, testcaseEnv):
     for entityId, mirrorMakerParentPid in testcaseEnv.entityMirrorMakerParentPidDict.items():
         stop_remote_entity(systemTestEnv, entityId, mirrorMakerParentPid)
 
+    for entityId, consumerParentPid in testcaseEnv.entityConsoleConsumerParentPidDict.items():
+        stop_remote_entity(systemTestEnv, entityId, consumerParentPid)
+
     for entityId, brokerParentPid in testcaseEnv.entityBrokerParentPidDict.items():
         stop_remote_entity(systemTestEnv, entityId, brokerParentPid)
 
@@ -2119,7 +2231,6 @@ def validate_data_matched_in_multi_topics_from_single_consumer_producer(systemTe
     clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList
 
     prodPerfCfgList = system_test_utils.get_dict_from_list_of_dicts(clusterEntityConfigDictList, "role", "producer_performance")
-    consumerCfgList = system_test_utils.get_dict_from_list_of_dicts(clusterEntityConfigDictList, "role", "console_consumer")
 
     for prodPerfCfg in prodPerfCfgList:
         producerEntityId = prodPerfCfg["entity_id"]
@@ -2147,7 +2258,7 @@ def validate_data_matched_in_multi_topics_from_single_consumer_producer(systemTe
         topicList = topicStr.split(',')
         for topic in topicList:
             consumerDuplicateCount = 0
-            msgIdMissingInConsumerLogPathName = get_testcase_config_log_dir_pathname( 
+            msgIdMissingInConsumerLogPathName = get_testcase_config_log_dir_pathname(
                                                 testcaseEnv, "console_consumer", matchingConsumerEntityId, "default") \
                                                 + "/msg_id_missing_in_consumer_" + topic + ".log"
             producerMsgIdList  = get_message_id(producerLogPathName, topic)
@@ -2163,8 +2274,11 @@ def validate_data_matched_in_multi_topics_from_single_consumer_producer(systemTe
                 outfile.write(id + "\n")
             outfile.close()
 
+            logger.info("Producer entity id " + producerEntityId, extra=d)
+            logger.info("Consumer entity id " + matchingConsumerEntityId, extra=d)
             logger.info("no. of unique messages on topic [" + topic + "] sent from publisher  : " + str(len(producerMsgIdSet)), extra=d)
             logger.info("no. of unique messages on topic [" + topic + "] received by consumer : " + str(len(consumerMsgIdSet)), extra=d)
+            logger.info("no. of duplicate messages on topic [" + topic + "] received by consumer: " + str(consumerDuplicateCount), extra=d)
             validationStatusDict["Unique messages from producer on [" + topic + "]"] = str(len(producerMsgIdSet))
             validationStatusDict["Unique messages from consumer on [" + topic + "]"] = str(len(consumerMsgIdSet))
 
@@ -2263,6 +2377,43 @@ def validate_index_log(systemTestEnv, testcaseEnv, clusterName="source"):
     else:
         validationStatusDict["Validate index log in cluster [" + clusterName + "]"] = "FAILED"
 
+def get_leader_for(systemTestEnv, testcaseEnv, topic, partition):
+    logger.info("Querying Zookeeper for leader info for topic " + topic, extra=d)
+    clusterConfigsList = systemTestEnv.clusterEntityConfigDictList
+    tcConfigsList      = testcaseEnv.testcaseConfigsList
+
+    zkDictList         = system_test_utils.get_dict_from_list_of_dicts(clusterConfigsList, "role", "zookeeper")
+    firstZkDict        = zkDictList[0]
+    hostname           = firstZkDict["hostname"]
+    zkEntityId         = firstZkDict["entity_id"]
+    clientPort         = system_test_utils.get_data_by_lookup_keyval(tcConfigsList, "entity_id", zkEntityId, "clientPort")
+    kafkaHome          = system_test_utils.get_data_by_lookup_keyval(clusterConfigsList, "entity_id", zkEntityId, "kafka_home")
+    javaHome           = system_test_utils.get_data_by_lookup_keyval(clusterConfigsList, "entity_id", zkEntityId, "java_home")
+    kafkaRunClassBin   = kafkaHome + "/bin/kafka-run-class.sh"
+
+    zkQueryStr = "get /brokers/topics/" + topic + "/partitions/" + str(partition) + "/state"
+    brokerid   = ''
+    leaderEntityId = ''
+
+    cmdStrList = ["ssh " + hostname,
+                  "\"JAVA_HOME=" + javaHome,
+                  kafkaRunClassBin + " org.apache.zookeeper.ZooKeeperMain",
+                  "-server " + testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"],
+                  zkQueryStr + " 2> /dev/null | tail -1\""]
+    cmdStr = " ".join(cmdStrList)
+    logger.info("executing command [" + cmdStr + "]", extra=d)
+    subproc = system_test_utils.sys_call_return_subproc(cmdStr)
+    for line in subproc.stdout.readlines():
+        if "\"leader\"" in line:
+            line = line.rstrip('\n')
+            json_data = json.loads(line)
+            for key,val in json_data.items():
+                if key == 'leader':
+                    brokerid = str(val)
+            leaderEntityId = system_test_utils.get_data_by_lookup_keyval(tcConfigsList, "broker.id", brokerid, "entity_id")
+            break
+    return leaderEntityId
+
 def get_leader_attributes(systemTestEnv, testcaseEnv):
 
     logger.info("Querying Zookeeper for leader info ...", extra=d)
@@ -2294,11 +2445,10 @@ def get_leader_attributes(systemTestEnv, testcaseEnv):
                   "-server " + testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"],
                   zkQueryStr + " 2> /dev/null | tail -1\""]
     cmdStr = " ".join(cmdStrList)
-    logger.debug("executing command [" + cmdStr + "]", extra=d)
+    logger.info("executing command [" + cmdStr + "]", extra=d)
 
     subproc = system_test_utils.sys_call_return_subproc(cmdStr)
     for line in subproc.stdout.readlines():
-        logger.debug("zk returned : " + line, extra=d)
         if "\"leader\"" in line:
             line = line.rstrip('\n')
             json_data = json.loads(line)

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/utils/testcase_env.py
----------------------------------------------------------------------
diff --git a/system_test/utils/testcase_env.py b/system_test/utils/testcase_env.py
index bee8716..b3c2910 100644
--- a/system_test/utils/testcase_env.py
+++ b/system_test/utils/testcase_env.py
@@ -51,6 +51,12 @@ class TestcaseEnv():
     # { 0: 12345, 1: 12389, ... }
     entityMirrorMakerParentPidDict = {}
 
+    # dictionary of entity_id to ppid for console-consumer entities
+    # key: entity_id
+    # val: ppid of console consumer associated to that entity_id
+    # { 0: 12345, 1: 12389, ... }
+    entityConsoleConsumerParentPidDict = {}
+
     # dictionary of entity_id to ppid for migration tool entities
     # key: entity_id
     # val: ppid of broker associated to that entity_id


[4/5] KAFKA-1012; Consumer offset management in Kafka; patched by Tejas Patil and Joel Koshy; feedback and reviews from Neha Narkhede, Jun Rao, Guozhang Wang, Sriram Subramanian, Joe Stein, Chris Riccomini

Posted by jj...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/producer/ProducerConfig.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/producer/ProducerConfig.scala b/core/src/main/scala/kafka/producer/ProducerConfig.scala
index 7947b18..3cdf23d 100644
--- a/core/src/main/scala/kafka/producer/ProducerConfig.scala
+++ b/core/src/main/scala/kafka/producer/ProducerConfig.scala
@@ -77,16 +77,7 @@ class ProducerConfig private (val props: VerifiableProperties)
    * This parameter allows you to specify the compression codec for all data generated *
    * by this producer. The default is NoCompressionCodec
    */
-  val compressionCodec = {
-    val prop = props.getString("compression.codec", NoCompressionCodec.name)
-    try {
-      CompressionCodec.getCompressionCodec(prop.toInt)
-    }
-    catch {
-      case nfe: NumberFormatException =>
-        CompressionCodec.getCompressionCodec(prop)
-    }
-  }
+  val compressionCodec = props.getCompressionCodec("compression.codec", NoCompressionCodec)
 
   /** This parameter allows you to set whether compression should be turned *
    *  on for particular topics

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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 215ac36..1d9922b 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -22,24 +22,23 @@ import kafka.api._
 import kafka.message._
 import kafka.network._
 import kafka.log._
-import kafka.utils.ZKGroupTopicDirs
 import scala.collection._
 import java.util.concurrent.TimeUnit
 import java.util.concurrent.atomic._
 import kafka.metrics.KafkaMetricsGroup
-import org.I0Itec.zkclient.ZkClient
 import kafka.common._
-import kafka.utils.{ZkUtils, Pool, SystemTime, Logging}
+import kafka.utils.{Pool, SystemTime, Logging}
 import kafka.network.RequestChannel.Response
 import kafka.cluster.Broker
 import kafka.controller.KafkaController
-
+import org.I0Itec.zkclient.ZkClient
 
 /**
  * Logic to handle the various Kafka requests
  */
 class KafkaApis(val requestChannel: RequestChannel,
                 val replicaManager: ReplicaManager,
+                val offsetManager: OffsetManager,
                 val zkClient: ZkClient,
                 val brokerId: Int,
                 val config: KafkaConfig,
@@ -65,7 +64,7 @@ class KafkaApis(val requestChannel: RequestChannel,
     try{
       trace("Handling request: " + request.requestObj + " from client: " + request.remoteAddress)
       request.requestId match {
-        case RequestKeys.ProduceKey => handleProducerRequest(request)
+        case RequestKeys.ProduceKey => handleProducerOrOffsetCommitRequest(request)
         case RequestKeys.FetchKey => handleFetchRequest(request)
         case RequestKeys.OffsetsKey => handleOffsetRequest(request)
         case RequestKeys.MetadataKey => handleTopicMetadataRequest(request)
@@ -73,8 +72,9 @@ class KafkaApis(val requestChannel: RequestChannel,
         case RequestKeys.StopReplicaKey => handleStopReplicaRequest(request)
         case RequestKeys.UpdateMetadataKey => handleUpdateMetadataRequest(request)
         case RequestKeys.ControlledShutdownKey => handleControlledShutdownRequest(request)
-        case RequestKeys.OffsetCommitKey => handleOffsetCommitRequest(request)
+        case RequestKeys.OffsetCommitKey => handleProducerOrOffsetCommitRequest(request)
         case RequestKeys.OffsetFetchKey => handleOffsetFetchRequest(request)
+        case RequestKeys.ConsumerMetadataKey => handleConsumerMetadataRequest(request)
         case requestId => throw new KafkaException("Unknown api code " + requestId)
       }
     } catch {
@@ -97,7 +97,7 @@ class KafkaApis(val requestChannel: RequestChannel,
     // stop serving data to clients for the topic being deleted
     val leaderAndIsrRequest = request.requestObj.asInstanceOf[LeaderAndIsrRequest]
     try {
-      val (response, error) = replicaManager.becomeLeaderOrFollower(leaderAndIsrRequest)
+      val (response, error) = replicaManager.becomeLeaderOrFollower(leaderAndIsrRequest, offsetManager)
       val leaderAndIsrResponse = new LeaderAndIsrResponse(leaderAndIsrRequest.correlationId, response, error)
       requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(leaderAndIsrResponse)))
     } catch {
@@ -192,14 +192,47 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  private def producerRequestFromOffsetCommit(offsetCommitRequest: OffsetCommitRequest) = {
+    val msgs = offsetCommitRequest.filterLargeMetadata(config.offsetMetadataMaxSize).map {
+      case (topicAndPartition, offset) =>
+        new Message(
+          bytes = OffsetManager.offsetCommitValue(offset),
+          key = OffsetManager.offsetCommitKey(offsetCommitRequest.groupId, topicAndPartition.topic, topicAndPartition.partition)
+        )
+    }.toSeq
+
+    val producerData = mutable.Map(
+      TopicAndPartition(OffsetManager.OffsetsTopicName, offsetManager.partitionFor(offsetCommitRequest.groupId)) ->
+        new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, msgs:_*)
+    )
+
+    val request = ProducerRequest(
+      correlationId = offsetCommitRequest.correlationId,
+      clientId = offsetCommitRequest.clientId,
+      requiredAcks = config.offsetCommitRequiredAcks,
+      ackTimeoutMs = config.offsetCommitTimeoutMs,
+      data = producerData)
+    trace("Created producer request %s for offset commit request %s.".format(request, offsetCommitRequest))
+    request
+  }
+
   /**
-   * Handle a produce request
+   * Handle a produce request or offset commit request (which is really a specialized producer request)
    */
-  def handleProducerRequest(request: RequestChannel.Request) {
-    val produceRequest = request.requestObj.asInstanceOf[ProducerRequest]
+  def handleProducerOrOffsetCommitRequest(request: RequestChannel.Request) {
+
+    val (produceRequest, offsetCommitRequestOpt) = if (request.requestId == RequestKeys.OffsetCommitKey) {
+      val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest]
+      (producerRequestFromOffsetCommit(offsetCommitRequest), Some(offsetCommitRequest))
+    }
+    else {
+      (request.requestObj.asInstanceOf[ProducerRequest], None)
+    }
+
     val sTime = SystemTime.milliseconds
     val localProduceResults = appendToLocalLog(produceRequest)
     debug("Produce to local log in %d ms".format(SystemTime.milliseconds - sTime))
+    val firstErrorCode = localProduceResults.find(_.errorCode != ErrorMapping.NoError).map(_.errorCode).getOrElse(ErrorMapping.NoError)
 
     val numPartitionsInError = localProduceResults.count(_.error.isDefined)
     produceRequest.data.foreach(partitionAndData =>
@@ -218,14 +251,29 @@ class KafkaApis(val requestChannel: RequestChannel,
           .format(produceRequest.clientId, produceRequest.correlationId, produceRequest.topicPartitionMessageSizeMap.keySet.mkString(",")))
         requestChannel.closeConnection(request.processor, request)
       } else {
-        requestChannel.noOperation(request.processor, request)
+
+        if (firstErrorCode == ErrorMapping.NoError)
+          offsetCommitRequestOpt.foreach(ocr => offsetManager.putOffsets(ocr.groupId, ocr.requestInfo))
+
+        if (offsetCommitRequestOpt.isDefined) {
+          val response = offsetCommitRequestOpt.get.responseFor(firstErrorCode, config.offsetMetadataMaxSize)
+          requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response)))
+        } else
+          requestChannel.noOperation(request.processor, request)
       }
     } else if (produceRequest.requiredAcks == 1 ||
         produceRequest.numPartitions <= 0 ||
         allPartitionHaveReplicationFactorOne ||
         numPartitionsInError == produceRequest.numPartitions) {
+
+      if (firstErrorCode == ErrorMapping.NoError) {
+        offsetCommitRequestOpt.foreach(ocr => offsetManager.putOffsets(ocr.groupId, ocr.requestInfo) )
+      }
+
       val statuses = localProduceResults.map(r => r.key -> ProducerResponseStatus(r.errorCode, r.start)).toMap
-      val response = ProducerResponse(produceRequest.correlationId, statuses)
+      val response = offsetCommitRequestOpt.map(_.responseFor(firstErrorCode, config.offsetMetadataMaxSize))
+                                           .getOrElse(ProducerResponse(produceRequest.correlationId, statuses))
+
       requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response)))
     } else {
       // create a list of (topic, partition) pairs to use as keys for this delayed request
@@ -233,12 +281,15 @@ class KafkaApis(val requestChannel: RequestChannel,
         topicAndPartition => new RequestKey(topicAndPartition)).toSeq
       val statuses = localProduceResults.map(r =>
         r.key -> DelayedProduceResponseStatus(r.end + 1, ProducerResponseStatus(r.errorCode, r.start))).toMap
-      val delayedProduce = new DelayedProduce(producerRequestKeys, 
-                                              request,
-                                              statuses,
-                                              produceRequest, 
-                                              produceRequest.ackTimeoutMs.toLong)
-      producerRequestPurgatory.watch(delayedProduce)
+      val delayedRequest =  new DelayedProduce(
+        producerRequestKeys,
+        request,
+        statuses,
+        produceRequest,
+        produceRequest.ackTimeoutMs.toLong,
+        offsetCommitRequestOpt)
+
+      producerRequestPurgatory.watch(delayedRequest)
 
       /*
        * Replica fetch requests may have arrived (and potentially satisfied)
@@ -252,6 +303,7 @@ class KafkaApis(val requestChannel: RequestChannel,
       debug(satisfiedProduceRequests.size +
         " producer requests unblocked during produce to local log.")
       satisfiedProduceRequests.foreach(_.respond())
+
       // we do not need the data anymore
       produceRequest.emptyData()
     }
@@ -265,7 +317,7 @@ class KafkaApis(val requestChannel: RequestChannel,
       "acksPending:%b, error: %d, startOffset: %d, requiredOffset: %d".format(
         acksPending, status.error, status.offset, requiredOffset)
   }
-
+  
   case class ProduceResult(key: TopicAndPartition, start: Long, end: Long, error: Option[Throwable] = None) {
     def this(key: TopicAndPartition, throwable: Throwable) = 
       this(key, -1L, -1L, Some(throwable))
@@ -557,120 +609,85 @@ class KafkaApis(val requestChannel: RequestChannel,
     ret.toSeq.sortBy(- _)
   }
 
-  /**
-   * Service the topic metadata request API
-   */
-  def handleTopicMetadataRequest(request: RequestChannel.Request) {
-    val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest]
-    val topicsMetadata = new mutable.ArrayBuffer[TopicMetadata]()
+  private def getTopicMetadata(topics: Set[String]): Seq[TopicMetadata] = {
     val config = replicaManager.config
-    var uniqueTopics = Set.empty[String]
-    uniqueTopics = {
-      if(metadataRequest.topics.size > 0)
-        metadataRequest.topics.toSet
-      else {
-        partitionMetadataLock synchronized {
-          metadataCache.keySet.map(_.topic)
-        }
-      }
-    }
-    val topicMetadataList =
-      partitionMetadataLock synchronized {
-        uniqueTopics.map { topic =>
-          if(metadataCache.keySet.map(_.topic).contains(topic)) {
-            debug("Topic %s exists in metadata cache on broker %d".format(topic, config.brokerId))
-            val partitionStateInfo = metadataCache.filter(p => p._1.topic.equals(topic))
-            val sortedPartitions = partitionStateInfo.toList.sortWith((m1,m2) => m1._1.partition < m2._1.partition)
-            val partitionMetadata = sortedPartitions.map { case(topicAndPartition, partitionState) =>
-              val replicas = metadataCache(topicAndPartition).allReplicas
-              var replicaInfo: Seq[Broker] = replicas.map(aliveBrokers.getOrElse(_, null)).filter(_ != null).toSeq
-              var leaderInfo: Option[Broker] = None
-              var isrInfo: Seq[Broker] = Nil
-              val leaderIsrAndEpoch = partitionState.leaderIsrAndControllerEpoch
-              val leader = leaderIsrAndEpoch.leaderAndIsr.leader
-              val isr = leaderIsrAndEpoch.leaderAndIsr.isr
-              debug("%s".format(topicAndPartition) + ";replicas = " + replicas + ", in sync replicas = " + isr + ", leader = " + leader)
-              try {
-                if(aliveBrokers.keySet.contains(leader))
-                  leaderInfo = Some(aliveBrokers(leader))
-                else throw new LeaderNotAvailableException("Leader not available for partition %s".format(topicAndPartition))
-                isrInfo = isr.map(aliveBrokers.getOrElse(_, null)).filter(_ != null)
-                if(replicaInfo.size < replicas.size)
-                  throw new ReplicaNotAvailableException("Replica information not available for following brokers: " +
-                    replicas.filterNot(replicaInfo.map(_.id).contains(_)).mkString(","))
-                if(isrInfo.size < isr.size)
-                  throw new ReplicaNotAvailableException("In Sync Replica information not available for following brokers: " +
-                    isr.filterNot(isrInfo.map(_.id).contains(_)).mkString(","))
-                new PartitionMetadata(topicAndPartition.partition, leaderInfo, replicaInfo, isrInfo, ErrorMapping.NoError)
-              } catch {
-                case e: Throwable =>
-                  error("Error while fetching metadata for partition %s".format(topicAndPartition), e)
-                  new PartitionMetadata(topicAndPartition.partition, leaderInfo, replicaInfo, isrInfo,
-                    ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
-              }
+
+    partitionMetadataLock synchronized {
+      topics.map { topic =>
+        if(metadataCache.keySet.map(_.topic).contains(topic)) {
+          val partitionStateInfo = metadataCache.filter(p => p._1.topic.equals(topic))
+          val sortedPartitions = partitionStateInfo.toList.sortWith((m1,m2) => m1._1.partition < m2._1.partition)
+          val partitionMetadata = sortedPartitions.map { case(topicAndPartition, partitionState) =>
+            val replicas = metadataCache(topicAndPartition).allReplicas
+            val replicaInfo: Seq[Broker] = replicas.map(aliveBrokers.getOrElse(_, null)).filter(_ != null).toSeq
+            var leaderInfo: Option[Broker] = None
+            var isrInfo: Seq[Broker] = Nil
+            val leaderIsrAndEpoch = partitionState.leaderIsrAndControllerEpoch
+            val leader = leaderIsrAndEpoch.leaderAndIsr.leader
+            val isr = leaderIsrAndEpoch.leaderAndIsr.isr
+            debug("%s".format(topicAndPartition) + ";replicas = " + replicas + ", in sync replicas = " + isr + ", leader = " + leader)
+            try {
+              if(aliveBrokers.keySet.contains(leader))
+                leaderInfo = Some(aliveBrokers(leader))
+              else throw new LeaderNotAvailableException("Leader not available for partition %s".format(topicAndPartition))
+              isrInfo = isr.map(aliveBrokers.getOrElse(_, null)).filter(_ != null)
+              if(replicaInfo.size < replicas.size)
+                throw new ReplicaNotAvailableException("Replica information not available for following brokers: " +
+                  replicas.filterNot(replicaInfo.map(_.id).contains(_)).mkString(","))
+              if(isrInfo.size < isr.size)
+                throw new ReplicaNotAvailableException("In Sync Replica information not available for following brokers: " +
+                  isr.filterNot(isrInfo.map(_.id).contains(_)).mkString(","))
+              new PartitionMetadata(topicAndPartition.partition, leaderInfo, replicaInfo, isrInfo, ErrorMapping.NoError)
+            } catch {
+              case e: Throwable =>
+                error("Error while fetching metadata for partition %s".format(topicAndPartition), e)
+                new PartitionMetadata(topicAndPartition.partition, leaderInfo, replicaInfo, isrInfo,
+                  ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
             }
-            new TopicMetadata(topic, partitionMetadata)
-          } else {
-            debug("Topic %s does not exist in metadata cache on broker %d".format(topic, config.brokerId))
-            // topic doesn't exist, send appropriate error code
-            new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.UnknownTopicOrPartitionCode)
           }
-        }
-      }
-
-    // handle auto create topics
-    topicMetadataList.foreach { topicMetadata =>
-      topicMetadata.errorCode match {
-        case ErrorMapping.NoError => topicsMetadata += topicMetadata
-        case ErrorMapping.UnknownTopicOrPartitionCode =>
-          if (config.autoCreateTopicsEnable) {
+          new TopicMetadata(topic, partitionMetadata)
+        } else {
+          // topic doesn't exist, send appropriate error code after handling auto create topics
+          val isOffsetsTopic = topic == OffsetManager.OffsetsTopicName
+          if (config.autoCreateTopicsEnable || isOffsetsTopic) {
             try {
-              AdminUtils.createTopic(zkClient, topicMetadata.topic, config.numPartitions, config.defaultReplicationFactor)
+              if (isOffsetsTopic)
+                AdminUtils.createTopic(zkClient, topic,
+                  config.offsetsTopicPartitions, config.offsetsTopicReplicationFactor, offsetManager.offsetsTopicConfig)
+              else
+                AdminUtils.createTopic(zkClient, topic, config.numPartitions, config.defaultReplicationFactor)
               info("Auto creation of topic %s with %d partitions and replication factor %d is successful!"
-                .format(topicMetadata.topic, config.numPartitions, config.defaultReplicationFactor))
+                .format(topic, config.numPartitions, config.defaultReplicationFactor))
             } catch {
               case e: TopicExistsException => // let it go, possibly another broker created this topic
             }
-            topicsMetadata += new TopicMetadata(topicMetadata.topic, topicMetadata.partitionsMetadata, ErrorMapping.LeaderNotAvailableCode)
+            new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.LeaderNotAvailableCode)
           } else {
-            debug("Auto create topic skipped for %s".format(topicMetadata.topic))
-            topicsMetadata += topicMetadata
+            new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.UnknownTopicOrPartitionCode)
           }
-        case _ =>
-          debug("Error while fetching topic metadata for topic %s due to %s ".format(topicMetadata.topic,
-            ErrorMapping.exceptionFor(topicMetadata.errorCode).getClass.getName))
-          topicsMetadata += topicMetadata
+        }
       }
-    }
-    trace("Sending topic metadata %s for correlation id %d to client %s".format(topicsMetadata.mkString(","), metadataRequest.correlationId, metadataRequest.clientId))
-    val response = new TopicMetadataResponse(topicsMetadata.toSeq, metadataRequest.correlationId)
-    requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response)))
+    }.toSeq
   }
 
-  /* 
-   * Service the Offset commit API
+  /**
+   * Service the topic metadata request API
    */
-  def handleOffsetCommitRequest(request: RequestChannel.Request) {
-    val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest]
-    val responseInfo = offsetCommitRequest.requestInfo.map{
-      case (topicAndPartition, metaAndError) => {
-        val topicDirs = new ZKGroupTopicDirs(offsetCommitRequest.groupId, topicAndPartition.topic)
-        try {
-          ensureTopicExists(topicAndPartition.topic)
-          if(metaAndError.metadata != null && metaAndError.metadata.length > config.offsetMetadataMaxSize) {
-            (topicAndPartition, ErrorMapping.OffsetMetadataTooLargeCode)
-          } else {
-            ZkUtils.updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" +
-              topicAndPartition.partition, metaAndError.offset.toString)
-            (topicAndPartition, ErrorMapping.NoError)
-          }
-        } catch {
-          case e: Throwable => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
+  def handleTopicMetadataRequest(request: RequestChannel.Request) {
+    val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest]
+    var uniqueTopics = Set.empty[String]
+    uniqueTopics = {
+      if(metadataRequest.topics.size > 0)
+        metadataRequest.topics.toSet
+      else {
+        partitionMetadataLock synchronized {
+          metadataCache.keySet.map(_.topic)
         }
       }
     }
-    val response = new OffsetCommitResponse(responseInfo, 
-                                            offsetCommitRequest.correlationId)
+    val topicMetadata = getTopicMetadata(uniqueTopics)
+    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)
     requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response)))
   }
 
@@ -679,26 +696,38 @@ class KafkaApis(val requestChannel: RequestChannel,
    */
   def handleOffsetFetchRequest(request: RequestChannel.Request) {
     val offsetFetchRequest = request.requestObj.asInstanceOf[OffsetFetchRequest]
-    val responseInfo = offsetFetchRequest.requestInfo.map( t => {
-      val topicDirs = new ZKGroupTopicDirs(offsetFetchRequest.groupId, t.topic)
-      try {
-        ensureTopicExists(t.topic)
-        val payloadOpt = ZkUtils.readDataMaybeNull(zkClient, topicDirs.consumerOffsetDir + "/" + t.partition)._1
-        payloadOpt match {
-          case Some(payload) => {
-            (t, OffsetMetadataAndError(offset=payload.toLong, error=ErrorMapping.NoError))
-          } 
-          case None => (t, OffsetMetadataAndError(OffsetMetadataAndError.InvalidOffset, OffsetMetadataAndError.NoMetadata,
-                          ErrorMapping.UnknownTopicOrPartitionCode))
-        }
-      } catch {
-        case e: Throwable =>
-          (t, OffsetMetadataAndError(OffsetMetadataAndError.InvalidOffset, OffsetMetadataAndError.NoMetadata,
-             ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])))
-      }
-    })
-    val response = new OffsetFetchResponse(collection.immutable.Map(responseInfo: _*), 
-                                           offsetFetchRequest.correlationId)
+
+    val status = offsetManager.getOffsets(offsetFetchRequest.groupId, offsetFetchRequest.requestInfo).toMap
+
+    val response = OffsetFetchResponse(status, offsetFetchRequest.correlationId)
+
+    trace("Sending offset fetch response %s for correlation id %d to client %s."
+          .format(response, offsetFetchRequest.correlationId, offsetFetchRequest.clientId))
+    requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response)))
+  }
+
+  /*
+   * Service the consumer metadata API
+   */
+  def handleConsumerMetadataRequest(request: RequestChannel.Request) {
+    val consumerMetadataRequest = request.requestObj.asInstanceOf[ConsumerMetadataRequest]
+
+    val partition = offsetManager.partitionFor(consumerMetadataRequest.group)
+
+    // get metadata (and create the topic if necessary)
+    val offsetsTopicMetadata = getTopicMetadata(Set(OffsetManager.OffsetsTopicName)).head
+
+    val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, consumerMetadataRequest.correlationId)
+
+    val response =
+      offsetsTopicMetadata.partitionsMetadata.find(_.partitionId == partition).map { partitionMetadata =>
+        partitionMetadata.leader.map { leader =>
+          ConsumerMetadataResponse(Some(leader), ErrorMapping.NoError, consumerMetadataRequest.correlationId)
+        }.getOrElse(errorResponse)
+      }.getOrElse(errorResponse)
+
+    trace("Sending consumer metadata %s for correlation id %d to client %s."
+          .format(response, consumerMetadataRequest.correlationId, consumerMetadataRequest.clientId))
     requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response)))
   }
 
@@ -772,12 +801,13 @@ class KafkaApis(val requestChannel: RequestChannel,
 
   class DelayedProduce(keys: Seq[RequestKey],
                        request: RequestChannel.Request,
-                       val partitionStatus: Map[TopicAndPartition, DelayedProduceResponseStatus],
-                       val produce: ProducerRequest,
-                       delayMs: Long)
+                       val partitionStatus: immutable.Map[TopicAndPartition, DelayedProduceResponseStatus],
+                       produce: ProducerRequest,
+                       delayMs: Long,
+                       offsetCommitRequestOpt: Option[OffsetCommitRequest] = None)
           extends DelayedRequest(keys, request, delayMs) with Logging {
 
-    // first update the acks pending variable according to error code
+    // first update the acks pending variable according to the error code
     partitionStatus foreach { case (topicAndPartition, delayedStatus) =>
       if (delayedStatus.status.error == ErrorMapping.NoError) {
         // Timeout error state will be cleared when requiredAcks are received
@@ -790,13 +820,21 @@ class KafkaApis(val requestChannel: RequestChannel,
       trace("Initial partition status for %s is %s".format(topicAndPartition, delayedStatus))
     }
 
-
     def respond() {
       val responseStatus = partitionStatus.map { case (topicAndPartition, delayedStatus) =>
         topicAndPartition -> delayedStatus.status
       }
 
-      val response = ProducerResponse(produce.correlationId, responseStatus)
+      val errorCode = responseStatus.find { case (_, status) =>
+        status.error != ErrorMapping.NoError
+      }.map(_._2.error).getOrElse(ErrorMapping.NoError)
+
+      if (errorCode == ErrorMapping.NoError) {
+        offsetCommitRequestOpt.foreach(ocr => offsetManager.putOffsets(ocr.groupId, ocr.requestInfo) )
+      }
+
+      val response = offsetCommitRequestOpt.map(_.responseFor(errorCode, config.offsetMetadataMaxSize))
+                                           .getOrElse(ProducerResponse(produce.correlationId, responseStatus))
 
       requestChannel.sendResponse(new RequestChannel.Response(
         request, new BoundedByteBufferSend(response)))
@@ -828,7 +866,7 @@ class KafkaApis(val requestChannel: RequestChannel,
             (false, ErrorMapping.UnknownTopicOrPartitionCode)
         }
         if (errorCode != ErrorMapping.NoError) {
-          fetchPartitionStatus.acksPending = false
+          fetchPartitionStatus. acksPending = false
           fetchPartitionStatus.status.error = errorCode
         } else if (hasEnough) {
           fetchPartitionStatus.acksPending = false

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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 b871843..d07796e 100644
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -58,7 +58,7 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
   val numIoThreads = props.getIntInRange("num.io.threads", 8, (1, Int.MaxValue))
   
   /* the number of threads to use for various background processing tasks */
-  val backgroundThreads = props.getIntInRange("background.threads", 4, (1, Int.MaxValue))
+  val backgroundThreads = props.getIntInRange("background.threads", 10, (1, Int.MaxValue))
   
   /* the number of queued requests allowed before blocking the network threads */
   val queuedMaxRequests = props.getIntInRange("queued.max.requests", 500, (1, Int.MaxValue))
@@ -242,10 +242,46 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
   val controlledShutdownRetryBackoffMs = props.getInt("controlled.shutdown.retry.backoff.ms", 5000)
 
   /* enable controlled shutdown of the server */
-  val controlledShutdownEnable = props.getBoolean("controlled.shutdown.enable", false)
+  val controlledShutdownEnable = props.getBoolean("controlled.shutdown.enable", default = false)
 
-  /*********** Misc configuration ***********/
+  /*********** Offset management configuration ***********/
   
   /* the maximum size for a metadata entry associated with an offset commit */
-  val offsetMetadataMaxSize = props.getInt("offset.metadata.max.bytes", 1024)
+  val offsetMetadataMaxSize = props.getInt("offset.metadata.max.bytes", OffsetManagerConfig.DefaultMaxMetadataSize)
+
+  /** Batch size for reading from the offsets segments when loading offsets into the cache. */
+  val offsetsLoadBufferSize = props.getIntInRange("offsets.load.buffer.size",
+    OffsetManagerConfig.DefaultLoadBufferSize, (1, Integer.MAX_VALUE))
+
+  /** The replication factor for the offset commit topic (set higher to ensure availability). */
+  val offsetsTopicReplicationFactor: Short = props.getShortInRange("offsets.topic.replication.factor",
+    OffsetManagerConfig.DefaultOffsetsTopicReplicationFactor, (1, Short.MaxValue))
+
+  /** The number of partitions for the offset commit topic (should not change after deployment). */
+  val offsetsTopicPartitions: Int = props.getIntInRange("offsets.topic.num.partitions",
+    OffsetManagerConfig.DefaultOffsetsTopicNumPartitions, (1, Integer.MAX_VALUE))
+
+  /** The offsets topic segment bytes should be kept relatively small in order to facilitate faster log compaction and cache loads */
+  val offsetsTopicSegmentBytes: Int = props.getIntInRange("offsets.topic.segment.bytes",
+    OffsetManagerConfig.DefaultOffsetsTopicSegmentBytes, (1, Integer.MAX_VALUE))
+
+  /** Compression codec for the offsets topic - compression may be used to achieve "atomic" commits. */
+  val offsetsTopicCompressionCodec = props.getCompressionCodec("offsets.topic.compression.codec",
+    OffsetManagerConfig.DefaultOffsetsTopicCompressionCodec)
+
+  /** Offsets older than this retention period will be discarded. */
+  val offsetsRetentionMinutes: Int = props.getIntInRange("offsets.retention.minutes", 24*60, (1, Integer.MAX_VALUE))
+
+  /** Frequency at which to check for stale offsets. */
+  val offsetsRetentionCheckIntervalMs: Long = props.getLongInRange("offsets.retention.check.interval.ms",
+    OffsetManagerConfig.DefaultOffsetsRetentionCheckIntervalMs, (1, Long.MaxValue))
+
+  /* Offset commit will be delayed until all replicas for the offsets topic receive the commit or this timeout is
+   * reached. This is similar to the producer request timeout. */
+   val offsetCommitTimeoutMs = props.getIntInRange("offsets.commit.timeout.ms",
+    OffsetManagerConfig.DefaultOffsetCommitTimeoutMs, (1, Integer.MAX_VALUE))
+
+  /** The required acks before the commit can be accepted. In general, the default (-1) should not be overridden. */
+  val offsetCommitRequiredAcks = props.getShortInRange("offsets.commit.required.acks",
+    OffsetManagerConfig.DefaultOffsetCommitRequiredAcks, (-1, offsetsTopicReplicationFactor))
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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 feb2093..c208f83 100644
--- a/core/src/main/scala/kafka/server/KafkaServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaServer.scala
@@ -40,11 +40,12 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
   this.logIdent = "[Kafka Server " + config.brokerId + "], "
   private var isShuttingDown = new AtomicBoolean(false)
   private var shutdownLatch = new CountDownLatch(1)
-  private var startupComplete = new AtomicBoolean(false);
+  private var startupComplete = new AtomicBoolean(false)
   val correlationId: AtomicInteger = new AtomicInteger(0)
   var socketServer: SocketServer = null
   var requestHandlerPool: KafkaRequestHandlerPool = null
   var logManager: LogManager = null
+  var offsetManager: OffsetManager = null
   var kafkaHealthcheck: KafkaHealthcheck = null
   var topicConfigManager: TopicConfigManager = null
   var replicaManager: ReplicaManager = null
@@ -83,10 +84,14 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
     socketServer.startup()
 
     replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler, logManager, isShuttingDown)
+
+    /* start offset manager */
+    offsetManager = createOffsetManager()
+
     kafkaController = new KafkaController(config, zkClient)
     
     /* start processing requests */
-    apis = new KafkaApis(socketServer.requestChannel, replicaManager, zkClient, config.brokerId, config, kafkaController)
+    apis = new KafkaApis(socketServer.requestChannel, replicaManager, offsetManager, zkClient, config.brokerId, config, kafkaController)
     requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads)
    
     Mx4jLoader.maybeLoad()
@@ -104,7 +109,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
 
     
     registerStats()
-    startupComplete.set(true);
+    startupComplete.set(true)
     info("started")
   }
   
@@ -215,7 +220,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
    */
   def shutdown() {
     info("shutting down")
-    val canShutdown = isShuttingDown.compareAndSet(false, true);
+    val canShutdown = isShuttingDown.compareAndSet(false, true)
     if (canShutdown) {
       Utils.swallow(controlledShutdown())
       if(kafkaHealthcheck != null)
@@ -224,6 +229,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
         Utils.swallow(socketServer.shutdown())
       if(requestHandlerPool != null)
         Utils.swallow(requestHandlerPool.shutdown())
+      if(offsetManager != null)
+        offsetManager.shutdown()
       Utils.swallow(kafkaScheduler.shutdown())
       if(apis != null)
         Utils.swallow(apis.close())
@@ -237,7 +244,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
         Utils.swallow(zkClient.close())
 
       shutdownLatch.countDown()
-      startupComplete.set(false);
+      startupComplete.set(false)
       info("shut down completed")
     }
   }
@@ -285,6 +292,17 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
                    time = time)
   }
 
-}
+  private def createOffsetManager(): OffsetManager = {
+    val offsetManagerConfig = OffsetManagerConfig(
+      maxMetadataSize = config.offsetMetadataMaxSize,
+      loadBufferSize = config.offsetsLoadBufferSize,
+      offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L,
+      offsetsTopicNumPartitions = config.offsetsTopicPartitions,
+      offsetsTopicReplicationFactor = config.offsetsTopicReplicationFactor,
+      offsetCommitTimeoutMs = config.offsetCommitTimeoutMs,
+      offsetCommitRequiredAcks = config.offsetCommitRequiredAcks)
+    new OffsetManager(offsetManagerConfig, replicaManager, zkClient, kafkaScheduler)
+  }
 
+}
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/server/OffsetManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala
new file mode 100644
index 0000000..89a88a7
--- /dev/null
+++ b/core/src/main/scala/kafka/server/OffsetManager.scala
@@ -0,0 +1,480 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package kafka.server
+
+import kafka.utils._
+import kafka.common._
+import java.nio.ByteBuffer
+import java.util.Properties
+import kafka.log.{FileMessageSet, LogConfig}
+import org.I0Itec.zkclient.ZkClient
+import scala.collection._
+import kafka.message._
+import java.util.concurrent.TimeUnit
+import kafka.metrics.KafkaMetricsGroup
+import com.yammer.metrics.core.Gauge
+import scala.Some
+import kafka.common.TopicAndPartition
+import kafka.consumer.MessageFormatter
+import java.io.PrintStream
+import org.apache.kafka.common.protocol.types.{Struct, Schema, Field}
+import org.apache.kafka.common.protocol.types.Type.STRING
+import org.apache.kafka.common.protocol.types.Type.INT32
+import org.apache.kafka.common.protocol.types.Type.INT64
+import java.util.concurrent.atomic.AtomicBoolean
+
+
+/**
+ * Configuration settings for in-built offset management
+ * @param maxMetadataSize The maximum allowed metadata for any offset commit.
+ * @param loadBufferSize Batch size for reading from the offsets segments when loading offsets into the cache.
+ * @param offsetsRetentionMs Offsets older than this retention period will be discarded.
+ * @param offsetsRetentionCheckIntervalMs Frequency at which to check for stale offsets.
+ * @param offsetsTopicNumPartitions The number of partitions for the offset commit topic (should not change after deployment).
+ * @param offsetsTopicSegmentBytes The offsets topic segment bytes should be kept relatively small to facilitate faster
+ *                                 log compaction and faster offset loads
+ * @param offsetsTopicReplicationFactor The replication factor for the offset commit topic (set higher to ensure availability).
+ * @param offsetsTopicCompressionCodec Compression codec for the offsets topic - compression should be turned on in
+ *                                     order to achieve "atomic" commits.
+ * @param offsetCommitTimeoutMs The offset commit will be delayed until all replicas for the offsets topic receive the
+ *                              commit or this timeout is reached. (Similar to the producer request timeout.)
+ * @param offsetCommitRequiredAcks The required acks before the commit can be accepted. In general, the default (-1)
+ *                                 should not be overridden.
+ */
+case class OffsetManagerConfig(maxMetadataSize: Int = OffsetManagerConfig.DefaultMaxMetadataSize,
+                               loadBufferSize: Int = OffsetManagerConfig.DefaultLoadBufferSize,
+                               offsetsRetentionMs: Long = 24*60*60000L,
+                               offsetsRetentionCheckIntervalMs: Long = OffsetManagerConfig.DefaultOffsetsRetentionCheckIntervalMs,
+                               offsetsTopicNumPartitions: Int = OffsetManagerConfig.DefaultOffsetsTopicNumPartitions,
+                               offsetsTopicSegmentBytes: Int = OffsetManagerConfig.DefaultOffsetsTopicSegmentBytes,
+                               offsetsTopicReplicationFactor: Short = OffsetManagerConfig.DefaultOffsetsTopicReplicationFactor,
+                               offsetsTopicCompressionCodec: CompressionCodec = OffsetManagerConfig.DefaultOffsetsTopicCompressionCodec,
+                               offsetCommitTimeoutMs: Int = OffsetManagerConfig.DefaultOffsetCommitTimeoutMs,
+                               offsetCommitRequiredAcks: Short = OffsetManagerConfig.DefaultOffsetCommitRequiredAcks)
+
+object OffsetManagerConfig {
+  val DefaultMaxMetadataSize = 4096
+  val DefaultLoadBufferSize = 5*1024*1024
+  val DefaultOffsetsRetentionCheckIntervalMs = 600000L
+  val DefaultOffsetsTopicNumPartitions = 1
+  val DefaultOffsetsTopicSegmentBytes = 100*1024*1024
+  val DefaultOffsetsTopicReplicationFactor = 1.toShort
+  val DefaultOffsetsTopicCompressionCodec = NoCompressionCodec
+  val DefaultOffsetCommitTimeoutMs = 5000
+  val DefaultOffsetCommitRequiredAcks = (-1).toShort
+}
+
+class OffsetManager(val config: OffsetManagerConfig,
+                    replicaManager: ReplicaManager,
+                    zkClient: ZkClient,
+                    scheduler: Scheduler) extends Logging with KafkaMetricsGroup {
+
+  /* offsets and metadata cache */
+  private val offsetsCache = new Pool[GroupTopicPartition, OffsetAndMetadata]
+  private val followerTransitionLock = new Object
+
+  private val loadingPartitions: mutable.Set[Int] = mutable.Set()
+
+  private val shuttingDown = new AtomicBoolean(false)
+
+  scheduler.schedule(name = "offsets-cache-compactor",
+                     fun = compact,
+                     period = config.offsetsRetentionCheckIntervalMs,
+                     unit = TimeUnit.MILLISECONDS)
+
+  newGauge("NumOffsets",
+    new Gauge[Int] {
+      def value = offsetsCache.size
+    }
+  )
+
+  newGauge("NumGroups",
+    new Gauge[Int] {
+      def value = offsetsCache.keys.map(_.group).toSet.size
+    }
+  )
+
+  private def compact() {
+    debug("Compacting offsets cache.")
+    val startMs = SystemTime.milliseconds
+
+    val staleOffsets = offsetsCache.filter(startMs - _._2.timestamp > config.offsetsRetentionMs)
+
+    debug("Found %d stale offsets (older than %d ms).".format(staleOffsets.size, config.offsetsRetentionMs))
+
+    // delete the stale offsets from the table and generate tombstone messages to remove them from the log
+    val tombstonesForPartition = staleOffsets.map { case(groupTopicAndPartition, offsetAndMetadata) =>
+      val offsetsPartition = partitionFor(groupTopicAndPartition.group)
+      trace("Removing stale offset and metadata for %s: %s".format(groupTopicAndPartition, offsetAndMetadata))
+
+      offsetsCache.remove(groupTopicAndPartition)
+
+      val commitKey = OffsetManager.offsetCommitKey(groupTopicAndPartition.group,
+        groupTopicAndPartition.topicPartition.topic, groupTopicAndPartition.topicPartition.partition)
+
+      (offsetsPartition, new Message(bytes = null, key = commitKey))
+    }.groupBy{ case (partition, tombstone) => partition }
+
+    // Append the tombstone messages to the offset partitions. It is okay if the replicas don't receive these (say,
+    // if we crash or leaders move) since the new leaders will get rid of stale offsets during their own purge cycles.
+    val numRemoved = tombstonesForPartition.flatMap { case(offsetsPartition, tombstones) =>
+      val partitionOpt = replicaManager.getPartition(OffsetManager.OffsetsTopicName, offsetsPartition)
+      partitionOpt.map { partition =>
+        val appendPartition = TopicAndPartition(OffsetManager.OffsetsTopicName, offsetsPartition)
+        val messages = tombstones.map(_._2).toSeq
+
+        trace("Marked %d offsets in %s for deletion.".format(messages.size, appendPartition))
+
+        try {
+          partition.appendMessagesToLeader(new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*))
+          tombstones.size
+        }
+        catch {
+          case t: Throwable =>
+            error("Failed to mark %d stale offsets for deletion in %s.".format(messages.size, appendPartition), t)
+            // ignore and continue
+            0
+        }
+      }
+    }.sum
+
+    debug("Removed %d stale offsets in %d milliseconds.".format(numRemoved, SystemTime.milliseconds - startMs))
+  }
+
+  def offsetsTopicConfig: Properties = {
+    val props = new Properties
+    props.put(LogConfig.SegmentBytesProp, config.offsetsTopicSegmentBytes.toString)
+    props.put(LogConfig.CleanupPolicyProp, "dedupe")
+    props
+  }
+
+  def partitionFor(group: String): Int = Utils.abs(group.hashCode) % config.offsetsTopicNumPartitions
+
+  /**
+   * Fetch the current offset for the given group/topic/partition from the underlying offsets storage.
+   *
+   * @param key The requested group-topic-partition
+   * @return If the key is present, return the offset and metadata; otherwise return None
+   */
+  private def getOffset(key: GroupTopicPartition) = {
+    val offsetAndMetadata = offsetsCache.get(key)
+    if (offsetAndMetadata == null)
+      OffsetMetadataAndError.NoOffset
+    else
+      OffsetMetadataAndError(offsetAndMetadata.offset, offsetAndMetadata.metadata, ErrorMapping.NoError)
+  }
+
+  /**
+   * Put the (already committed) offset for the given group/topic/partition into the cache.
+   *
+   * @param key The group-topic-partition
+   * @param offsetAndMetadata The offset/metadata to be stored
+   */
+  private def putOffset(key: GroupTopicPartition, offsetAndMetadata: OffsetAndMetadata) {
+    offsetsCache.put(key, offsetAndMetadata)
+  }
+
+  def putOffsets(group: String, offsets: Map[TopicAndPartition, OffsetAndMetadata]) {
+    // this method is called _after_ the offsets have been durably appended to the commit log, so there is no need to
+    // check for current leadership as we do for the offset fetch
+    trace("Putting offsets %s for group %s in offsets partition %d.".format(offsets, group, partitionFor(group)))
+    offsets.foreach { case (topicAndPartition, offsetAndMetadata) =>
+      putOffset(GroupTopicPartition(group, topicAndPartition), offsetAndMetadata)
+    }
+  }
+
+  /**
+   * The most important guarantee that this API provides is that it should never return a stale offset. i.e., it either
+   * returns the current offset or it begins to sync the cache from the log (and returns an error code).
+   */
+  def getOffsets(group: String, topicPartitions: Seq[TopicAndPartition]): Map[TopicAndPartition, OffsetMetadataAndError] = {
+    trace("Getting offsets %s for group %s.".format(topicPartitions, group))
+
+    val offsetsPartition = partitionFor(group)
+
+    /**
+     * followerTransitionLock protects against fetching from an empty/cleared offset cache (i.e., cleared due to a
+     * leader->follower transition). i.e., even if leader-is-local is true a follower transition can occur right after
+     * the check and clear the cache. i.e., we would read from the empty cache and incorrectly return NoOffset.
+     */
+    followerTransitionLock synchronized {
+      if (leaderIsLocal(offsetsPartition)) {
+        if (loadingPartitions synchronized loadingPartitions.contains(offsetsPartition)) {
+          debug("Cannot fetch offsets for group %s due to ongoing offset load.".format(group))
+          topicPartitions.map { topicAndPartition =>
+            val groupTopicPartition = GroupTopicPartition(group, topicAndPartition)
+            (groupTopicPartition.topicPartition, OffsetMetadataAndError.OffsetsLoading)
+          }.toMap
+        } else {
+          if (topicPartitions.size == 0) {
+           // Return offsets for all partitions owned by this consumer group. (this only applies to consumers that commit offsets to Kafka.)
+            offsetsCache.filter(_._1.group == group).map { case(groupTopicPartition, offsetAndMetadata) =>
+              (groupTopicPartition.topicPartition, OffsetMetadataAndError(offsetAndMetadata.offset, offsetAndMetadata.metadata, ErrorMapping.NoError))
+            }.toMap
+          } else {
+            topicPartitions.map { topicAndPartition =>
+              val groupTopicPartition = GroupTopicPartition(group, topicAndPartition)
+              (groupTopicPartition.topicPartition, getOffset(groupTopicPartition))
+            }.toMap
+          }
+        }
+      } else {
+        debug("Could not fetch offsets for group %s (not offset coordinator).".format(group))
+        topicPartitions.map { topicAndPartition =>
+          val groupTopicPartition = GroupTopicPartition(group, topicAndPartition)
+          (groupTopicPartition.topicPartition, OffsetMetadataAndError.NotOffsetManagerForGroup)
+        }.toMap
+      }
+    }
+  }
+
+  /**
+   * Asynchronously read the partition from the offsets topic and populate the cache
+   */
+  def loadOffsetsFromLog(offsetsPartition: Int) {
+    
+    val topicPartition = TopicAndPartition(OffsetManager.OffsetsTopicName, offsetsPartition)
+
+    loadingPartitions synchronized {
+      if (loadingPartitions.contains(offsetsPartition)) {
+        info("Offset load from %s already in progress.".format(topicPartition))
+      } else {
+        loadingPartitions.add(offsetsPartition)
+        scheduler.schedule(topicPartition.toString, loadOffsets)
+      }
+    }
+
+    def loadOffsets() {
+      info("Loading offsets from " + topicPartition)
+
+      val startMs = SystemTime.milliseconds
+      try {
+        replicaManager.logManager.getLog(topicPartition) match {
+          case Some(log) =>
+            var currOffset = log.logSegments.head.baseOffset
+            val buffer = ByteBuffer.allocate(config.loadBufferSize)
+            // loop breaks if leader changes at any time during the load, since getHighWatermark is -1
+            while (currOffset < getHighWatermark(offsetsPartition) && !shuttingDown.get()) {
+              buffer.clear()
+              val messages = log.read(currOffset, config.loadBufferSize).asInstanceOf[FileMessageSet]
+              messages.readInto(buffer, 0)
+              val messageSet = new ByteBufferMessageSet(buffer)
+              messageSet.foreach { msgAndOffset =>
+                require(msgAndOffset.message.key != null, "Offset entry key should not be null")
+                val key = OffsetManager.readMessageKey(msgAndOffset.message.key)
+                if (msgAndOffset.message.payload == null) {
+                  if (offsetsCache.remove(key) != null)
+                    trace("Removed offset for %s due to tombstone entry.".format(key))
+                  else
+                    trace("Ignoring redundant tombstone for %s.".format(key))
+                } else {
+                  val value = OffsetManager.readMessageValue(msgAndOffset.message.payload)
+                  putOffset(key, value)
+                  trace("Loaded offset %s for %s.".format(value, key))
+                }
+                currOffset = msgAndOffset.nextOffset
+              }
+            }
+
+            if (!shuttingDown.get())
+              info("Finished loading offsets from %s in %d milliseconds."
+                   .format(topicPartition, SystemTime.milliseconds - startMs))
+          case None =>
+            warn("No log found for " + topicPartition)
+        }
+      }
+      catch {
+        case t: Throwable =>
+          error("Error in loading offsets from " + topicPartition, t)
+      }
+      finally {
+        loadingPartitions synchronized loadingPartitions.remove(offsetsPartition)
+      }
+    }
+  }
+
+  private def getHighWatermark(partitionId: Int): Long = {
+    val partitionOpt = replicaManager.getPartition(OffsetManager.OffsetsTopicName, partitionId)
+
+    val hw = partitionOpt.map { partition =>
+      partition.leaderReplicaIfLocal().map(_.highWatermark).getOrElse(-1L)
+    }.getOrElse(-1L)
+
+    hw
+  }
+
+  private def leaderIsLocal(partition: Int) = { getHighWatermark(partition) != -1L }
+
+  /**
+   * When this broker becomes a follower for an offsets topic partition clear out the cache for groups that belong to
+   * that partition.
+   * @param offsetsPartition Groups belonging to this partition of the offsets topic will be deleted from the cache.
+   */
+  def clearOffsetsInPartition(offsetsPartition: Int) {
+    debug("Deleting offset entries belonging to [%s,%d].".format(OffsetManager.OffsetsTopicName, offsetsPartition))
+
+    followerTransitionLock synchronized {
+      offsetsCache.keys.foreach { key =>
+        if (partitionFor(key.group) == offsetsPartition) {
+          offsetsCache.remove(key)
+        }
+      }
+    }
+  }
+
+  def shutdown() {
+    shuttingDown.set(true)
+  }
+
+}
+
+object OffsetManager {
+
+  val OffsetsTopicName = "__consumer_offsets"
+
+  private case class KeyAndValueSchemas(keySchema: Schema, valueSchema: Schema)
+
+  private val CURRENT_OFFSET_SCHEMA_VERSION = 0.toShort
+
+  private val OFFSET_COMMIT_KEY_SCHEMA_V0 = new Schema(new Field("group", STRING),
+                                                       new Field("topic", STRING),
+                                                       new Field("partition", INT32))
+  private val KEY_GROUP_FIELD = OFFSET_COMMIT_KEY_SCHEMA_V0.get("group")
+  private val KEY_TOPIC_FIELD = OFFSET_COMMIT_KEY_SCHEMA_V0.get("topic")
+  private val KEY_PARTITION_FIELD = OFFSET_COMMIT_KEY_SCHEMA_V0.get("partition")
+
+  private val OFFSET_COMMIT_VALUE_SCHEMA_V0 = new Schema(new Field("offset", INT64),
+                                                         new Field("metadata", STRING, "Associated metadata.", ""),
+                                                         new Field("timestamp", INT64))
+  private val VALUE_OFFSET_FIELD = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("offset")
+  private val VALUE_METADATA_FIELD = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("metadata")
+  private val VALUE_TIMESTAMP_FIELD = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("timestamp")
+
+  // map of versions to schemas
+  private val OFFSET_SCHEMAS = Map(0 -> KeyAndValueSchemas(OFFSET_COMMIT_KEY_SCHEMA_V0, OFFSET_COMMIT_VALUE_SCHEMA_V0))
+
+  private val CURRENT_SCHEMA = schemaFor(CURRENT_OFFSET_SCHEMA_VERSION)
+
+  private def schemaFor(version: Int) = {
+    val schemaOpt = OFFSET_SCHEMAS.get(version)
+    schemaOpt match {
+      case Some(schema) => schema
+      case _ => throw new KafkaException("Unknown offset schema version " + version)
+    }
+  }
+
+  /**
+   * Generates the key for offset commit message for given (group, topic, partition)
+   *
+   * @return key for offset commit message
+   */
+  def offsetCommitKey(group: String, topic: String, partition: Int, versionId: Short = 0): Array[Byte] = {
+    val key = new Struct(CURRENT_SCHEMA.keySchema)
+    key.set(KEY_GROUP_FIELD, group)
+    key.set(KEY_TOPIC_FIELD, topic)
+    key.set(KEY_PARTITION_FIELD, partition)
+
+    val byteBuffer = ByteBuffer.allocate(2 /* version */ + key.sizeOf)
+    byteBuffer.putShort(CURRENT_OFFSET_SCHEMA_VERSION)
+    key.writeTo(byteBuffer)
+    byteBuffer.array()
+  }
+
+  /**
+   * Generates the payload for offset commit message from given offset and metadata
+   *
+   * @param offsetAndMetadata consumer's current offset and metadata
+   * @return payload for offset commit message
+   */
+  def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata): Array[Byte] = {
+    val value = new Struct(CURRENT_SCHEMA.valueSchema)
+    value.set(VALUE_OFFSET_FIELD, offsetAndMetadata.offset)
+    value.set(VALUE_METADATA_FIELD, offsetAndMetadata.metadata)
+    value.set(VALUE_TIMESTAMP_FIELD, offsetAndMetadata.timestamp)
+
+    val byteBuffer = ByteBuffer.allocate(2 /* version */ + value.sizeOf)
+    byteBuffer.putShort(CURRENT_OFFSET_SCHEMA_VERSION)
+    value.writeTo(byteBuffer)
+    byteBuffer.array()
+  }
+
+  /**
+   * Decodes the offset messages' key
+   *
+   * @param buffer input byte-buffer
+   * @return an GroupTopicPartition object
+   */
+  def readMessageKey(buffer: ByteBuffer): GroupTopicPartition = {
+    val version = buffer.getShort()
+    val keySchema = schemaFor(version).keySchema
+    val key = keySchema.read(buffer).asInstanceOf[Struct]
+
+    val group = key.get(KEY_GROUP_FIELD).asInstanceOf[String]
+    val topic = key.get(KEY_TOPIC_FIELD).asInstanceOf[String]
+    val partition = key.get(KEY_PARTITION_FIELD).asInstanceOf[Int]
+
+    GroupTopicPartition(group, TopicAndPartition(topic, partition))
+  }
+
+  /**
+   * Decodes the offset messages' payload and retrieves offset and metadata from it
+   *
+   * @param buffer input byte-buffer
+   * @return an offset-metadata object from the message
+   */
+  def readMessageValue(buffer: ByteBuffer): OffsetAndMetadata = {
+    if(buffer == null) { // tombstone
+      null
+    } else {
+      val version = buffer.getShort()
+      val valueSchema = schemaFor(version).valueSchema
+      val value = valueSchema.read(buffer).asInstanceOf[Struct]
+
+      val offset = value.get(VALUE_OFFSET_FIELD).asInstanceOf[Long]
+      val metadata = value.get(VALUE_METADATA_FIELD).asInstanceOf[String]
+      val timestamp = value.get(VALUE_TIMESTAMP_FIELD).asInstanceOf[Long]
+
+      OffsetAndMetadata(offset, metadata, timestamp)
+    }
+  }
+
+  // Formatter for use with tools such as console consumer: Consumer should also set exclude.internal.topics to false.
+  // (specify --formatter "kafka.server.OffsetManager\$OffsetsMessageFormatter" when consuming __consumer_offsets)
+  class OffsetsMessageFormatter extends MessageFormatter {
+    def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream) {
+      val formattedKey = if (key == null) "NULL" else OffsetManager.readMessageKey(ByteBuffer.wrap(key)).toString
+      val formattedValue = if (value == null) "NULL" else OffsetManager.readMessageValue(ByteBuffer.wrap(value)).toString
+      output.write(formattedKey.getBytes)
+      output.write("::".getBytes)
+      output.write(formattedValue.getBytes)
+      output.write("\n".getBytes)
+    }
+  }
+
+}
+
+case class GroupTopicPartition(group: String, topicPartition: TopicAndPartition) {
+
+  def this(group: String, topic: String, partition: Int) =
+    this(group, new TopicAndPartition(topic, partition))
+
+  override def toString =
+    "[%s,%s,%d]".format(group, topicPartition.topic, topicPartition.partition)
+
+}
+

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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 fb759d9..f16fbe6 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -205,7 +205,8 @@ class ReplicaManager(val config: KafkaConfig,
     }
   }
 
-  def becomeLeaderOrFollower(leaderAndISRRequest: LeaderAndIsrRequest): (collection.Map[(String, Int), Short], Short) = {
+  def becomeLeaderOrFollower(leaderAndISRRequest: LeaderAndIsrRequest,
+                             offsetManager: OffsetManager): (collection.Map[(String, Int), Short], Short) = {
     leaderAndISRRequest.partitionStateInfos.foreach { case ((topic, partition), stateInfo) =>
       stateChangeLogger.trace("Broker %d received LeaderAndIsr request %s correlation id %d from controller %d epoch %d for partition [%s,%d]"
                                 .format(localBrokerId, stateInfo, leaderAndISRRequest.correlationId,
@@ -255,8 +256,10 @@ class ReplicaManager(val config: KafkaConfig,
           .filter{ case (partition, partitionStateInfo) => partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader == config.brokerId}
         val partitionsToBeFollower = (partitionState -- partitionsTobeLeader.keys)
 
-        if (!partitionsTobeLeader.isEmpty) makeLeaders(controllerId, controllerEpoch, partitionsTobeLeader, leaderAndISRRequest.correlationId, responseMap)
-        if (!partitionsToBeFollower.isEmpty) makeFollowers(controllerId, controllerEpoch, partitionsToBeFollower, leaderAndISRRequest.leaders, leaderAndISRRequest.correlationId, responseMap)
+        if (!partitionsTobeLeader.isEmpty)
+          makeLeaders(controllerId, controllerEpoch, partitionsTobeLeader, leaderAndISRRequest.correlationId, responseMap, offsetManager)
+        if (!partitionsToBeFollower.isEmpty)
+          makeFollowers(controllerId, controllerEpoch, partitionsToBeFollower, leaderAndISRRequest.leaders, leaderAndISRRequest.correlationId, responseMap, offsetManager)
 
         // we initialize highwatermark thread after the first leaderisrrequest. This ensures that all the partitions
         // have been completely populated before starting the checkpointing there by avoiding weird race conditions
@@ -283,7 +286,8 @@ class ReplicaManager(val config: KafkaConfig,
    */
   private def makeLeaders(controllerId: Int, epoch: Int,
                           partitionState: Map[Partition, PartitionStateInfo],
-                          correlationId: Int, responseMap: mutable.Map[(String, Int), Short]) = {
+                          correlationId: Int, responseMap: mutable.Map[(String, Int), Short],
+                          offsetManager: OffsetManager) = {
     partitionState.foreach(state =>
       stateChangeLogger.trace(("Broker %d handling LeaderAndIsr request correlationId %d from controller %d epoch %d " +
         "starting the become-leader transition for partition %s")
@@ -302,7 +306,7 @@ class ReplicaManager(val config: KafkaConfig,
       }
       // Update the partition information to be the leader
       partitionState.foreach{ case (partition, partitionStateInfo) =>
-        partition.makeLeader(controllerId, partitionStateInfo, correlationId)}
+        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 {
@@ -344,7 +348,8 @@ class ReplicaManager(val config: KafkaConfig,
    * the error message will be set on each partition since we do not know which partition caused it
    */
   private def makeFollowers(controllerId: Int, epoch: Int, partitionState: Map[Partition, PartitionStateInfo],
-                            leaders: Set[Broker], correlationId: Int, responseMap: mutable.Map[(String, Int), Short]) {
+                            leaders: Set[Broker], correlationId: Int, responseMap: mutable.Map[(String, Int), Short],
+                            offsetManager: OffsetManager) {
     partitionState.foreach { state =>
       stateChangeLogger.trace(("Broker %d handling LeaderAndIsr request correlationId %d from controller %d epoch %d " +
         "starting the become-follower transition for partition %s")
@@ -367,7 +372,7 @@ class ReplicaManager(val config: KafkaConfig,
         val newLeaderBrokerId = leaderIsrAndControllerEpoch.leaderAndIsr.leader
         leaders.find(_.id == newLeaderBrokerId) match {
           case Some(leaderBroker) =>
-            if (partition.makeFollower(controllerId, partitionStateInfo, correlationId))
+            if (partition.makeFollower(controllerId, partitionStateInfo, correlationId, offsetManager))
               partitionsToMakeFollower += partition
             else
               stateChangeLogger.info(("Broker %d skipped the become-follower state change after marking its partition as follower with correlation id %d from " +

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala
index 33d7c2c..88f824f 100644
--- a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala
+++ b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala
@@ -20,16 +20,22 @@ package kafka.tools
 
 import joptsimple._
 import org.I0Itec.zkclient.ZkClient
-import kafka.utils.{Json, ZkUtils, ZKStringSerializer, Logging}
+import kafka.utils._
 import kafka.consumer.SimpleConsumer
-import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest}
-import kafka.common.{BrokerNotAvailableException, TopicAndPartition}
+import kafka.api.{OffsetFetchResponse, OffsetFetchRequest, OffsetRequest}
+import kafka.common.{OffsetMetadataAndError, ErrorMapping, BrokerNotAvailableException, TopicAndPartition}
 import scala.collection._
+import kafka.client.ClientUtils
+import kafka.network.BlockingChannel
+import kafka.api.PartitionOffsetRequestInfo
+import scala.Some
 
 
 object ConsumerOffsetChecker extends Logging {
 
   private val consumerMap: mutable.Map[Int, Option[SimpleConsumer]] = mutable.Map()
+  private val offsetMap: mutable.Map[TopicAndPartition, Long] = mutable.Map()
+  private var topicPidMap: immutable.Map[String, Seq[Int]] = immutable.Map()
 
   private def getConsumer(zkClient: ZkClient, bid: Int): Option[SimpleConsumer] = {
     try {
@@ -49,18 +55,17 @@ object ConsumerOffsetChecker extends Logging {
       }
     } catch {
       case t: Throwable =>
-        error("Could not parse broker info", t)
+        println("Could not parse broker info due to " + t.getCause)
         None
     }
   }
 
   private def processPartition(zkClient: ZkClient,
                                group: String, topic: String, pid: Int) {
-    val offset = ZkUtils.readData(zkClient, "/consumers/%s/offsets/%s/%s".
-            format(group, topic, pid))._1.toLong
-    val owner = ZkUtils.readDataMaybeNull(zkClient, "/consumers/%s/owners/%s/%s".
-            format(group, topic, pid))._1
-
+    val topicPartition = TopicAndPartition(topic, pid)
+    val offsetOpt = offsetMap.get(topicPartition)
+    val groupDirs = new ZKGroupTopicDirs(group, topic)
+    val owner = ZkUtils.readDataMaybeNull(zkClient, groupDirs.consumerOwnerDir + "/%s".format(pid))._1
     ZkUtils.getLeaderForPartition(zkClient, topic, pid) match {
       case Some(bid) =>
         val consumerOpt = consumerMap.getOrElseUpdate(bid, getConsumer(zkClient, bid))
@@ -71,19 +76,18 @@ object ConsumerOffsetChecker extends Logging {
               OffsetRequest(immutable.Map(topicAndPartition -> PartitionOffsetRequestInfo(OffsetRequest.LatestTime, 1)))
             val logSize = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head
 
-            val lag = logSize - offset
-            println("%-15s %-30s %-3s %-15s %-15s %-15s %s".format(group, topic, pid, offset, logSize, lag,
-              owner match {case Some(ownerStr) => ownerStr case None => "none"}))
+            val lagString = offsetOpt.map(o => if (o == -1) "unknown" else (logSize - o).toString)
+            println("%-15s %-30s %-3s %-15s %-15s %-15s %s".format(group, topic, pid, offsetOpt.getOrElse("unknown"), logSize, lagString.getOrElse("unknown"),
+                                                                   owner match {case Some(ownerStr) => ownerStr case None => "none"}))
           case None => // ignore
         }
       case None =>
-        error("No broker for partition %s - %s".format(topic, pid))
+        println("No broker for partition %s - %s".format(topic, pid))
     }
   }
 
   private def processTopic(zkClient: ZkClient, group: String, topic: String) {
-    val pidMap = ZkUtils.getPartitionsForTopics(zkClient, Seq(topic))
-    pidMap.get(topic) match {
+    topicPidMap.get(topic) match {
       case Some(pids) =>
         pids.sorted.foreach {
           pid => processPartition(zkClient, group, topic, pid)
@@ -105,13 +109,18 @@ object ConsumerOffsetChecker extends Logging {
   def main(args: Array[String]) {
     val parser = new OptionParser()
 
-    val zkConnectOpt = parser.accepts("zkconnect", "ZooKeeper connect string.").
-            withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String]);
+    val zkConnectOpt = parser.accepts("zookeeper", "ZooKeeper connect string.").
+            withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String])
     val topicsOpt = parser.accepts("topic",
             "Comma-separated list of consumer topics (all topics if absent).").
             withRequiredArg().ofType(classOf[String])
     val groupOpt = parser.accepts("group", "Consumer group.").
             withRequiredArg().ofType(classOf[String])
+    val channelSocketTimeoutMsOpt = parser.accepts("socket.timeout.ms", "Socket timeout to use when querying for offsets.").
+            withRequiredArg().ofType(classOf[java.lang.Integer]).defaultsTo(6000)
+    val channelRetryBackoffMsOpt = parser.accepts("retry.backoff.ms", "Retry back-off to use for failed offset queries.").
+            withRequiredArg().ofType(classOf[java.lang.Integer]).defaultsTo(3000)
+
     parser.accepts("broker-info", "Print broker info")
     parser.accepts("help", "Print this message.")
 
@@ -122,7 +131,7 @@ object ConsumerOffsetChecker extends Logging {
        System.exit(0)
     }
 
-    for (opt <- List(groupOpt))
+    for (opt <- List(groupOpt, zkConnectOpt))
       if (!options.has(opt)) {
         System.err.println("Missing required argument: %s".format(opt))
         parser.printHelpOn(System.err)
@@ -130,23 +139,50 @@ object ConsumerOffsetChecker extends Logging {
       }
 
     val zkConnect = options.valueOf(zkConnectOpt)
+
     val group = options.valueOf(groupOpt)
-    val topics = if (options.has(topicsOpt)) Some(options.valueOf(topicsOpt))
-      else None
+    val groupDirs = new ZKGroupDirs(group)
 
+    val channelSocketTimeoutMs = options.valueOf(channelSocketTimeoutMsOpt).intValue()
+    val channelRetryBackoffMs = options.valueOf(channelRetryBackoffMsOpt).intValue()
+
+    val topics = if (options.has(topicsOpt)) Some(options.valueOf(topicsOpt)) else None
 
     var zkClient: ZkClient = null
+    var channel: BlockingChannel = null
     try {
       zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer)
 
       val topicList = topics match {
         case Some(x) => x.split(",").view.toList
-        case None => ZkUtils.getChildren(
-          zkClient, "/consumers/%s/offsets".format(group)).toList
+        case None => ZkUtils.getChildren(zkClient, groupDirs.consumerGroupDir +  "/owners").toList
       }
 
-      debug("zkConnect = %s; topics = %s; group = %s".format(
-        zkConnect, topicList.toString(), group))
+      topicPidMap = immutable.Map(ZkUtils.getPartitionsForTopics(zkClient, topicList).toSeq:_*)
+      val topicPartitions = topicPidMap.flatMap { case(topic, partitionSeq) => partitionSeq.map(TopicAndPartition(topic, _)) }.toSeq
+
+      val channel = ClientUtils.channelToOffsetManager(group, zkClient, channelSocketTimeoutMs, channelRetryBackoffMs)
+      
+      debug("Sending offset fetch request to coordinator %s:%d.".format(channel.host, channel.port))
+      channel.send(OffsetFetchRequest(group, topicPartitions))
+      val offsetFetchResponse = OffsetFetchResponse.readFrom(channel.receive().buffer)
+      debug("Received offset fetch response %s.".format(offsetFetchResponse))
+      
+      offsetFetchResponse.requestInfo.foreach { case (topicAndPartition, offsetAndMetadata) =>
+        if (offsetAndMetadata == OffsetMetadataAndError.NoOffset) {
+          val topicDirs = new ZKGroupTopicDirs(group, topicAndPartition.topic)
+          // this group may not have migrated off zookeeper for offsets storage (we don't expose the dual-commit option in this tool
+          // (meaning the lag may be off until all the consumers in the group have the same setting for offsets storage)
+          val offset = ZkUtils.readData(zkClient, topicDirs.consumerOffsetDir + "/%d".format(topicAndPartition.partition))._1.toLong
+          offsetMap.put(topicAndPartition, offset)
+        }
+        else if (offsetAndMetadata.error == ErrorMapping.NoError)
+          offsetMap.put(topicAndPartition, offsetAndMetadata.offset)
+        else {
+          println("Could not fetch offset for %s due to %s.".format(topicAndPartition, ErrorMapping.exceptionFor(offsetAndMetadata.error)))
+        }
+      }
+      channel.disconnect()
 
       println("%-15s %-30s %-3s %-15s %-15s %-15s %s".format("Group", "Topic", "Pid", "Offset", "logSize", "Lag", "Owner"))
       topicList.sorted.foreach {
@@ -154,7 +190,7 @@ object ConsumerOffsetChecker extends Logging {
       }
 
       if (options.has("broker-info"))
-        printBrokerInfo();
+        printBrokerInfo()
 
       for ((_, consumerOpt) <- consumerMap)
         consumerOpt match {
@@ -162,6 +198,10 @@ object ConsumerOffsetChecker extends Logging {
           case None => // ignore
         }
     }
+    catch {
+      case t: Throwable =>
+        println("Exiting due to: %s.".format(t.getMessage))
+    }
     finally {
       for (consumerOpt <- consumerMap.values) {
         consumerOpt match {
@@ -171,6 +211,9 @@ object ConsumerOffsetChecker extends Logging {
       }
       if (zkClient != null)
         zkClient.close()
+
+      if (channel != null)
+        channel.disconnect()
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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 14f44d9..f0ab02a 100644
--- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala
+++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala
@@ -62,7 +62,7 @@ object DumpLogSegments {
       val file = new File(arg)
       if(file.getName.endsWith(Log.LogFileSuffix)) {
         println("Dumping " + file)
-        dumpLog(file, print, nonConsecutivePairsForLogFilesMap, isDeepIteration)
+        dumpLog(file, print, nonConsecutivePairsForLogFilesMap, isDeepIteration, maxMessageSize)
       } else if(file.getName.endsWith(Log.IndexFileSuffix)) {
         println("Dumping " + file)
         dumpIndex(file, verifyOnly, misMatchesForIndexFilesMap, maxMessageSize)
@@ -117,13 +117,15 @@ object DumpLogSegments {
   private def dumpLog(file: File,
                       printContents: Boolean,
                       nonConsecutivePairsForLogFilesMap: mutable.HashMap[String, List[(Long, Long)]],
-                      isDeepIteration: Boolean) {
+                      isDeepIteration: Boolean,
+                      maxMessageSize: Int) {
     val startOffset = file.getName().split("\\.")(0).toLong
     println("Starting offset: " + startOffset)
     val messageSet = new FileMessageSet(file, false)
     var validBytes = 0L
     var lastOffset = -1l
-    for(shallowMessageAndOffset <- messageSet) { // this only does shallow iteration
+    val shallowIterator = messageSet.iterator(maxMessageSize)
+    for(shallowMessageAndOffset <- shallowIterator) { // this only does shallow iteration
       val itr = getIterator(shallowMessageAndOffset, isDeepIteration)
       for (messageAndOffset <- itr) {
         val msg = messageAndOffset.message

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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 f0f871c..e4d1a86 100644
--- a/core/src/main/scala/kafka/tools/MirrorMaker.scala
+++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala
@@ -22,7 +22,6 @@ import kafka.utils.{Utils, CommandLineUtils, Logging}
 import kafka.producer.{KeyedMessage, ProducerConfig, Producer}
 import scala.collection.JavaConversions._
 import java.util.concurrent.CountDownLatch
-import java.nio.ByteBuffer
 import kafka.consumer._
 import kafka.serializer._
 import collection.mutable.ListBuffer

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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 5e8c56d..a649461 100644
--- a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
+++ b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
@@ -118,7 +118,10 @@ object ReplicaVerificationTool extends Logging {
     val topicsMetadataResponse = ClientUtils.fetchTopicMetadata(Set[String](), metadataTargetBrokers, clientId, maxWaitMs)
     val brokerMap = topicsMetadataResponse.extractBrokers(topicsMetadataResponse.topicsMetadata)
     val filteredTopicMetadata = topicsMetadataResponse.topicsMetadata.filter(
-        topicMetadata => if (topicWhiteListFiler.isTopicAllowed(topicMetadata.topic)) true else false
+        topicMetadata => if (topicWhiteListFiler.isTopicAllowed(topicMetadata.topic, excludeInternalTopics = false))
+          true
+        else
+          false
     )
     val topicPartitionReplicaList: Seq[TopicPartitionReplica] = filteredTopicMetadata.flatMap(
       topicMetadataResponse =>

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala b/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala
index eac9af2..92c0d1f 100644
--- a/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala
+++ b/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala
@@ -26,7 +26,7 @@ object VerifyConsumerRebalance extends Logging {
     val parser = new OptionParser()
 
     val zkConnectOpt = parser.accepts("zookeeper.connect", "ZooKeeper connect string.").
-      withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String]);
+      withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String])
     val groupOpt = parser.accepts("group", "Consumer group.").
       withRequiredArg().ofType(classOf[String])
     parser.accepts("help", "Print this message.")
@@ -78,7 +78,7 @@ object VerifyConsumerRebalance extends Logging {
      * This means that for each partition registered under /brokers/topics/[topic]/[broker-id], an owner exists
      * under /consumers/[consumer_group]/owners/[topic]/[broker_id-partition_id]
      */
-    val consumersPerTopicMap = ZkUtils.getConsumersPerTopic(zkClient, group)
+    val consumersPerTopicMap = ZkUtils.getConsumersPerTopic(zkClient, group, excludeInternalTopics = false)
     val partitionsPerTopicMap = ZkUtils.getPartitionsForTopics(zkClient, consumersPerTopicMap.keySet.toSeq)
 
     partitionsPerTopicMap.foreach { partitionsForTopic =>

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/utils/VerifiableProperties.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/utils/VerifiableProperties.scala b/core/src/main/scala/kafka/utils/VerifiableProperties.scala
index b070bb4..d40b03c 100644
--- a/core/src/main/scala/kafka/utils/VerifiableProperties.scala
+++ b/core/src/main/scala/kafka/utils/VerifiableProperties.scala
@@ -20,6 +20,8 @@ package kafka.utils
 import java.util.Properties
 import java.util.Collections
 import scala.collection._
+import kafka.message.{CompressionCodec, NoCompressionCodec}
+
 
 class VerifiableProperties(val props: Properties) extends Logging {
   private val referenceSet = mutable.HashSet[String]()
@@ -193,6 +195,24 @@ class VerifiableProperties(val props: Properties) extends Logging {
     }
   }
 
+  /**
+   * Parse compression codec from a property list in either. Codecs may be specified as integers, or as strings.
+   * See [[kafka.message.CompressionCodec]] for more details.
+   * @param name The property name
+   * @param default Default compression codec
+   * @return compression codec
+   */
+  def getCompressionCodec(name: String, default: CompressionCodec) = {
+    val prop = getString(name, NoCompressionCodec.name)
+    try {
+      CompressionCodec.getCompressionCodec(prop.toInt)
+    }
+    catch {
+      case nfe: NumberFormatException =>
+        CompressionCodec.getCompressionCodec(prop)
+    }
+  }
+
   def verify() {
     info("Verifying properties")
     val propNames = {

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/utils/ZkUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala
index a198628..16bf7e3 100644
--- a/core/src/main/scala/kafka/utils/ZkUtils.scala
+++ b/core/src/main/scala/kafka/utils/ZkUtils.scala
@@ -676,12 +676,12 @@ object ZkUtils extends Logging {
     getChildren(zkClient, dirs.consumerRegistryDir)
   }
 
-  def getConsumersPerTopic(zkClient: ZkClient, group: String) : mutable.Map[String, List[String]] = {
+  def getConsumersPerTopic(zkClient: ZkClient, group: String, excludeInternalTopics: Boolean) : mutable.Map[String, List[String]] = {
     val dirs = new ZKGroupDirs(group)
     val consumers = getChildrenParentMayNotExist(zkClient, dirs.consumerRegistryDir)
     val consumersPerTopicMap = new mutable.HashMap[String, List[String]]
     for (consumer <- consumers) {
-      val topicCount = TopicCount.constructTopicCount(group, consumer, zkClient)
+      val topicCount = TopicCount.constructTopicCount(group, consumer, zkClient, excludeInternalTopics)
       for ((topic, consumerThreadIdSet) <- topicCount.getConsumerThreadIdsPerTopic) {
         for (consumerThreadId <- consumerThreadIdSet)
           consumersPerTopicMap.get(topic) match {


[2/5] KAFKA-1012; Consumer offset management in Kafka; patched by Tejas Patil and Joel Koshy; feedback and reviews from Neha Narkhede, Jun Rao, Guozhang Wang, Sriram Subramanian, Joe Stein, Chris Riccomini

Posted by jj...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/mirror_maker/config/whitelisttest_2.consumer.properties
----------------------------------------------------------------------
diff --git a/system_test/mirror_maker/config/whitelisttest_2.consumer.properties b/system_test/mirror_maker/config/whitelisttest_2.consumer.properties
deleted file mode 100644
index f1a902b..0000000
--- a/system_test/mirror_maker/config/whitelisttest_2.consumer.properties
+++ /dev/null
@@ -1,28 +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.
-# see kafka.consumer.ConsumerConfig for more details
-
-# zk connection string
-# comma separated host:port pairs, each corresponding to a zk
-# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"
-zk.connect=localhost:2182
-
-# timeout in ms for connecting to zookeeper
-zk.connection.timeout.ms=1000000
-
-#consumer group id
-group.id=group1
-shallow.iterator.enable=true
-

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/mirror_maker/config/zookeeper_source_1.properties
----------------------------------------------------------------------
diff --git a/system_test/mirror_maker/config/zookeeper_source_1.properties b/system_test/mirror_maker/config/zookeeper_source_1.properties
deleted file mode 100644
index f851796..0000000
--- a/system_test/mirror_maker/config/zookeeper_source_1.properties
+++ /dev/null
@@ -1,18 +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.
-# the directory where the snapshot is stored.
-dataDir=/tmp/zookeeper_source-1
-# the port at which the clients will connect
-clientPort=2181

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/mirror_maker/config/zookeeper_source_2.properties
----------------------------------------------------------------------
diff --git a/system_test/mirror_maker/config/zookeeper_source_2.properties b/system_test/mirror_maker/config/zookeeper_source_2.properties
deleted file mode 100644
index d534d18..0000000
--- a/system_test/mirror_maker/config/zookeeper_source_2.properties
+++ /dev/null
@@ -1,18 +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.
-# the directory where the snapshot is stored.
-dataDir=/tmp/zookeeper_source-2
-# the port at which the clients will connect
-clientPort=2182

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/mirror_maker/config/zookeeper_target.properties
----------------------------------------------------------------------
diff --git a/system_test/mirror_maker/config/zookeeper_target.properties b/system_test/mirror_maker/config/zookeeper_target.properties
deleted file mode 100644
index 55a7eb1..0000000
--- a/system_test/mirror_maker/config/zookeeper_target.properties
+++ /dev/null
@@ -1,18 +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.
-# the directory where the snapshot is stored.
-dataDir=/tmp/zookeeper_target
-# the port at which the clients will connect
-clientPort=2183

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/mirror_maker_testsuite/mirror_maker_test.py
----------------------------------------------------------------------
diff --git a/system_test/mirror_maker_testsuite/mirror_maker_test.py b/system_test/mirror_maker_testsuite/mirror_maker_test.py
index fd18088..c0117c6 100644
--- a/system_test/mirror_maker_testsuite/mirror_maker_test.py
+++ b/system_test/mirror_maker_testsuite/mirror_maker_test.py
@@ -166,7 +166,7 @@ class MirrorMakerTest(ReplicationUtils, SetupUtils):
                 time.sleep(5)
 
                 self.log_message("creating topics")
-                kafka_system_test_utils.create_topic(self.systemTestEnv, self.testcaseEnv)
+                kafka_system_test_utils.create_topic_for_producer_performance(self.systemTestEnv, self.testcaseEnv)
                 self.anonLogger.info("sleeping for 5s")
                 time.sleep(5)
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/offset_management_testsuite/cluster_config.json
----------------------------------------------------------------------
diff --git a/system_test/offset_management_testsuite/cluster_config.json b/system_test/offset_management_testsuite/cluster_config.json
new file mode 100644
index 0000000..dcca200
--- /dev/null
+++ b/system_test/offset_management_testsuite/cluster_config.json
@@ -0,0 +1,103 @@
+{
+    "cluster_config": [
+        {
+            "entity_id": "0",
+            "hostname": "localhost",
+            "role": "zookeeper",
+            "cluster_name":"source",
+            "kafka_home": "default",
+            "java_home": "default",
+            "jmx_port": "9100"
+        },
+        {
+            "entity_id": "1",
+            "hostname": "localhost",
+            "role": "broker",
+            "cluster_name":"source",
+            "kafka_home": "default",
+            "java_home": "default",
+            "jmx_port": "9101"
+        },
+        {
+            "entity_id": "2",
+            "hostname": "localhost",
+            "role": "broker",
+            "cluster_name":"source",
+            "kafka_home": "default",
+            "java_home": "default",
+            "jmx_port": "9102"
+        },
+        {
+            "entity_id": "3",
+            "hostname": "localhost",
+            "role": "broker",
+            "cluster_name":"source",
+            "kafka_home": "default",
+            "java_home": "default",
+            "jmx_port": "9103"
+        },
+        {
+            "entity_id": "4",
+            "hostname": "localhost",
+            "role": "broker",
+            "cluster_name":"source",
+            "kafka_home": "default",
+            "java_home": "default",
+            "jmx_port": "9104"
+        },
+        {
+            "entity_id": "5",
+            "hostname": "localhost",
+            "role": "producer_performance",
+            "cluster_name":"source",
+            "kafka_home": "default",
+            "java_home": "default",
+            "jmx_port": "9105"
+        },
+        {
+            "entity_id": "6",
+            "hostname": "localhost",
+            "role": "console_consumer",
+            "cluster_name":"source",
+            "kafka_home": "default",
+            "java_home": "default",
+            "jmx_port": "9106"
+        },
+        {
+            "entity_id": "7",
+            "hostname": "localhost",
+            "role": "console_consumer",
+            "cluster_name":"source",
+            "kafka_home": "default",
+            "java_home": "default",
+            "jmx_port": "9107"
+        },
+        {
+            "entity_id": "8",
+            "hostname": "localhost",
+            "role": "console_consumer",
+            "cluster_name":"source",
+            "kafka_home": "default",
+            "java_home": "default",
+            "jmx_port": "9108"
+        },
+        {
+            "entity_id": "9",
+            "hostname": "localhost",
+            "role": "console_consumer",
+            "cluster_name":"source",
+            "kafka_home": "default",
+            "java_home": "default",
+            "jmx_port": "9109"
+        },
+        {
+            "entity_id": "10",
+            "hostname": "localhost",
+            "role": "console_consumer",
+            "cluster_name":"source",
+            "kafka_home": "default",
+            "java_home": "default",
+            "jmx_port": "9110"
+        }
+    ]
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/offset_management_testsuite/config/console_consumer.properties
----------------------------------------------------------------------
diff --git a/system_test/offset_management_testsuite/config/console_consumer.properties b/system_test/offset_management_testsuite/config/console_consumer.properties
new file mode 100644
index 0000000..a2ab8b9
--- /dev/null
+++ b/system_test/offset_management_testsuite/config/console_consumer.properties
@@ -0,0 +1,2 @@
+auto.offset.reset=smallest
+auto.commit.interval.ms=1000

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/offset_management_testsuite/config/producer_performance.properties
----------------------------------------------------------------------
diff --git a/system_test/offset_management_testsuite/config/producer_performance.properties b/system_test/offset_management_testsuite/config/producer_performance.properties
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/offset_management_testsuite/config/server.properties
----------------------------------------------------------------------
diff --git a/system_test/offset_management_testsuite/config/server.properties b/system_test/offset_management_testsuite/config/server.properties
new file mode 100644
index 0000000..2b988f8
--- /dev/null
+++ b/system_test/offset_management_testsuite/config/server.properties
@@ -0,0 +1,144 @@
+# 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.
+# see kafka.server.KafkaConfig for additional details and defaults
+
+############################# Server Basics #############################
+
+# The id of the broker. This must be set to a unique integer for each broker.
+broker.id=0
+
+# Hostname the broker will advertise to consumers. If not set, kafka will use the value returned
+# from InetAddress.getLocalHost().  If there are multiple interfaces getLocalHost
+# may not be what you want.
+#host.name=
+
+
+############################# Socket Server Settings #############################
+
+# The port the socket server listens on
+port=9091
+
+# The number of threads handling network requests
+num.network.threads=2
+ 
+# The number of threads doing disk I/O
+num.io.threads=2
+
+# The send buffer (SO_SNDBUF) used by the socket server
+socket.send.buffer.bytes=1048576
+
+# The receive buffer (SO_RCVBUF) used by the socket server
+socket.receive.buffer.bytes=1048576
+
+# The maximum size of a request that the socket server will accept (protection against OOM)
+socket.request.max.bytes=104857600
+
+
+############################# Log Basics #############################
+
+# The directory under which to store log files
+log.dir=/tmp/kafka_server_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=5
+
+# Overrides for for the default given by num.partitions on a per-topic basis
+#topic.partition.count.map=topic1:3, topic2:4
+
+############################# Log Flush Policy #############################
+
+# The following configurations control the flush of data to disk. This is the most
+# important performance knob in kafka.
+# There are a few important trade-offs here:
+#    1. Durability: Unflushed data is at greater risk of loss in the event of a crash.
+#    2. Latency: Data is not made available to consumers until it is flushed (which adds latency).
+#    3. Throughput: The flush is generally the most expensive operation. 
+# The settings below allow one to configure the flush policy to flush data after a period of time or
+# every N messages (or both). This can be done globally and overridden on a per-topic basis.
+
+# The number of messages to accept before forcing a flush of data to disk
+log.flush.interval.messages=10000
+
+# The maximum amount of time a message can sit in a log before we force a flush
+log.flush.interval.ms=1000
+
+# Per-topic overrides for log.flush.interval.ms
+#log.flush.intervals.ms.per.topic=topic1:1000, topic2:3000
+
+# The interval (in ms) at which logs are checked to see if they need to be flushed to disk.
+log.flush.scheduler.interval.ms=1000
+
+############################# Log Retention Policy #############################
+
+# The following configurations control the disposal of log segments. The policy can
+# be set to delete segments after a period of time, or after a given size has accumulated.
+# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens
+# from the end of the log.
+
+# The minimum age of a log file to be eligible for deletion
+log.retention.hours=168
+
+# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining
+# segments don't drop below log.retention.bytes.
+#log.retention.bytes=1073741824
+log.retention.bytes=-1
+
+# The maximum size of a log segment file. When this size is reached a new log segment will be created.
+#log.segment.size=536870912
+log.segment.bytes=102400
+
+# The interval at which log segments are checked to see if they can be deleted according 
+# to the retention policies
+log.cleanup.interval.mins=1
+
+############################# Zookeeper #############################
+
+# Enable connecting to zookeeper
+enable.zookeeper=true
+
+# Zk connection string (see zk docs for details).
+# This is a comma separated host:port pairs, each corresponding to a zk
+# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002".
+# You can also append an optional chroot string to the urls to specify the
+# root directory for all kafka znodes.
+zookeeper.connect=localhost:2181
+
+# Timeout in ms for connecting to zookeeper
+zookeeper.connection.timeout.ms=1000000
+
+monitoring.period.secs=1
+message.max.bytes=1000000
+queued.max.requests=500
+log.roll.hours=168
+log.index.size.max.bytes=10485760
+log.index.interval.bytes=4096
+auto.create.topics.enable=true
+controller.socket.timeout.ms=30000
+controller.message.queue.size=10
+default.replication.factor=1
+replica.lag.time.max.ms=10000
+replica.lag.max.messages=4000
+replica.socket.timeout.ms=30000
+replica.socket.receive.buffer.bytes=65536
+replica.fetch.max.bytes=1048576
+replica.fetch.wait.max.ms=500
+replica.fetch.min.bytes=4096
+num.replica.fetchers=1
+
+offsets.topic.num.partitions=2
+offsets.topic.replication.factor=4
+

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/offset_management_testsuite/config/zookeeper.properties
----------------------------------------------------------------------
diff --git a/system_test/offset_management_testsuite/config/zookeeper.properties b/system_test/offset_management_testsuite/config/zookeeper.properties
new file mode 100644
index 0000000..5474a72
--- /dev/null
+++ b/system_test/offset_management_testsuite/config/zookeeper.properties
@@ -0,0 +1,23 @@
+# 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.
+# the directory where the snapshot is stored.
+dataDir=/tmp/zookeeper
+# the port at which the clients will connect
+clientPort=2181
+# disable the per-ip limit on the number of connections since this is a non-production config
+maxClientCnxns=0
+syncLimit=5
+initLimit=10
+tickTime=2000

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/offset_management_testsuite/offset_management_test.py
----------------------------------------------------------------------
diff --git a/system_test/offset_management_testsuite/offset_management_test.py b/system_test/offset_management_testsuite/offset_management_test.py
new file mode 100644
index 0000000..12b5cd2
--- /dev/null
+++ b/system_test/offset_management_testsuite/offset_management_test.py
@@ -0,0 +1,298 @@
+# 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.
+#!/usr/bin/env python
+
+# ===================================
+# offset_management_test.py
+# ===================================
+
+import os
+import signal
+import sys
+import time
+import traceback
+
+from   system_test_env    import SystemTestEnv
+sys.path.append(SystemTestEnv.SYSTEM_TEST_UTIL_DIR)
+
+from   setup_utils        import SetupUtils
+from   replication_utils  import ReplicationUtils
+import system_test_utils
+from   testcase_env       import TestcaseEnv
+
+# product specific: Kafka
+import kafka_system_test_utils
+import metrics
+
+class OffsetManagementTest(ReplicationUtils, SetupUtils):
+
+    testModuleAbsPathName = os.path.realpath(__file__)
+    testSuiteAbsPathName  = os.path.abspath(os.path.dirname(testModuleAbsPathName))
+
+    def __init__(self, systemTestEnv):
+
+        # SystemTestEnv - provides cluster level environment settings
+        #     such as entity_id, hostname, kafka_home, java_home which
+        #     are available in a list of dictionary named 
+        #     "clusterEntityConfigDictList"
+        self.systemTestEnv = systemTestEnv
+
+        super(OffsetManagementTest, self).__init__(self)
+
+        # dict to pass user-defined attributes to logger argument: "extra"
+        d = {'name_of_class': self.__class__.__name__}
+
+    def signal_handler(self, signal, frame):
+        self.log_message("Interrupt detected - User pressed Ctrl+c")
+
+        # perform the necessary cleanup here when user presses Ctrl+c and it may be product specific
+        self.log_message("stopping all entities - please wait ...")
+        kafka_system_test_utils.stop_all_remote_running_processes(self.systemTestEnv, self.testcaseEnv)
+        sys.exit(1) 
+
+    def runTest(self):
+
+        # ======================================================================
+        # get all testcase directories under this testsuite
+        # ======================================================================
+        testCasePathNameList = system_test_utils.get_dir_paths_with_prefix(
+            self.testSuiteAbsPathName, SystemTestEnv.SYSTEM_TEST_CASE_PREFIX)
+        testCasePathNameList.sort()
+
+        replicationUtils = ReplicationUtils(self)
+
+        # =============================================================
+        # launch each testcase one by one: testcase_1, testcase_2, ...
+        # =============================================================
+        for testCasePathName in testCasePathNameList:
+   
+            skipThisTestCase = False
+
+            try: 
+                # ======================================================================
+                # A new instance of TestcaseEnv to keep track of this testcase's env vars
+                # and initialize some env vars as testCasePathName is available now
+                # ======================================================================
+                self.testcaseEnv = TestcaseEnv(self.systemTestEnv, self)
+                self.testcaseEnv.testSuiteBaseDir = self.testSuiteAbsPathName
+                self.testcaseEnv.initWithKnownTestCasePathName(testCasePathName)
+                self.testcaseEnv.testcaseArgumentsDict = self.testcaseEnv.testcaseNonEntityDataDict["testcase_args"]
+
+                # ======================================================================
+                # SKIP if this case is IN testcase_to_skip.json or NOT IN testcase_to_run.json
+                # ======================================================================
+                testcaseDirName = self.testcaseEnv.testcaseResultsDict["_test_case_name"]
+
+                if self.systemTestEnv.printTestDescriptionsOnly:
+                    self.testcaseEnv.printTestCaseDescription(testcaseDirName)
+                    continue
+                elif self.systemTestEnv.isTestCaseToSkip(self.__class__.__name__, testcaseDirName):
+                    self.log_message("Skipping : " + testcaseDirName)
+                    skipThisTestCase = True
+                    continue
+                else:
+                    self.testcaseEnv.printTestCaseDescription(testcaseDirName)
+                    system_test_utils.setup_remote_hosts_with_testcase_level_cluster_config(self.systemTestEnv, testCasePathName)
+
+                # ============================================================================== #
+                # ============================================================================== #
+                #                   Product Specific Testing Code Starts Here:                   #
+                # ============================================================================== #
+                # ============================================================================== #
+    
+                # initialize self.testcaseEnv with user-defined environment variables (product specific)
+                self.testcaseEnv.userDefinedEnvVarDict["stopBackgroundProducer"]    = False
+                self.testcaseEnv.userDefinedEnvVarDict["backgroundProducerStopped"] = False
+
+                # initialize signal handler
+                signal.signal(signal.SIGINT, self.signal_handler)
+
+                # TestcaseEnv.testcaseConfigsList initialized by reading testcase properties file:
+                #   system_test/<suite_name>_testsuite/testcase_<n>/testcase_<n>_properties.json
+                self.testcaseEnv.testcaseConfigsList = system_test_utils.get_json_list_data(
+                    self.testcaseEnv.testcasePropJsonPathName)
+                 
+                # clean up data directories specified in zookeeper.properties and kafka_server_<n>.properties
+                kafka_system_test_utils.cleanup_data_at_remote_hosts(self.systemTestEnv, self.testcaseEnv)
+
+                # create "LOCAL" log directories for metrics, dashboards for each entity under this testcase
+                # for collecting logs from remote machines
+                kafka_system_test_utils.generate_testcase_log_dirs(self.systemTestEnv, self.testcaseEnv)
+
+                # TestcaseEnv - initialize producer & consumer config / log file pathnames
+                kafka_system_test_utils.init_entity_props(self.systemTestEnv, self.testcaseEnv)
+
+                # generate remote hosts log/config dirs if not exist
+                kafka_system_test_utils.generate_testcase_log_dirs_in_remote_hosts(self.systemTestEnv, self.testcaseEnv)
+    
+                # generate properties files for zookeeper, kafka, producer, and consumer:
+                # 1. copy system_test/<suite_name>_testsuite/config/*.properties to 
+                #    system_test/<suite_name>_testsuite/testcase_<n>/config/
+                # 2. update all properties files in system_test/<suite_name>_testsuite/testcase_<n>/config
+                #    by overriding the settings specified in:
+                #    system_test/<suite_name>_testsuite/testcase_<n>/testcase_<n>_properties.json
+                kafka_system_test_utils.generate_overriden_props_files(self.testSuiteAbsPathName,
+                    self.testcaseEnv, self.systemTestEnv)
+               
+                # =============================================
+                # preparing all entities to start the test
+                # =============================================
+                self.log_message("starting zookeepers")
+                kafka_system_test_utils.start_zookeepers(self.systemTestEnv, self.testcaseEnv)
+                self.anonLogger.info("sleeping for 2s")
+                time.sleep(2)
+
+                self.log_message("starting brokers")
+                kafka_system_test_utils.start_brokers(self.systemTestEnv, self.testcaseEnv)
+                self.anonLogger.info("sleeping for 5s")
+                time.sleep(5)
+
+                self.log_message("creating offset topic")
+                kafka_system_test_utils.create_topic(self.systemTestEnv, self.testcaseEnv, "__consumer_offsets", 3, 2)
+                self.anonLogger.info("sleeping for 5s")
+                time.sleep(5)
+
+                # =============================================
+                # starting producer 
+                # =============================================
+                self.log_message("starting producer in the background")
+                kafka_system_test_utils.start_producer_performance(self.systemTestEnv, self.testcaseEnv, False)
+                msgProducingFreeTimeSec = self.testcaseEnv.testcaseArgumentsDict["message_producing_free_time_sec"]
+                self.anonLogger.info("sleeping for " + msgProducingFreeTimeSec + " sec to produce some messages")
+                time.sleep(int(msgProducingFreeTimeSec))
+
+                kafka_system_test_utils.start_console_consumers(self.systemTestEnv, self.testcaseEnv)
+
+                kafka_system_test_utils.get_leader_for(self.systemTestEnv, self.testcaseEnv, "__consumer_offsets", 0)
+
+                # =============================================
+                # A while-loop to bounce consumers as specified
+                # by "num_iterations" in testcase_n_properties.json
+                # =============================================
+                i = 1
+                numIterations = int(self.testcaseEnv.testcaseArgumentsDict["num_iteration"])
+                bouncedEntityDownTimeSec = 10
+                try:
+                    bouncedEntityDownTimeSec = int(self.testcaseEnv.testcaseArgumentsDict["bounced_entity_downtime_sec"])
+                except:
+                    pass
+
+                # group1 -> offsets partition 0 // has one consumer; eid: 6
+                # group2 -> offsets partition 1 // has four consumers; eid: 7, 8, 9, 10
+
+                offsets_0_leader_entity = kafka_system_test_utils.get_leader_for(self.systemTestEnv, self.testcaseEnv, "__consumer_offsets", 0)
+                offsets_1_leader_entity = kafka_system_test_utils.get_leader_for(self.systemTestEnv, self.testcaseEnv, "__consumer_offsets", 1)
+
+                while i <= numIterations:
+
+                    self.log_message("Iteration " + str(i) + " of " + str(numIterations))
+                    kafka_system_test_utils.stop_remote_entity(self.systemTestEnv, offsets_0_leader_entity, self.testcaseEnv.entityBrokerParentPidDict[offsets_0_leader_entity])
+                    kafka_system_test_utils.stop_remote_entity(self.systemTestEnv, offsets_1_leader_entity, self.testcaseEnv.entityBrokerParentPidDict[offsets_1_leader_entity])
+
+                    # =============================================
+                    # Bounce consumers if specified in testcase config
+                    # =============================================
+                    bounceConsumers = self.testcaseEnv.testcaseArgumentsDict["bounce_consumers"]
+                    self.log_message("bounce_consumers flag : " + bounceConsumers)
+                    if (bounceConsumers.lower() == "true"):
+
+                        clusterConfigList       = self.systemTestEnv.clusterEntityConfigDictList
+                        consumerEntityIdList    = system_test_utils.get_data_from_list_of_dicts( clusterConfigList, "role", "console_consumer", "entity_id")
+
+                        for stoppedConsumerEntityId in consumerEntityIdList:
+                            consumerPPID = self.testcaseEnv.entityConsoleConsumerParentPidDict[stoppedConsumerEntityId]
+                            self.log_message("stopping consumer: " + consumerPPID)
+                            kafka_system_test_utils.stop_remote_entity(self.systemTestEnv, stoppedConsumerEntityId, consumerPPID)
+
+                        self.anonLogger.info("sleeping for " + str(bouncedEntityDownTimeSec) + " sec")
+                        time.sleep(bouncedEntityDownTimeSec)
+                        # leaders would have changed during the above bounce.
+                        self.log_message("starting the previously terminated consumers.")
+                        for stoppedConsumerEntityId in consumerEntityIdList:
+                            # starting previously terminated consumer
+                            kafka_system_test_utils.start_console_consumers(self.systemTestEnv, self.testcaseEnv, stoppedConsumerEntityId)
+
+                        self.log_message("starting the previously terminated brokers")
+                        kafka_system_test_utils.start_entity_in_background(self.systemTestEnv, self.testcaseEnv, offsets_0_leader_entity)
+                        kafka_system_test_utils.start_entity_in_background(self.systemTestEnv, self.testcaseEnv, offsets_1_leader_entity)
+
+                    self.anonLogger.info("sleeping for 15s")
+                    time.sleep(15)
+                    i += 1
+                # while loop
+
+                # =============================================
+                # tell producer to stop
+                # =============================================
+                self.testcaseEnv.lock.acquire()
+                self.testcaseEnv.userDefinedEnvVarDict["stopBackgroundProducer"] = True
+                time.sleep(1)
+                self.testcaseEnv.lock.release()
+                time.sleep(1)
+
+                # =============================================
+                # wait for producer thread's update of
+                # "backgroundProducerStopped" to be "True"
+                # =============================================
+                while 1:
+                    self.testcaseEnv.lock.acquire()
+                    self.logger.info("status of backgroundProducerStopped : [" + \
+                        str(self.testcaseEnv.userDefinedEnvVarDict["backgroundProducerStopped"]) + "]", extra=self.d)
+                    if self.testcaseEnv.userDefinedEnvVarDict["backgroundProducerStopped"]:
+                        time.sleep(1)
+                        self.logger.info("all producer threads completed", extra=self.d)
+                        break
+                    time.sleep(1)
+                    self.testcaseEnv.lock.release()
+                    time.sleep(2)
+
+                self.anonLogger.info("sleeping for 15s")
+                time.sleep(15)
+
+                # =============================================
+                # this testcase is completed - stop all entities
+                # =============================================
+                self.log_message("stopping all entities")
+                for entityId, parentPid in self.testcaseEnv.entityBrokerParentPidDict.items():
+                    kafka_system_test_utils.stop_remote_entity(self.systemTestEnv, entityId, parentPid)
+
+                for entityId, parentPid in self.testcaseEnv.entityZkParentPidDict.items():
+                    kafka_system_test_utils.stop_remote_entity(self.systemTestEnv, entityId, parentPid)
+
+                # make sure all entities are stopped
+                kafka_system_test_utils.ps_grep_terminate_running_entity(self.systemTestEnv)
+
+                # =============================================
+                # collect logs from remote hosts
+                # =============================================
+                kafka_system_test_utils.collect_logs_from_remote_hosts(self.systemTestEnv, self.testcaseEnv)
+
+                # =============================================
+                # validate the data matched and checksum
+                # =============================================
+                self.log_message("validating data matched")
+                kafka_system_test_utils.validate_data_matched_in_multi_topics_from_single_consumer_producer(self.systemTestEnv, self.testcaseEnv, replicationUtils)
+
+            except Exception as e:
+                self.log_message("Exception while running test {0}".format(e))
+                traceback.print_exc()
+
+            finally:
+                if not skipThisTestCase and not self.systemTestEnv.printTestDescriptionsOnly:
+                    self.log_message("stopping all entities - please wait ...")
+                    kafka_system_test_utils.stop_all_remote_running_processes(self.systemTestEnv, self.testcaseEnv)
+

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json
----------------------------------------------------------------------
diff --git a/system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json b/system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json
new file mode 100644
index 0000000..02af3e8
--- /dev/null
+++ b/system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json
@@ -0,0 +1,95 @@
+{
+  "description": {"01":"To Test : 'Basic offset management test.'",
+                  "02":"Set up a Zk and Kafka cluster.",
+                  "03":"Produce messages to a multiple topics - various partition counts.",
+                  "04":"Start multiple consumer groups to read various subsets of above topics.",
+                  "05":"Bounce consumers.",
+                  "06":"Verify that there are no duplicate messages or lost messages on any consumer group.",
+                  "07":"Producer dimensions : mode:sync, acks:-1, comp:0"
+  },
+  "testcase_args": {
+    "bounce_leaders": "false",
+    "bounce_consumers": "true",
+    "replica_factor": "3",
+    "num_partition": "1",
+    "num_iteration": "1",
+    "sleep_seconds_between_producer_calls": "1",
+    "message_producing_free_time_sec": "15",
+    "num_messages_to_produce_per_producer_call": "50",
+    "num_topics_for_auto_generated_string":"1"
+  },
+  "entities": [
+    {
+      "entity_id": "0",
+      "clientPort": "2108",
+      "dataDir": "/tmp/zookeeper_0",
+      "log_filename": "zookeeper_0.log",
+      "config_filename": "zookeeper_0.properties"
+    },
+    {
+      "entity_id": "1",
+      "port": "9091",
+      "broker.id": "1",
+      "log.segment.bytes": "10240",
+      "log.dir": "/tmp/kafka_server_1_logs",
+      "default.replication.factor": "3",
+      "num.partitions": "5",
+      "log_filename": "kafka_server_1.log",
+      "config_filename": "kafka_server_1.properties"
+    },
+    {
+      "entity_id": "2",
+      "port": "9092",
+      "broker.id": "2",
+      "log.segment.bytes": "10240",
+      "log.dir": "/tmp/kafka_server_2_logs",
+      "default.replication.factor": "3",
+      "num.partitions": "5",
+      "log_filename": "kafka_server_2.log",
+      "config_filename": "kafka_server_2.properties"
+    },
+    {
+      "entity_id": "3",
+      "port": "9093",
+      "broker.id": "3",
+      "log.segment.bytes": "10240",
+      "log.dir": "/tmp/kafka_server_3_logs",
+      "default.replication.factor": "3",
+      "num.partitions": "5",
+      "log_filename": "kafka_server_3.log",
+      "config_filename": "kafka_server_3.properties"
+    },
+    {
+      "entity_id": "4",
+      "port": "9094",
+      "broker.id": "4",
+      "log.segment.bytes": "10240",
+      "log.dir": "/tmp/kafka_server_4_logs",
+      "default.replication.factor": "3",
+      "num.partitions": "5",
+      "log_filename": "kafka_server_4.log",
+      "config_filename": "kafka_server_4.properties"
+    },
+    {
+      "entity_id": "5",
+      "topic": "test",
+      "threads": "3",
+      "compression-codec": "0",
+      "message-size": "500",
+      "message": "1000",
+      "request-num-acks": "-1",
+      "sync":"true",
+      "producer-num-retries":"5",
+      "log_filename": "producer_performance_10.log",
+      "config_filename": "producer_performance_10.properties"
+    },
+    {
+      "entity_id": "6",
+      "topic": "test_0001",
+      "group.id": "group1",
+      "consumer-timeout-ms": "30000",
+      "log_filename": "console_consumer.log",
+      "config_filename": "console_consumer_6.properties"
+    }
+   ]
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties
----------------------------------------------------------------------
diff --git a/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties
new file mode 100644
index 0000000..41ec6e4
--- /dev/null
+++ b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties
@@ -0,0 +1,148 @@
+# 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.
+# see kafka.server.KafkaConfig for additional details and defaults
+
+############################# Server Basics #############################
+
+# The id of the broker. This must be set to a unique integer for each broker.
+broker.id=1
+
+# Hostname the broker will advertise to consumers. If not set, kafka will use the value returned
+# from InetAddress.getLocalHost().  If there are multiple interfaces getLocalHost
+# may not be what you want.
+#host.name=
+
+
+############################# Socket Server Settings #############################
+
+# The port the socket server listens on
+port=9091
+
+# The number of threads handling network requests
+num.network.threads=2
+ 
+# The number of threads doing disk I/O
+num.io.threads=2
+
+# The send buffer (SO_SNDBUF) used by the socket server
+socket.send.buffer.bytes=1048576
+
+# The receive buffer (SO_RCVBUF) used by the socket server
+socket.receive.buffer.bytes=1048576
+
+# The maximum size of a request that the socket server will accept (protection against OOM)
+socket.request.max.bytes=104857600
+
+
+############################# Log Basics #############################
+
+# The directory under which to store log files
+log.dir=/tmp/kafka_server_1_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=5
+
+# Overrides for for the default given by num.partitions on a per-topic basis
+#topic.partition.count.map=topic1:3, topic2:4
+
+############################# Log Flush Policy #############################
+
+# The following configurations control the flush of data to disk. This is the most
+# important performance knob in kafka.
+# There are a few important trade-offs here:
+#    1. Durability: Unflushed data is at greater risk of loss in the event of a crash.
+#    2. Latency: Data is not made available to consumers until it is flushed (which adds latency).
+#    3. Throughput: The flush is generally the most expensive operation. 
+# The settings below allow one to configure the flush policy to flush data after a period of time or
+# every N messages (or both). This can be done globally and overridden on a per-topic basis.
+
+# The number of messages to accept before forcing a flush of data to disk
+log.flush.interval.messages=10000
+
+# The maximum amount of time a message can sit in a log before we force a flush
+log.flush.interval.ms=1000
+
+# Per-topic overrides for log.flush.interval.ms
+#log.flush.intervals.ms.per.topic=topic1:1000, topic2:3000
+
+# The interval (in ms) at which logs are checked to see if they need to be flushed to disk.
+log.flush.scheduler.interval.ms=1000
+
+############################# Log Retention Policy #############################
+
+# The following configurations control the disposal of log segments. The policy can
+# be set to delete segments after a period of time, or after a given size has accumulated.
+# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens
+# from the end of the log.
+
+# The minimum age of a log file to be eligible for deletion
+log.retention.hours=168
+
+# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining
+# segments don't drop below log.retention.bytes.
+#log.retention.bytes=1073741824
+log.retention.bytes=-1
+
+# The maximum size of a log segment file. When this size is reached a new log segment will be created.
+#log.segment.size=536870912
+log.segment.bytes=10240
+
+# The interval at which log segments are checked to see if they can be deleted according 
+# to the retention policies
+log.cleanup.interval.mins=1
+
+############################# Zookeeper #############################
+
+# Enable connecting to zookeeper
+enable.zookeeper=true
+
+# Zk connection string (see zk docs for details).
+# This is a comma separated host:port pairs, each corresponding to a zk
+# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002".
+# You can also append an optional chroot string to the urls to specify the
+# root directory for all kafka znodes.
+zookeeper.connect=localhost:2108
+
+# Timeout in ms for connecting to zookeeper
+zookeeper.connection.timeout.ms=1000000
+
+monitoring.period.secs=1
+message.max.bytes=1000000
+queued.max.requests=500
+log.roll.hours=168
+log.index.size.max.bytes=10485760
+log.index.interval.bytes=4096
+auto.create.topics.enable=true
+controller.socket.timeout.ms=30000
+controller.message.queue.size=10
+default.replication.factor=3
+replica.lag.time.max.ms=10000
+replica.lag.max.messages=4000
+replica.socket.timeout.ms=30000
+replica.socket.receive.buffer.bytes=65536
+replica.fetch.max.bytes=1048576
+replica.fetch.wait.max.ms=500
+replica.fetch.min.bytes=4096
+num.replica.fetchers=1
+
+offsets.topic.num.partitions=2
+offsets.topic.replication.factor=4
+
+kafka.csv.metrics.dir=/home/jkoshy/Projects/kafka/system_test/offset_management_testsuite/testcase_7002/logs/broker-1/metrics
+kafka.csv.metrics.reporter.enabled=true
+kafka.metrics.polling.interval.secs=5
+kafka.metrics.reporters=kafka.metrics.KafkaCSVMetricsReporter

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties
----------------------------------------------------------------------
diff --git a/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties
new file mode 100644
index 0000000..727e237
--- /dev/null
+++ b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties
@@ -0,0 +1,148 @@
+# 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.
+# see kafka.server.KafkaConfig for additional details and defaults
+
+############################# Server Basics #############################
+
+# The id of the broker. This must be set to a unique integer for each broker.
+broker.id=2
+
+# Hostname the broker will advertise to consumers. If not set, kafka will use the value returned
+# from InetAddress.getLocalHost().  If there are multiple interfaces getLocalHost
+# may not be what you want.
+#host.name=
+
+
+############################# Socket Server Settings #############################
+
+# The port the socket server listens on
+port=9092
+
+# The number of threads handling network requests
+num.network.threads=2
+ 
+# The number of threads doing disk I/O
+num.io.threads=2
+
+# The send buffer (SO_SNDBUF) used by the socket server
+socket.send.buffer.bytes=1048576
+
+# The receive buffer (SO_RCVBUF) used by the socket server
+socket.receive.buffer.bytes=1048576
+
+# The maximum size of a request that the socket server will accept (protection against OOM)
+socket.request.max.bytes=104857600
+
+
+############################# Log Basics #############################
+
+# The directory under which to store log files
+log.dir=/tmp/kafka_server_2_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=5
+
+# Overrides for for the default given by num.partitions on a per-topic basis
+#topic.partition.count.map=topic1:3, topic2:4
+
+############################# Log Flush Policy #############################
+
+# The following configurations control the flush of data to disk. This is the most
+# important performance knob in kafka.
+# There are a few important trade-offs here:
+#    1. Durability: Unflushed data is at greater risk of loss in the event of a crash.
+#    2. Latency: Data is not made available to consumers until it is flushed (which adds latency).
+#    3. Throughput: The flush is generally the most expensive operation. 
+# The settings below allow one to configure the flush policy to flush data after a period of time or
+# every N messages (or both). This can be done globally and overridden on a per-topic basis.
+
+# The number of messages to accept before forcing a flush of data to disk
+log.flush.interval.messages=10000
+
+# The maximum amount of time a message can sit in a log before we force a flush
+log.flush.interval.ms=1000
+
+# Per-topic overrides for log.flush.interval.ms
+#log.flush.intervals.ms.per.topic=topic1:1000, topic2:3000
+
+# The interval (in ms) at which logs are checked to see if they need to be flushed to disk.
+log.flush.scheduler.interval.ms=1000
+
+############################# Log Retention Policy #############################
+
+# The following configurations control the disposal of log segments. The policy can
+# be set to delete segments after a period of time, or after a given size has accumulated.
+# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens
+# from the end of the log.
+
+# The minimum age of a log file to be eligible for deletion
+log.retention.hours=168
+
+# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining
+# segments don't drop below log.retention.bytes.
+#log.retention.bytes=1073741824
+log.retention.bytes=-1
+
+# The maximum size of a log segment file. When this size is reached a new log segment will be created.
+#log.segment.size=536870912
+log.segment.bytes=10240
+
+# The interval at which log segments are checked to see if they can be deleted according 
+# to the retention policies
+log.cleanup.interval.mins=1
+
+############################# Zookeeper #############################
+
+# Enable connecting to zookeeper
+enable.zookeeper=true
+
+# Zk connection string (see zk docs for details).
+# This is a comma separated host:port pairs, each corresponding to a zk
+# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002".
+# You can also append an optional chroot string to the urls to specify the
+# root directory for all kafka znodes.
+zookeeper.connect=localhost:2108
+
+# Timeout in ms for connecting to zookeeper
+zookeeper.connection.timeout.ms=1000000
+
+monitoring.period.secs=1
+message.max.bytes=1000000
+queued.max.requests=500
+log.roll.hours=168
+log.index.size.max.bytes=10485760
+log.index.interval.bytes=4096
+auto.create.topics.enable=true
+controller.socket.timeout.ms=30000
+controller.message.queue.size=10
+default.replication.factor=3
+replica.lag.time.max.ms=10000
+replica.lag.max.messages=4000
+replica.socket.timeout.ms=30000
+replica.socket.receive.buffer.bytes=65536
+replica.fetch.max.bytes=1048576
+replica.fetch.wait.max.ms=500
+replica.fetch.min.bytes=4096
+num.replica.fetchers=1
+
+offsets.topic.num.partitions=2
+offsets.topic.replication.factor=4
+
+kafka.csv.metrics.dir=/home/jkoshy/Projects/kafka/system_test/offset_management_testsuite/testcase_7002/logs/broker-2/metrics
+kafka.csv.metrics.reporter.enabled=true
+kafka.metrics.polling.interval.secs=5
+kafka.metrics.reporters=kafka.metrics.KafkaCSVMetricsReporter

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties
----------------------------------------------------------------------
diff --git a/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties
new file mode 100644
index 0000000..e6fbbe1
--- /dev/null
+++ b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties
@@ -0,0 +1,148 @@
+# 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.
+# see kafka.server.KafkaConfig for additional details and defaults
+
+############################# Server Basics #############################
+
+# The id of the broker. This must be set to a unique integer for each broker.
+broker.id=3
+
+# Hostname the broker will advertise to consumers. If not set, kafka will use the value returned
+# from InetAddress.getLocalHost().  If there are multiple interfaces getLocalHost
+# may not be what you want.
+#host.name=
+
+
+############################# Socket Server Settings #############################
+
+# The port the socket server listens on
+port=9093
+
+# The number of threads handling network requests
+num.network.threads=2
+ 
+# The number of threads doing disk I/O
+num.io.threads=2
+
+# The send buffer (SO_SNDBUF) used by the socket server
+socket.send.buffer.bytes=1048576
+
+# The receive buffer (SO_RCVBUF) used by the socket server
+socket.receive.buffer.bytes=1048576
+
+# The maximum size of a request that the socket server will accept (protection against OOM)
+socket.request.max.bytes=104857600
+
+
+############################# Log Basics #############################
+
+# The directory under which to store log files
+log.dir=/tmp/kafka_server_3_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=5
+
+# Overrides for for the default given by num.partitions on a per-topic basis
+#topic.partition.count.map=topic1:3, topic2:4
+
+############################# Log Flush Policy #############################
+
+# The following configurations control the flush of data to disk. This is the most
+# important performance knob in kafka.
+# There are a few important trade-offs here:
+#    1. Durability: Unflushed data is at greater risk of loss in the event of a crash.
+#    2. Latency: Data is not made available to consumers until it is flushed (which adds latency).
+#    3. Throughput: The flush is generally the most expensive operation. 
+# The settings below allow one to configure the flush policy to flush data after a period of time or
+# every N messages (or both). This can be done globally and overridden on a per-topic basis.
+
+# The number of messages to accept before forcing a flush of data to disk
+log.flush.interval.messages=10000
+
+# The maximum amount of time a message can sit in a log before we force a flush
+log.flush.interval.ms=1000
+
+# Per-topic overrides for log.flush.interval.ms
+#log.flush.intervals.ms.per.topic=topic1:1000, topic2:3000
+
+# The interval (in ms) at which logs are checked to see if they need to be flushed to disk.
+log.flush.scheduler.interval.ms=1000
+
+############################# Log Retention Policy #############################
+
+# The following configurations control the disposal of log segments. The policy can
+# be set to delete segments after a period of time, or after a given size has accumulated.
+# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens
+# from the end of the log.
+
+# The minimum age of a log file to be eligible for deletion
+log.retention.hours=168
+
+# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining
+# segments don't drop below log.retention.bytes.
+#log.retention.bytes=1073741824
+log.retention.bytes=-1
+
+# The maximum size of a log segment file. When this size is reached a new log segment will be created.
+#log.segment.size=536870912
+log.segment.bytes=10240
+
+# The interval at which log segments are checked to see if they can be deleted according 
+# to the retention policies
+log.cleanup.interval.mins=1
+
+############################# Zookeeper #############################
+
+# Enable connecting to zookeeper
+enable.zookeeper=true
+
+# Zk connection string (see zk docs for details).
+# This is a comma separated host:port pairs, each corresponding to a zk
+# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002".
+# You can also append an optional chroot string to the urls to specify the
+# root directory for all kafka znodes.
+zookeeper.connect=localhost:2108
+
+# Timeout in ms for connecting to zookeeper
+zookeeper.connection.timeout.ms=1000000
+
+monitoring.period.secs=1
+message.max.bytes=1000000
+queued.max.requests=500
+log.roll.hours=168
+log.index.size.max.bytes=10485760
+log.index.interval.bytes=4096
+auto.create.topics.enable=true
+controller.socket.timeout.ms=30000
+controller.message.queue.size=10
+default.replication.factor=3
+replica.lag.time.max.ms=10000
+replica.lag.max.messages=4000
+replica.socket.timeout.ms=30000
+replica.socket.receive.buffer.bytes=65536
+replica.fetch.max.bytes=1048576
+replica.fetch.wait.max.ms=500
+replica.fetch.min.bytes=4096
+num.replica.fetchers=1
+
+offsets.topic.num.partitions=2
+offsets.topic.replication.factor=4
+
+kafka.csv.metrics.dir=/home/jkoshy/Projects/kafka/system_test/offset_management_testsuite/testcase_7002/logs/broker-3/metrics
+kafka.csv.metrics.reporter.enabled=true
+kafka.metrics.polling.interval.secs=5
+kafka.metrics.reporters=kafka.metrics.KafkaCSVMetricsReporter

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties
----------------------------------------------------------------------
diff --git a/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties
new file mode 100644
index 0000000..fee65bc
--- /dev/null
+++ b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties
@@ -0,0 +1,148 @@
+# 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.
+# see kafka.server.KafkaConfig for additional details and defaults
+
+############################# Server Basics #############################
+
+# The id of the broker. This must be set to a unique integer for each broker.
+broker.id=4
+
+# Hostname the broker will advertise to consumers. If not set, kafka will use the value returned
+# from InetAddress.getLocalHost().  If there are multiple interfaces getLocalHost
+# may not be what you want.
+#host.name=
+
+
+############################# Socket Server Settings #############################
+
+# The port the socket server listens on
+port=9094
+
+# The number of threads handling network requests
+num.network.threads=2
+ 
+# The number of threads doing disk I/O
+num.io.threads=2
+
+# The send buffer (SO_SNDBUF) used by the socket server
+socket.send.buffer.bytes=1048576
+
+# The receive buffer (SO_RCVBUF) used by the socket server
+socket.receive.buffer.bytes=1048576
+
+# The maximum size of a request that the socket server will accept (protection against OOM)
+socket.request.max.bytes=104857600
+
+
+############################# Log Basics #############################
+
+# The directory under which to store log files
+log.dir=/tmp/kafka_server_4_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=5
+
+# Overrides for for the default given by num.partitions on a per-topic basis
+#topic.partition.count.map=topic1:3, topic2:4
+
+############################# Log Flush Policy #############################
+
+# The following configurations control the flush of data to disk. This is the most
+# important performance knob in kafka.
+# There are a few important trade-offs here:
+#    1. Durability: Unflushed data is at greater risk of loss in the event of a crash.
+#    2. Latency: Data is not made available to consumers until it is flushed (which adds latency).
+#    3. Throughput: The flush is generally the most expensive operation. 
+# The settings below allow one to configure the flush policy to flush data after a period of time or
+# every N messages (or both). This can be done globally and overridden on a per-topic basis.
+
+# The number of messages to accept before forcing a flush of data to disk
+log.flush.interval.messages=10000
+
+# The maximum amount of time a message can sit in a log before we force a flush
+log.flush.interval.ms=1000
+
+# Per-topic overrides for log.flush.interval.ms
+#log.flush.intervals.ms.per.topic=topic1:1000, topic2:3000
+
+# The interval (in ms) at which logs are checked to see if they need to be flushed to disk.
+log.flush.scheduler.interval.ms=1000
+
+############################# Log Retention Policy #############################
+
+# The following configurations control the disposal of log segments. The policy can
+# be set to delete segments after a period of time, or after a given size has accumulated.
+# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens
+# from the end of the log.
+
+# The minimum age of a log file to be eligible for deletion
+log.retention.hours=168
+
+# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining
+# segments don't drop below log.retention.bytes.
+#log.retention.bytes=1073741824
+log.retention.bytes=-1
+
+# The maximum size of a log segment file. When this size is reached a new log segment will be created.
+#log.segment.size=536870912
+log.segment.bytes=10240
+
+# The interval at which log segments are checked to see if they can be deleted according 
+# to the retention policies
+log.cleanup.interval.mins=1
+
+############################# Zookeeper #############################
+
+# Enable connecting to zookeeper
+enable.zookeeper=true
+
+# Zk connection string (see zk docs for details).
+# This is a comma separated host:port pairs, each corresponding to a zk
+# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002".
+# You can also append an optional chroot string to the urls to specify the
+# root directory for all kafka znodes.
+zookeeper.connect=localhost:2108
+
+# Timeout in ms for connecting to zookeeper
+zookeeper.connection.timeout.ms=1000000
+
+monitoring.period.secs=1
+message.max.bytes=1000000
+queued.max.requests=500
+log.roll.hours=168
+log.index.size.max.bytes=10485760
+log.index.interval.bytes=4096
+auto.create.topics.enable=true
+controller.socket.timeout.ms=30000
+controller.message.queue.size=10
+default.replication.factor=3
+replica.lag.time.max.ms=10000
+replica.lag.max.messages=4000
+replica.socket.timeout.ms=30000
+replica.socket.receive.buffer.bytes=65536
+replica.fetch.max.bytes=1048576
+replica.fetch.wait.max.ms=500
+replica.fetch.min.bytes=4096
+num.replica.fetchers=1
+
+offsets.topic.num.partitions=2
+offsets.topic.replication.factor=4
+
+kafka.csv.metrics.dir=/home/jkoshy/Projects/kafka/system_test/offset_management_testsuite/testcase_7002/logs/broker-4/metrics
+kafka.csv.metrics.reporter.enabled=true
+kafka.metrics.polling.interval.secs=5
+kafka.metrics.reporters=kafka.metrics.KafkaCSVMetricsReporter

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties
----------------------------------------------------------------------
diff --git a/system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties b/system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties
new file mode 100644
index 0000000..97c07b9
--- /dev/null
+++ b/system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties
@@ -0,0 +1,24 @@
+# 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.
+# the directory where the snapshot is stored.
+dataDir=/tmp/zookeeper_0
+# the port at which the clients will connect
+clientPort=2108
+# disable the per-ip limit on the number of connections since this is a non-production config
+maxClientCnxns=0
+syncLimit=5
+initLimit=10
+tickTime=2000
+server.1=localhost:2107:2109

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json
----------------------------------------------------------------------
diff --git a/system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json b/system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json
new file mode 100644
index 0000000..fdab69b
--- /dev/null
+++ b/system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json
@@ -0,0 +1,127 @@
+{
+  "description": {"01":"To Test : 'Basic offset management test.'",
+                  "02":"Set up a Zk and Kafka cluster.",
+                  "03":"Produce messages to a multiple topics - various partition counts.",
+                  "04":"Start multiple consumer groups to read various subsets of above topics.",
+                  "05":"Bounce consumers.",
+                  "06":"Verify that there are no duplicate messages or lost messages on any consumer group.",
+                  "07":"Producer dimensions : mode:sync, acks:-1, comp:0"
+  },
+  "testcase_args": {
+    "bounce_leaders": "false",
+    "bounce_consumers": "true",
+    "replica_factor": "3",
+    "num_partition": "1",
+    "num_iteration": "1",
+    "sleep_seconds_between_producer_calls": "1",
+    "message_producing_free_time_sec": "15",
+    "num_messages_to_produce_per_producer_call": "50",
+    "num_topics_for_auto_generated_string":"3"
+  },
+  "entities": [
+    {
+      "entity_id": "0",
+      "clientPort": "2108",
+      "dataDir": "/tmp/zookeeper_0",
+      "log_filename": "zookeeper_0.log",
+      "config_filename": "zookeeper_0.properties"
+    },
+    {
+      "entity_id": "1",
+      "port": "9091",
+      "broker.id": "1",
+      "log.segment.bytes": "10240",
+      "log.dir": "/tmp/kafka_server_1_logs",
+      "default.replication.factor": "3",
+      "num.partitions": "5",
+      "log_filename": "kafka_server_1.log",
+      "config_filename": "kafka_server_1.properties"
+    },
+    {
+      "entity_id": "2",
+      "port": "9092",
+      "broker.id": "2",
+      "log.segment.bytes": "10240",
+      "log.dir": "/tmp/kafka_server_2_logs",
+      "default.replication.factor": "3",
+      "num.partitions": "5",
+      "log_filename": "kafka_server_2.log",
+      "config_filename": "kafka_server_2.properties"
+    },
+    {
+      "entity_id": "3",
+      "port": "9093",
+      "broker.id": "3",
+      "log.segment.bytes": "10240",
+      "log.dir": "/tmp/kafka_server_3_logs",
+      "default.replication.factor": "3",
+      "num.partitions": "5",
+      "log_filename": "kafka_server_3.log",
+      "config_filename": "kafka_server_3.properties"
+    },
+    {
+      "entity_id": "4",
+      "port": "9094",
+      "broker.id": "4",
+      "log.segment.bytes": "10240",
+      "log.dir": "/tmp/kafka_server_4_logs",
+      "default.replication.factor": "3",
+      "num.partitions": "5",
+      "log_filename": "kafka_server_4.log",
+      "config_filename": "kafka_server_4.properties"
+    },
+    {
+      "entity_id": "5",
+      "topic": "test",
+      "threads": "5",
+      "compression-codec": "0",
+      "message-size": "500",
+      "message": "1000",
+      "request-num-acks": "-1",
+      "sync":"true",
+      "producer-num-retries":"5",
+      "log_filename": "producer_performance_10.log",
+      "config_filename": "producer_performance_10.properties"
+    },
+    {
+      "entity_id": "6",
+      "topic": "test_0001",
+      "group.id": "group1",
+      "consumer-timeout-ms": "30000",
+      "log_filename": "console_consumer.log",
+      "config_filename": "console_consumer_6.properties"
+    },
+    {
+      "entity_id": "7",
+      "topic": "test_0002",
+      "group.id": "group2",
+      "consumer-timeout-ms": "30000",
+      "log_filename": "console_consumer.log",
+      "config_filename": "console_consumer_7.properties"
+    },
+    {
+      "entity_id": "8",
+      "topic": "test_0002",
+      "group.id": "group2",
+      "consumer-timeout-ms": "30000",
+      "log_filename": "console_consumer.log",
+      "config_filename": "console_consumer_8.properties"
+    },
+    {
+      "entity_id": "9",
+      "topic": "test_0002",
+      "group.id": "group2",
+      "consumer-timeout-ms": "30000",
+      "log_filename": "console_consumer.log",
+      "config_filename": "console_consumer_9.properties"
+    },
+    {
+      "entity_id": "10",
+      "topic": "test_0003",
+      "group.id": "group2",
+      "consumer-timeout-ms": "30000",
+      "log_filename": "console_consumer.log",
+      "config_filename": "console_consumer_10.properties"
+    }
+   ]
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/replication_testsuite/replica_basic_test.py
----------------------------------------------------------------------
diff --git a/system_test/replication_testsuite/replica_basic_test.py b/system_test/replication_testsuite/replica_basic_test.py
index 5d3d93e..660006c 100644
--- a/system_test/replication_testsuite/replica_basic_test.py
+++ b/system_test/replication_testsuite/replica_basic_test.py
@@ -188,7 +188,7 @@ class ReplicaBasicTest(ReplicationUtils, SetupUtils):
 
                 if autoCreateTopic.lower() == "false":
                     self.log_message("creating topics")
-                    kafka_system_test_utils.create_topic(self.systemTestEnv, self.testcaseEnv)
+                    kafka_system_test_utils.create_topic_for_producer_performance(self.systemTestEnv, self.testcaseEnv)
                     self.anonLogger.info("sleeping for 5s")
                     time.sleep(5)
 


[5/5] git commit: KAFKA-1012; Consumer offset management in Kafka; patched by Tejas Patil and Joel Koshy; feedback and reviews from Neha Narkhede, Jun Rao, Guozhang Wang, Sriram Subramanian, Joe Stein, Chris Riccomini

Posted by jj...@apache.org.
KAFKA-1012; Consumer offset management in Kafka; patched by Tejas Patil and Joel Koshy; feedback and reviews from Neha Narkhede, Jun Rao, Guozhang Wang, Sriram Subramanian, Joe Stein, Chris Riccomini


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

Branch: refs/heads/trunk
Commit: a670537aa33732b15b56644d8ccc1681e16395f5
Parents: 84a3a9a
Author: Joel Koshy <jj...@gmail.com>
Authored: Fri Mar 14 15:14:33 2014 -0700
Committer: Joel Koshy <jj...@gmail.com>
Committed: Fri Mar 14 15:14:33 2014 -0700

----------------------------------------------------------------------
 .../main/scala/kafka/admin/TopicCommand.scala   |   7 +-
 .../kafka/api/ConsumerMetadataRequest.scala     |  79 +++
 .../kafka/api/ConsumerMetadataResponse.scala    |  57 +++
 .../scala/kafka/api/OffsetCommitRequest.scala   |  57 ++-
 .../scala/kafka/api/OffsetCommitResponse.scala  |  40 +-
 .../scala/kafka/api/OffsetFetchRequest.scala    |  20 +-
 core/src/main/scala/kafka/api/RequestKeys.scala |   4 +-
 .../main/scala/kafka/client/ClientUtils.scala   | 100 +++-
 .../main/scala/kafka/cluster/Partition.scala    |  28 +-
 ...nsumerCoordinatorNotAvailableException.scala |  22 +
 .../main/scala/kafka/common/ErrorMapping.scala  |   8 +-
 .../NotCoordinatorForConsumerException.scala    |  22 +
 .../kafka/common/OffsetMetadataAndError.scala   |  41 +-
 .../common/OffsetsLoadInProgressException.scala |  26 +
 core/src/main/scala/kafka/common/Topic.scala    |   4 +
 .../scala/kafka/consumer/ConsoleConsumer.scala  |  16 +-
 .../scala/kafka/consumer/ConsumerConfig.scala   |  39 ++
 .../kafka/consumer/ConsumerConnector.scala      |   2 +-
 .../scala/kafka/consumer/SimpleConsumer.scala   |  11 +-
 .../main/scala/kafka/consumer/TopicCount.scala  |  14 +-
 .../main/scala/kafka/consumer/TopicFilter.scala |  11 +-
 .../consumer/ZookeeperConsumerConnector.scala   | 300 +++++++++---
 .../kafka/controller/KafkaController.scala      |   4 +-
 .../javaapi/ConsumerMetadataResponse.scala      |  42 ++
 .../kafka/javaapi/OffsetCommitRequest.scala     |  13 +-
 .../kafka/javaapi/OffsetCommitResponse.scala    |   2 +-
 .../javaapi/consumer/ConsumerConnector.java     |   6 +-
 .../consumer/ZookeeperConsumerConnector.scala   |   8 +-
 .../main/scala/kafka/log/FileMessageSet.scala   |   4 +-
 .../scala/kafka/producer/ProducerConfig.scala   |  11 +-
 .../src/main/scala/kafka/server/KafkaApis.scala | 324 +++++++------
 .../main/scala/kafka/server/KafkaConfig.scala   |  44 +-
 .../main/scala/kafka/server/KafkaServer.scala   |  30 +-
 .../main/scala/kafka/server/OffsetManager.scala | 480 +++++++++++++++++++
 .../scala/kafka/server/ReplicaManager.scala     |  19 +-
 .../kafka/tools/ConsumerOffsetChecker.scala     |  93 +++-
 .../scala/kafka/tools/DumpLogSegments.scala     |   8 +-
 .../main/scala/kafka/tools/MirrorMaker.scala    |   1 -
 .../kafka/tools/ReplicaVerificationTool.scala   |   5 +-
 .../kafka/tools/VerifyConsumerRebalance.scala   |   4 +-
 .../kafka/utils/VerifiableProperties.scala      |  20 +
 core/src/main/scala/kafka/utils/ZkUtils.scala   |   4 +-
 .../scala/other/kafka/TestOffsetManager.scala   | 291 +++++++++++
 .../other/kafka/TestZKConsumerOffsets.scala     |  73 ---
 .../unit/kafka/admin/DeleteTopicTest.scala      |   7 -
 .../api/RequestResponseSerializationTest.scala  | 155 ++----
 .../unit/kafka/consumer/TopicFilterTest.scala   |  24 +-
 .../ZookeeperConsumerConnectorTest.scala        |   4 +-
 .../unit/kafka/server/OffsetCommitTest.scala    | 111 ++---
 .../unit/kafka/server/SimpleFetchTest.scala     |   9 +-
 .../migration_tool_test.py                      |   2 +-
 system_test/mirror_maker/README                 |  22 -
 system_test/mirror_maker/bin/expected.out       |  18 -
 system_test/mirror_maker/bin/run-test.sh        | 357 --------------
 .../config/blacklisttest.consumer.properties    |  28 --
 .../config/mirror_producer.properties           |  30 --
 .../config/server_source_1_1.properties         |  76 ---
 .../config/server_source_1_2.properties         |  76 ---
 .../config/server_source_2_1.properties         |  76 ---
 .../config/server_source_2_2.properties         |  76 ---
 .../config/server_target_1_1.properties         |  78 ---
 .../config/server_target_1_2.properties         |  78 ---
 .../config/whitelisttest_1.consumer.properties  |  28 --
 .../config/whitelisttest_2.consumer.properties  |  28 --
 .../config/zookeeper_source_1.properties        |  18 -
 .../config/zookeeper_source_2.properties        |  18 -
 .../config/zookeeper_target.properties          |  18 -
 .../mirror_maker_testsuite/mirror_maker_test.py |   2 +-
 .../cluster_config.json                         | 103 ++++
 .../config/console_consumer.properties          |   2 +
 .../config/producer_performance.properties      |   0
 .../config/server.properties                    | 144 ++++++
 .../config/zookeeper.properties                 |  23 +
 .../offset_management_test.py                   | 298 ++++++++++++
 .../testcase_7001/testcase_7001_properties.json |  95 ++++
 .../config/kafka_server_1.properties            | 148 ++++++
 .../config/kafka_server_2.properties            | 148 ++++++
 .../config/kafka_server_3.properties            | 148 ++++++
 .../config/kafka_server_4.properties            | 148 ++++++
 .../testcase_7002/config/zookeeper_0.properties |  24 +
 .../testcase_7002/testcase_7002_properties.json | 127 +++++
 .../replication_testsuite/replica_basic_test.py |   2 +-
 system_test/utils/kafka_system_test_utils.py    | 170 ++++++-
 system_test/utils/testcase_env.py               |   6 +
 84 files changed, 3642 insertions(+), 1677 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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 dc9b092..6fef9df 100644
--- a/core/src/main/scala/kafka/admin/TopicCommand.scala
+++ b/core/src/main/scala/kafka/admin/TopicCommand.scala
@@ -26,6 +26,8 @@ import scala.collection.JavaConversions._
 import kafka.cluster.Broker
 import kafka.log.LogConfig
 import kafka.consumer.Whitelist
+import kafka.server.OffsetManager
+
 
 object TopicCommand {
 
@@ -70,7 +72,7 @@ object TopicCommand {
     if (opts.options.has(opts.topicOpt)) {
       val topicsSpec = opts.options.valueOf(opts.topicOpt)
       val topicsFilter = new Whitelist(topicsSpec)
-      allTopics.filter(topicsFilter.isTopicAllowed)
+      allTopics.filter(topicsFilter.isTopicAllowed(_, excludeInternalTopics = false))
     } else
       allTopics
   }
@@ -104,6 +106,9 @@ object TopicCommand {
         println("Updated config for topic \"%s\".".format(topic))
       }
       if(opts.options.has(opts.partitionsOpt)) {
+        if (topic == OffsetManager.OffsetsTopicName) {
+          throw new IllegalArgumentException("The number of partitions for the offsets topic cannot be changed.")
+        }
         println("WARNING: If partitions are increased for a topic that has a key, the partition " +
           "logic or ordering of the messages will be affected")
         val nPartitions = opts.options.valueOf(opts.partitionsOpt).intValue

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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
new file mode 100644
index 0000000..dfad6e6
--- /dev/null
+++ b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    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.network.RequestChannel.Response
+import kafka.common.ErrorMapping
+
+object ConsumerMetadataRequest {
+  val CurrentVersion = 0.shortValue
+  val DefaultClientId = ""
+
+  def readFrom(buffer: ByteBuffer) = {
+    // envelope
+    val versionId = buffer.getShort
+    val correlationId = buffer.getInt
+    val clientId = ApiUtils.readShortString(buffer)
+
+    // request
+    val group = ApiUtils.readShortString(buffer)
+    ConsumerMetadataRequest(group, versionId, correlationId, clientId)
+  }
+
+}
+
+case class ConsumerMetadataRequest(group: String,
+                                   versionId: Short = ConsumerMetadataRequest.CurrentVersion,
+                                   override val correlationId: Int = 0,
+                                   clientId: String = ConsumerMetadataRequest.DefaultClientId)
+  extends RequestOrResponse(Some(RequestKeys.ConsumerMetadataKey), correlationId) {
+
+  def sizeInBytes =
+    2 + /* versionId */
+    4 + /* correlationId */
+    ApiUtils.shortStringLength(clientId) +
+    ApiUtils.shortStringLength(group)
+
+  def writeTo(buffer: ByteBuffer) {
+    // envelope
+    buffer.putShort(versionId)
+    buffer.putInt(correlationId)
+    ApiUtils.writeShortString(buffer, clientId)
+
+    // consumer metadata request
+    ApiUtils.writeShortString(buffer, group)
+  }
+
+  override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
+    // return ConsumerCoordinatorNotAvailable for all uncaught errors
+    val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode)
+    requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse)))
+  }
+
+  def describe(details: Boolean) = {
+    val consumerMetadataRequest = new StringBuilder
+    consumerMetadataRequest.append("Name: " + this.getClass.getSimpleName)
+    consumerMetadataRequest.append("; Version: " + versionId)
+    consumerMetadataRequest.append("; CorrelationId: " + correlationId)
+    consumerMetadataRequest.append("; ClientId: " + clientId)
+    consumerMetadataRequest.append("; Group: " + group)
+    consumerMetadataRequest.toString()
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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
new file mode 100644
index 0000000..6807f98
--- /dev/null
+++ b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package kafka.api
+
+import java.nio.ByteBuffer
+import kafka.cluster.Broker
+import kafka.common.ErrorMapping
+
+object ConsumerMetadataResponse {
+  val CurrentVersion = 0
+  
+  def readFrom(buffer: ByteBuffer) = {
+    val correlationId = buffer.getInt
+    val errorCode = buffer.getShort
+    val coordinatorOpt = if (errorCode == ErrorMapping.NoError)
+      Some(Broker.readFrom(buffer))
+    else
+      None
+
+    ConsumerMetadataResponse(coordinatorOpt, errorCode, correlationId)
+  }
+  
+}
+
+case class ConsumerMetadataResponse (coordinator: Option[Broker], errorCode: Short, override val correlationId: Int = 0)
+  extends RequestOrResponse(correlationId = correlationId) {
+
+  def sizeInBytes =
+    4 + /* correlationId */
+    2 + /* error code */
+    coordinator.map(_.sizeInBytes).getOrElse(0)
+
+  def writeTo(buffer: ByteBuffer) {
+    buffer.putInt(correlationId)
+    buffer.putShort(errorCode)
+    if (errorCode == ErrorMapping.NoError) {
+      coordinator.get.writeTo(buffer)
+    }
+  }
+
+  def describe(details: Boolean) = toString
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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 4d1fa5c..9f6956e 100644
--- a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala
+++ b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala
@@ -18,17 +18,20 @@
 package kafka.api
 
 import java.nio.ByteBuffer
-
 import kafka.api.ApiUtils._
-import kafka.utils.Logging
+import kafka.utils.{SystemTime, Logging}
 import kafka.network.{RequestChannel, BoundedByteBufferSend}
-import kafka.common.{ErrorMapping, TopicAndPartition, OffsetMetadataAndError}
+import kafka.common.{OffsetAndMetadata, ErrorMapping, TopicAndPartition}
 import kafka.network.RequestChannel.Response
+import scala.collection._
+
 object OffsetCommitRequest extends Logging {
   val CurrentVersion: Short = 0
   val DefaultClientId = ""
 
   def readFrom(buffer: ByteBuffer): OffsetCommitRequest = {
+    val now = SystemTime.milliseconds
+
     // Read values from the envelope
     val versionId = buffer.getShort
     val correlationId = buffer.getInt
@@ -43,23 +46,45 @@ object OffsetCommitRequest extends Logging {
       (1 to partitionCount).map(_ => {
         val partitionId = buffer.getInt
         val offset = buffer.getLong
+        val timestamp = {
+          val given = buffer.getLong
+          if (given == -1L) now else given
+        }
         val metadata = readShortString(buffer)
-        (TopicAndPartition(topic, partitionId), OffsetMetadataAndError(offset, metadata))
+        (TopicAndPartition(topic, partitionId), OffsetAndMetadata(offset, metadata, timestamp))
       })
     })
-    OffsetCommitRequest(consumerGroupId, Map(pairs:_*), versionId, correlationId, clientId)
+    OffsetCommitRequest(consumerGroupId, mutable.Map(pairs:_*), versionId, correlationId, clientId)
   }
 }
 
 case class OffsetCommitRequest(groupId: String,
-                               requestInfo: Map[TopicAndPartition, OffsetMetadataAndError],
+                               requestInfo: mutable.Map[TopicAndPartition, OffsetAndMetadata],
                                versionId: Short = OffsetCommitRequest.CurrentVersion,
                                override val correlationId: Int = 0,
                                clientId: String = OffsetCommitRequest.DefaultClientId)
     extends RequestOrResponse(Some(RequestKeys.OffsetCommitKey), correlationId) {
 
   lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic)
-  
+
+  def filterLargeMetadata(maxMetadataSize: Int) =
+    requestInfo.filter(info => info._2.metadata == null || info._2.metadata.length <= maxMetadataSize)
+
+  def responseFor(errorCode: Short, offsetMetadataMaxSize: Int) = {
+    val commitStatus = requestInfo.map {info =>
+      (info._1, if (info._2.metadata != null && info._2.metadata.length > offsetMetadataMaxSize)
+                  ErrorMapping.OffsetMetadataTooLargeCode
+                else if (errorCode == ErrorMapping.UnknownTopicOrPartitionCode)
+                  ErrorMapping.ConsumerCoordinatorNotAvailableCode
+                else if (errorCode == ErrorMapping.NotLeaderForPartitionCode)
+                  ErrorMapping.NotCoordinatorForConsumerCode
+                else
+                  errorCode)
+    }.toMap
+    OffsetCommitResponse(commitStatus, correlationId)
+  }
+
+
   def writeTo(buffer: ByteBuffer) {
     // Write envelope
     buffer.putShort(versionId)
@@ -73,9 +98,10 @@ case class OffsetCommitRequest(groupId: String,
       writeShortString(buffer, t1._1) // topic
       buffer.putInt(t1._2.size)       // number of partitions for this topic
       t1._2.foreach( t2 => {
-        buffer.putInt(t2._1.partition)  // partition
-        buffer.putLong(t2._2.offset)    // offset
-        writeShortString(buffer, t2._2.metadata) // metadata
+        buffer.putInt(t2._1.partition)
+        buffer.putLong(t2._2.offset)
+        buffer.putLong(t2._2.timestamp)
+        writeShortString(buffer, t2._2.metadata)
       })
     })
   }
@@ -95,15 +121,14 @@ case class OffsetCommitRequest(groupId: String,
         innerCount +
         4 /* partition */ +
         8 /* offset */ +
+        8 /* timestamp */ +
         shortStringLength(offsetAndMetadata._2.metadata)
       })
     })
 
   override  def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
-    val responseMap = requestInfo.map {
-      case (topicAndPartition, offset) => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
-    }.toMap
-    val errorResponse = OffsetCommitResponse(requestInfo=responseMap, correlationId=correlationId)
+    val errorCode = ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])
+    val errorResponse = responseFor(errorCode, Int.MaxValue)
     requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse)))
   }
 
@@ -119,7 +144,7 @@ case class OffsetCommitRequest(groupId: String,
     offsetCommitRequest.toString()
   }
 
-  override def toString(): String = {
-    describe(true)
+  override def toString = {
+    describe(details = true)
   }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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 9e1795f..4946e97 100644
--- a/core/src/main/scala/kafka/api/OffsetCommitResponse.scala
+++ b/core/src/main/scala/kafka/api/OffsetCommitResponse.scala
@@ -19,9 +19,8 @@ package kafka.api
 
 import java.nio.ByteBuffer
 
-import kafka.api.ApiUtils._
-import kafka.common.TopicAndPartition
 import kafka.utils.Logging
+import kafka.common.TopicAndPartition
 
 object OffsetCommitResponse extends Logging {
   val CurrentVersion: Short = 0
@@ -30,7 +29,7 @@ object OffsetCommitResponse extends Logging {
     val correlationId = buffer.getInt
     val topicCount = buffer.getInt
     val pairs = (1 to topicCount).flatMap(_ => {
-      val topic = readShortString(buffer)
+      val topic = ApiUtils.readShortString(buffer)
       val partitionCount = buffer.getInt
       (1 to partitionCount).map(_ => {
         val partitionId = buffer.getInt
@@ -42,37 +41,34 @@ object OffsetCommitResponse extends Logging {
   }
 }
 
-case class OffsetCommitResponse(requestInfo: Map[TopicAndPartition, Short],
+case class OffsetCommitResponse(commitStatus: Map[TopicAndPartition, Short],
                                override val correlationId: Int = 0)
     extends RequestOrResponse(correlationId=correlationId) {
 
-  lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic)
+  lazy val commitStatusGroupedByTopic = commitStatus.groupBy(_._1.topic)
 
   def writeTo(buffer: ByteBuffer) {
     buffer.putInt(correlationId)
-    buffer.putInt(requestInfoGroupedByTopic.size) // number of topics
-    requestInfoGroupedByTopic.foreach( t1 => { // topic -> Map[TopicAndPartition, Short]
-      writeShortString(buffer, t1._1) // topic
-      buffer.putInt(t1._2.size)       // number of partitions for this topic
-      t1._2.foreach( t2 => {  // TopicAndPartition -> Short
-        buffer.putInt(t2._1.partition)
-        buffer.putShort(t2._2)  //error
-      })
-    })
+    buffer.putInt(commitStatusGroupedByTopic.size)
+    commitStatusGroupedByTopic.foreach { case(topic, statusMap) =>
+      ApiUtils.writeShortString(buffer, topic)
+      buffer.putInt(statusMap.size) // partition count
+      statusMap.foreach { case(topicAndPartition, errorCode) =>
+        buffer.putInt(topicAndPartition.partition)
+        buffer.putShort(errorCode)
+      }
+    }
   }
 
   override def sizeInBytes = 
     4 + /* correlationId */
     4 + /* topic count */
-    requestInfoGroupedByTopic.foldLeft(0)((count, topicAndOffsets) => {
-      val (topic, offsets) = topicAndOffsets
+    commitStatusGroupedByTopic.foldLeft(0)((count, partitionStatusMap) => {
+      val (topic, partitionStatus) = partitionStatusMap
       count +
-      shortStringLength(topic) + /* topic */
-      4 + /* number of partitions */
-      offsets.size * (
-        4 + /* partition */
-        2 /* error */
-      )
+      ApiUtils.shortStringLength(topic) +
+      4 + /* partition count */
+      partitionStatus.size * ( 4 /* partition */  + 2 /* error code */)
     })
 
   override def describe(details: Boolean):String = { toString }

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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 7036532..a32f858 100644
--- a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala
+++ b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala
@@ -23,7 +23,7 @@ import kafka.api.ApiUtils._
 import kafka.utils.Logging
 import kafka.network.{BoundedByteBufferSend, RequestChannel}
 import kafka.network.RequestChannel.Response
-import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
+import kafka.common.{OffsetAndMetadata, ErrorMapping, OffsetMetadataAndError, TopicAndPartition}
 object OffsetFetchRequest extends Logging {
   val CurrentVersion: Short = 0
   val DefaultClientId = ""
@@ -50,10 +50,10 @@ object OffsetFetchRequest extends Logging {
 }
 
 case class OffsetFetchRequest(groupId: String,
-                               requestInfo: Seq[TopicAndPartition],
-                               versionId: Short = OffsetFetchRequest.CurrentVersion,
-                               override val correlationId: Int = 0,
-                               clientId: String = OffsetFetchRequest.DefaultClientId)
+                              requestInfo: Seq[TopicAndPartition],
+                              versionId: Short = OffsetFetchRequest.CurrentVersion,
+                              override val correlationId: Int = 0,
+                              clientId: String = OffsetFetchRequest.DefaultClientId)
     extends RequestOrResponse(Some(RequestKeys.OffsetFetchKey), correlationId) {
 
   lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_.topic)
@@ -91,8 +91,8 @@ case class OffsetFetchRequest(groupId: String,
   override  def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
     val responseMap = requestInfo.map {
       case (topicAndPartition) => (topicAndPartition, OffsetMetadataAndError(
-        offset=OffsetMetadataAndError.InvalidOffset,
-        error=ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])
+        offset = OffsetAndMetadata.InvalidOffset,
+        error = ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])
       ))
     }.toMap
     val errorResponse = OffsetFetchResponse(requestInfo=responseMap, correlationId=correlationId)
@@ -111,7 +111,7 @@ case class OffsetFetchRequest(groupId: String,
     offsetFetchRequest.toString()
   }
 
-  override def toString(): String = {
-    describe(true)
+  override def toString: String = {
+    describe(details = true)
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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 c81214f..fbfc9d3 100644
--- a/core/src/main/scala/kafka/api/RequestKeys.scala
+++ b/core/src/main/scala/kafka/api/RequestKeys.scala
@@ -31,6 +31,7 @@ object RequestKeys {
   val ControlledShutdownKey: Short = 7
   val OffsetCommitKey: Short = 8
   val OffsetFetchKey: Short = 9
+  val ConsumerMetadataKey: Short = 10
 
   val keyToNameAndDeserializerMap: Map[Short, (String, (ByteBuffer) => RequestOrResponse)]=
     Map(ProduceKey -> ("Produce", ProducerRequest.readFrom),
@@ -42,7 +43,8 @@ object RequestKeys {
         UpdateMetadataKey -> ("UpdateMetadata", UpdateMetadataRequest.readFrom),
         ControlledShutdownKey -> ("ControlledShutdown", ControlledShutdownRequest.readFrom),
         OffsetCommitKey -> ("OffsetCommit", OffsetCommitRequest.readFrom),
-        OffsetFetchKey -> ("OffsetFetch", OffsetFetchRequest.readFrom))
+        OffsetFetchKey -> ("OffsetFetch", OffsetFetchRequest.readFrom),
+        ConsumerMetadataKey -> ("ConsumerMetadata", ConsumerMetadataRequest.readFrom))
 
   def nameForKey(key: Short): String = {
     keyToNameAndDeserializerMap.get(key) match {

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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 1d2f81b..fc9e084 100644
--- a/core/src/main/scala/kafka/client/ClientUtils.scala
+++ b/core/src/main/scala/kafka/client/ClientUtils.scala
@@ -20,12 +20,16 @@ import scala.collection._
 import kafka.cluster._
 import kafka.api._
 import kafka.producer._
-import kafka.common.KafkaException
+import kafka.common.{ErrorMapping, KafkaException}
 import kafka.utils.{Utils, Logging}
 import java.util.Properties
 import util.Random
+ import kafka.network.BlockingChannel
+ import kafka.utils.ZkUtils._
+ import org.I0Itec.zkclient.ZkClient
+ import java.io.IOException
 
-/**
+ /**
  * Helper functions common to clients (producer, consumer, or admin)
  */
 object ClientUtils extends Logging{
@@ -103,5 +107,93 @@ object ClientUtils extends Logging{
       new Broker(brokerId, hostName, port)
     })
   }
-  
-}
\ No newline at end of file
+
+   /**
+    * Creates a blocking channel to a random broker
+    */
+   def channelToAnyBroker(zkClient: ZkClient, socketTimeoutMs: Int = 3000) : BlockingChannel = {
+     var channel: BlockingChannel = null
+     var connected = false
+     while (!connected) {
+       val allBrokers = getAllBrokersInCluster(zkClient)
+       Random.shuffle(allBrokers).find { broker =>
+         trace("Connecting to broker %s:%d.".format(broker.host, broker.port))
+         try {
+           channel = new BlockingChannel(broker.host, broker.port, BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, socketTimeoutMs)
+           channel.connect()
+           debug("Created channel to broker %s:%d.".format(channel.host, channel.port))
+           true
+         } catch {
+           case e: Exception =>
+             if (channel != null) channel.disconnect()
+             channel = null
+             info("Error while creating channel to %s:%d.".format(broker.host, broker.port))
+             false
+         }
+       }
+       connected = if (channel == null) false else true
+     }
+
+     channel
+   }
+
+   /**
+    * Creates a blocking channel to the offset manager of the given group
+    */
+   def channelToOffsetManager(group: String, zkClient: ZkClient, socketTimeoutMs: Int = 3000, retryBackOffMs: Int = 1000) = {
+     var queryChannel = channelToAnyBroker(zkClient)
+
+     var offsetManagerChannelOpt: Option[BlockingChannel] = None
+
+     while (!offsetManagerChannelOpt.isDefined) {
+
+       var coordinatorOpt: Option[Broker] = None
+
+       while (!coordinatorOpt.isDefined) {
+         try {
+           if (!queryChannel.isConnected)
+             queryChannel = channelToAnyBroker(zkClient)
+           debug("Querying %s:%d to locate offset manager for %s.".format(queryChannel.host, queryChannel.port, group))
+           queryChannel.send(ConsumerMetadataRequest(group))
+           val response = queryChannel.receive()
+           val consumerMetadataResponse =  ConsumerMetadataResponse.readFrom(response.buffer)
+           debug("Consumer metadata response: " + consumerMetadataResponse.toString)
+           if (consumerMetadataResponse.errorCode == ErrorMapping.NoError)
+             coordinatorOpt = consumerMetadataResponse.coordinator
+         }
+         catch {
+           case ioe: IOException =>
+             info("Failed to fetch consumer metadata from %s:%d.".format(queryChannel.host, queryChannel.port))
+             queryChannel.disconnect()
+         }
+       }
+
+       val coordinator = coordinatorOpt.get
+       if (coordinator.host == queryChannel.host && coordinator.port == queryChannel.port) {
+         offsetManagerChannelOpt = Some(queryChannel)
+       } else {
+         val connectString = "%s:%d".format(coordinator.host, coordinator.port)
+         var offsetManagerChannel: BlockingChannel = null
+         try {
+           debug("Connecting to offset manager %s.".format(connectString))
+           offsetManagerChannel = new BlockingChannel(coordinator.host, coordinator.port,
+                                                      BlockingChannel.UseDefaultBufferSize,
+                                                      BlockingChannel.UseDefaultBufferSize,
+                                                      socketTimeoutMs)
+           offsetManagerChannel.connect()
+           offsetManagerChannelOpt = Some(offsetManagerChannel)
+           queryChannel.disconnect()
+         }
+         catch {
+           case ioe: IOException => // offsets manager may have moved
+             info("Error while connecting to %s.".format(connectString))
+             if (offsetManagerChannel != null) offsetManagerChannel.disconnect()
+             Thread.sleep(retryBackOffMs)
+             offsetManagerChannelOpt = None // just in case someone decides to change shutdownChannel to not swallow exceptions
+         }
+       }
+     }
+
+     offsetManagerChannelOpt.get
+   }
+ }

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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 882b6da..0b88f14 100644
--- a/core/src/main/scala/kafka/cluster/Partition.scala
+++ b/core/src/main/scala/kafka/cluster/Partition.scala
@@ -22,7 +22,7 @@ import kafka.utils._
 import java.lang.Object
 import kafka.api.{PartitionStateInfo, LeaderAndIsr}
 import kafka.log.LogConfig
-import kafka.server.ReplicaManager
+import kafka.server.{OffsetManager, ReplicaManager}
 import com.yammer.metrics.core.Gauge
 import kafka.metrics.KafkaMetricsGroup
 import kafka.controller.KafkaController
@@ -165,7 +165,8 @@ class Partition(val topic: String,
    *  and setting the new leader and ISR
    */
   def makeLeader(controllerId: Int,
-                 partitionStateInfo: PartitionStateInfo, correlationId: Int): Boolean = {
+                 partitionStateInfo: PartitionStateInfo, correlationId: Int,
+                 offsetManager: OffsetManager): Boolean = {
     leaderIsrUpdateLock synchronized {
       val allReplicas = partitionStateInfo.allReplicas
       val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch
@@ -186,6 +187,8 @@ class Partition(val topic: String,
       leaderReplicaIdOpt = Some(localBrokerId)
       // we may need to increment high watermark since ISR could be down to 1
       maybeIncrementLeaderHW(getReplica().get)
+      if (topic == OffsetManager.OffsetsTopicName)
+        offsetManager.loadOffsetsFromLog(partitionId)
       true
     }
   }
@@ -196,7 +199,7 @@ class Partition(val topic: String,
    */
   def makeFollower(controllerId: Int,
                    partitionStateInfo: PartitionStateInfo,
-                   correlationId: Int): Boolean = {
+                   correlationId: Int, offsetManager: OffsetManager): Boolean = {
     leaderIsrUpdateLock synchronized {
       val allReplicas = partitionStateInfo.allReplicas
       val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch
@@ -212,12 +215,21 @@ class Partition(val topic: String,
       inSyncReplicas = Set.empty[Replica]
       leaderEpoch = leaderAndIsr.leaderEpoch
       zkVersion = leaderAndIsr.zkVersion
+      
+      leaderReplicaIdOpt.foreach { leaderReplica =>
+        if (topic == OffsetManager.OffsetsTopicName &&
+           /* if we are making a leader->follower transition */
+           leaderReplica == localBrokerId)
+          offsetManager.clearOffsetsInPartition(partitionId)
+      }
 
-      if (leaderReplicaIdOpt.isDefined && leaderReplicaIdOpt.get == newLeaderBrokerId)
-        return false;
-
-      leaderReplicaIdOpt = Some(newLeaderBrokerId)
-      true
+      if (leaderReplicaIdOpt.isDefined && leaderReplicaIdOpt.get == newLeaderBrokerId) {
+        false
+      }
+      else {
+        leaderReplicaIdOpt = Some(newLeaderBrokerId)
+        true
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala b/core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala
new file mode 100644
index 0000000..8e02d26
--- /dev/null
+++ b/core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala
@@ -0,0 +1,22 @@
+/**
+ * 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.common
+
+class ConsumerCoordinatorNotAvailableException(message: String) extends RuntimeException(message) {
+  def this() = this(null)
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/common/ErrorMapping.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/common/ErrorMapping.scala b/core/src/main/scala/kafka/common/ErrorMapping.scala
index b0b5dce..5559d26 100644
--- a/core/src/main/scala/kafka/common/ErrorMapping.scala
+++ b/core/src/main/scala/kafka/common/ErrorMapping.scala
@@ -43,6 +43,9 @@ object ErrorMapping {
   val StaleControllerEpochCode: Short = 11
   val OffsetMetadataTooLargeCode: Short = 12
   val StaleLeaderEpochCode: Short = 13
+  val OffsetsLoadInProgressCode: Short = 14
+  val ConsumerCoordinatorNotAvailableCode: Short = 15
+  val NotCoordinatorForConsumerCode: Short = 16
 
   private val exceptionToCode = 
     Map[Class[Throwable], Short](
@@ -57,7 +60,10 @@ object ErrorMapping {
       classOf[ReplicaNotAvailableException].asInstanceOf[Class[Throwable]] -> ReplicaNotAvailableCode,
       classOf[MessageSizeTooLargeException].asInstanceOf[Class[Throwable]] -> MessageSizeTooLargeCode,
       classOf[ControllerMovedException].asInstanceOf[Class[Throwable]] -> StaleControllerEpochCode,
-      classOf[OffsetMetadataTooLargeException].asInstanceOf[Class[Throwable]] -> OffsetMetadataTooLargeCode
+      classOf[OffsetMetadataTooLargeException].asInstanceOf[Class[Throwable]] -> OffsetMetadataTooLargeCode,
+      classOf[OffsetsLoadInProgressException].asInstanceOf[Class[Throwable]] -> OffsetsLoadInProgressCode,
+      classOf[ConsumerCoordinatorNotAvailableException].asInstanceOf[Class[Throwable]] -> ConsumerCoordinatorNotAvailableCode,
+      classOf[NotCoordinatorForConsumerException].asInstanceOf[Class[Throwable]] -> NotCoordinatorForConsumerCode
     ).withDefaultValue(UnknownCode)
   
   /* invert the mapping */

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala b/core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala
new file mode 100644
index 0000000..1eb74be
--- /dev/null
+++ b/core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala
@@ -0,0 +1,22 @@
+/**
+ * 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.common
+
+class NotCoordinatorForConsumerException(message: String) extends RuntimeException(message) {
+  def this() = this(null)
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala
index 59608a3..1586243 100644
--- a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala
+++ b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala
@@ -1,5 +1,3 @@
-package kafka.common
-
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -17,20 +15,41 @@ package kafka.common
  * limitations under the License.
  */
 
-/**
- * Convenience case class since (topic, partition) pairs are ubiquitous.
- */
-case class OffsetMetadataAndError(offset: Long, metadata: String = OffsetMetadataAndError.NoMetadata, error: Short = ErrorMapping.NoError) {
+package kafka.common
 
-  def this(tuple: (Long, String, Short)) = this(tuple._1, tuple._2, tuple._3)
+case class OffsetAndMetadata(offset: Long,
+                             metadata: String = OffsetAndMetadata.NoMetadata,
+                             timestamp: Long = -1L) {
+  override def toString = "OffsetAndMetadata[%d,%s%s]"
+                          .format(offset,
+                                  if (metadata != null && metadata.length > 0) metadata else "NO_METADATA",
+                                  if (timestamp == -1) "" else "," + timestamp.toString)
+}
 
-  def asTuple = (offset, metadata, error)
+object OffsetAndMetadata {
+  val InvalidOffset: Long = -1L
+  val NoMetadata: String = ""
+  val InvalidTime: Long = -1L
+}
+
+case class OffsetMetadataAndError(offset: Long,
+                                  metadata: String = OffsetAndMetadata.NoMetadata,
+                                  error: Short = ErrorMapping.NoError) {
+
+  def this(offsetMetadata: OffsetAndMetadata, error: Short) =
+    this(offsetMetadata.offset, offsetMetadata.metadata, error)
 
-  override def toString = "OffsetAndMetadata[%d,%s,%d]".format(offset, metadata, error)
+  def this(error: Short) =
+    this(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, error)
 
+  def asTuple = (offset, metadata, error)
+
+  override def toString = "OffsetMetadataAndError[%d,%s,%d]".format(offset, metadata, error)
 }
 
 object OffsetMetadataAndError {
-  val InvalidOffset: Long = -1L;
-  val NoMetadata: String = "";
+  val NoOffset = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.NoError)
+  val OffsetsLoading = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.OffsetsLoadInProgressCode)
+  val NotOffsetManagerForGroup = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.NotCoordinatorForConsumerCode)
 }
+

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala b/core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala
new file mode 100644
index 0000000..1c8e96e
--- /dev/null
+++ b/core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala
@@ -0,0 +1,26 @@
+/**
+ * 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.common
+
+/**
+ * Indicates that offsets are currently being loaded from disk into the cache so offset fetch requests cannot be satisfied.
+ */
+class OffsetsLoadInProgressException(message: String) extends RuntimeException(message) {
+  def this() = this(null)
+}
+

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/common/Topic.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/common/Topic.scala b/core/src/main/scala/kafka/common/Topic.scala
index c1b9f65..ad75978 100644
--- a/core/src/main/scala/kafka/common/Topic.scala
+++ b/core/src/main/scala/kafka/common/Topic.scala
@@ -18,12 +18,16 @@
 package kafka.common
 
 import util.matching.Regex
+import kafka.server.OffsetManager
+
 
 object Topic {
   val legalChars = "[a-zA-Z0-9\\._\\-]"
   private val maxNameLength = 255
   private val rgx = new Regex(legalChars + "+")
 
+  val InternalTopics = Set(OffsetManager.OffsetsTopicName)
+
   def validate(topic: String) {
     if (topic.length <= 0)
       throw new InvalidTopicException("topic name is illegal, can't be empty")

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala b/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala
index dc066c2..0f62819 100644
--- a/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala
+++ b/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala
@@ -123,7 +123,13 @@ object ConsoleConsumer extends Logging {
       .withRequiredArg
       .describedAs("metrics dictory")
       .ofType(classOf[java.lang.String])
-
+    val includeInternalTopicsOpt = parser.accepts("include-internal-topics", "Allow consuming internal topics.")
+    val offsetsStorageOpt = parser.accepts("offsets-storage", "Specify offsets storage backend (kafka/zookeeper).")
+            .withRequiredArg
+            .describedAs("Offsets storage method.")
+            .ofType(classOf[String])
+            .defaultsTo("zookeeper")
+    val dualCommitEnabledOpt = parser.accepts("dual-commit-enabled", "If offsets storage is kafka and this is set, then commit to zookeeper as well.")
 
     val options: OptionSet = tryParse(parser, args)
     CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt)
@@ -153,6 +159,7 @@ object ConsoleConsumer extends Logging {
       KafkaMetricsReporter.startReporters(verifiableProps)
     }
 
+    val offsetsStorage = options.valueOf(offsetsStorageOpt)
     val props = new Properties()
     props.put("group.id", options.valueOf(groupIdOpt))
     props.put("socket.receive.buffer.bytes", options.valueOf(socketBufferSizeOpt).toString)
@@ -166,6 +173,13 @@ object ConsoleConsumer extends Logging {
     props.put("zookeeper.connect", options.valueOf(zkConnectOpt))
     props.put("consumer.timeout.ms", options.valueOf(consumerTimeoutMsOpt).toString)
     props.put("refresh.leader.backoff.ms", options.valueOf(refreshMetadataBackoffMsOpt).toString)
+    props.put("offsets.storage", offsetsStorage)
+    if (options.has(includeInternalTopicsOpt))
+      props.put("exclude.internal.topics", "false")
+    if (options.has(dualCommitEnabledOpt))
+      props.put("dual.commit.enabled", "true")
+    else
+      props.put("dual.commit.enabled", "false")
 
     val config = new ConsumerConfig(props)
     val skipMessageOnError = if (options.has(skipMessageOnErrorOpt)) true else false

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala
index e6875d6..1cf2f62 100644
--- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala
+++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala
@@ -41,9 +41,14 @@ object ConsumerConfig extends Config {
   val MirrorTopicsWhitelist = ""
   val MirrorTopicsBlacklist = ""
   val MirrorConsumerNumThreads = 1
+  val OffsetsChannelBackoffMs = 1000
+  val OffsetsChannelSocketTimeoutMs = 10000
+  val OffsetsCommitMaxRetries = 5
+  val OffsetsStorage = "zookeeper"
 
   val MirrorTopicsWhitelistProp = "mirror.topics.whitelist"
   val MirrorTopicsBlacklistProp = "mirror.topics.blacklist"
+  val ExcludeInternalTopics = true
   val MirrorConsumerNumThreadsProp = "mirror.consumer.numthreads"
   val DefaultClientId = ""
 
@@ -51,6 +56,7 @@ object ConsumerConfig extends Config {
     validateClientId(config.clientId)
     validateGroupId(config.groupId)
     validateAutoOffsetReset(config.autoOffsetReset)
+    validateOffsetsStorage(config.offsetsStorage)
   }
 
   def validateClientId(clientId: String) {
@@ -69,6 +75,15 @@ object ConsumerConfig extends Config {
                                                  "Valid values are " + OffsetRequest.SmallestTimeString + " and " + OffsetRequest.LargestTimeString)
     }
   }
+
+  def validateOffsetsStorage(storage: String) {
+    storage match {
+      case "zookeeper" =>
+      case "kafka" =>
+      case _ => throw new InvalidConfigException("Wrong value " + storage + " of offsets.storage in consumer config; " +
+                                                 "Valid values are 'zookeeper' and 'kafka'")
+    }
+  }
 }
 
 class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig(props) {
@@ -122,6 +137,27 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig(
   /** backoff time to refresh the leader of a partition after it loses the current leader */
   val refreshLeaderBackoffMs = props.getInt("refresh.leader.backoff.ms", RefreshMetadataBackoffMs)
 
+  /** backoff time to reconnect the offsets channel or to retry offset fetches/commits */
+  val offsetsChannelBackoffMs = props.getInt("offsets.channel.backoff.ms", OffsetsChannelBackoffMs)
+  /** socket timeout to use when reading responses for Offset Fetch/Commit requests. This timeout will also be used for
+   *  the ConsumerMetdata requests that are used to query for the offset coordinator. */
+  val offsetsChannelSocketTimeoutMs = props.getInt("offsets.channel.socket.timeout.ms", OffsetsChannelSocketTimeoutMs)
+
+  /** Retry the offset commit up to this many times on failure. This retry count only applies to offset commits during
+    * shut-down. It does not apply to commits from the auto-commit thread. It also does not apply to attempts to query
+    * for the offset coordinator before committing offsets. i.e., if a consumer metadata request fails for any reason,
+    * it is retried and that retry does not count toward this limit. */
+  val offsetsCommitMaxRetries = props.getInt("offsets.commit.max.retries", OffsetsCommitMaxRetries)
+
+  /** Specify whether offsets should be committed to "zookeeper" (default) or "kafka" */
+  val offsetsStorage = props.getString("offsets.storage", OffsetsStorage).toLowerCase
+
+  /** If you are using "kafka" as offsets.storage, you can dual commit offsets to ZooKeeper (in addition to Kafka). This
+    * is required during migration from zookeeper-based offset storage to kafka-based offset storage. With respect to any
+    * given consumer group, it is safe to turn this off after all instances within that group have been migrated to
+    * the new jar that commits offsets to the broker (instead of directly to ZooKeeper). */
+  val dualCommitEnabled = props.getBoolean("dual.commit.enabled", if (offsetsStorage == "kafka") true else false)
+
   /* what to do if an offset is out of range.
      smallest : automatically reset the offset to the smallest offset
      largest : automatically reset the offset to the largest offset
@@ -136,6 +172,9 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig(
    */
   val clientId = props.getString("client.id", groupId)
 
+  /** Whether messages from internal topics (such as offsets) should be exposed to the consumer. */
+  val excludeInternalTopics = props.getBoolean("exclude.internal.topics", ExcludeInternalTopics)
+
   validate(this)
 }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/consumer/ConsumerConnector.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/consumer/ConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ConsumerConnector.scala
index 13c3f77..07677c1 100644
--- a/core/src/main/scala/kafka/consumer/ConsumerConnector.scala
+++ b/core/src/main/scala/kafka/consumer/ConsumerConnector.scala
@@ -70,7 +70,7 @@ trait ConsumerConnector {
   /**
    *  Commit the offsets of all broker partitions connected by this connector.
    */
-  def commitOffsets
+  def commitOffsets(retryOnFailure: Boolean = true)
   
   /**
    *  Shut down the connector

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala
index fa7caa7..0e64632 100644
--- a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala
+++ b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala
@@ -92,6 +92,11 @@ class SimpleConsumer(val host: String,
     TopicMetadataResponse.readFrom(response.buffer)
   }
 
+  def send(request: ConsumerMetadataRequest): ConsumerMetadataResponse = {
+    val response = sendRequest(request)
+    ConsumerMetadataResponse.readFrom(response.buffer)
+  }
+
   /**
    *  Fetch a set of messages from a topic.
    *
@@ -126,7 +131,11 @@ class SimpleConsumer(val host: String,
    * @param request a [[kafka.api.OffsetCommitRequest]] object.
    * @return a [[kafka.api.OffsetCommitResponse]] object.
    */
-  def commitOffsets(request: OffsetCommitRequest) = OffsetCommitResponse.readFrom(sendRequest(request).buffer)
+  def commitOffsets(request: OffsetCommitRequest) = {
+    // TODO: With KAFKA-1012, we have to first issue a ConsumerMetadataRequest and connect to the coordinator before
+    // we can commit offsets.
+    OffsetCommitResponse.readFrom(sendRequest(request).buffer)
+  }
 
   /**
    * Fetch offsets for a topic

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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 e332633..c793110 100644
--- a/core/src/main/scala/kafka/consumer/TopicCount.scala
+++ b/core/src/main/scala/kafka/consumer/TopicCount.scala
@@ -47,7 +47,7 @@ private[kafka] object TopicCount extends Logging {
   val blackListPattern = "black_list"
   val staticPattern = "static"
 
-  def constructTopicCount(group: String, consumerId: String, zkClient: ZkClient) : TopicCount = {
+  def constructTopicCount(group: String, consumerId: String, zkClient: ZkClient, excludeInternalTopics: Boolean) : TopicCount = {
     val dirs = new ZKGroupDirs(group)
     val topicCountString = ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId)._1
     var subscriptionPattern: String = null
@@ -85,15 +85,15 @@ private[kafka] object TopicCount extends Logging {
           new Whitelist(regex)
         else
           new Blacklist(regex)
-      new WildcardTopicCount(zkClient, consumerId, filter, numStreams)
+      new WildcardTopicCount(zkClient, consumerId, filter, numStreams, excludeInternalTopics)
     }
   }
 
   def constructTopicCount(consumerIdString: String, topicCount: Map[String, Int]) =
     new StaticTopicCount(consumerIdString, topicCount)
 
-  def constructTopicCount(consumerIdString: String, filter: TopicFilter, numStreams: Int, zkClient: ZkClient) =
-    new WildcardTopicCount(zkClient, consumerIdString, filter, numStreams)
+  def constructTopicCount(consumerIdString: String, filter: TopicFilter, numStreams: Int, zkClient: ZkClient, excludeInternalTopics: Boolean) =
+    new WildcardTopicCount(zkClient, consumerIdString, filter, numStreams, excludeInternalTopics)
 
 }
 
@@ -119,9 +119,11 @@ private[kafka] class StaticTopicCount(val consumerIdString: String,
 private[kafka] class WildcardTopicCount(zkClient: ZkClient,
                                         consumerIdString: String,
                                         topicFilter: TopicFilter,
-                                        numStreams: Int) extends TopicCount {
+                                        numStreams: Int,
+                                        excludeInternalTopics: Boolean) extends TopicCount {
   def getConsumerThreadIdsPerTopic = {
-    val wildcardTopics = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.BrokerTopicsPath).filter(topicFilter.isTopicAllowed(_))
+    val wildcardTopics = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.BrokerTopicsPath)
+                         .filter(topic => topicFilter.isTopicAllowed(topic, excludeInternalTopics))
     makeConsumerThreadIdsPerTopic(consumerIdString, Map(wildcardTopics.map((_, numStreams)): _*))
   }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/consumer/TopicFilter.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/consumer/TopicFilter.scala b/core/src/main/scala/kafka/consumer/TopicFilter.scala
index 4f20823..5a13540 100644
--- a/core/src/main/scala/kafka/consumer/TopicFilter.scala
+++ b/core/src/main/scala/kafka/consumer/TopicFilter.scala
@@ -20,6 +20,7 @@ package kafka.consumer
 
 import kafka.utils.Logging
 import java.util.regex.{PatternSyntaxException, Pattern}
+import kafka.common.Topic
 
 
 sealed abstract class TopicFilter(rawRegex: String) extends Logging {
@@ -41,12 +42,12 @@ sealed abstract class TopicFilter(rawRegex: String) extends Logging {
 
   override def toString = regex
 
-  def isTopicAllowed(topic: String): Boolean
+  def isTopicAllowed(topic: String, excludeInternalTopics: Boolean): Boolean
 }
 
 case class Whitelist(rawRegex: String) extends TopicFilter(rawRegex) {
-  override def isTopicAllowed(topic: String) = {
-    val allowed = topic.matches(regex)
+  override def isTopicAllowed(topic: String, excludeInternalTopics: Boolean) = {
+    val allowed = topic.matches(regex) && !(Topic.InternalTopics.contains(topic) && excludeInternalTopics)
 
     debug("%s %s".format(
       topic, if (allowed) "allowed" else "filtered"))
@@ -58,8 +59,8 @@ case class Whitelist(rawRegex: String) extends TopicFilter(rawRegex) {
 }
 
 case class Blacklist(rawRegex: String) extends TopicFilter(rawRegex) {
-  override def isTopicAllowed(topic: String) = {
-    val allowed = !topic.matches(regex)
+  override def isTopicAllowed(topic: String, excludeInternalTopics: Boolean) = {
+    val allowed = (!topic.matches(regex)) && !(Topic.InternalTopics.contains(topic) && excludeInternalTopics)
 
     debug("%s %s".format(
       topic, if (allowed) "allowed" else "filtered"))

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
index 703b2e2..9a3db90 100644
--- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
+++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
@@ -34,7 +34,11 @@ import kafka.utils.Utils.inLock
 import kafka.common._
 import com.yammer.metrics.core.Gauge
 import kafka.metrics._
+import kafka.network.BlockingChannel
+import kafka.client.ClientUtils
+import kafka.api._
 import scala.Some
+import kafka.common.TopicAndPartition
 
 
 /**
@@ -85,7 +89,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
   private var fetcher: Option[ConsumerFetcherManager] = None
   private var zkClient: ZkClient = null
   private var topicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]]
-  private var checkpointedOffsets = new Pool[TopicAndPartition, Long]
+  private val checkpointedOffsets = new Pool[TopicAndPartition, Long]
   private val topicThreadIdAndQueues = new Pool[(String,String), BlockingQueue[FetchedDataChunk]]
   private val scheduler = new KafkaScheduler(threads = 1, threadNamePrefix = "kafka-consumer-scheduler-")
   private val messageStreamCreated = new AtomicBoolean(false)
@@ -94,8 +98,15 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
   private var topicPartitionChangeListener: ZKTopicPartitionChangeListener = null
   private var loadBalancerListener: ZKRebalancerListener = null
 
+  private var offsetsChannel: BlockingChannel = null
+  private val offsetsChannelLock = new Object
+
   private var wildcardTopicWatcher: ZookeeperTopicEventWatcher = null
 
+  // useful for tracking migration of consumers to store offsets in kafka
+  private val kafkaCommitMeter = newMeter(config.clientId + "-KafkaCommitsPerSec", "commits", TimeUnit.SECONDS)
+  private val zkCommitMeter = newMeter(config.clientId + "-ZooKeeperCommitsPerSec", "commits", TimeUnit.SECONDS)
+
   val consumerIdString = {
     var consumerUuid : String = null
     config.consumerId match {
@@ -113,6 +124,8 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
 
   connectZk()
   createFetcher()
+  ensureOffsetManagerConnected()
+
   if (config.autoCommitEnable) {
     scheduler.startup
     info("starting auto committer every " + config.autoCommitIntervalMs + " ms")
@@ -156,12 +169,22 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
     zkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, ZKStringSerializer)
   }
 
+  // Blocks until the offset manager is located and a channel is established to it.
+  private def ensureOffsetManagerConnected() {
+    if (config.offsetsStorage == "kafka") {
+      if (offsetsChannel == null || !offsetsChannel.isConnected)
+        offsetsChannel = ClientUtils.channelToOffsetManager(config.groupId, zkClient, config.offsetsChannelSocketTimeoutMs, config.offsetsChannelBackoffMs)
+
+      debug("Connected to offset manager %s:%d.".format(offsetsChannel.host, offsetsChannel.port))
+    }
+  }
+
   def shutdown() {
-    rebalanceLock synchronized {
-      val canShutdown = isShuttingDown.compareAndSet(false, true);
-      if (canShutdown) {
-        info("ZKConsumerConnector shutting down")
+    val canShutdown = isShuttingDown.compareAndSet(false, true)
+    if (canShutdown) {
+      info("ZKConsumerConnector shutting down")
 
+      rebalanceLock synchronized {
         if (wildcardTopicWatcher != null)
           wildcardTopicWatcher.shutdown()
         try {
@@ -178,6 +201,8 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
             zkClient.close()
             zkClient = null
           }
+
+          if (offsetsChannel != null) offsetsChannel.disconnect()
         } catch {
           case e: Throwable =>
             fatal("error during consumer connector shutdown", e)
@@ -240,7 +265,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
   def autoCommit() {
     trace("auto committing")
     try {
-      commitOffsets()
+      commitOffsets(isAutoCommit = false)
     }
     catch {
       case t: Throwable =>
@@ -249,30 +274,184 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
     }
   }
 
-  def commitOffsets() {
-    if (zkClient == null) {
-      error("zk client is null. Cannot commit offsets")
-      return
+  def commitOffsetToZooKeeper(topicPartition: TopicAndPartition, offset: Long) {
+    val topicDirs = new ZKGroupTopicDirs(config.groupId, topicPartition.topic)
+    updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + topicPartition.partition, offset.toString)
+    zkCommitMeter.mark()
+  }
+
+  def commitOffsets(isAutoCommit: Boolean = true) {
+    var retriesRemaining = 1 + (if (isAutoCommit) config.offsetsCommitMaxRetries else 0) // no retries for commits from auto-commit
+    var done = false
+
+    while (!done) {
+      val committed = offsetsChannelLock synchronized { // committed when we receive either no error codes or only MetadataTooLarge errors
+        val offsetsToCommit = mutable.Map(topicRegistry.flatMap { case (topic, partitionTopicInfos) =>
+          partitionTopicInfos.filterNot { case (partition, info) =>
+            val newOffset = info.getConsumeOffset()
+            newOffset == checkpointedOffsets.get(TopicAndPartition(topic, info.partitionId))
+          }.map { case (partition, info) =>
+            TopicAndPartition(info.topic, info.partitionId) -> OffsetAndMetadata(info.getConsumeOffset())
+          }
+        }.toSeq:_*)
+
+        if (offsetsToCommit.size > 0) {
+          if (config.offsetsStorage == "zookeeper") {
+            offsetsToCommit.foreach { case(topicAndPartition, offsetAndMetadata) =>
+              commitOffsetToZooKeeper(topicAndPartition, offsetAndMetadata.offset)
+              checkpointedOffsets.put(topicAndPartition, offsetAndMetadata.offset)
+            }
+            true
+          } else {
+            val offsetCommitRequest = OffsetCommitRequest(config.groupId, offsetsToCommit, clientId = config.clientId)
+            ensureOffsetManagerConnected()
+            try {
+              kafkaCommitMeter.mark(offsetsToCommit.size)
+              offsetsChannel.send(offsetCommitRequest)
+              val offsetCommitResponse = OffsetCommitResponse.readFrom(offsetsChannel.receive().buffer)
+              trace("Offset commit response: %s.".format(offsetCommitResponse))
+
+              val (commitFailed, retryableIfFailed, shouldRefreshCoordinator, errorCount) = {
+                offsetCommitResponse.commitStatus.foldLeft(false, false, false, 0) { case(folded, (topicPartition, errorCode)) =>
+
+                  if (errorCode == ErrorMapping.NoError) {
+                    val offset = offsetsToCommit(topicPartition).offset
+                    checkpointedOffsets.put(topicPartition, offset)
+                    if (config.dualCommitEnabled) {
+                      commitOffsetToZooKeeper(topicPartition, offset)
+                    }
+                  }
+
+                  (folded._1 || // update commitFailed
+                     errorCode != ErrorMapping.NoError,
+
+                  folded._2 || // update retryableIfFailed - (only metadata too large is not retryable)
+                    (errorCode != ErrorMapping.NoError && errorCode != ErrorMapping.OffsetMetadataTooLargeCode),
+
+                  folded._3 || // update shouldRefreshCoordinator
+                    errorCode == ErrorMapping.NotCoordinatorForConsumerCode ||
+                    errorCode == ErrorMapping.ConsumerCoordinatorNotAvailableCode,
+
+                  // update error count
+                  folded._4 + (if (errorCode != ErrorMapping.NoError) 1 else 0))
+                }
+              }
+              debug(errorCount + " errors in offset commit response.")
+
+
+              if (shouldRefreshCoordinator) {
+                debug("Could not commit offsets (because offset coordinator has moved or is unavailable).")
+                offsetsChannel.disconnect()
+              }
+
+              if (commitFailed && retryableIfFailed)
+                false
+              else
+                true
+            }
+            catch {
+              case t: Throwable =>
+                error("Error while committing offsets.", t)
+                offsetsChannel.disconnect()
+                false
+            }
+          }
+        } else {
+          debug("No updates to offsets since last commit.")
+          true
+        }
+      }
+
+      done = if (isShuttingDown.get() && isAutoCommit) { // should not retry indefinitely if shutting down
+        retriesRemaining -= 1
+        retriesRemaining == 0 || committed
+      } else
+        true
+
+      if (!done) {
+        debug("Retrying offset commit in %d ms".format(config.offsetsChannelBackoffMs))
+        Thread.sleep(config.offsetsChannelBackoffMs)
+      }
+    }
+  }
+
+  private def fetchOffsetFromZooKeeper(topicPartition: TopicAndPartition) = {
+    val dirs = new ZKGroupTopicDirs(config.groupId, topicPartition.topic)
+    val offsetString = readDataMaybeNull(zkClient, dirs.consumerOffsetDir + "/" + topicPartition.partition)._1
+    offsetString match {
+      case Some(offsetStr) => (topicPartition, OffsetMetadataAndError(offsetStr.toLong, OffsetAndMetadata.NoMetadata, ErrorMapping.NoError))
+      case None => (topicPartition, OffsetMetadataAndError.NoOffset)
     }
-    for ((topic, infos) <- topicRegistry) {
-      val topicDirs = new ZKGroupTopicDirs(config.groupId, topic)
-      for (info <- infos.values) {
-        val newOffset = info.getConsumeOffset
-        if (newOffset != checkpointedOffsets.get(TopicAndPartition(topic, info.partitionId))) {
+  }
+
+  private def fetchOffsets(partitions: Seq[TopicAndPartition]) = {
+    if (partitions.isEmpty)
+      Some(OffsetFetchResponse(Map.empty))
+    else if (config.offsetsStorage == "zookeeper") {
+      val offsets = partitions.map(fetchOffsetFromZooKeeper)
+      Some(OffsetFetchResponse(immutable.Map(offsets:_*)))
+    } else {
+      val offsetFetchRequest = OffsetFetchRequest(groupId = config.groupId, requestInfo = partitions, clientId = config.clientId)
+
+      var offsetFetchResponseOpt: Option[OffsetFetchResponse] = None
+      while (!isShuttingDown.get && !offsetFetchResponseOpt.isDefined) {
+        offsetFetchResponseOpt = offsetsChannelLock synchronized {
+          ensureOffsetManagerConnected()
           try {
-            updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + info.partitionId, newOffset.toString)
-            checkpointedOffsets.put(TopicAndPartition(topic, info.partitionId), newOffset)
-          } catch {
-            case t: Throwable =>
-              // log it and let it go
-              warn("exception during commitOffsets",  t)
+            offsetsChannel.send(offsetFetchRequest)
+            val offsetFetchResponse = OffsetFetchResponse.readFrom(offsetsChannel.receive().buffer)
+            trace("Offset fetch response: %s.".format(offsetFetchResponse))
+
+            val (leaderChanged, loadInProgress) =
+              offsetFetchResponse.requestInfo.foldLeft(false, false) { case(folded, (topicPartition, offsetMetadataAndError)) =>
+                (folded._1 || (offsetMetadataAndError.error == ErrorMapping.NotCoordinatorForConsumerCode),
+                 folded._2 || (offsetMetadataAndError.error == ErrorMapping.OffsetsLoadInProgressCode))
+              }
+
+            if (leaderChanged) {
+              offsetsChannel.disconnect()
+              debug("Could not fetch offsets (because offset manager has moved).")
+              None // retry
+            }
+            else if (loadInProgress) {
+              debug("Could not fetch offsets (because offset cache is being loaded).")
+              None // retry
+            }
+            else {
+              if (config.dualCommitEnabled) {
+                // if dual-commit is enabled (i.e., if a consumer group is migrating offsets to kafka), then pick the
+                // maximum between offsets in zookeeper and kafka.
+                val kafkaOffsets = offsetFetchResponse.requestInfo
+                val mostRecentOffsets = kafkaOffsets.map { case (topicPartition, kafkaOffset) =>
+                  val zkOffset = fetchOffsetFromZooKeeper(topicPartition)._2.offset
+                  val mostRecentOffset = zkOffset.max(kafkaOffset.offset)
+                  (topicPartition, OffsetMetadataAndError(mostRecentOffset, kafkaOffset.metadata, ErrorMapping.NoError))
+                }
+                Some(OffsetFetchResponse(mostRecentOffsets))
+              }
+              else
+                Some(offsetFetchResponse)
+            }
+          }
+          catch {
+            case e: Exception =>
+              error("Error while fetching offsets from %s:%d.".format(offsetsChannel.host, offsetsChannel.port), e)
+              offsetsChannel.disconnect()
+              None // retry
           }
-          debug("Committed offset " + newOffset + " for topic " + info)
+        }
+
+        if (offsetFetchResponseOpt.isEmpty) {
+          debug("Retrying offset fetch in %d ms".format(config.offsetsChannelBackoffMs))
+          Thread.sleep(config.offsetsChannelBackoffMs)
         }
       }
+
+      offsetFetchResponseOpt
     }
   }
 
+
   class ZKSessionExpireListener(val dirs: ZKGroupDirs,
                                  val consumerIdString: String,
                                  val topicCount: TopicCount,
@@ -433,8 +612,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
     }
 
     private def rebalance(cluster: Cluster): Boolean = {
-      val myTopicThreadIdsMap = TopicCount.constructTopicCount(group, consumerIdString, zkClient).getConsumerThreadIdsPerTopic
-      val consumersPerTopicMap = getConsumersPerTopic(zkClient, group)
+      val myTopicThreadIdsMap = TopicCount.constructTopicCount(
+        group, consumerIdString, zkClient, config.excludeInternalTopics).getConsumerThreadIdsPerTopic
+      val consumersPerTopicMap = getConsumersPerTopic(zkClient, group, config.excludeInternalTopics)
       val brokers = getAllBrokersInCluster(zkClient)
       if (brokers.size == 0) {
         // This can happen in a rare case when there are no brokers available in the cluster when the consumer is started.
@@ -458,13 +638,12 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
 
         releasePartitionOwnership(topicRegistry)
 
-        var partitionOwnershipDecision = new collection.mutable.HashMap[(String, Int), String]()
+        var partitionOwnershipDecision = new collection.mutable.HashMap[TopicAndPartition, String]()
         val currentTopicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]]
 
         for ((topic, consumerThreadIdSet) <- myTopicThreadIdsMap) {
           currentTopicRegistry.put(topic, new Pool[Int, PartitionTopicInfo])
 
-          val topicDirs = new ZKGroupTopicDirs(group, topic)
           val curConsumers = consumersPerTopicMap.get(topic).get
           val curPartitions: Seq[Int] = partitionsPerTopicMap.get(topic).get
 
@@ -490,27 +669,42 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
               for (i <- startPart until startPart + nParts) {
                 val partition = curPartitions(i)
                 info(consumerThreadId + " attempting to claim partition " + partition)
-                addPartitionTopicInfo(currentTopicRegistry, topicDirs, partition, topic, consumerThreadId)
                 // record the partition ownership decision
-                partitionOwnershipDecision += ((topic, partition) -> consumerThreadId)
+                partitionOwnershipDecision += (TopicAndPartition(topic, partition) -> consumerThreadId)
               }
             }
           }
         }
 
-        /**
-         * move the partition ownership here, since that can be used to indicate a truly successful rebalancing attempt
-         * A rebalancing attempt is completed successfully only after the fetchers have been started correctly
-         */
-        if(reflectPartitionOwnershipDecision(partitionOwnershipDecision.toMap)) {
-          info("Updating the cache")
-          debug("Partitions per topic cache " + partitionsPerTopicMap)
-          debug("Consumers per topic cache " + consumersPerTopicMap)
-          topicRegistry = currentTopicRegistry
-          updateFetcher(cluster)
-          true
-        } else {
+        // fetch current offsets for all topic-partitions
+        val topicPartitions = partitionOwnershipDecision.keySet.toSeq
+        val offsetFetchResponseOpt = fetchOffsets(topicPartitions)
+
+        if (isShuttingDown.get || !offsetFetchResponseOpt.isDefined)
           false
+        else {
+          val offsetFetchResponse = offsetFetchResponseOpt.get
+          topicPartitions.foreach { topicAndPartition =>
+            val (topic, partition) = topicAndPartition.asTuple
+            val offset = offsetFetchResponse.requestInfo(topicAndPartition).offset
+            val threadId = partitionOwnershipDecision(topicAndPartition)
+            addPartitionTopicInfo(currentTopicRegistry, partition, topic, offset, threadId)
+          }
+
+          /**
+           * move the partition ownership here, since that can be used to indicate a truly successful rebalancing attempt
+           * A rebalancing attempt is completed successfully only after the fetchers have been started correctly
+           */
+          if(reflectPartitionOwnershipDecision(partitionOwnershipDecision.toMap)) {
+            info("Updating the cache")
+            debug("Partitions per topic cache " + partitionsPerTopicMap)
+            debug("Consumers per topic cache " + consumersPerTopicMap)
+            topicRegistry = currentTopicRegistry
+            updateFetcher(cluster)
+            true
+          } else {
+            false
+          }
         }
       }
     }
@@ -533,7 +727,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
           * successfully and the fetchers restart to fetch more data chunks
           **/
         if (config.autoCommitEnable)
-          commitOffsets
+          commitOffsets()
         case None =>
       }
     }
@@ -578,11 +772,11 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
       }
     }
 
-    private def reflectPartitionOwnershipDecision(partitionOwnershipDecision: Map[(String, Int), String]): Boolean = {
+    private def reflectPartitionOwnershipDecision(partitionOwnershipDecision: Map[TopicAndPartition, String]): Boolean = {
       var successfullyOwnedPartitions : List[(String, Int)] = Nil
       val partitionOwnershipSuccessful = partitionOwnershipDecision.map { partitionOwner =>
-        val topic = partitionOwner._1._1
-        val partition = partitionOwner._1._2
+        val topic = partitionOwner._1.topic
+        val partition = partitionOwner._1.partition
         val consumerThreadId = partitionOwner._2
         val partitionOwnerPath = getConsumerPartitionOwnerPath(group, topic, partition)
         try {
@@ -609,18 +803,10 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
     }
 
     private def addPartitionTopicInfo(currentTopicRegistry: Pool[String, Pool[Int, PartitionTopicInfo]],
-                                      topicDirs: ZKGroupTopicDirs, partition: Int,
-                                      topic: String, consumerThreadId: String) {
+                                      partition: Int, topic: String,
+                                      offset: Long, consumerThreadId: String) {
       val partTopicInfoMap = currentTopicRegistry.get(topic)
 
-      val znode = topicDirs.consumerOffsetDir + "/" + partition
-      val offsetString = readDataMaybeNull(zkClient, znode)._1
-      // If first time starting a consumer, set the initial offset to -1
-      val offset =
-        offsetString match {
-          case Some(offsetStr) => offsetStr.toLong
-          case None => PartitionTopicInfo.InvalidOffset
-        }
       val queue = topicThreadIdAndQueues.get((topic, consumerThreadId))
       val consumedOffset = new AtomicLong(offset)
       val fetchedOffset = new AtomicLong(offset)
@@ -746,10 +932,10 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
      // bootstrap with existing topics
     private var wildcardTopics =
       getChildrenParentMayNotExist(zkClient, BrokerTopicsPath)
-        .filter(topicFilter.isTopicAllowed)
+        .filter(topic => topicFilter.isTopicAllowed(topic, config.excludeInternalTopics))
 
     private val wildcardTopicCount = TopicCount.constructTopicCount(
-      consumerIdString, topicFilter, numStreams, zkClient)
+      consumerIdString, topicFilter, numStreams, zkClient, config.excludeInternalTopics)
 
     val dirs = new ZKGroupDirs(config.groupId)
     registerConsumerInZK(dirs, consumerIdString, wildcardTopicCount)
@@ -764,7 +950,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
     def handleTopicEvent(allTopics: Seq[String]) {
       debug("Handling topic event")
 
-      val updatedTopics = allTopics.filter(topicFilter.isTopicAllowed)
+      val updatedTopics = allTopics.filter(topic => topicFilter.isTopicAllowed(topic, config.excludeInternalTopics))
 
       val addedTopics = updatedTopics filterNot (wildcardTopics contains)
       if (addedTopics.nonEmpty)

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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 4deff9d..5db24a7 100644
--- a/core/src/main/scala/kafka/controller/KafkaController.scala
+++ b/core/src/main/scala/kafka/controller/KafkaController.scala
@@ -604,7 +604,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg
     }
   }
 
-  def onPreferredReplicaElection(partitions: Set[TopicAndPartition], isTriggeredByAutoRebalance: Boolean = true) {
+  def onPreferredReplicaElection(partitions: Set[TopicAndPartition], isTriggeredByAutoRebalance: Boolean = false) {
     info("Starting preferred replica leader election for partitions %s".format(partitions.mkString(",")))
     try {
       controllerContext.partitionsUndergoingPreferredReplicaElection ++= partitions
@@ -1116,7 +1116,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg
                       controllerContext.partitionsUndergoingPreferredReplicaElection.size == 0 &&
                       !deleteTopicManager.isTopicQueuedUpForDeletion(topicPartition.topic) &&
                       controllerContext.allTopics.contains(topicPartition.topic)) {
-                    onPreferredReplicaElection(Set(topicPartition), false)
+                    onPreferredReplicaElection(Set(topicPartition), true)
                   }
                 }
               }

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala
new file mode 100644
index 0000000..dfa9c42
--- /dev/null
+++ b/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala
@@ -0,0 +1,42 @@
+/**
+ * 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.javaapi
+
+import kafka.cluster.Broker
+
+class ConsumerMetadataResponse(private val underlying: kafka.api.ConsumerMetadataResponse) {
+
+  def errorCode = underlying.errorCode
+
+  def coordinator: Broker = {
+    import kafka.javaapi.Implicits._
+    underlying.coordinator
+  }
+
+  override def equals(other: Any) = canEqual(other) && {
+    val otherConsumerMetadataResponse = other.asInstanceOf[kafka.javaapi.ConsumerMetadataResponse]
+    this.underlying.equals(otherConsumerMetadataResponse.underlying)
+  }
+
+  def canEqual(other: Any) = other.isInstanceOf[kafka.javaapi.ConsumerMetadataResponse]
+
+  override def hashCode = underlying.hashCode
+
+  override def toString = underlying.toString
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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 57b9d2a..6de320d 100644
--- a/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala
+++ b/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala
@@ -17,19 +17,18 @@
 
 package kafka.javaapi
 
-import kafka.common.{TopicAndPartition, OffsetMetadataAndError}
-import collection.JavaConversions
-import java.nio.ByteBuffer
+import kafka.common.{OffsetAndMetadata, TopicAndPartition}
 
 class OffsetCommitRequest(groupId: String,
-                          requestInfo: java.util.Map[TopicAndPartition, OffsetMetadataAndError],
+                          requestInfo: java.util.Map[TopicAndPartition, OffsetAndMetadata],
                           versionId: Short,
                           correlationId: Int,
                           clientId: String) {
   val underlying = {
-    val scalaMap: Map[TopicAndPartition, OffsetMetadataAndError] = {
-      import JavaConversions._
-      requestInfo.toMap
+    val scalaMap: collection.mutable.Map[TopicAndPartition, OffsetAndMetadata] = {
+      import collection.JavaConversions._
+
+      collection.JavaConversions.asMap(requestInfo)
     }
     kafka.api.OffsetCommitRequest(
       groupId = groupId,

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala b/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala
index 570bf31..c2d3d11 100644
--- a/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala
+++ b/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala
@@ -24,7 +24,7 @@ class OffsetCommitResponse(private val underlying: kafka.api.OffsetCommitRespons
 
   def errors: java.util.Map[TopicAndPartition, Short] = {
     import JavaConversions._
-    underlying.requestInfo
+    underlying.commitStatus
   }
 
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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 c45c803..44d3d35 100644
--- a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java
+++ b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java
@@ -18,13 +18,12 @@
 package kafka.javaapi.consumer;
 
 
+import java.util.List;
+import java.util.Map;
 import kafka.consumer.KafkaStream;
 import kafka.consumer.TopicFilter;
 import kafka.serializer.Decoder;
 
-import java.util.List;
-import java.util.Map;
-
 public interface ConsumerConnector {
   /**
    *  Create a list of MessageStreams of type T for each topic.
@@ -62,6 +61,7 @@ public interface ConsumerConnector {
    *  Commit the offsets of all broker partitions connected by this connector.
    */
   public void commitOffsets();
+  public void commitOffsets(boolean retryOnFailure);
 
   /**
    *  Shut down the connector

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala
index 58e83f6..1f95d9b 100644
--- a/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala
+++ b/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala
@@ -101,9 +101,13 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
     
   def createMessageStreamsByFilter(topicFilter: TopicFilter) = 
     createMessageStreamsByFilter(topicFilter, 1, new DefaultDecoder(), new DefaultDecoder())
-    
+
   def commitOffsets() {
-    underlying.commitOffsets
+    underlying.commitOffsets()
+  }
+
+  def commitOffsets(retryOnFailure: Boolean) {
+    underlying.commitOffsets(retryOnFailure)
   }
 
   def shutdown() {

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/main/scala/kafka/log/FileMessageSet.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/FileMessageSet.scala b/core/src/main/scala/kafka/log/FileMessageSet.scala
index e1f8b97..b2652dd 100644
--- a/core/src/main/scala/kafka/log/FileMessageSet.scala
+++ b/core/src/main/scala/kafka/log/FileMessageSet.scala
@@ -255,8 +255,8 @@ class FileMessageSet private[kafka](@volatile var file: File,
   /**
    * Read from the underlying file into the buffer starting at the given position
    */
-  def readInto(buffer: ByteBuffer, position: Int): ByteBuffer = {
-    channel.read(buffer, position)
+  def readInto(buffer: ByteBuffer, relativePosition: Int): ByteBuffer = {
+    channel.read(buffer, relativePosition + this.start)
     buffer.flip()
     buffer
   }


[3/5] KAFKA-1012; Consumer offset management in Kafka; patched by Tejas Patil and Joel Koshy; feedback and reviews from Neha Narkhede, Jun Rao, Guozhang Wang, Sriram Subramanian, Joe Stein, Chris Riccomini

Posted by jj...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/test/scala/other/kafka/TestOffsetManager.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/other/kafka/TestOffsetManager.scala b/core/src/test/scala/other/kafka/TestOffsetManager.scala
new file mode 100644
index 0000000..83317f0
--- /dev/null
+++ b/core/src/test/scala/other/kafka/TestOffsetManager.scala
@@ -0,0 +1,291 @@
+package other.kafka
+
+import org.I0Itec.zkclient.ZkClient
+import kafka.api._
+import kafka.utils.{ShutdownableThread, ZKStringSerializer}
+import scala.collection._
+import kafka.client.ClientUtils
+import joptsimple.OptionParser
+import kafka.common.{ErrorMapping, OffsetAndMetadata, TopicAndPartition}
+import kafka.network.BlockingChannel
+import scala.util.Random
+import java.io.IOException
+import kafka.metrics.{KafkaTimer, KafkaMetricsGroup}
+import java.util.concurrent.TimeUnit
+import com.yammer.metrics.core.Gauge
+import java.util.concurrent.atomic.AtomicInteger
+import java.nio.channels.ClosedByInterruptException
+
+
+object TestOffsetManager {
+
+  val random = new Random
+  val SocketTimeoutMs = 10000
+
+  class StatsThread(reportingIntervalMs: Long, commitThreads: Seq[CommitThread], fetchThread: FetchThread)
+        extends ShutdownableThread("stats-thread") {
+
+    def printStats() {
+      println("--------------------------------------------------------------------------------")
+      println("Aggregate stats for commits:")
+      println("Error count: %d; Max:%f; Min: %f; Mean: %f; Commit count: %d".format(
+        commitThreads.map(_.numErrors.get).sum,
+        commitThreads.map(_.timer.max()).max,
+        commitThreads.map(_.timer.min()).min,
+        commitThreads.map(_.timer.mean()).sum / commitThreads.size,
+        commitThreads.map(_.numCommits.get).sum))
+      println("--------------------------------------------------------------------------------")
+      commitThreads.foreach(t => println(t.stats))
+      println(fetchThread.stats)
+    }
+
+    override def doWork() {
+      printStats()
+      Thread.sleep(reportingIntervalMs)
+    }
+
+  }
+
+  class CommitThread(id: Int, partitionCount: Int, commitIntervalMs: Long, zkClient: ZkClient)
+        extends ShutdownableThread("commit-thread")
+        with KafkaMetricsGroup {
+
+    private val group = "group-" + id
+    private val metadata = "Metadata from commit thread " + id
+    private var offsetsChannel = ClientUtils.channelToOffsetManager(group, zkClient, SocketTimeoutMs)
+    private var offset = 0L
+    val numErrors = new AtomicInteger(0)
+    val numCommits = new AtomicInteger(0)
+    val timer = newTimer("commit-thread", TimeUnit.MILLISECONDS, TimeUnit.SECONDS)
+    private val commitTimer = new KafkaTimer(timer)
+    val shutdownLock = new Object
+
+    private def ensureConnected() {
+      if (!offsetsChannel.isConnected)
+        offsetsChannel = ClientUtils.channelToOffsetManager(group, zkClient, SocketTimeoutMs)
+    }
+
+    override def doWork() {
+      val commitRequest = OffsetCommitRequest(group, mutable.Map((1 to partitionCount).map(TopicAndPartition("topic-" + id, _) -> OffsetAndMetadata(offset, metadata)):_*))
+      try {
+        ensureConnected()
+        offsetsChannel.send(commitRequest)
+        numCommits.getAndIncrement
+        commitTimer.time {
+          val response = OffsetCommitResponse.readFrom(offsetsChannel.receive().buffer)
+          if (response.commitStatus.exists(_._2 != ErrorMapping.NoError)) numErrors.getAndIncrement
+        }
+        offset += 1
+      }
+      catch {
+        case e1: ClosedByInterruptException =>
+          offsetsChannel.disconnect()
+        case e2: IOException =>
+          println("Commit thread %d: Error while committing offsets to %s:%d for group %s due to %s.".format(id, offsetsChannel.host, offsetsChannel.port, group, e2))
+          offsetsChannel.disconnect()
+      }
+      finally {
+        Thread.sleep(commitIntervalMs)
+      }
+    }
+
+    override def shutdown() {
+      super.shutdown()
+      awaitShutdown()
+      offsetsChannel.disconnect()
+      println("Commit thread %d ended. Last committed offset: %d.".format(id, offset))
+    }
+
+    def stats = {
+      "Commit thread %d :: Error count: %d; Max:%f; Min: %f; Mean: %f; Commit count: %d"
+      .format(id, numErrors.get(), timer.max(), timer.min(), timer.mean(), numCommits.get())
+    }
+  }
+
+  class FetchThread(numGroups: Int, fetchIntervalMs: Long, zkClient: ZkClient)
+        extends ShutdownableThread("fetch-thread")
+        with KafkaMetricsGroup {
+
+    private val timer = newTimer("fetch-thread", TimeUnit.MILLISECONDS, TimeUnit.SECONDS)
+    private val fetchTimer = new KafkaTimer(timer)
+
+    private val channels = mutable.Map[Int, BlockingChannel]()
+    private var metadataChannel = ClientUtils.channelToAnyBroker(zkClient, SocketTimeoutMs)
+
+    private val numErrors = new AtomicInteger(0)
+
+    override def doWork() {
+      val id = random.nextInt().abs % numGroups
+      val group = "group-" + id
+      try {
+        metadataChannel.send(ConsumerMetadataRequest(group))
+        val coordinatorId = ConsumerMetadataResponse.readFrom(metadataChannel.receive().buffer).coordinator.map(_.id).getOrElse(-1)
+
+        val channel = if (channels.contains(coordinatorId))
+          channels(coordinatorId)
+        else {
+          val newChannel = ClientUtils.channelToOffsetManager(group, zkClient, SocketTimeoutMs)
+          channels.put(coordinatorId, newChannel)
+          newChannel
+        }
+
+        try {
+          // send the offset fetch request
+          val fetchRequest = OffsetFetchRequest(group, Seq(TopicAndPartition("topic-"+id, 1)))
+          channel.send(fetchRequest)
+
+          fetchTimer.time {
+            val response = OffsetFetchResponse.readFrom(channel.receive().buffer)
+            if (response.requestInfo.exists(_._2.error != ErrorMapping.NoError)) {
+              numErrors.getAndIncrement
+            }
+          }
+        }
+        catch {
+          case e1: ClosedByInterruptException =>
+            channel.disconnect()
+            channels.remove(coordinatorId)
+          case e2: IOException =>
+            println("Error while fetching offset from %s:%d due to %s.".format(channel.host, channel.port, e2))
+            channel.disconnect()
+            channels.remove(coordinatorId)
+        }
+      }
+      catch {
+        case e: IOException =>
+          println("Error while querying %s:%d - shutting down query channel.".format(metadataChannel.host, metadataChannel.port))
+          metadataChannel.disconnect()
+          println("Creating new query channel.")
+          metadataChannel = ClientUtils.channelToAnyBroker(zkClient, SocketTimeoutMs)
+      }
+      finally {
+        Thread.sleep(fetchIntervalMs)
+      }
+
+    }
+
+    override def shutdown() {
+      super.shutdown()
+      awaitShutdown()
+      channels.foreach(_._2.disconnect())
+      metadataChannel.disconnect()
+    }
+
+    def stats = {
+      "Fetch thread :: Error count: %d; Max:%f; Min: %f; Mean: %f; Fetch count: %d"
+      .format(numErrors.get(), timer.max(), timer.min(), timer.mean(), timer.count())
+    }
+  }
+
+  def main(args: Array[String]) {
+    val parser = new OptionParser
+    val zookeeperOpt = parser.accepts("zookeeper", "The ZooKeeper connection URL.")
+      .withRequiredArg
+      .describedAs("ZooKeeper URL")
+      .ofType(classOf[java.lang.String])
+      .defaultsTo("localhost:2181")
+
+    val commitIntervalOpt = parser.accepts("commit-interval-ms", "Offset commit interval.")
+      .withRequiredArg
+      .describedAs("interval")
+      .ofType(classOf[java.lang.Integer])
+      .defaultsTo(100)
+
+    val fetchIntervalOpt = parser.accepts("fetch-interval-ms", "Offset fetch interval.")
+      .withRequiredArg
+      .describedAs("interval")
+      .ofType(classOf[java.lang.Integer])
+      .defaultsTo(1000)
+
+    val numPartitionsOpt = parser.accepts("partition-count", "Number of partitions per commit.")
+      .withRequiredArg
+      .describedAs("interval")
+      .ofType(classOf[java.lang.Integer])
+      .defaultsTo(1)
+
+    val numThreadsOpt = parser.accepts("thread-count", "Number of commit threads.")
+      .withRequiredArg
+      .describedAs("threads")
+      .ofType(classOf[java.lang.Integer])
+      .defaultsTo(1)
+
+    val reportingIntervalOpt = parser.accepts("reporting-interval-ms", "Interval at which stats are reported.")
+      .withRequiredArg
+      .describedAs("interval (ms)")
+      .ofType(classOf[java.lang.Integer])
+      .defaultsTo(3000)
+
+    val helpOpt = parser.accepts("help", "Print this message.")
+
+    val options = parser.parse(args : _*)
+
+    if (options.has(helpOpt)) {
+      parser.printHelpOn(System.out)
+      System.exit(0)
+    }
+
+    val commitIntervalMs = options.valueOf(commitIntervalOpt).intValue()
+    val fetchIntervalMs = options.valueOf(fetchIntervalOpt).intValue()
+    val threadCount = options.valueOf(numThreadsOpt).intValue()
+    val partitionCount = options.valueOf(numPartitionsOpt).intValue()
+    val zookeeper = options.valueOf(zookeeperOpt)
+    val reportingIntervalMs = options.valueOf(reportingIntervalOpt).intValue()
+    println("Commit thread count: %d; Partition count: %d, Commit interval: %d ms; Fetch interval: %d ms; Reporting interval: %d ms"
+            .format(threadCount, partitionCount, commitIntervalMs, fetchIntervalMs, reportingIntervalMs))
+
+    var zkClient: ZkClient = null
+    var commitThreads: Seq[CommitThread] = Seq()
+    var fetchThread: FetchThread = null
+    var statsThread: StatsThread = null
+    try {
+      zkClient = new ZkClient(zookeeper, 6000, 2000, ZKStringSerializer)
+      commitThreads = (0 to (threadCount-1)).map { threadId =>
+        new CommitThread(threadId, partitionCount, commitIntervalMs, zkClient)
+      }
+
+      fetchThread = new FetchThread(threadCount, fetchIntervalMs, zkClient)
+
+      val statsThread = new StatsThread(reportingIntervalMs, commitThreads, fetchThread)
+
+      Runtime.getRuntime.addShutdownHook(new Thread() {
+        override def run() {
+          cleanShutdown()
+          statsThread.printStats()
+        }
+      })
+
+      commitThreads.foreach(_.start())
+
+      fetchThread.start()
+
+      statsThread.start()
+
+      commitThreads.foreach(_.join())
+      fetchThread.join()
+      statsThread.join()
+    }
+    catch {
+      case e: Throwable =>
+        println("Error: ", e)
+    }
+    finally {
+      cleanShutdown()
+    }
+
+    def cleanShutdown() {
+      commitThreads.foreach(_.shutdown())
+      commitThreads.foreach(_.join())
+      if (fetchThread != null) {
+        fetchThread.shutdown()
+        fetchThread.join()
+      }
+      if (statsThread != null) {
+        statsThread.shutdown()
+        statsThread.join()
+      }
+      zkClient.close()
+    }
+
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala b/core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala
deleted file mode 100644
index 31534ca..0000000
--- a/core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala
+++ /dev/null
@@ -1,73 +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
-
-import consumer._
-import utils.Utils
-import java.util.concurrent.CountDownLatch
-
-object TestZKConsumerOffsets {
-  def main(args: Array[String]): Unit = {
-    if(args.length < 1) {
-      println("USAGE: " + TestZKConsumerOffsets.getClass.getName + " consumer.properties topic latest")
-      System.exit(1)
-    }
-    println("Starting consumer...")
-    val topic = args(1)
-    val autoOffsetReset = args(2)    
-    val props = Utils.loadProps(args(0))
-    props.put("auto.offset.reset", "largest")
-    
-    val config = new ConsumerConfig(props)
-    val consumerConnector: ConsumerConnector = Consumer.create(config)
-    val topicMessageStreams = consumerConnector.createMessageStreams(Predef.Map(topic -> 1))
-    var threadList = List[ConsumerThread]()
-    for ((topic, streamList) <- topicMessageStreams)
-      for (stream <- streamList)
-        threadList ::= new ConsumerThread(stream)
-
-    for (thread <- threadList)
-      thread.start
-
-    // attach shutdown handler to catch control-c
-    Runtime.getRuntime().addShutdownHook(new Thread() {
-      override def run() = {
-        consumerConnector.shutdown
-        threadList.foreach(_.shutdown)
-        println("consumer threads shutted down")
-      }
-    })
-  }
-}
-
-private class ConsumerThread(stream: KafkaStream[Array[Byte], Array[Byte]]) extends Thread {
-  val shutdownLatch = new CountDownLatch(1)
-
-  override def run() {
-    println("Starting consumer thread..")
-    for (messageAndMetadata <- stream) {
-      println("consumed: " + new String(messageAndMetadata.message, "UTF-8"))
-    }
-    shutdownLatch.countDown
-    println("thread shutdown !" )
-  }
-
-  def shutdown() {
-    shutdownLatch.await
-  }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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 dbe078c..6db76a5 100644
--- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
+++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
@@ -142,13 +142,6 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness {
       val offsetResponse = consumer.getOffsetsBefore(offsetRequest)
       val errorCode = offsetResponse.partitionErrorAndOffsets(topicAndPartition).error
       assertTrue("Offset request should fail with UnknownTopicOrPartitionCode", errorCode == ErrorMapping.UnknownTopicOrPartitionCode)
-      // test if offset fetch requests fail during delete topic
-      val offsetFetchRequest = new OffsetFetchRequest("test-group", Seq(topicAndPartition))
-      val offsetFetchResponse = consumer.fetchOffsets(offsetFetchRequest)
-      val offsetFetchErrorCode = offsetFetchResponse.requestInfo(topicAndPartition).error
-      assertTrue("Offset fetch request should fail with UnknownTopicOrPartitionCode",
-        offsetFetchErrorCode == ErrorMapping.UnknownTopicOrPartitionCode)
-      // TODO: test if offset commit requests fail during delete topic
     }
     // restart follower replica
     follower.startup()

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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 eb274d1..5378446 100644
--- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
+++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
@@ -23,12 +23,12 @@ import junit.framework.Assert._
 import java.nio.ByteBuffer
 import kafka.message.{Message, ByteBufferMessageSet}
 import kafka.cluster.Broker
-import collection.mutable._
-import kafka.common.{TopicAndPartition, ErrorMapping, OffsetMetadataAndError}
+import kafka.common.{OffsetAndMetadata, TopicAndPartition, ErrorMapping, OffsetMetadataAndError}
 import kafka.controller.LeaderIsrAndControllerEpoch
+import kafka.utils.SystemTime
 
 
-object SerializationTestUtils{
+object SerializationTestUtils {
   private val topic1 = "test1"
   private val topic2 = "test2"
   private val leader1 = 0
@@ -147,17 +147,15 @@ object SerializationTestUtils{
   }
 
   def createTestOffsetCommitRequest: OffsetCommitRequest = {
-    new OffsetCommitRequest("group 1", collection.immutable.Map(
-      TopicAndPartition(topic1, 0) -> OffsetMetadataAndError(offset=42L, metadata="some metadata"),
-      TopicAndPartition(topic1, 1) -> OffsetMetadataAndError(offset=100L, metadata=OffsetMetadataAndError.NoMetadata)
+    new OffsetCommitRequest("group 1", collection.mutable.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.UnknownTopicOrPartitionCode
-    ))
+    new OffsetCommitResponse(collection.immutable.Map(TopicAndPartition(topic1, 0) -> ErrorMapping.NoError,
+                                 TopicAndPartition(topic1, 1) -> ErrorMapping.NoError))
   }
 
   def createTestOffsetFetchRequest: OffsetFetchRequest = {
@@ -170,11 +168,18 @@ object SerializationTestUtils{
   def createTestOffsetFetchResponse: OffsetFetchResponse = {
     new OffsetFetchResponse(collection.immutable.Map(
       TopicAndPartition(topic1, 0) -> OffsetMetadataAndError(42L, "some metadata", ErrorMapping.NoError),
-      TopicAndPartition(topic1, 1) -> OffsetMetadataAndError(100L, OffsetMetadataAndError.NoMetadata,
-        ErrorMapping.UnknownTopicOrPartitionCode)
+      TopicAndPartition(topic1, 1) -> OffsetMetadataAndError(100L, OffsetAndMetadata.NoMetadata, ErrorMapping.UnknownTopicOrPartitionCode)
     ))
   }
 
+  def createConsumerMetadataRequest: ConsumerMetadataRequest = {
+    ConsumerMetadataRequest("group 1", clientId = "client 1")
+  }
+
+  def createConsumerMetadataResponse: ConsumerMetadataResponse = {
+    ConsumerMetadataResponse(Some(brokers.head), ErrorMapping.NoError)
+  }
+
 }
 
 class RequestResponseSerializationTest extends JUnitSuite {
@@ -193,114 +198,30 @@ class RequestResponseSerializationTest extends JUnitSuite {
   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
 
   @Test
   def testSerializationAndDeserialization() {
-    var buffer: ByteBuffer = ByteBuffer.allocate(leaderAndIsrRequest.sizeInBytes())
-    leaderAndIsrRequest.writeTo(buffer)
-    buffer.rewind()
-    val deserializedLeaderAndIsrRequest = LeaderAndIsrRequest.readFrom(buffer)
-    assertEquals("The original and deserialzed leaderAndISRRequest should be the same", leaderAndIsrRequest,
-                 deserializedLeaderAndIsrRequest)
-
-    buffer = ByteBuffer.allocate(leaderAndIsrResponse.sizeInBytes())
-    leaderAndIsrResponse.writeTo(buffer)
-    buffer.rewind()
-    val deserializedLeaderAndIsrResponse = LeaderAndIsrResponse.readFrom(buffer)
-    assertEquals("The original and deserialzed leaderAndISRResponse should be the same", leaderAndIsrResponse,
-                 deserializedLeaderAndIsrResponse)
-
-    buffer = ByteBuffer.allocate(stopReplicaRequest.sizeInBytes())
-    stopReplicaRequest.writeTo(buffer)
-    buffer.rewind()
-    val deserializedStopReplicaRequest = StopReplicaRequest.readFrom(buffer)
-    assertEquals("The original and deserialzed stopReplicaRequest should be the same", stopReplicaRequest,
-                 deserializedStopReplicaRequest)
-
-    buffer = ByteBuffer.allocate(stopReplicaResponse.sizeInBytes())
-    stopReplicaResponse.writeTo(buffer)
-    buffer.rewind()
-    val deserializedStopReplicaResponse = StopReplicaResponse.readFrom(buffer)
-    assertEquals("The original and deserialzed stopReplicaResponse should be the same", stopReplicaResponse,
-                 deserializedStopReplicaResponse)
-
-    buffer = ByteBuffer.allocate(producerRequest.sizeInBytes)
-    producerRequest.writeTo(buffer)
-    buffer.rewind()
-    val deserializedProducerRequest = ProducerRequest.readFrom(buffer)
-    assertEquals("The original and deserialzed producerRequest should be the same", producerRequest,
-                 deserializedProducerRequest)
-
-    buffer = ByteBuffer.allocate(producerResponse.sizeInBytes)
-    producerResponse.writeTo(buffer)
-    buffer.rewind()
-    val deserializedProducerResponse = ProducerResponse.readFrom(buffer)
-    assertEquals("The original and deserialzed producerResponse should be the same: [%s], [%s]".format(producerResponse, deserializedProducerResponse), producerResponse,
-                 deserializedProducerResponse)
-
-    buffer = ByteBuffer.allocate(fetchRequest.sizeInBytes)
-    fetchRequest.writeTo(buffer)
-    buffer.rewind()
-    val deserializedFetchRequest = FetchRequest.readFrom(buffer)
-    assertEquals("The original and deserialzed fetchRequest should be the same", fetchRequest,
-                 deserializedFetchRequest)
-
-    buffer = ByteBuffer.allocate(offsetRequest.sizeInBytes)
-    offsetRequest.writeTo(buffer)
-    buffer.rewind()
-    val deserializedOffsetRequest = OffsetRequest.readFrom(buffer)
-    assertEquals("The original and deserialzed offsetRequest should be the same", offsetRequest,
-                 deserializedOffsetRequest)
-
-    buffer = ByteBuffer.allocate(offsetResponse.sizeInBytes)
-    offsetResponse.writeTo(buffer)
-    buffer.rewind()
-    val deserializedOffsetResponse = OffsetResponse.readFrom(buffer)
-    assertEquals("The original and deserialzed offsetResponse should be the same", offsetResponse,
-                 deserializedOffsetResponse)
-
-    buffer = ByteBuffer.allocate(topicMetadataRequest.sizeInBytes())
-    topicMetadataRequest.writeTo(buffer)
-    buffer.rewind()
-    val deserializedTopicMetadataRequest = TopicMetadataRequest.readFrom(buffer)
-    assertEquals("The original and deserialzed topicMetadataRequest should be the same", topicMetadataRequest,
-                 deserializedTopicMetadataRequest)
-
-    buffer = ByteBuffer.allocate(topicMetadataResponse.sizeInBytes)
-    topicMetadataResponse.writeTo(buffer)
-    buffer.rewind()
-    val deserializedTopicMetadataResponse = TopicMetadataResponse.readFrom(buffer)
-    assertEquals("The original and deserialzed topicMetadataResponse should be the same", topicMetadataResponse,
-                 deserializedTopicMetadataResponse)
-
-    buffer = ByteBuffer.allocate(offsetCommitRequest.sizeInBytes)
-    offsetCommitRequest.writeTo(buffer)
-    buffer.rewind()
-    val deserializedOffsetCommitRequest = OffsetCommitRequest.readFrom(buffer)
-    assertEquals("The original and deserialzed offsetCommitRequest should be the same", offsetCommitRequest, 
-      deserializedOffsetCommitRequest)
-
-    buffer = ByteBuffer.allocate(offsetCommitResponse.sizeInBytes)
-    offsetCommitResponse.writeTo(buffer)
-    buffer.rewind()
-    val deserializedOffsetCommitResponse = OffsetCommitResponse.readFrom(buffer)
-    assertEquals("The original and deserialzed offsetCommitResponse should be the same", offsetCommitResponse, 
-      deserializedOffsetCommitResponse)
-
-    buffer = ByteBuffer.allocate(offsetFetchRequest.sizeInBytes)
-    offsetFetchRequest.writeTo(buffer)
-    buffer.rewind()
-    val deserializedOffsetFetchRequest = OffsetFetchRequest.readFrom(buffer)
-    assertEquals("The original and deserialzed offsetFetchRequest should be the same", offsetFetchRequest, 
-      deserializedOffsetFetchRequest)
-
-    buffer = ByteBuffer.allocate(offsetFetchResponse.sizeInBytes)
-    offsetFetchResponse.writeTo(buffer)
-    buffer.rewind()
-    val deserializedOffsetFetchResponse = OffsetFetchResponse.readFrom(buffer)
-    assertEquals("The original and deserialzed offsetFetchResponse should be the same", offsetFetchResponse, 
-      deserializedOffsetFetchResponse)
 
+    val requestsAndResponses =
+      collection.immutable.Seq(leaderAndIsrRequest, leaderAndIsrResponse,
+                               stopReplicaRequest, stopReplicaResponse,
+                               producerRequest, producerResponse,
+                               fetchRequest,
+                               offsetRequest, offsetResponse,
+                               topicMetadataRequest, topicMetadataResponse,
+                               offsetCommitRequest, offsetCommitResponse,
+                               offsetFetchRequest, offsetFetchResponse,
+                               consumerMetadataRequest, consumerMetadataResponse)
+
+    requestsAndResponses.foreach { original =>
+      val buffer = ByteBuffer.allocate(original.sizeInBytes)
+      original.writeTo(buffer)
+      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)
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/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 cf2724b..d903a6f 100644
--- a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala
+++ b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala
@@ -21,6 +21,7 @@ package kafka.consumer
 import junit.framework.Assert._
 import org.scalatest.junit.JUnitSuite
 import org.junit.Test
+import kafka.server.OffsetManager
 
 
 class TopicFilterTest extends JUnitSuite {
@@ -29,19 +30,30 @@ class TopicFilterTest extends JUnitSuite {
   def testWhitelists() {
 
     val topicFilter1 = new Whitelist("white1,white2")
-    assertTrue(topicFilter1.isTopicAllowed("white2"))
-    assertFalse(topicFilter1.isTopicAllowed("black1"))
+    assertTrue(topicFilter1.isTopicAllowed("white2", excludeInternalTopics = true))
+    assertTrue(topicFilter1.isTopicAllowed("white2", excludeInternalTopics = false))
+    assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = true))
+    assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = false))
 
     val topicFilter2 = new Whitelist(".+")
-    assertTrue(topicFilter2.isTopicAllowed("alltopics"))
-    
+    assertTrue(topicFilter2.isTopicAllowed("alltopics", excludeInternalTopics = true))
+    assertFalse(topicFilter2.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = true))
+    assertTrue(topicFilter2.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = false))
+
     val topicFilter3 = new Whitelist("white_listed-topic.+")
-    assertTrue(topicFilter3.isTopicAllowed("white_listed-topic1"))
-    assertFalse(topicFilter3.isTopicAllowed("black1"))
+    assertTrue(topicFilter3.isTopicAllowed("white_listed-topic1", excludeInternalTopics = true))
+    assertFalse(topicFilter3.isTopicAllowed("black1", excludeInternalTopics = true))
   }
 
   @Test
   def testBlacklists() {
     val topicFilter1 = new Blacklist("black1")
+    assertTrue(topicFilter1.isTopicAllowed("white2", excludeInternalTopics = true))
+    assertTrue(topicFilter1.isTopicAllowed("white2", excludeInternalTopics = false))
+    assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = true))
+    assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = false))
+
+    assertFalse(topicFilter1.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = true))
+    assertTrue(topicFilter1.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = false))
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala
index 8fe7259..258dd25 100644
--- a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala
+++ b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala
@@ -115,7 +115,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
     assertEquals(expected_1, actual_1)
 
     // commit consumed offsets
-    zkConsumerConnector1.commitOffsets
+    zkConsumerConnector1.commitOffsets()
 
     // create a consumer
     val consumerConfig2 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer2)) {
@@ -194,7 +194,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
     assertEquals(expected_1, actual_1)
 
     // commit consumed offsets
-    zkConsumerConnector1.commitOffsets
+    zkConsumerConnector1.commitOffsets()
 
     // create a consumer
     val consumerConfig2 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer2)) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala
index 6a96d80..e632997 100644
--- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala
+++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala
@@ -23,13 +23,13 @@ import junit.framework.Assert._
 import java.util.Properties
 import kafka.consumer.SimpleConsumer
 import org.junit.{After, Before, Test}
-import kafka.message.{NoCompressionCodec, ByteBufferMessageSet, Message}
 import kafka.zk.ZooKeeperTestHarness
 import org.scalatest.junit.JUnit3Suite
-import kafka.api.{OffsetCommitRequest, OffsetFetchRequest}
+import kafka.api.{ConsumerMetadataRequest, OffsetCommitRequest, OffsetFetchRequest}
 import kafka.utils.TestUtils._
-import kafka.common.{ErrorMapping, TopicAndPartition, OffsetMetadataAndError}
+import kafka.common.{OffsetMetadataAndError, OffsetAndMetadata, ErrorMapping, TopicAndPartition}
 import scala.util.Random
+import scala.collection._
 import kafka.admin.AdminUtils
 
 class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness {
@@ -39,6 +39,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness {
   var server: KafkaServer = null
   var logSize: Int = 100
   val brokerPort: Int = 9099
+  val group = "test-group"
   var simpleConsumer: SimpleConsumer = null
   var time: Time = new MockTime()
 
@@ -51,6 +52,14 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness {
     time = new MockTime()
     server = TestUtils.createServer(new KafkaConfig(config), time)
     simpleConsumer = new SimpleConsumer("localhost", brokerPort, 1000000, 64*1024, "test-client")
+    val consumerMetadataRequest = ConsumerMetadataRequest(group)
+    Stream.continually {
+      val consumerMetadataResponse = simpleConsumer.send(consumerMetadataRequest)
+      consumerMetadataResponse.coordinator.isDefined
+    }.dropWhile(success => {
+      if (!success) Thread.sleep(1000)
+      !success
+    })
   }
 
   @After
@@ -72,34 +81,34 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness {
     AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment)
     val leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000)
     assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined)
-    val commitRequest = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError(offset=42L)))
+    val commitRequest = OffsetCommitRequest("test-group", mutable.Map(topicAndPartition -> OffsetAndMetadata(offset=42L)))
     val commitResponse = simpleConsumer.commitOffsets(commitRequest)
 
-    assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(topicAndPartition).get)
+    assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(topicAndPartition).get)
 
     // Fetch it and verify
-    val fetchRequest = OffsetFetchRequest("test-group", Seq(topicAndPartition))
+    val fetchRequest = OffsetFetchRequest(group, Seq(topicAndPartition))
     val fetchResponse = simpleConsumer.fetchOffsets(fetchRequest)
 
     assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(topicAndPartition).get.error)
-    //assertEquals(OffsetMetadataAndError.NoMetadata, fetchResponse.requestInfo.get(topicAndPartition).get.metadata)
+    assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(topicAndPartition).get.metadata)
     assertEquals(42L, fetchResponse.requestInfo.get(topicAndPartition).get.offset)
 
     // Commit a new offset
-    val commitRequest1 = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError(
+    val commitRequest1 = OffsetCommitRequest(group, mutable.Map(topicAndPartition -> OffsetAndMetadata(
       offset=100L,
       metadata="some metadata"
     )))
     val commitResponse1 = simpleConsumer.commitOffsets(commitRequest1)
 
-    assertEquals(ErrorMapping.NoError, commitResponse1.requestInfo.get(topicAndPartition).get)
+    assertEquals(ErrorMapping.NoError, commitResponse1.commitStatus.get(topicAndPartition).get)
 
     // Fetch it and verify
-    val fetchRequest1 = OffsetFetchRequest("test-group", Seq(topicAndPartition))
+    val fetchRequest1 = OffsetFetchRequest(group, Seq(topicAndPartition))
     val fetchResponse1 = simpleConsumer.fetchOffsets(fetchRequest1)
-    
+
     assertEquals(ErrorMapping.NoError, fetchResponse1.requestInfo.get(topicAndPartition).get.error)
-    //assertEquals("some metadata", fetchResponse1.requestInfo.get(topicAndPartition).get.metadata)
+    assertEquals("some metadata", fetchResponse1.requestInfo.get(topicAndPartition).get.metadata)
     assertEquals(100L, fetchResponse1.requestInfo.get(topicAndPartition).get.offset)
 
   }
@@ -111,34 +120,19 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness {
     val topic3 = "topic-3"
     val topic4 = "topic-4"
 
-    val expectedReplicaAssignment = Map(0  -> List(1))
-    // create the topic
-    AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic1, expectedReplicaAssignment)
-    AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic2, expectedReplicaAssignment)
-    AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic3, expectedReplicaAssignment)
-    AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic4, expectedReplicaAssignment)
-    var leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 0, 1000)
-    assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined)
-    leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 0, 1000)
-    assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined)
-    leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic3, 0, 1000)
-    assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined)
-    leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic4, 0, 1000)
-    assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined)
-
-    val commitRequest = OffsetCommitRequest("test-group", Map(
-      TopicAndPartition(topic1, 0) -> OffsetMetadataAndError(offset=42L, metadata="metadata one"),
-      TopicAndPartition(topic2, 0) -> OffsetMetadataAndError(offset=43L, metadata="metadata two"),
-      TopicAndPartition(topic3, 0) -> OffsetMetadataAndError(offset=44L, metadata="metadata three"),
-      TopicAndPartition(topic2, 1) -> OffsetMetadataAndError(offset=45L)
+    val commitRequest = OffsetCommitRequest("test-group", mutable.Map(
+      TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L, metadata="metadata one"),
+      TopicAndPartition(topic2, 0) -> OffsetAndMetadata(offset=43L, metadata="metadata two"),
+      TopicAndPartition(topic3, 0) -> OffsetAndMetadata(offset=44L, metadata="metadata three"),
+      TopicAndPartition(topic2, 1) -> OffsetAndMetadata(offset=45L)
     ))
     val commitResponse = simpleConsumer.commitOffsets(commitRequest)
-    assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get)
-    assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get)
-    assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get)
-    assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get)
+    assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(TopicAndPartition(topic1, 0)).get)
+    assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(TopicAndPartition(topic2, 0)).get)
+    assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(TopicAndPartition(topic3, 0)).get)
+    assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(TopicAndPartition(topic2, 1)).get)
 
-    val fetchRequest = OffsetFetchRequest("test-group", Seq(
+    val fetchRequest = OffsetFetchRequest(group, Seq(
       TopicAndPartition(topic1, 0),
       TopicAndPartition(topic2, 0),
       TopicAndPartition(topic3, 0),
@@ -152,22 +146,22 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness {
     assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.error)
     assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.error)
     assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.error)
-    assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.error)
-    assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.error)
+    assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get)
+    assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get)
 
-    //assertEquals("metadata one", fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.metadata)
-    //assertEquals("metadata two", fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.metadata)
-    //assertEquals("metadata three", fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.metadata)
-    //assertEquals(OffsetMetadataAndError.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.metadata)
-    //assertEquals(OffsetMetadataAndError.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.metadata)
-    //assertEquals(OffsetMetadataAndError.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.metadata)
+    assertEquals("metadata one", fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.metadata)
+    assertEquals("metadata two", fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.metadata)
+    assertEquals("metadata three", fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.metadata)
+    assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.metadata)
+    assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.metadata)
+    assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.metadata)
 
     assertEquals(42L, fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.offset)
     assertEquals(43L, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.offset)
     assertEquals(44L, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.offset)
     assertEquals(45L, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.offset)
-    assertEquals(OffsetMetadataAndError.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.offset)
-    assertEquals(OffsetMetadataAndError.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.offset)
+    assertEquals(OffsetAndMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.offset)
+    assertEquals(OffsetAndMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.offset)
   }
 
   @Test
@@ -178,36 +172,21 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness {
     var leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topicAndPartition.topic, 0, 1000)
     assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined)
 
-    val commitRequest = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError(
+    val commitRequest = OffsetCommitRequest("test-group", mutable.Map(topicAndPartition -> OffsetAndMetadata(
       offset=42L,
       metadata=random.nextString(server.config.offsetMetadataMaxSize)
     )))
     val commitResponse = simpleConsumer.commitOffsets(commitRequest)
 
-    assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(topicAndPartition).get)
+    assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(topicAndPartition).get)
 
-    val commitRequest1 = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError(
+    val commitRequest1 = OffsetCommitRequest(group, mutable.Map(topicAndPartition -> OffsetAndMetadata(
       offset=42L,
       metadata=random.nextString(server.config.offsetMetadataMaxSize + 1)
     )))
     val commitResponse1 = simpleConsumer.commitOffsets(commitRequest1)
 
-    assertEquals(ErrorMapping.OffsetMetadataTooLargeCode, commitResponse1.requestInfo.get(topicAndPartition).get)
-
-  }
+    assertEquals(ErrorMapping.OffsetMetadataTooLargeCode, commitResponse1.commitStatus.get(topicAndPartition).get)
 
-  @Test
-  def testNullMetadata() {
-    val topicAndPartition = TopicAndPartition("null-metadata", 0)
-    val expectedReplicaAssignment = Map(0  -> List(1))
-    AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topicAndPartition.topic, expectedReplicaAssignment)
-    var leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topicAndPartition.topic, 0, 1000)
-    assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined)
-    val commitRequest = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError(
-      offset=42L,
-      metadata=null
-    )))
-    val commitResponse = simpleConsumer.commitOffsets(commitRequest)
-    assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(topicAndPartition).get)
   }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
index 1317b4c..22bb6f2 100644
--- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
+++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
@@ -86,12 +86,15 @@ class SimpleFetchTest extends JUnit3Suite {
     EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(topic, partitionId)).andReturn(partition.leaderReplicaIfLocal().get).anyTimes()
     EasyMock.replay(replicaManager)
 
+    val offsetManager = EasyMock.createMock(classOf[kafka.server.OffsetManager])
+
     val controller = EasyMock.createMock(classOf[kafka.controller.KafkaController])
 
     // start a request channel with 2 processors and a queue size of 5 (this is more or less arbitrary)
     // don't provide replica or leader callbacks since they will not be tested here
     val requestChannel = new RequestChannel(2, 5)
-    val apis = new KafkaApis(requestChannel, replicaManager, zkClient, configs.head.brokerId, configs.head, controller)
+    val apis = new KafkaApis(requestChannel, replicaManager, offsetManager, zkClient, configs.head.brokerId, configs.head, controller)
+
     val partitionStateInfo = EasyMock.createNiceMock(classOf[PartitionStateInfo])
     apis.metadataCache.put(TopicAndPartition(topic, partitionId), partitionStateInfo)
     EasyMock.replay(partitionStateInfo)
@@ -159,10 +162,12 @@ class SimpleFetchTest extends JUnit3Suite {
     EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(topic, partitionId)).andReturn(partition.leaderReplicaIfLocal().get).anyTimes()
     EasyMock.replay(replicaManager)
 
+    val offsetManager = EasyMock.createMock(classOf[kafka.server.OffsetManager])
+
     val controller = EasyMock.createMock(classOf[kafka.controller.KafkaController])
 
     val requestChannel = new RequestChannel(2, 5)
-    val apis = new KafkaApis(requestChannel, replicaManager, zkClient, configs.head.brokerId, configs.head, controller)
+    val apis = new KafkaApis(requestChannel, replicaManager, offsetManager, zkClient, configs.head.brokerId, configs.head, controller)
     val partitionStateInfo = EasyMock.createNiceMock(classOf[PartitionStateInfo])
     apis.metadataCache.put(TopicAndPartition(topic, partitionId), partitionStateInfo)
     EasyMock.replay(partitionStateInfo)

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/migration_tool_testsuite/migration_tool_test.py
----------------------------------------------------------------------
diff --git a/system_test/migration_tool_testsuite/migration_tool_test.py b/system_test/migration_tool_testsuite/migration_tool_test.py
index 2fecd19..2386a58 100644
--- a/system_test/migration_tool_testsuite/migration_tool_test.py
+++ b/system_test/migration_tool_testsuite/migration_tool_test.py
@@ -171,7 +171,7 @@ class MigrationToolTest(ReplicationUtils, SetupUtils):
                 time.sleep(5)
 
                 self.log_message("creating topics")
-                kafka_system_test_utils.create_topic(self.systemTestEnv, self.testcaseEnv)
+                kafka_system_test_utils.create_topic_for_producer_performance(self.systemTestEnv, self.testcaseEnv)
                 self.anonLogger.info("sleeping for 5s")
                 time.sleep(5)
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/mirror_maker/README
----------------------------------------------------------------------
diff --git a/system_test/mirror_maker/README b/system_test/mirror_maker/README
deleted file mode 100644
index da53c14..0000000
--- a/system_test/mirror_maker/README
+++ /dev/null
@@ -1,22 +0,0 @@
-This test replicates messages from two source kafka clusters into one target
-kafka cluster using the mirror-maker tool.  At the end, the messages produced
-at the source brokers should match that at the target brokers.
-
-To run this test, do
-bin/run-test.sh
-
-In the event of failure, by default the brokers and zookeepers remain running
-to make it easier to debug the issue - hit Ctrl-C to shut them down. You can
-change this behavior by setting the action_on_fail flag in the script to "exit"
-or "proceed", in which case a snapshot of all the logs and directories is
-placed in the test's base directory.
-
-It is a good idea to run the test in a loop. E.g.:
-
-:>/tmp/mirrormaker_test.log
-for i in {1..10}; do echo "run $i"; ./bin/run-test.sh 2>1 >> /tmp/mirrormaker_test.log; done
-tail -F /tmp/mirrormaker_test.log
-
-grep -ic passed /tmp/mirrormaker_test.log
-grep -ic failed /tmp/mirrormaker_test.log
-

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/mirror_maker/bin/expected.out
----------------------------------------------------------------------
diff --git a/system_test/mirror_maker/bin/expected.out b/system_test/mirror_maker/bin/expected.out
deleted file mode 100644
index 0a1bbaf..0000000
--- a/system_test/mirror_maker/bin/expected.out
+++ /dev/null
@@ -1,18 +0,0 @@
-start the servers ...
-start producing messages ...
-wait for consumer to finish consuming ...
-[2011-05-17 14:49:11,605] INFO Creating async producer for broker id = 2 at localhost:9091 (kafka.producer.ProducerPool)
-[2011-05-17 14:49:11,606] INFO Creating async producer for broker id = 1 at localhost:9092 (kafka.producer.ProducerPool)
-[2011-05-17 14:49:11,607] INFO Creating async producer for broker id = 3 at localhost:9090 (kafka.producer.ProducerPool)
-thread 0: 400000 messages sent 3514012.1233 nMsg/sec 3.3453 MBs/sec
-[2011-05-17 14:49:34,382] INFO Closing all async producers (kafka.producer.ProducerPool)
-[2011-05-17 14:49:34,383] INFO Closed AsyncProducer (kafka.producer.async.AsyncProducer)
-[2011-05-17 14:49:34,384] INFO Closed AsyncProducer (kafka.producer.async.AsyncProducer)
-[2011-05-17 14:49:34,385] INFO Closed AsyncProducer (kafka.producer.async.AsyncProducer)
-Total Num Messages: 400000 bytes: 79859641 in 22.93 secs
-Messages/sec: 17444.3960
-MB/sec: 3.3214
-test passed
-stopping the servers
-bin/../../../bin/zookeeper-server-start.sh: line 9: 22584 Terminated              $(dirname $0)/kafka-run-class.sh org.apache.zookeeper.server.quorum.QuorumPeerMain $@
-bin/../../../bin/zookeeper-server-start.sh: line 9: 22585 Terminated              $(dirname $0)/kafka-run-class.sh org.apache.zookeeper.server.quorum.QuorumPeerMain $@

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/mirror_maker/bin/run-test.sh
----------------------------------------------------------------------
diff --git a/system_test/mirror_maker/bin/run-test.sh b/system_test/mirror_maker/bin/run-test.sh
deleted file mode 100644
index e5e6c08..0000000
--- a/system_test/mirror_maker/bin/run-test.sh
+++ /dev/null
@@ -1,357 +0,0 @@
-#!/bin/bash
-# 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.
-
-readonly num_messages=10000
-readonly message_size=100
-readonly action_on_fail="proceed"
-# readonly action_on_fail="wait"
-
-readonly test_start_time="$(date +%s)"
-
-readonly base_dir=$(dirname $0)/..
-
-info() {
-    echo -e "$(date +"%Y-%m-%d %H:%M:%S") $*"
-}
-
-kill_child_processes() {
-    isTopmost=$1
-    curPid=$2
-    childPids=$(ps a -o pid= -o ppid= | grep "${curPid}$" | awk '{print $1;}')
-    for childPid in $childPids
-    do
-        kill_child_processes 0 $childPid
-    done
-    if [ $isTopmost -eq 0 ]; then
-        kill -15 $curPid 2> /dev/null
-    fi
-}
-
-cleanup() {
-    info "cleaning up"
-
-    pid_zk_source1=
-    pid_zk_source2=
-    pid_zk_target=
-    pid_kafka_source_1_1=
-    pid_kafka_source_1_2=
-    pid_kafka_source_2_1=
-    pid_kafka_source_2_2=
-    pid_kafka_target_1_1=
-    pid_kafka_target_1_2=
-    pid_producer=
-    pid_mirrormaker_1=
-    pid_mirrormaker_2=
-
-    rm -rf /tmp/zookeeper*
-
-    rm -rf /tmp/kafka*
-}
-
-begin_timer() {
-    t_begin=$(date +%s)
-}
-
-end_timer() {
-    t_end=$(date +%s)
-}
-
-start_zk() {
-    info "starting zookeepers"
-    $base_dir/../../bin/zookeeper-server-start.sh $base_dir/config/zookeeper_source_1.properties 2>&1 > $base_dir/zookeeper_source-1.log &
-    pid_zk_source1=$!
-    $base_dir/../../bin/zookeeper-server-start.sh $base_dir/config/zookeeper_source_2.properties 2>&1 > $base_dir/zookeeper_source-2.log &
-    pid_zk_source2=$!
-    $base_dir/../../bin/zookeeper-server-start.sh $base_dir/config/zookeeper_target.properties 2>&1 > $base_dir/zookeeper_target.log &
-    pid_zk_target=$!
-}
-
-start_source_servers() {
-    info "starting source cluster"
-
-    JMX_PORT=1111 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source_1_1.properties 2>&1 > $base_dir/kafka_source-1-1.log &
-    pid_kafka_source_1_1=$!
-    JMX_PORT=2222 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source_1_2.properties 2>&1 > $base_dir/kafka_source-1-2.log &
-    pid_kafka_source_1_2=$!
-    JMX_PORT=3333 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source_2_1.properties 2>&1 > $base_dir/kafka_source-2-1.log &
-    pid_kafka_source_2_1=$!
-    JMX_PORT=4444 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source_2_2.properties 2>&1 > $base_dir/kafka_source-2-2.log &
-    pid_kafka_source_2_2=$!
-}
-
-start_target_servers() {
-    info "starting mirror cluster"
-    JMX_PORT=5555 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_target_1_1.properties 2>&1 > $base_dir/kafka_target-1-1.log &
-    pid_kafka_target_1_1=$!
-    JMX_PORT=6666 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_target_1_2.properties 2>&1 > $base_dir/kafka_target-1-2.log &
-    pid_kafka_target_1_2=$!
-}
-
-shutdown_servers() {
-    info "stopping mirror-maker"
-    if [ "x${pid_mirrormaker_1}" != "x" ]; then kill_child_processes 0 ${pid_mirrormaker_1}; fi
-    # sleep to avoid rebalancing during shutdown
-    sleep 2
-    if [ "x${pid_mirrormaker_2}" != "x" ]; then kill_child_processes 0 ${pid_mirrormaker_2}; fi
-
-    info "stopping producer"
-    if [ "x${pid_producer}" != "x" ]; then kill_child_processes 0 ${pid_producer}; fi
-
-    info "shutting down target servers"
-    if [ "x${pid_kafka_target_1_1}" != "x" ]; then kill_child_processes 0 ${pid_kafka_target_1_1}; fi
-    if [ "x${pid_kafka_target_1_2}" != "x" ]; then kill_child_processes 0 ${pid_kafka_target_1_2}; fi
-    sleep 2
-
-    info "shutting down source servers"
-    if [ "x${pid_kafka_source_1_1}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source_1_1}; fi
-    if [ "x${pid_kafka_source_1_2}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source_1_2}; fi
-    if [ "x${pid_kafka_source_2_1}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source_2_1}; fi
-    if [ "x${pid_kafka_source_2_2}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source_2_2}; fi
-
-    info "shutting down zookeeper servers"
-    if [ "x${pid_zk_target}" != "x" ]; then kill_child_processes 0 ${pid_zk_target}; fi
-    if [ "x${pid_zk_source1}" != "x" ]; then kill_child_processes 0 ${pid_zk_source1}; fi
-    if [ "x${pid_zk_source2}" != "x" ]; then kill_child_processes 0 ${pid_zk_source2}; fi
-}
-
-start_producer() {
-    topic=$1
-    zk=$2
-    info "start producing messages for topic $topic to zookeeper $zk ..."
-    $base_dir/../../bin/kafka-run-class.sh kafka.perf.ProducerPerformance --brokerinfo zk.connect=$zk --topics $topic --messages $num_messages --message-size $message_size --batch-size 200 --vary-message-size --threads 1 --reporting-interval $num_messages --async 2>&1 > $base_dir/producer_performance.log &
-    pid_producer=$!
-}
-
-# Usage: wait_partition_done ([kafka-server] [topic] [partition-id])+
-wait_partition_done() {
-    n_tuples=$(($# / 3))
-
-    i=1
-    while (($#)); do
-        kafka_server[i]=$1
-        topic[i]=$2
-        partitionid[i]=$3
-        prev_offset[i]=0
-        info "\twaiting for partition on server ${kafka_server[i]}, topic ${topic[i]}, partition ${partitionid[i]}"
-        i=$((i+1))
-        shift 3
-    done
-
-    all_done=0
-
-    # set -x
-    while [[ $all_done != 1 ]]; do
-        sleep 4
-        i=$n_tuples
-        all_done=1
-        for ((i=1; i <= $n_tuples; i++)); do
-            cur_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server ${kafka_server[i]} --topic ${topic[i]} --partition ${partitionid[i]} --time -1 --offsets 1 | tail -1)
-            if [ "x$cur_size" != "x${prev_offset[i]}" ]; then
-                all_done=0
-                prev_offset[i]=$cur_size
-            fi
-        done
-    done
-
-}
-
-cmp_logs() {
-    topic=$1
-    info "comparing source and target logs for topic $topic"
-    source_part0_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9090 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1)
-    source_part1_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9091 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1)
-    source_part2_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9092 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1)
-    source_part3_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9093 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1)
-    target_part0_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9094 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1)
-    target_part1_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9095 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1)
-    if [ "x$source_part0_size" == "x" ]; then source_part0_size=0; fi
-    if [ "x$source_part1_size" == "x" ]; then source_part1_size=0; fi
-    if [ "x$source_part2_size" == "x" ]; then source_part2_size=0; fi
-    if [ "x$source_part3_size" == "x" ]; then source_part3_size=0; fi
-    if [ "x$target_part0_size" == "x" ]; then target_part0_size=0; fi
-    if [ "x$target_part1_size" == "x" ]; then target_part1_size=0; fi
-    expected_size=$(($source_part0_size + $source_part1_size + $source_part2_size + $source_part3_size))
-    actual_size=$(($target_part0_size + $target_part1_size))
-    if [ "x$expected_size" != "x$actual_size" ]
-    then
-        info "source size: $expected_size target size: $actual_size"
-        return 1
-    else
-        return 0
-    fi
-}
-
-take_fail_snapshot() {
-    snapshot_dir="$base_dir/failed-${snapshot_prefix}-${test_start_time}"
-    mkdir $snapshot_dir
-    for dir in /tmp/zookeeper_source{1..2} /tmp/zookeeper_target /tmp/kafka-source-{1..2}-{1..2}-logs /tmp/kafka-target{1..2}-logs; do
-        if [ -d $dir ]; then
-            cp -r $dir $snapshot_dir
-        fi
-    done
-}
-
-# Usage: process_test_result <result> <action_on_fail>
-# result: last test result
-# action_on_fail: (exit|wait|proceed)
-# ("wait" is useful if you want to troubleshoot using zookeeper)
-process_test_result() {
-    result=$1
-    if [ $1 -eq 0 ]; then
-        info "test passed"
-    else
-        info "test failed"
-        case "$2" in
-            "wait") info "waiting: hit Ctrl-c to quit"
-                wait
-                ;;
-            "exit") shutdown_servers
-                take_fail_snapshot
-                exit $result
-                ;;
-            *) shutdown_servers
-                take_fail_snapshot
-                info "proceeding"
-                ;;
-        esac
-    fi
-}
-
-test_whitelists() {
-    info "### Testing whitelists"
-    snapshot_prefix="whitelist-test"
-
-    cleanup
-    start_zk
-    start_source_servers
-    start_target_servers
-    sleep 4
-
-    info "starting mirror makers"
-    JMX_PORT=7777 $base_dir/../../bin/kafka-run-class.sh kafka.tools.MirrorMaker --consumer.config $base_dir/config/whitelisttest_1.consumer.properties --consumer.config $base_dir/config/whitelisttest_2.consumer.properties --producer.config $base_dir/config/mirror_producer.properties --whitelist="white.*" --num.streams 2 2>&1 > $base_dir/kafka_mirrormaker_1.log &
-    pid_mirrormaker_1=$!
-    JMX_PORT=8888 $base_dir/../../bin/kafka-run-class.sh kafka.tools.MirrorMaker --consumer.config $base_dir/config/whitelisttest_1.consumer.properties --consumer.config $base_dir/config/whitelisttest_2.consumer.properties --producer.config $base_dir/config/mirror_producer.properties --whitelist="white.*" --num.streams 2 2>&1 > $base_dir/kafka_mirrormaker_2.log &
-    pid_mirrormaker_2=$!
-
-    begin_timer
-
-    start_producer whitetopic01 localhost:2181
-    start_producer whitetopic01 localhost:2182
-    info "waiting for whitetopic01 producers to finish producing ..."
-    wait_partition_done kafka://localhost:9090 whitetopic01 0 kafka://localhost:9091 whitetopic01 0 kafka://localhost:9092 whitetopic01 0 kafka://localhost:9093 whitetopic01 0
-
-    start_producer whitetopic02 localhost:2181
-    start_producer whitetopic03 localhost:2181
-    start_producer whitetopic04 localhost:2182
-    info "waiting for whitetopic02,whitetopic03,whitetopic04 producers to finish producing ..."
-    wait_partition_done kafka://localhost:9090 whitetopic02 0 kafka://localhost:9091 whitetopic02 0 kafka://localhost:9090 whitetopic03 0 kafka://localhost:9091 whitetopic03 0 kafka://localhost:9092 whitetopic04 0 kafka://localhost:9093 whitetopic04 0
-
-    start_producer blacktopic01 localhost:2182
-    info "waiting for blacktopic01 producer to finish producing ..."
-    wait_partition_done kafka://localhost:9092 blacktopic01 0 kafka://localhost:9093 blacktopic01 0
-
-    info "waiting for consumer to finish consuming ..."
-
-    wait_partition_done kafka://localhost:9094 whitetopic01 0 kafka://localhost:9095 whitetopic01 0 kafka://localhost:9094 whitetopic02 0 kafka://localhost:9095 whitetopic02 0 kafka://localhost:9094 whitetopic03 0 kafka://localhost:9095 whitetopic03 0 kafka://localhost:9094 whitetopic04 0 kafka://localhost:9095 whitetopic04 0
-
-    end_timer
-    info "embedded consumer took $((t_end - t_begin)) seconds"
-
-    sleep 2
-
-    # if [[ -d /tmp/kafka-target-1-1-logs/blacktopic01 || /tmp/kafka-target-1-2-logs/blacktopic01 ]]; then
-    #     echo "blacktopic01 found on target cluster"
-    #     result=1
-    # else
-    #     cmp_logs whitetopic01 && cmp_logs whitetopic02 && cmp_logs whitetopic03 && cmp_logs whitetopic04
-    #     result=$?
-    # fi
-
-    cmp_logs blacktopic01
-
-    cmp_logs whitetopic01 && cmp_logs whitetopic02 && cmp_logs whitetopic03 && cmp_logs whitetopic04
-    result=$?
-
-    return $result
-}
-
-test_blacklists() {
-    info "### Testing blacklists"
-    snapshot_prefix="blacklist-test"
-    cleanup
-    start_zk
-    start_source_servers
-    start_target_servers
-    sleep 4
-
-    info "starting mirror maker"
-    $base_dir/../../bin/kafka-run-class.sh kafka.tools.MirrorMaker --consumer.config $base_dir/config/blacklisttest.consumer.properties --producer.config $base_dir/config/mirror_producer.properties --blacklist="black.*" --num.streams 2 2>&1 > $base_dir/kafka_mirrormaker_1.log &
-    pid_mirrormaker_1=$!
-
-    start_producer blacktopic01 localhost:2181
-    start_producer blacktopic02 localhost:2181
-    info "waiting for producer to finish producing blacktopic01,blacktopic02 ..."
-    wait_partition_done kafka://localhost:9090 blacktopic01 0 kafka://localhost:9091 blacktopic01 0 kafka://localhost:9090 blacktopic02 0 kafka://localhost:9091 blacktopic02 0
-
-    begin_timer
-
-    start_producer whitetopic01 localhost:2181
-    info "waiting for producer to finish producing whitetopic01 ..."
-    wait_partition_done kafka://localhost:9090 whitetopic01 0 kafka://localhost:9091 whitetopic01 0
-
-    info "waiting for consumer to finish consuming ..."
-    wait_partition_done kafka://localhost:9094 whitetopic01 0 kafka://localhost:9095 whitetopic01 0
-
-    end_timer
-
-    info "embedded consumer took $((t_end - t_begin)) seconds"
-
-    sleep 2
-
-    cmp_logs blacktopic01 || cmp_logs blacktopic02
-    if [ $? -eq 0 ]; then
-        return 1
-    fi
-    
-    cmp_logs whitetopic01
-    return $?
-}
-
-# main test begins
-
-echo "Test-$test_start_time"
-
-# Ctrl-c trap. Catches INT signal
-trap "shutdown_servers; exit 0" INT
-
-test_whitelists
-result=$?
-
-process_test_result $result $action_on_fail
-
-shutdown_servers
- 
-sleep 2
- 
-test_blacklists
-result=$?
-
-process_test_result $result $action_on_fail
-
-shutdown_servers
-
-exit $result
-

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/mirror_maker/config/blacklisttest.consumer.properties
----------------------------------------------------------------------
diff --git a/system_test/mirror_maker/config/blacklisttest.consumer.properties b/system_test/mirror_maker/config/blacklisttest.consumer.properties
deleted file mode 100644
index ff12015..0000000
--- a/system_test/mirror_maker/config/blacklisttest.consumer.properties
+++ /dev/null
@@ -1,28 +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.
-# see kafka.consumer.ConsumerConfig for more details
-
-# zk connection string
-# comma separated host:port pairs, each corresponding to a zk
-# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"
-zk.connect=localhost:2181
-
-# timeout in ms for connecting to zookeeper
-zk.connection.timeout.ms=1000000
-
-#consumer group id
-group.id=group1
-shallow.iterator.enable=true
-

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/mirror_maker/config/mirror_producer.properties
----------------------------------------------------------------------
diff --git a/system_test/mirror_maker/config/mirror_producer.properties b/system_test/mirror_maker/config/mirror_producer.properties
deleted file mode 100644
index aa8be65..0000000
--- a/system_test/mirror_maker/config/mirror_producer.properties
+++ /dev/null
@@ -1,30 +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.
-# zk connection string
-# comma separated host:port pairs, each corresponding to a zk
-# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"
-zk.connect=localhost:2183
-# broker.list=1:localhost:9094,2:localhost:9095
-
-# timeout in ms for connecting to zookeeper
-# zk.connection.timeout.ms=1000000
-
-producer.type=async
-
-# to avoid dropping events if the queue is full, wait indefinitely
-queue.enqueue.timeout.ms=-1
-
-num.producers.per.broker=2
-

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/mirror_maker/config/server_source_1_1.properties
----------------------------------------------------------------------
diff --git a/system_test/mirror_maker/config/server_source_1_1.properties b/system_test/mirror_maker/config/server_source_1_1.properties
deleted file mode 100644
index 2f070a7..0000000
--- a/system_test/mirror_maker/config/server_source_1_1.properties
+++ /dev/null
@@ -1,76 +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.
-# see kafka.server.KafkaConfig for additional details and defaults
-
-# the id of the broker
-broker.id=1
-
-# hostname of broker. If not set, will pick up from the value returned
-# from getLocalHost.  If there are multiple interfaces getLocalHost
-# may not be what you want.
-# host.name=
-
-# number of logical partitions on this broker
-num.partitions=1
-
-# the port the socket server runs on
-port=9090
-
-# the number of processor threads the socket server uses. Defaults to the number of cores on the machine
-num.threads=8
-
-# the directory in which to store log files
-log.dir=/tmp/kafka-source-1-1-logs
-
-# the send buffer used by the socket server 
-socket.send.buffer.bytes=1048576
-
-# the receive buffer used by the socket server
-socket.receive.buffer.bytes=1048576
-
-# the maximum size of a log segment
-log.segment.bytes=10000000
-
-# the interval between running cleanup on the logs
-log.cleanup.interval.mins=1
-
-# the minimum age of a log file to eligible for deletion
-log.retention.hours=168
-
-#the number of messages to accept without flushing the log to disk
-log.flush.interval.messages=600
-
-#set the following properties to use zookeeper
-
-# enable connecting to zookeeper
-enable.zookeeper=true
-
-# zk connection string
-# comma separated host:port pairs, each corresponding to a zk
-# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"
-zk.connect=localhost:2181
-
-# timeout in ms for connecting to zookeeper
-zk.connection.timeout.ms=1000000
-
-# time based topic flush intervals in ms
-#log.flush.intervals.ms.per.topic=topic:1000
-
-# default time based flush interval in ms
-log.flush.interval.ms=1000
-
-# time based topic flasher time rate in ms
-log.flush.scheduler.interval.ms=1000
-

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/mirror_maker/config/server_source_1_2.properties
----------------------------------------------------------------------
diff --git a/system_test/mirror_maker/config/server_source_1_2.properties b/system_test/mirror_maker/config/server_source_1_2.properties
deleted file mode 100644
index f9353e8..0000000
--- a/system_test/mirror_maker/config/server_source_1_2.properties
+++ /dev/null
@@ -1,76 +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.
-# see kafka.server.KafkaConfig for additional details and defaults
-
-# the id of the broker
-broker.id=2
-
-# hostname of broker. If not set, will pick up from the value returned
-# from getLocalHost.  If there are multiple interfaces getLocalHost
-# may not be what you want.
-# host.name=
-
-# number of logical partitions on this broker
-num.partitions=1
-
-# the port the socket server runs on
-port=9091
-
-# the number of processor threads the socket server uses. Defaults to the number of cores on the machine
-num.threads=8
-
-# the directory in which to store log files
-log.dir=/tmp/kafka-source-1-2-logs
-
-# the send buffer used by the socket server 
-socket.send.buffer.bytes=1048576
-
-# the receive buffer used by the socket server
-socket.receive.buffer.bytes=1048576
-
-# the maximum size of a log segment
-log.segment.bytes=536870912
-
-# the interval between running cleanup on the logs
-log.cleanup.interval.mins=1
-
-# the minimum age of a log file to eligible for deletion
-log.retention.hours=168
-
-#the number of messages to accept without flushing the log to disk
-log.flush.interval.messages=600
-
-#set the following properties to use zookeeper
-
-# enable connecting to zookeeper
-enable.zookeeper=true
-
-# zk connection string
-# comma separated host:port pairs, each corresponding to a zk
-# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"
-zk.connect=localhost:2181
-
-# timeout in ms for connecting to zookeeper
-zk.connection.timeout.ms=1000000
-
-# time based topic flush intervals in ms
-#log.flush.intervals.ms.per.topic=topic:1000
-
-# default time based flush interval in ms
-log.flush.interval.ms=1000
-
-# time based topic flasher time rate in ms
-log.flush.scheduler.interval.ms=1000
-

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/mirror_maker/config/server_source_2_1.properties
----------------------------------------------------------------------
diff --git a/system_test/mirror_maker/config/server_source_2_1.properties b/system_test/mirror_maker/config/server_source_2_1.properties
deleted file mode 100644
index daa01ad..0000000
--- a/system_test/mirror_maker/config/server_source_2_1.properties
+++ /dev/null
@@ -1,76 +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.
-# see kafka.server.KafkaConfig for additional details and defaults
-
-# the id of the broker
-broker.id=1
-
-# hostname of broker. If not set, will pick up from the value returned
-# from getLocalHost.  If there are multiple interfaces getLocalHost
-# may not be what you want.
-# host.name=
-
-# number of logical partitions on this broker
-num.partitions=1
-
-# the port the socket server runs on
-port=9092
-
-# the number of processor threads the socket server uses. Defaults to the number of cores on the machine
-num.threads=8
-
-# the directory in which to store log files
-log.dir=/tmp/kafka-source-2-1-logs
-
-# the send buffer used by the socket server 
-socket.send.buffer.bytes=1048576
-
-# the receive buffer used by the socket server
-socket.receive.buffer.bytes=1048576
-
-# the maximum size of a log segment
-log.segment.bytes=536870912
-
-# the interval between running cleanup on the logs
-log.cleanup.interval.mins=1
-
-# the minimum age of a log file to eligible for deletion
-log.retention.hours=168
-
-#the number of messages to accept without flushing the log to disk
-log.flush.interval.messages=600
-
-#set the following properties to use zookeeper
-
-# enable connecting to zookeeper
-enable.zookeeper=true
-
-# zk connection string
-# comma separated host:port pairs, each corresponding to a zk
-# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"
-zk.connect=localhost:2182
-
-# timeout in ms for connecting to zookeeper
-zk.connection.timeout.ms=1000000
-
-# time based topic flush intervals in ms
-#log.flush.intervals.ms.per.topic=topic:1000
-
-# default time based flush interval in ms
-log.flush.interval.ms=1000
-
-# time based topic flasher time rate in ms
-log.flush.scheduler.interval.ms=1000
-

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/mirror_maker/config/server_source_2_2.properties
----------------------------------------------------------------------
diff --git a/system_test/mirror_maker/config/server_source_2_2.properties b/system_test/mirror_maker/config/server_source_2_2.properties
deleted file mode 100644
index be6fdfc..0000000
--- a/system_test/mirror_maker/config/server_source_2_2.properties
+++ /dev/null
@@ -1,76 +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.
-# see kafka.server.KafkaConfig for additional details and defaults
-
-# the id of the broker
-broker.id=2
-
-# hostname of broker. If not set, will pick up from the value returned
-# from getLocalHost.  If there are multiple interfaces getLocalHost
-# may not be what you want.
-# host.name=
-
-# number of logical partitions on this broker
-num.partitions=1
-
-# the port the socket server runs on
-port=9093
-
-# the number of processor threads the socket server uses. Defaults to the number of cores on the machine
-num.threads=8
-
-# the directory in which to store log files
-log.dir=/tmp/kafka-source-2-2-logs
-
-# the send buffer used by the socket server 
-socket.send.buffer.bytes=1048576
-
-# the receive buffer used by the socket server
-socket.receive.buffer.bytes=1048576
-
-# the maximum size of a log segment
-log.segment.bytes=536870912
-
-# the interval between running cleanup on the logs
-log.cleanup.interval.mins=1
-
-# the minimum age of a log file to eligible for deletion
-log.retention.hours=168
-
-#the number of messages to accept without flushing the log to disk
-log.flush.interval.messages=600
-
-#set the following properties to use zookeeper
-
-# enable connecting to zookeeper
-enable.zookeeper=true
-
-# zk connection string
-# comma separated host:port pairs, each corresponding to a zk
-# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"
-zk.connect=localhost:2182
-
-# timeout in ms for connecting to zookeeper
-zk.connection.timeout.ms=1000000
-
-# time based topic flush intervals in ms
-#log.flush.intervals.ms.per.topic=topic:1000
-
-# default time based flush interval in ms
-log.flush.interval.ms=1000
-
-# time based topic flasher time rate in ms
-log.flush.scheduler.interval.ms=1000
-

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/mirror_maker/config/server_target_1_1.properties
----------------------------------------------------------------------
diff --git a/system_test/mirror_maker/config/server_target_1_1.properties b/system_test/mirror_maker/config/server_target_1_1.properties
deleted file mode 100644
index d37955a..0000000
--- a/system_test/mirror_maker/config/server_target_1_1.properties
+++ /dev/null
@@ -1,78 +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.
-# see kafka.server.KafkaConfig for additional details and defaults
-
-# the id of the broker
-broker.id=1
-
-# hostname of broker. If not set, will pick up from the value returned
-# from getLocalHost.  If there are multiple interfaces getLocalHost
-# may not be what you want.
-# host.name=
-
-# number of logical partitions on this broker
-num.partitions=1
-
-# the port the socket server runs on
-port=9094
-
-# the number of processor threads the socket server uses. Defaults to the number of cores on the machine
-num.threads=8
-
-# the directory in which to store log files
-log.dir=/tmp/kafka-target-1-1-logs
-
-# the send buffer used by the socket server 
-socket.send.buffer.bytes=1048576
-
-# the receive buffer used by the socket server
-socket.receive.buffer.bytes=1048576
-
-# the maximum size of a log segment
-log.segment.bytes=536870912
-
-# the interval between running cleanup on the logs
-log.cleanup.interval.mins=1
-
-# the minimum age of a log file to eligible for deletion
-log.retention.hours=168
-
-#the number of messages to accept without flushing the log to disk
-log.flush.interval.messages=600
-
-#set the following properties to use zookeeper
-
-# enable connecting to zookeeper
-enable.zookeeper=true
-
-# zk connection string
-# comma separated host:port pairs, each corresponding to a zk
-# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"
-zk.connect=localhost:2183
-
-# timeout in ms for connecting to zookeeper
-zk.connection.timeout.ms=1000000
-
-# time based topic flush intervals in ms
-#log.flush.intervals.ms.per.topic=topic:1000
-
-# default time based flush interval in ms
-log.flush.interval.ms=1000
-
-# time based topic flasher time rate in ms
-log.flush.scheduler.interval.ms=1000
-
-# topic partition count map
-# topic.partition.count.map=topic1:3, topic2:4

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/mirror_maker/config/server_target_1_2.properties
----------------------------------------------------------------------
diff --git a/system_test/mirror_maker/config/server_target_1_2.properties b/system_test/mirror_maker/config/server_target_1_2.properties
deleted file mode 100644
index aa7546c..0000000
--- a/system_test/mirror_maker/config/server_target_1_2.properties
+++ /dev/null
@@ -1,78 +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.
-# see kafka.server.KafkaConfig for additional details and defaults
-
-# the id of the broker
-broker.id=2
-
-# hostname of broker. If not set, will pick up from the value returned
-# from getLocalHost.  If there are multiple interfaces getLocalHost
-# may not be what you want.
-# host.name=
-
-# number of logical partitions on this broker
-num.partitions=1
-
-# the port the socket server runs on
-port=9095
-
-# the number of processor threads the socket server uses. Defaults to the number of cores on the machine
-num.threads=8
-
-# the directory in which to store log files
-log.dir=/tmp/kafka-target-1-2-logs
-
-# the send buffer used by the socket server 
-socket.send.buffer.bytes=1048576
-
-# the receive buffer used by the socket server
-socket.receive.buffer.bytes=1048576
-
-# the maximum size of a log segment
-log.segment.bytes=536870912
-
-# the interval between running cleanup on the logs
-log.cleanup.interval.mins=1
-
-# the minimum age of a log file to eligible for deletion
-log.retention.hours=168
-
-#the number of messages to accept without flushing the log to disk
-log.flush.interval.messages=600
-
-#set the following properties to use zookeeper
-
-# enable connecting to zookeeper
-enable.zookeeper=true
-
-# zk connection string
-# comma separated host:port pairs, each corresponding to a zk
-# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"
-zk.connect=localhost:2183
-
-# timeout in ms for connecting to zookeeper
-zk.connection.timeout.ms=1000000
-
-# time based topic flush intervals in ms
-#log.flush.intervals.ms.per.topic=topic:1000
-
-# default time based flush interval in ms
-log.flush.interval.ms=1000
-
-# time based topic flasher time rate in ms
-log.flush.scheduler.interval.ms=1000
-
-# topic partition count map
-# topic.partition.count.map=topic1:3, topic2:4

http://git-wip-us.apache.org/repos/asf/kafka/blob/a670537a/system_test/mirror_maker/config/whitelisttest_1.consumer.properties
----------------------------------------------------------------------
diff --git a/system_test/mirror_maker/config/whitelisttest_1.consumer.properties b/system_test/mirror_maker/config/whitelisttest_1.consumer.properties
deleted file mode 100644
index ff12015..0000000
--- a/system_test/mirror_maker/config/whitelisttest_1.consumer.properties
+++ /dev/null
@@ -1,28 +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.
-# see kafka.consumer.ConsumerConfig for more details
-
-# zk connection string
-# comma separated host:port pairs, each corresponding to a zk
-# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"
-zk.connect=localhost:2181
-
-# timeout in ms for connecting to zookeeper
-zk.connection.timeout.ms=1000000
-
-#consumer group id
-group.id=group1
-shallow.iterator.enable=true
-