diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index decba0e..e055d67 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -80,4 +80,4 @@ else JAVA="$JAVA_HOME/bin/java" fi -$JAVA $KAFKA_OPTS $KAFKA_JMX_OPTS -cp $CLASSPATH $@ +$JAVA $KAFKA_OPTS $KAFKA_JMX_OPTS -cp $CLASSPATH "$@" diff --git a/config/log4j.properties b/config/log4j.properties index 1891f38..c611786 100644 --- a/config/log4j.properties +++ b/config/log4j.properties @@ -42,6 +42,12 @@ log4j.appender.cleanerAppender.File=log-cleaner.log log4j.appender.cleanerAppender.layout=org.apache.log4j.PatternLayout log4j.appender.cleanerAppender.layout.ConversionPattern=[%d] %p %m (%c)%n +log4j.appender.controllerAppender=org.apache.log4j.DailyRollingFileAppender +log4j.appender.controllerAppender.DatePattern='.'yyyy-MM-dd-HH +log4j.appender.controllerAppender.File=controller.log +log4j.appender.controllerAppender.layout=org.apache.log4j.PatternLayout +log4j.appender.controllerAppender.layout.ConversionPattern=[%d] %p %m (%c)%n + # Turn on all our debugging info #log4j.logger.kafka.producer.async.DefaultEventHandler=DEBUG, kafkaAppender #log4j.logger.kafka.client.ClientUtils=DEBUG, kafkaAppender @@ -59,10 +65,13 @@ log4j.additivity.kafka.network.RequestChannel$=false log4j.logger.kafka.request.logger=WARN, requestAppender log4j.additivity.kafka.request.logger=false -log4j.logger.kafka.controller=TRACE, stateChangeAppender +log4j.logger.kafka.controller=TRACE, controllerAppender log4j.additivity.kafka.controller=false log4j.logger.kafka.log.LogCleaner=INFO, cleanerAppender log4j.additivity.kafka.log.LogCleaner=false log4j.logger.kafka.log.Cleaner=INFO, cleanerAppender log4j.additivity.kafka.log.Cleaner=false + +log4j.logger.state.change.logger=TRACE, stateChangeAppender +log4j.additivity.state.change.logger=false diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 6479385..b896182 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -28,6 +28,7 @@ import kafka.utils.{Logging, Utils, ZkUtils, Json} import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.exception.ZkNodeExistsException import scala.collection._ +import mutable.ListBuffer import scala.collection.mutable import kafka.common._ import scala.Some @@ -208,13 +209,14 @@ object AdminUtils extends Logging { var replicaInfo: Seq[Broker] = Nil var isrInfo: Seq[Broker] = Nil try { - try { - leaderInfo = leader match { - case Some(l) => Some(getBrokerInfoFromCache(zkClient, cachedBrokerInfo, List(l)).head) - case None => throw new LeaderNotAvailableException("No leader exists for partition " + partition) - } - } catch { - case e => throw new LeaderNotAvailableException("Leader not available for topic %s partition %d".format(topic, partition), e) + leaderInfo = leader match { + case Some(l) => + try { + Some(getBrokerInfoFromCache(zkClient, cachedBrokerInfo, List(l)).head) + } catch { + case e => throw new LeaderNotAvailableException("Leader not available for topic %s partition %d".format(topic, partition), e) + } + case None => throw new LeaderNotAvailableException("No leader exists for partition " + partition) } try { replicaInfo = getBrokerInfoFromCache(zkClient, cachedBrokerInfo, replicas.map(id => id.toInt)) @@ -222,12 +224,18 @@ object AdminUtils extends Logging { } catch { case e => throw new ReplicaNotAvailableException(e) } + 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 < inSyncReplicas.size) + throw new ReplicaNotAvailableException("In Sync Replica information not available for following brokers: " + + inSyncReplicas.filterNot(isrInfo.map(_.id).contains(_)).mkString(",")) new PartitionMetadata(partition, leaderInfo, replicaInfo, isrInfo, ErrorMapping.NoError) } catch { case e => error("Error while fetching metadata for partition [%s,%d]".format(topic, partition), e) new PartitionMetadata(partition, leaderInfo, replicaInfo, isrInfo, - ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) + ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) } } new TopicMetadata(topic, partitionMetadata) @@ -240,19 +248,23 @@ object AdminUtils extends Logging { private def getBrokerInfoFromCache(zkClient: ZkClient, cachedBrokerInfo: scala.collection.mutable.Map[Int, Broker], brokerIds: Seq[Int]): Seq[Broker] = { - brokerIds.map { id => + var failedBrokerIds: ListBuffer[Int] = new ListBuffer() + val brokerMetadata = brokerIds.map { id => val optionalBrokerInfo = cachedBrokerInfo.get(id) optionalBrokerInfo match { - case Some(brokerInfo) => brokerInfo // return broker info from the cache + case Some(brokerInfo) => Some(brokerInfo) // return broker info from the cache case None => // fetch it from zookeeper ZkUtils.getBrokerInfo(zkClient, id) match { case Some(brokerInfo) => cachedBrokerInfo += (id -> brokerInfo) - brokerInfo - case None => throw new BrokerNotAvailableException("Failed to fetch broker info for broker " + id) + Some(brokerInfo) + case None => + failedBrokerIds += id + None } } } + brokerMetadata.filter(_.isDefined).map(_.get) } private def replicaIndex(firstReplicaIndex: Int, secondReplicaShift: Int, replicaIndex: Int, nBrokers: Int): Int = { diff --git a/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala b/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala index 49342c6..a2afd16 100644 --- a/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala +++ b/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala @@ -21,6 +21,8 @@ import kafka.utils._ import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.exception.ZkNodeExistsException import kafka.common.{TopicAndPartition, AdminCommandFailedException} +import collection._ +import mutable.ListBuffer object PreferredReplicaLeaderElectionCommand extends Logging { @@ -28,12 +30,11 @@ object PreferredReplicaLeaderElectionCommand extends Logging { val parser = new OptionParser val jsonFileOpt = parser.accepts("path-to-json-file", "The JSON file with the list of partitions " + "for which preferred replica leader election should be done, in the following format - \n" + - "[{\"topic\": \"foo\", \"partition\": \"1\"}, {\"topic\": \"foobar\", \"partition\": \"2\"}]. \n" + + "{\"partitions\":\n\t[{\"topic\": \"foo\", \"partition\": 1},\n\t {\"topic\": \"foobar\", \"partition\": 2}]\n}\n" + "Defaults to all existing partitions") .withRequiredArg .describedAs("list of partitions for which preferred replica leader election needs to be triggered") .ofType(classOf[String]) - .defaultsTo("") val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the " + "form host:port. Multiple URLS can be given to allow fail-over.") .withRequiredArg @@ -42,28 +43,20 @@ object PreferredReplicaLeaderElectionCommand extends Logging { val options = parser.parse(args : _*) - CommandLineUtils.checkRequiredArgs(parser, options, jsonFileOpt, zkConnectOpt) + CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt) - val jsonFile = options.valueOf(jsonFileOpt) val zkConnect = options.valueOf(zkConnectOpt) - val jsonString = Utils.readFileAsString(jsonFile) var zkClient: ZkClient = null try { zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) val partitionsForPreferredReplicaElection = - if(jsonFile == "") ZkUtils.getAllPartitions(zkClient) else parsePreferredReplicaJsonData(jsonString) + if (!options.has(jsonFileOpt)) + ZkUtils.getAllPartitions(zkClient) + else + parsePreferredReplicaJsonData(Utils.readFileAsString(options.valueOf(jsonFileOpt))) val preferredReplicaElectionCommand = new PreferredReplicaLeaderElectionCommand(zkClient, partitionsForPreferredReplicaElection) - // attach shutdown handler to catch control-c - Runtime.getRuntime().addShutdownHook(new Thread() { - override def run() = { - // delete the admin path so it can be retried - ZkUtils.deletePathRecursive(zkClient, ZkUtils.PreferredReplicaLeaderElectionPath) - zkClient.close() - } - }) - preferredReplicaElectionCommand.moveLeaderToPreferredReplica() println("Successfully started preferred replica election for partitions %s".format(partitionsForPreferredReplicaElection)) } catch { @@ -76,14 +69,18 @@ object PreferredReplicaLeaderElectionCommand extends Logging { } } - def parsePreferredReplicaJsonData(jsonString: String): Set[TopicAndPartition] = { + def parsePreferredReplicaJsonData(jsonString: String): immutable.Set[TopicAndPartition] = { Json.parseFull(jsonString) match { - case Some(partitionList) => - val partitions = (partitionList.asInstanceOf[List[Any]]) - Set.empty[TopicAndPartition] ++ partitions.map { m => - val topic = m.asInstanceOf[Map[String, String]].get("topic").get - val partition = m.asInstanceOf[Map[String, String]].get("partition").get.toInt - TopicAndPartition(topic, partition) + case Some(m) => + m.asInstanceOf[Map[String, Any]].get("partitions") match { + case Some(partitionsList) => + val partitions = partitionsList.asInstanceOf[List[Map[String, Any]]] + partitions.map { p => + val topic = p.get("topic").get.asInstanceOf[String] + val partition = p.get("partition").get.asInstanceOf[Int] + TopicAndPartition(topic, partition) + }.toSet + case None => throw new AdminOperationException("Preferred replica election data is empty") } case None => throw new AdminOperationException("Preferred replica election data is empty") } @@ -92,9 +89,13 @@ object PreferredReplicaLeaderElectionCommand extends Logging { def writePreferredReplicaElectionData(zkClient: ZkClient, partitionsUndergoingPreferredReplicaElection: scala.collection.Set[TopicAndPartition]) { val zkPath = ZkUtils.PreferredReplicaLeaderElectionPath - val jsonData = Utils.seqToJson(partitionsUndergoingPreferredReplicaElection.map { p => - Utils.mapToJson(Map(("topic" -> p.topic), ("partition" -> p.partition.toString)), valueInQuotes = true) - }.toSeq.sorted, valueInQuotes = false) + var partitionsData: mutable.ListBuffer[String] = ListBuffer[String]() + for (p <- partitionsUndergoingPreferredReplicaElection) { + partitionsData += Utils.mergeJsonFields(Utils.mapToJsonFields(Map("topic" -> p.topic), valueInQuotes = true) ++ + Utils.mapToJsonFields(Map("partition" -> p.partition.toString), valueInQuotes = false)) + } + val jsonPartitionsData = Utils.seqToJson(partitionsData, valueInQuotes = false) + val jsonData = Utils.mapToJson(Map("version" -> 1.toString, "partitions" -> jsonPartitionsData), valueInQuotes = false) try { ZkUtils.createPersistentPath(zkClient, zkPath, jsonData) info("Created preferred replica election path with %s".format(jsonData)) diff --git a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala index b2204b8..8d287f4 100644 --- a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala +++ b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala @@ -28,7 +28,7 @@ object ReassignPartitionsCommand extends Logging { val parser = new OptionParser val jsonFileOpt = parser.accepts("path-to-json-file", "REQUIRED: The JSON file with the list of partitions and the " + "new replicas they should be reassigned to in the following format - \n" + - "[{\"topic\": \"foo\", \"partition\": \"1\", \"replicas\": \"1,2,3\" }]") + "{\"partitions\":\n\t[{\"topic\": \"foo\",\n\t \"partition\": 1,\n\t \"replicas\": [1,2,3] }]\n}") .withRequiredArg .describedAs("partition reassignment json file path") .ofType(classOf[String]) @@ -55,30 +55,13 @@ object ReassignPartitionsCommand extends Logging { try { // read the json file into a string - val partitionsToBeReassigned = Json.parseFull(jsonString) match { - case Some(reassignedPartitions) => - val partitions = reassignedPartitions.asInstanceOf[Array[Map[String, String]]] - partitions.map { m => - val topic = m.asInstanceOf[Map[String, String]].get("topic").get - val partition = m.asInstanceOf[Map[String, String]].get("partition").get.toInt - val replicasList = m.asInstanceOf[Map[String, String]].get("replicas").get - val newReplicas = replicasList.split(",").map(_.toInt) - (TopicAndPartition(topic, partition), newReplicas.toSeq) - }.toMap - case None => throw new AdminCommandFailedException("Partition reassignment data file %s is empty".format(jsonFile)) - } + val partitionsToBeReassigned = ZkUtils.parsePartitionReassignmentData(jsonString) + if (partitionsToBeReassigned.isEmpty) + throw new AdminCommandFailedException("Partition reassignment data file %s is empty".format(jsonFile)) zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, partitionsToBeReassigned) - // attach shutdown handler to catch control-c - Runtime.getRuntime().addShutdownHook(new Thread() { - override def run() = { - // delete the admin path so it can be retried - ZkUtils.deletePathRecursive(zkClient, ZkUtils.ReassignPartitionsPath) - } - }) - if(reassignPartitionsCommand.reassignPartitions()) println("Successfully started reassignment of partitions %s".format(partitionsToBeReassigned)) else @@ -94,15 +77,15 @@ object ReassignPartitionsCommand extends Logging { } } -class ReassignPartitionsCommand(zkClient: ZkClient, partitions: collection.immutable.Map[TopicAndPartition, Seq[Int]]) +class ReassignPartitionsCommand(zkClient: ZkClient, partitions: collection.Map[TopicAndPartition, collection.Seq[Int]]) extends Logging { def reassignPartitions(): Boolean = { try { val validPartitions = partitions.filter(p => validatePartition(zkClient, p._1.topic, p._1.partition)) - val jsonReassignmentData = Utils.mapWithSeqValuesToJson(validPartitions.map(p => ("%s,%s".format(p._1.topic, p._1.partition)) -> p._2)) + val jsonReassignmentData = ZkUtils.getPartitionReassignmentZkData(validPartitions) ZkUtils.createPersistentPath(zkClient, ZkUtils.ReassignPartitionsPath, jsonReassignmentData) true - }catch { + } catch { case ze: ZkNodeExistsException => val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient) throw new AdminCommandFailedException("Partition reassignment currently in " + diff --git a/core/src/main/scala/kafka/api/FetchRequest.scala b/core/src/main/scala/kafka/api/FetchRequest.scala index dc4ed8e..a807c1f 100644 --- a/core/src/main/scala/kafka/api/FetchRequest.scala +++ b/core/src/main/scala/kafka/api/FetchRequest.scala @@ -59,13 +59,13 @@ object FetchRequest { } case class FetchRequest private[kafka] (versionId: Short = FetchRequest.CurrentVersion, - correlationId: Int = FetchRequest.DefaultCorrelationId, + override val correlationId: Int = FetchRequest.DefaultCorrelationId, clientId: String = ConsumerConfig.DefaultClientId, replicaId: Int = Request.OrdinaryConsumerId, maxWait: Int = FetchRequest.DefaultMaxWait, minBytes: Int = FetchRequest.DefaultMinBytes, requestInfo: Map[TopicAndPartition, PartitionFetchInfo]) - extends RequestOrResponse(Some(RequestKeys.FetchKey)) { + extends RequestOrResponse(Some(RequestKeys.FetchKey), correlationId) { /** * Partitions the request info into a map of maps (one for each topic). diff --git a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala index d146b14..b40522d 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala @@ -93,6 +93,7 @@ object LeaderAndIsrRequest { val correlationId = buffer.getInt val clientId = readShortString(buffer) val ackTimeoutMs = buffer.getInt + val controllerId = buffer.getInt val controllerEpoch = buffer.getInt val partitionStateInfosCount = buffer.getInt val partitionStateInfos = new collection.mutable.HashMap[(String, Int), PartitionStateInfo] @@ -110,23 +111,24 @@ object LeaderAndIsrRequest { for (i <- 0 until leadersCount) leaders += Broker.readFrom(buffer) - new LeaderAndIsrRequest(versionId, correlationId, clientId, ackTimeoutMs, partitionStateInfos.toMap, leaders, controllerEpoch) + new LeaderAndIsrRequest(versionId, correlationId, clientId, ackTimeoutMs, controllerId, controllerEpoch, partitionStateInfos.toMap, leaders) } } case class LeaderAndIsrRequest (versionId: Short, - correlationId: Int, + override val correlationId: Int, clientId: String, ackTimeoutMs: Int, + controllerId: Int, + controllerEpoch: Int, partitionStateInfos: Map[(String, Int), PartitionStateInfo], - leaders: Set[Broker], - controllerEpoch: Int) - extends RequestOrResponse(Some(RequestKeys.LeaderAndIsrKey)) { + leaders: Set[Broker]) + extends RequestOrResponse(Some(RequestKeys.LeaderAndIsrKey), correlationId) { - def this(partitionStateInfos: Map[(String, Int), PartitionStateInfo], liveBrokers: Set[Broker], + def this(partitionStateInfos: Map[(String, Int), PartitionStateInfo], liveBrokers: Set[Broker], controllerId: Int, controllerEpoch: Int, correlationId: Int) = { this(LeaderAndIsrRequest.CurrentVersion, correlationId, LeaderAndIsrRequest.DefaultClientId, LeaderAndIsrRequest.DefaultAckTimeout, - partitionStateInfos, liveBrokers, controllerEpoch) + controllerId, controllerEpoch, partitionStateInfos, liveBrokers) } def writeTo(buffer: ByteBuffer) { @@ -134,6 +136,7 @@ case class LeaderAndIsrRequest (versionId: Short, buffer.putInt(correlationId) writeShortString(buffer, clientId) buffer.putInt(ackTimeoutMs) + buffer.putInt(controllerId) buffer.putInt(controllerEpoch) buffer.putInt(partitionStateInfos.size) for((key, value) <- partitionStateInfos){ @@ -151,6 +154,7 @@ case class LeaderAndIsrRequest (versionId: Short, 4 /* correlation id */ + (2 + clientId.length) /* client id */ + 4 /* ack timeout */ + + 4 /* controller id */ + 4 /* controller epoch */ + 4 /* number of partitions */ for((key, value) <- partitionStateInfos) @@ -165,10 +169,11 @@ case class LeaderAndIsrRequest (versionId: Short, val leaderAndIsrRequest = new StringBuilder leaderAndIsrRequest.append("Name: " + this.getClass.getSimpleName) leaderAndIsrRequest.append("; Version: " + versionId) + leaderAndIsrRequest.append("; Controller: " + controllerId) + leaderAndIsrRequest.append("; ControllerEpoch: " + controllerEpoch) leaderAndIsrRequest.append("; CorrelationId: " + correlationId) leaderAndIsrRequest.append("; ClientId: " + clientId) leaderAndIsrRequest.append("; AckTimeoutMs: " + ackTimeoutMs + " ms") - leaderAndIsrRequest.append("; ControllerEpoch: " + controllerEpoch) leaderAndIsrRequest.append("; PartitionStateInfo: " + partitionStateInfos.mkString(",")) leaderAndIsrRequest.append("; Leaders: " + leaders.mkString(",")) leaderAndIsrRequest.toString() diff --git a/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala b/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala index dbd85d0..b4cfae8 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala @@ -41,10 +41,10 @@ object LeaderAndIsrResponse { } -case class LeaderAndIsrResponse(correlationId: Int, +case class LeaderAndIsrResponse(override val correlationId: Int, responseMap: Map[(String, Int), Short], errorCode: Short = ErrorMapping.NoError) - extends RequestOrResponse { + extends RequestOrResponse(correlationId = correlationId) { def sizeInBytes(): Int ={ var size = 4 /* correlation id */ + diff --git a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala index 1ca37e2..1cbe6e8 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala @@ -54,9 +54,9 @@ object OffsetCommitRequest extends Logging { case class OffsetCommitRequest(groupId: String, requestInfo: Map[TopicAndPartition, OffsetMetadataAndError], versionId: Short = OffsetCommitRequest.CurrentVersion, - correlationId: Int = 0, + override val correlationId: Int = 0, clientId: String = OffsetCommitRequest.DefaultClientId) - extends RequestOrResponse(Some(RequestKeys.OffsetCommitKey)) { + extends RequestOrResponse(Some(RequestKeys.OffsetCommitKey), correlationId) { lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) diff --git a/core/src/main/scala/kafka/api/OffsetCommitResponse.scala b/core/src/main/scala/kafka/api/OffsetCommitResponse.scala index 3b0d861..cbb5fa1 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitResponse.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitResponse.scala @@ -48,9 +48,9 @@ object OffsetCommitResponse extends Logging { } case class OffsetCommitResponse(requestInfo: Map[TopicAndPartition, Short], - correlationId: Int = 0, + override val correlationId: Int = 0, clientId: String = OffsetCommitResponse.DefaultClientId) - extends RequestOrResponse { + extends RequestOrResponse(correlationId = correlationId) { lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) diff --git a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala index fe94f17..a4c5623 100644 --- a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala @@ -52,9 +52,9 @@ object OffsetFetchRequest extends Logging { case class OffsetFetchRequest(groupId: String, requestInfo: Seq[TopicAndPartition], versionId: Short = OffsetFetchRequest.CurrentVersion, - correlationId: Int = 0, + override val correlationId: Int = 0, clientId: String = OffsetFetchRequest.DefaultClientId) - extends RequestOrResponse(Some(RequestKeys.OffsetFetchKey)) { + extends RequestOrResponse(Some(RequestKeys.OffsetFetchKey), correlationId) { lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_.topic) diff --git a/core/src/main/scala/kafka/api/OffsetFetchResponse.scala b/core/src/main/scala/kafka/api/OffsetFetchResponse.scala index 3d4ce2a..71c2efb 100644 --- a/core/src/main/scala/kafka/api/OffsetFetchResponse.scala +++ b/core/src/main/scala/kafka/api/OffsetFetchResponse.scala @@ -50,9 +50,9 @@ object OffsetFetchResponse extends Logging { } case class OffsetFetchResponse(requestInfo: Map[TopicAndPartition, OffsetMetadataAndError], - correlationId: Int = 0, + override val correlationId: Int = 0, clientId: String = OffsetFetchResponse.DefaultClientId) - extends RequestOrResponse { + extends RequestOrResponse(correlationId = correlationId) { lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) diff --git a/core/src/main/scala/kafka/api/OffsetRequest.scala b/core/src/main/scala/kafka/api/OffsetRequest.scala index 6360a98..32ebfd4 100644 --- a/core/src/main/scala/kafka/api/OffsetRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetRequest.scala @@ -57,10 +57,10 @@ case class PartitionOffsetRequestInfo(time: Long, maxNumOffsets: Int) case class OffsetRequest(requestInfo: Map[TopicAndPartition, PartitionOffsetRequestInfo], versionId: Short = OffsetRequest.CurrentVersion, - correlationId: Int = 0, + override val correlationId: Int = 0, clientId: String = OffsetRequest.DefaultClientId, replicaId: Int = Request.OrdinaryConsumerId) - extends RequestOrResponse(Some(RequestKeys.OffsetsKey)) { + extends RequestOrResponse(Some(RequestKeys.OffsetsKey), correlationId) { def this(requestInfo: Map[TopicAndPartition, PartitionOffsetRequestInfo], correlationId: Int, replicaId: Int) = this(requestInfo, OffsetRequest.CurrentVersion, correlationId, OffsetRequest.DefaultClientId, replicaId) diff --git a/core/src/main/scala/kafka/api/OffsetResponse.scala b/core/src/main/scala/kafka/api/OffsetResponse.scala index 264e200..08dc3cd 100644 --- a/core/src/main/scala/kafka/api/OffsetResponse.scala +++ b/core/src/main/scala/kafka/api/OffsetResponse.scala @@ -47,9 +47,9 @@ object OffsetResponse { case class PartitionOffsetsResponse(error: Short, offsets: Seq[Long]) -case class OffsetResponse(correlationId: Int, +case class OffsetResponse(override val correlationId: Int, partitionErrorAndOffsets: Map[TopicAndPartition, PartitionOffsetsResponse]) - extends RequestOrResponse { + extends RequestOrResponse(correlationId = correlationId) { lazy val offsetsGroupedByTopic = partitionErrorAndOffsets.groupBy(_._1.topic) diff --git a/core/src/main/scala/kafka/api/ProducerRequest.scala b/core/src/main/scala/kafka/api/ProducerRequest.scala index 916fb59..fda3e39 100644 --- a/core/src/main/scala/kafka/api/ProducerRequest.scala +++ b/core/src/main/scala/kafka/api/ProducerRequest.scala @@ -19,7 +19,6 @@ package kafka.api import java.nio._ import kafka.message._ -import scala.collection.Map import kafka.api.ApiUtils._ import kafka.common._ import kafka.network.RequestChannel.Response @@ -54,12 +53,12 @@ object ProducerRequest { } case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion, - correlationId: Int, + override val correlationId: Int, clientId: String, requiredAcks: Short, ackTimeoutMs: Int, data: collection.mutable.Map[TopicAndPartition, ByteBufferMessageSet]) - extends RequestOrResponse(Some(RequestKeys.ProduceKey)) { + extends RequestOrResponse(Some(RequestKeys.ProduceKey), correlationId) { /** * Partitions the data into a map of maps (one for each topic). diff --git a/core/src/main/scala/kafka/api/ProducerResponse.scala b/core/src/main/scala/kafka/api/ProducerResponse.scala index 5bff709..d59c5bb 100644 --- a/core/src/main/scala/kafka/api/ProducerResponse.scala +++ b/core/src/main/scala/kafka/api/ProducerResponse.scala @@ -43,8 +43,9 @@ object ProducerResponse { case class ProducerResponseStatus(error: Short, offset: Long) -case class ProducerResponse(correlationId: Int, - status: Map[TopicAndPartition, ProducerResponseStatus]) extends RequestOrResponse { +case class ProducerResponse(override val correlationId: Int, + status: Map[TopicAndPartition, ProducerResponseStatus]) + extends RequestOrResponse(correlationId = correlationId) { /** * Partitions the status map into a map of maps (one for each topic). diff --git a/core/src/main/scala/kafka/api/RequestOrResponse.scala b/core/src/main/scala/kafka/api/RequestOrResponse.scala index 3175e1c..b62330b 100644 --- a/core/src/main/scala/kafka/api/RequestOrResponse.scala +++ b/core/src/main/scala/kafka/api/RequestOrResponse.scala @@ -27,7 +27,7 @@ object Request { } -private[kafka] abstract class RequestOrResponse(val requestId: Option[Short] = None) extends Logging{ +private[kafka] abstract class RequestOrResponse(val requestId: Option[Short] = None, val correlationId: Int) extends Logging{ def sizeInBytes: Int diff --git a/core/src/main/scala/kafka/api/StopReplicaRequest.scala b/core/src/main/scala/kafka/api/StopReplicaRequest.scala index be3c7be..cd55db4 100644 --- a/core/src/main/scala/kafka/api/StopReplicaRequest.scala +++ b/core/src/main/scala/kafka/api/StopReplicaRequest.scala @@ -36,6 +36,7 @@ object StopReplicaRequest extends Logging { val correlationId = buffer.getInt val clientId = readShortString(buffer) val ackTimeoutMs = buffer.getInt + val controllerId = buffer.getInt val controllerEpoch = buffer.getInt val deletePartitions = buffer.get match { case 1 => true @@ -48,22 +49,24 @@ object StopReplicaRequest extends Logging { (1 to topicPartitionPairCount) foreach { _ => topicPartitionPairSet.add(readShortString(buffer), buffer.getInt) } - StopReplicaRequest(versionId, correlationId, clientId, ackTimeoutMs, deletePartitions, topicPartitionPairSet.toSet, controllerEpoch) + StopReplicaRequest(versionId, correlationId, clientId, ackTimeoutMs, controllerId, controllerEpoch, + deletePartitions, topicPartitionPairSet.toSet) } } case class StopReplicaRequest(versionId: Short, - correlationId: Int, + override val correlationId: Int, clientId: String, ackTimeoutMs: Int, + controllerId: Int, + controllerEpoch: Int, deletePartitions: Boolean, - partitions: Set[(String, Int)], - controllerEpoch: Int) - extends RequestOrResponse(Some(RequestKeys.StopReplicaKey)) { + partitions: Set[(String, Int)]) + extends RequestOrResponse(Some(RequestKeys.StopReplicaKey), correlationId) { - def this(deletePartitions: Boolean, partitions: Set[(String, Int)], controllerEpoch: Int, correlationId: Int) = { + def this(deletePartitions: Boolean, partitions: Set[(String, Int)], controllerId: Int, controllerEpoch: Int, correlationId: Int) = { this(StopReplicaRequest.CurrentVersion, correlationId, StopReplicaRequest.DefaultClientId, StopReplicaRequest.DefaultAckTimeout, - deletePartitions, partitions, controllerEpoch) + controllerId, controllerEpoch, deletePartitions, partitions) } def writeTo(buffer: ByteBuffer) { @@ -71,6 +74,7 @@ case class StopReplicaRequest(versionId: Short, buffer.putInt(correlationId) writeShortString(buffer, clientId) buffer.putInt(ackTimeoutMs) + buffer.putInt(controllerId) buffer.putInt(controllerEpoch) buffer.put(if (deletePartitions) 1.toByte else 0.toByte) buffer.putInt(partitions.size) @@ -86,6 +90,7 @@ case class StopReplicaRequest(versionId: Short, 4 + /* correlation id */ ApiUtils.shortStringLength(clientId) + 4 + /* ackTimeoutMs */ + 4 + /* controller id*/ 4 + /* controller epoch */ 1 + /* deletePartitions */ 4 /* partition count */ @@ -104,6 +109,7 @@ case class StopReplicaRequest(versionId: Short, stopReplicaRequest.append("; ClientId: " + clientId) stopReplicaRequest.append("; AckTimeoutMs: " + ackTimeoutMs + " ms") stopReplicaRequest.append("; DeletePartitions: " + deletePartitions) + stopReplicaRequest.append("; ControllerId: " + controllerId) stopReplicaRequest.append("; ControllerEpoch: " + controllerEpoch) stopReplicaRequest.append("; Partitions: " + partitions.mkString(",")) stopReplicaRequest.toString() diff --git a/core/src/main/scala/kafka/api/StopReplicaResponse.scala b/core/src/main/scala/kafka/api/StopReplicaResponse.scala index fa66b99..c82eadd 100644 --- a/core/src/main/scala/kafka/api/StopReplicaResponse.scala +++ b/core/src/main/scala/kafka/api/StopReplicaResponse.scala @@ -42,9 +42,10 @@ object StopReplicaResponse { } -case class StopReplicaResponse(val correlationId: Int, +case class StopReplicaResponse(override val correlationId: Int, val responseMap: Map[(String, Int), Short], - val errorCode: Short = ErrorMapping.NoError) extends RequestOrResponse{ + val errorCode: Short = ErrorMapping.NoError) + extends RequestOrResponse(correlationId = correlationId) { def sizeInBytes(): Int ={ var size = 4 /* correlation id */ + diff --git a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala index 88007b1..7477cfd 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala @@ -47,10 +47,10 @@ object TopicMetadataRequest extends Logging { } case class TopicMetadataRequest(val versionId: Short, - val correlationId: Int, + override val correlationId: Int, val clientId: String, val topics: Seq[String]) - extends RequestOrResponse(Some(RequestKeys.MetadataKey)){ + extends RequestOrResponse(Some(RequestKeys.MetadataKey), correlationId){ def this(topics: Seq[String], correlationId: Int) = this(TopicMetadataRequest.CurrentVersion, correlationId, TopicMetadataRequest.DefaultClientId, topics) diff --git a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala index af76776..290f263 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala @@ -34,7 +34,8 @@ object TopicMetadataResponse { } case class TopicMetadataResponse(topicsMetadata: Seq[TopicMetadata], - correlationId: Int) extends RequestOrResponse { + override val correlationId: Int) + extends RequestOrResponse(correlationId = correlationId) { val sizeInBytes: Int = { val brokers = extractBrokers(topicsMetadata).values 4 + 4 + brokers.map(_.sizeInBytes).sum + 4 + topicsMetadata.map(_.sizeInBytes).sum diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 367ccd5..f79a622 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -27,6 +27,7 @@ import com.yammer.metrics.core.Gauge import kafka.metrics.KafkaMetricsGroup import kafka.common._ import kafka.controller.{LeaderIsrAndControllerEpoch, KafkaController} +import org.apache.log4j.Logger /** @@ -53,7 +54,8 @@ class Partition(val topic: String, * In addition to the leader, the controller can also send the epoch of the controller that elected the leader for * each partition. */ private var controllerEpoch: Int = KafkaController.InitialControllerEpoch - 1 - this.logIdent = "Partition [%s, %d] on broker %d: ".format(topic, partitionId, localBrokerId) + this.logIdent = "Partition [%s,%d] on broker %d: ".format(topic, partitionId, localBrokerId) + private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) private def isReplicaLocal(replicaId: Int) : Boolean = (replicaId == localBrokerId) @@ -67,7 +69,9 @@ class Partition(val topic: String, ) def isUnderReplicated(): Boolean = { - inSyncReplicas.size < replicationFactor + leaderIsrUpdateLock synchronized { + inSyncReplicas.size < replicationFactor + } } def getOrCreateReplica(replicaId: Int = localBrokerId): Replica = { @@ -127,15 +131,17 @@ class Partition(val topic: String, * 3. reset LogEndOffset for remote replicas (there could be old LogEndOffset from the time when this broker was the leader last time) * 4. set the new leader and ISR */ - def makeLeader(topic: String, partitionId: Int, leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch): Boolean = { + def makeLeader(controllerId: Int, topic: String, partitionId: Int, + leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, correlationId: Int): Boolean = { leaderIsrUpdateLock synchronized { val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr if (leaderEpoch >= leaderAndIsr.leaderEpoch){ - info("Current leader epoch [%d] is larger or equal to the requested leader epoch [%d], discard the become leader request" - .format(leaderEpoch, leaderAndIsr.leaderEpoch)) + stateChangeLogger.trace(("Broker %d discarded the become-leader request with correlation id %d from " + + "controller %d epoch %d for partition [%s,%d] since current leader epoch %d is >= the request's leader epoch %d") + .format(localBrokerId, correlationId, controllerId, leaderIsrAndControllerEpoch.controllerEpoch, topic, + partitionId, leaderEpoch, leaderAndIsr.leaderEpoch)) return false } - trace("Started to become leader at the request %s".format(leaderAndIsr.toString())) // record the epoch of the controller that made the leadership decision. This is useful while updating the isr // to maintain the decision maker controller's epoch in the zookeeper path controllerEpoch = leaderIsrAndControllerEpoch.controllerEpoch @@ -162,22 +168,21 @@ class Partition(val topic: String, * 3. set the leader and set ISR to empty * 4. start a fetcher to the new leader */ - def makeFollower(topic: String, partitionId: Int, leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, - liveBrokers: Set[Broker]): Boolean = { + def makeFollower(controllerId: Int, topic: String, partitionId: Int, leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, + liveBrokers: Set[Broker], correlationId: Int): Boolean = { leaderIsrUpdateLock synchronized { val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr - if (leaderEpoch >= leaderAndIsr.leaderEpoch){ - info("Current leader epoch [%d] is larger or equal to the requested leader epoch [%d], discard the become follower request" - .format(leaderEpoch, leaderAndIsr.leaderEpoch)) + if (leaderEpoch >= leaderAndIsr.leaderEpoch) { + stateChangeLogger.trace(("Broker %d discarded the become-follower request with correlation id %d from " + + "controller %d epoch %d for partition [%s,%d] since current leader epoch %d is >= the request's leader epoch %d") + .format(localBrokerId, correlationId, controllerId, leaderIsrAndControllerEpoch.controllerEpoch, topic, + partitionId, leaderEpoch, leaderAndIsr.leaderEpoch)) return false } - trace("Started to become follower at the request %s".format(leaderAndIsr.toString())) // record the epoch of the controller that made the leadership decision. This is useful while updating the isr // to maintain the decision maker controller's epoch in the zookeeper path controllerEpoch = leaderIsrAndControllerEpoch.controllerEpoch val newLeaderBrokerId: Int = leaderAndIsr.leader - info("Starting the follower state transition to follow leader %d for topic %s partition %d" - .format(newLeaderBrokerId, topic, partitionId)) liveBrokers.find(_.id == newLeaderBrokerId) match { case Some(leaderBroker) => // stop fetcher thread to previous leader @@ -192,8 +197,10 @@ class Partition(val topic: String, // start fetcher thread to current leader replicaFetcherManager.addFetcher(topic, partitionId, localReplica.logEndOffset, leaderBroker) case None => // leader went down - warn("Aborting become follower state change on %d since leader %d for ".format(localBrokerId, newLeaderBrokerId) + - " topic %s partition %d became unavailble during the state change operation".format(topic, partitionId)) + stateChangeLogger.trace("Broker %d aborted the become-follower state change with correlation id %d from " + + " controller %d epoch %d since leader %d for partition [%s,%d] became unavailable during the state change operation" + .format(localBrokerId, correlationId, controllerId, leaderIsrAndControllerEpoch.controllerEpoch, + newLeaderBrokerId, topic, partitionId)) } true } @@ -201,7 +208,7 @@ class Partition(val topic: String, def updateLeaderHWAndMaybeExpandIsr(replicaId: Int, offset: Long) { leaderIsrUpdateLock synchronized { - debug("Recording follower %d position %d for topic %s partition %d.".format(replicaId, offset, topic, partitionId)) + debug("Recording follower %d position %d for partition [%s,%d].".format(replicaId, offset, topic, partitionId)) val replica = getOrCreateReplica(replicaId) replica.logEndOffset = offset @@ -337,12 +344,14 @@ class Partition(val topic: String, } override def toString(): String = { - val partitionString = new StringBuilder - partitionString.append("Topic: " + topic) - partitionString.append("; Partition: " + partitionId) - partitionString.append("; Leader: " + leaderReplicaIdOpt) - partitionString.append("; AssignedReplicas: " + assignedReplicaMap.keys.mkString(",")) - partitionString.append("; InSyncReplicas: " + inSyncReplicas.map(_.brokerId).mkString(",")) - partitionString.toString() + leaderIsrUpdateLock synchronized { + val partitionString = new StringBuilder + partitionString.append("Topic: " + topic) + partitionString.append("; Partition: " + partitionId) + partitionString.append("; Leader: " + leaderReplicaIdOpt) + partitionString.append("; AssignedReplicas: " + assignedReplicaMap.keys.mkString(",")) + partitionString.append("; InSyncReplicas: " + inSyncReplicas.map(_.brokerId).mkString(",")) + partitionString.toString() + } } } diff --git a/core/src/main/scala/kafka/common/Topic.scala b/core/src/main/scala/kafka/common/Topic.scala index 5bd8f6b..c1b9f65 100644 --- a/core/src/main/scala/kafka/common/Topic.scala +++ b/core/src/main/scala/kafka/common/Topic.scala @@ -20,7 +20,7 @@ package kafka.common import util.matching.Regex object Topic { - private val legalChars = "[a-zA-Z0-9\\._\\-]" + val legalChars = "[a-zA-Z0-9\\._\\-]" private val maxNameLength = 255 private val rgx = new Regex(legalChars + "+") diff --git a/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala b/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala index 7e84043..d6c4a51 100644 --- a/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala @@ -290,6 +290,11 @@ class DefaultMessageFormatter extends MessageFormatter { } } +class NoOpMessageFormatter extends MessageFormatter { + override def init(props: Properties) {} + def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream) {} +} + class ChecksumMessageFormatter extends MessageFormatter { private var topicStr: String = _ diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala index 45db07b..e9cfd10 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala @@ -63,7 +63,8 @@ object ConsumerConfig extends Config { autoOffsetReset match { case OffsetRequest.SmallestTimeString => case OffsetRequest.LargestTimeString => - case _ => throw new InvalidConfigException("Wrong value " + autoOffsetReset + " of autoOffsetReset in ConsumerConfig") + case _ => throw new InvalidConfigException("Wrong value " + autoOffsetReset + " of auto.offset.reset in ConsumerConfig; " + + "Valid values are " + OffsetRequest.SmallestTimeString + " and " + OffsetRequest.LargestTimeString) } } } @@ -125,12 +126,6 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig( /** throw a timeout exception to the consumer if no message is available for consumption after the specified interval */ val consumerTimeoutMs = props.getInt("consumer.timeout.ms", ConsumerTimeoutMs) - /** Use shallow iterator over compressed messages directly. This feature should be used very carefully. - * Typically, it's only used for mirroring raw messages from one kafka cluster to another to save the - * overhead of decompression. - * */ - val shallowIteratorEnable = props.getBoolean("shallow.iterator.enable", false) - /** * Client id is specified by the kafka consumer client, used to distinguish different clients */ diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala index 1dfc75c..80df1b5 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala @@ -58,12 +58,7 @@ class ConsumerFetcherThread(name: String, case OffsetRequest.LargestTimeString => startTimestamp = OffsetRequest.LatestTime case _ => startTimestamp = OffsetRequest.LatestTime } - val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(startTimestamp, 1))) - val partitionErrorAndOffset = simpleConsumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition) - val newOffset = partitionErrorAndOffset.error match { - case ErrorMapping.NoError => partitionErrorAndOffset.offsets.head - case _ => throw ErrorMapping.exceptionFor(partitionErrorAndOffset.error) - } + val newOffset = simpleConsumer.earliestOrLatestOffset(topicAndPartition, startTimestamp, Request.OrdinaryConsumerId) val pti = partitionMap(topicAndPartition) pti.resetFetchOffset(newOffset) pti.resetConsumeOffset(newOffset) diff --git a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala index 963a3a9..7423141 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala @@ -34,7 +34,6 @@ class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk consumerTimeoutMs: Int, private val keyDecoder: Decoder[K], private val valueDecoder: Decoder[V], - val enableShallowIterator: Boolean, val clientId: String) extends IteratorTemplate[MessageAndMetadata[K, V]] with Logging { @@ -83,11 +82,7 @@ class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk .format(ctiConsumeOffset, cdcFetchOffset, currentTopicInfo)) currentTopicInfo.resetConsumeOffset(cdcFetchOffset) } - localCurrent = - if (enableShallowIterator) - currentDataChunk.messages.shallowIterator - else - currentDataChunk.messages.iterator + localCurrent = currentDataChunk.messages.iterator current.set(localCurrent) } diff --git a/core/src/main/scala/kafka/consumer/KafkaStream.scala b/core/src/main/scala/kafka/consumer/KafkaStream.scala index d4e0e96..31eaf86 100644 --- a/core/src/main/scala/kafka/consumer/KafkaStream.scala +++ b/core/src/main/scala/kafka/consumer/KafkaStream.scala @@ -26,12 +26,11 @@ class KafkaStream[K,V](private val queue: BlockingQueue[FetchedDataChunk], consumerTimeoutMs: Int, private val keyDecoder: Decoder[K], private val valueDecoder: Decoder[V], - val enableShallowIterator: Boolean, val clientId: String) extends Iterable[MessageAndMetadata[K,V]] with java.lang.Iterable[MessageAndMetadata[K,V]] { private val iter: ConsumerIterator[K,V] = - new ConsumerIterator[K,V](queue, consumerTimeoutMs, keyDecoder, valueDecoder, enableShallowIterator, clientId) + new ConsumerIterator[K,V](queue, consumerTimeoutMs, keyDecoder, valueDecoder, clientId) /** * Create an iterator over messages in the stream. diff --git a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala index cd8ef0b..a0f5770 100644 --- a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala @@ -22,59 +22,10 @@ import kafka.network._ import kafka.utils._ import kafka.utils.ZkUtils._ import collection.immutable -import kafka.common.{TopicAndPartition, KafkaException} +import kafka.common.{ErrorMapping, TopicAndPartition, KafkaException} import org.I0Itec.zkclient.ZkClient import kafka.cluster.Broker - -object SimpleConsumer extends Logging { - def earliestOrLatestOffset(broker: Broker, - topic: String, - partitionId: Int, - earliestOrLatest: Long, - clientId: String, - isFromOrdinaryConsumer: Boolean): Long = { - var simpleConsumer: SimpleConsumer = null - var producedOffset: Long = -1L - try { - simpleConsumer = new SimpleConsumer(broker.host, broker.port, ConsumerConfig.SocketTimeout, - ConsumerConfig.SocketBufferSize, clientId) - val topicAndPartition = TopicAndPartition(topic, partitionId) - val request = if(isFromOrdinaryConsumer) - new OffsetRequest(immutable.Map(topicAndPartition -> PartitionOffsetRequestInfo(earliestOrLatest, 1))) - else - new OffsetRequest(immutable.Map(topicAndPartition -> PartitionOffsetRequestInfo(earliestOrLatest, 1)), - 0, Request.DebuggingConsumerId) - producedOffset = simpleConsumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head - } catch { - case e => - error("error in earliestOrLatestOffset() ", e) - } - finally { - if (simpleConsumer != null) - simpleConsumer.close() - } - producedOffset - } - - def earliestOrLatestOffset(zkClient: ZkClient, - topic: String, - brokerId: Int, - partitionId: Int, - earliestOrLatest: Long, - clientId: String, - isFromOrdinaryConsumer: Boolean = true): Long = { - val cluster = getCluster(zkClient) - val broker = cluster.getBroker(brokerId) match { - case Some(b) => b - case None => throw new KafkaException("Broker " + brokerId + " is unavailable. Cannot issue " + - "getOffsetsBefore request") - } - earliestOrLatestOffset(broker, topic, partitionId, earliestOrLatest, clientId, isFromOrdinaryConsumer) - } -} - - /** * A consumer of kafka messages */ @@ -194,5 +145,24 @@ class SimpleConsumer(val host: String, connect() } } + + /** + * Get the earliest or latest offset of a given topic, partition. + * @param topicAndPartition Topic and partition of which the offset is needed. + * @param earliestOrLatest A value to indicate earliest or latest offset. + * @param consumerId Id of the consumer which could be a consumer client, SimpleConsumerShell or a follower broker. + * @return Requested offset. + */ + def earliestOrLatestOffset(topicAndPartition: TopicAndPartition, earliestOrLatest: Long, consumerId: Int): Long = { + val request = OffsetRequest(requestInfo = Map(topicAndPartition -> PartitionOffsetRequestInfo(earliestOrLatest, 1)), + clientId = clientId, + replicaId = consumerId) + val partitionErrorAndOffset = getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition) + val offset = partitionErrorAndOffset.error match { + case ErrorMapping.NoError => partitionErrorAndOffset.offsets.head + case _ => throw ErrorMapping.exceptionFor(partitionErrorAndOffset.error) + } + offset + } } diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index 972d33d..0921ce6 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -198,7 +198,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, threadIdSet.map(_ => { val queue = new LinkedBlockingQueue[FetchedDataChunk](config.queuedMaxMessages) val stream = new KafkaStream[K,V]( - queue, config.consumerTimeoutMs, keyDecoder, valueDecoder, config.shallowIteratorEnable, config.clientId) + queue, config.consumerTimeoutMs, keyDecoder, valueDecoder, config.clientId) (queue, stream) }) ).flatten.toList @@ -216,8 +216,8 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private def registerConsumerInZK(dirs: ZKGroupDirs, consumerIdString: String, topicCount: TopicCount) = { info("begin registering consumer " + consumerIdString + " in ZK") val consumerRegistrationInfo = - Utils.mergeJsonObjects(Seq(Utils.mapToJson(Map("version" -> 1.toString, "subscription" -> topicCount.dbString), valueInQuotes = false), - Utils.mapToJson(Map("pattern" -> topicCount.pattern), valueInQuotes = true))) + Utils.mergeJsonFields(Utils.mapToJsonFields(Map("version" -> 1.toString, "subscription" -> topicCount.dbString), valueInQuotes = false) + ++ Utils.mapToJsonFields(Map("pattern" -> topicCount.pattern), valueInQuotes = true)) createEphemeralPathExpectConflict(zkClient, dirs.consumerRegistryDir + "/" + consumerIdString, consumerRegistrationInfo) info("end registering consumer " + consumerIdString + " in ZK") } @@ -698,7 +698,6 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, config.consumerTimeoutMs, keyDecoder, valueDecoder, - config.shallowIteratorEnable, config.clientId) (queue, stream) }).toList diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index e2ca1d6..6e563d2 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -24,16 +24,14 @@ import java.util.concurrent.{LinkedBlockingQueue, BlockingQueue} import kafka.server.KafkaConfig import collection.mutable import kafka.api._ +import org.apache.log4j.Logger -class ControllerChannelManager private (config: KafkaConfig) extends Logging { +class ControllerChannelManager (private val controllerContext: ControllerContext, config: KafkaConfig) extends Logging { private val brokerStateInfo = new HashMap[Int, ControllerBrokerStateInfo] private val brokerLock = new Object - this.logIdent = "[Channel manager on controller " + config.brokerId + "], " + this.logIdent = "[Channel manager on controller " + config.brokerId + "]: " - def this(allBrokers: Set[Broker], config : KafkaConfig) { - this(config) - allBrokers.foreach(addNewBroker(_)) - } + controllerContext.liveBrokers.foreach(addNewBroker(_)) def startup() = { brokerLock synchronized { @@ -82,7 +80,7 @@ class ControllerChannelManager private (config: KafkaConfig) extends Logging { BlockingChannel.UseDefaultBufferSize, config.controllerSocketTimeoutMs) channel.connect() - val requestThread = new RequestSendThread(config.brokerId, broker.id, messageQueue, channel) + val requestThread = new RequestSendThread(config.brokerId, controllerContext, broker.id, messageQueue, channel) requestThread.setDaemon(false) brokerStateInfo.put(broker.id, new ControllerBrokerStateInfo(channel, broker, messageQueue, requestThread)) } @@ -105,11 +103,13 @@ class ControllerChannelManager private (config: KafkaConfig) extends Logging { } class RequestSendThread(val controllerId: Int, + val controllerContext: ControllerContext, val toBrokerId: Int, val queue: BlockingQueue[(RequestOrResponse, (RequestOrResponse) => Unit)], val channel: BlockingChannel) extends ShutdownableThread("Controller-%d-to-broker-%d-send-thread".format(controllerId, toBrokerId)) { private val lock = new Object() + private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) override def doWork(): Unit = { val queueItem = queue.take() @@ -129,7 +129,8 @@ class RequestSendThread(val controllerId: Int, case RequestKeys.StopReplicaKey => response = StopReplicaResponse.readFrom(receive.buffer) } - trace("Controller %d request to broker %d got a response %s".format(controllerId, toBrokerId, response)) + stateChangeLogger.trace("Controller %d epoch %d received response correlationId %d for a request sent to broker %d" + .format(controllerId, controllerContext.epoch, response.correlationId, toBrokerId)) if(callback != null){ callback(response) @@ -143,11 +144,12 @@ class RequestSendThread(val controllerId: Int, } } -class ControllerBrokerRequestBatch(sendRequest: (Int, RequestOrResponse, (RequestOrResponse) => Unit) => Unit) +class ControllerBrokerRequestBatch(sendRequest: (Int, RequestOrResponse, (RequestOrResponse) => Unit) => Unit, controllerId: Int) extends Logging { val leaderAndIsrRequestMap = new mutable.HashMap[Int, mutable.HashMap[(String, Int), PartitionStateInfo]] val stopReplicaRequestMap = new mutable.HashMap[Int, Seq[(String, Int)]] val stopAndDeleteReplicaRequestMap = new mutable.HashMap[Int, Seq[(String, Int)]] + private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) def newBatch() { // raise error if the previous batch is not empty @@ -162,10 +164,8 @@ class ControllerBrokerRequestBatch(sendRequest: (Int, RequestOrResponse, (Reques def addLeaderAndIsrRequestForBrokers(brokerIds: Seq[Int], topic: String, partition: Int, leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, replicationFactor: Int) { brokerIds.foreach { brokerId => - leaderAndIsrRequestMap.getOrElseUpdate(brokerId, - new mutable.HashMap[(String, Int), PartitionStateInfo]) - leaderAndIsrRequestMap(brokerId).put((topic, partition), - PartitionStateInfo(leaderIsrAndControllerEpoch, replicationFactor)) + leaderAndIsrRequestMap.getOrElseUpdate(brokerId, new mutable.HashMap[(String, Int), PartitionStateInfo]) + leaderAndIsrRequestMap(brokerId).put((topic, partition), PartitionStateInfo(leaderIsrAndControllerEpoch, replicationFactor)) } } @@ -190,8 +190,13 @@ class ControllerBrokerRequestBatch(sendRequest: (Int, RequestOrResponse, (Reques val partitionStateInfos = m._2.toMap val leaderIds = partitionStateInfos.map(_._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader).toSet val leaders = liveBrokers.filter(b => leaderIds.contains(b.id)) - val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfos, leaders, controllerEpoch, correlationId) - debug("The leaderAndIsr request sent to broker %d is %s".format(broker, leaderAndIsrRequest)) + val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfos, leaders, controllerId, controllerEpoch, correlationId) + for (p <- partitionStateInfos) { + val typeOfRequest = if (broker == p._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader) "become-leader" else "become-follower" + stateChangeLogger.trace(("Controller %d epoch %d sending %s LeaderAndIsr request with correlationId %d to broker %d " + + "for partition [%s,%d]").format(controllerId, controllerEpoch, typeOfRequest, correlationId, broker, + p._1._1, p._1._2)) + } sendRequest(broker, leaderAndIsrRequest, null) } leaderAndIsrRequestMap.clear() @@ -202,8 +207,9 @@ class ControllerBrokerRequestBatch(sendRequest: (Int, RequestOrResponse, (Reques if (replicas.size > 0) { debug("The stop replica request (delete = %s) sent to broker %d is %s" .format(deletePartitions, broker, replicas.mkString(","))) - sendRequest(broker, new StopReplicaRequest(deletePartitions, - Set.empty[(String, Int)] ++ replicas, controllerEpoch, correlationId), null) + val stopReplicaRequest = new StopReplicaRequest(deletePartitions, Set.empty[(String, Int)] ++ replicas, controllerId, + controllerEpoch, correlationId) + sendRequest(broker, stopReplicaRequest, null) } } m.clear() diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 48eae7e..25a8cfe 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -74,6 +74,7 @@ trait KafkaControllerMBean { object KafkaController { val MBeanName = "kafka.controller:type=KafkaController,name=ControllerOps" + val stateChangeLogger = "state.change.logger" val InitialControllerEpoch = 1 val InitialControllerEpochZkVersion = 1 } @@ -89,7 +90,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg private val reassignedPartitionLeaderSelector = new ReassignedPartitionLeaderSelector(controllerContext) private val preferredReplicaPartitionLeaderSelector = new PreferredReplicaPartitionLeaderSelector(controllerContext) private val controlledShutdownPartitionLeaderSelector = new ControlledShutdownLeaderSelector(controllerContext) - private val brokerRequestBatch = new ControllerBrokerRequestBatch(sendRequest) + private val brokerRequestBatch = new ControllerBrokerRequestBatch(sendRequest, config.brokerId) registerControllerChangedListener() newGauge( @@ -491,7 +492,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } private def startChannelManager() { - controllerContext.controllerChannelManager = new ControllerChannelManager(controllerContext.liveBrokers, config) + controllerContext.controllerChannelManager = new ControllerChannelManager(controllerContext, config) controllerContext.controllerChannelManager.startup() } diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala index 4078604..b25e9f4 100644 --- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala +++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala @@ -24,6 +24,7 @@ import kafka.common.{TopicAndPartition, StateChangeFailedException, PartitionOff import kafka.utils.{Logging, ZkUtils} import org.I0Itec.zkclient.IZkChildListener import org.I0Itec.zkclient.exception.ZkNodeExistsException +import org.apache.log4j.Logger /** * This class represents the state machine for partitions. It defines the states that a partition can be in, and @@ -38,13 +39,15 @@ import org.I0Itec.zkclient.exception.ZkNodeExistsException * moves to the OfflinePartition state. Valid previous states are NewPartition/OnlinePartition */ class PartitionStateMachine(controller: KafkaController) extends Logging { - this.logIdent = "[Partition state machine on Controller " + controller.config.brokerId + "]: " private val controllerContext = controller.controllerContext + private val controllerId = controller.config.brokerId private val zkClient = controllerContext.zkClient var partitionState: mutable.Map[TopicAndPartition, PartitionState] = mutable.Map.empty - val brokerRequestBatch = new ControllerBrokerRequestBatch(controller.sendRequest) + val brokerRequestBatch = new ControllerBrokerRequestBatch(controller.sendRequest, controllerId) val offlinePartitionSelector = new OfflinePartitionLeaderSelector(controllerContext) private val isShuttingDown = new AtomicBoolean(false) + this.logIdent = "[Partition state machine on Controller " + controllerId + "]: " + private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) /** * Invoked on successful controller election. First registers a topic change listener since that triggers all @@ -126,12 +129,13 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { targetState match { case NewPartition => // pre: partition did not exist before this - // post: partition has been assigned replicas assertValidPreviousStates(topicAndPartition, List(NonExistentPartition), NewPartition) assignReplicasToPartitions(topic, partition) partitionState.put(topicAndPartition, NewPartition) - info("Partition %s state changed from NotExists to New with assigned replicas ".format(topicAndPartition) + - "%s".format(controllerContext.partitionReplicaAssignment(topicAndPartition).mkString(","))) + val assignedReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition).mkString(",") + stateChangeLogger.trace("Controller %d epoch %d changed partition %s state from NotExists to New with assigned replicas %s" + .format(controllerId, controller.epoch, topicAndPartition, assignedReplicas)) + // post: partition has been assigned replicas case OnlinePartition => assertValidPreviousStates(topicAndPartition, List(NewPartition, OnlinePartition, OfflinePartition), OnlinePartition) partitionState(topicAndPartition) match { @@ -144,27 +148,31 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { electLeaderForPartition(topic, partition, leaderSelector) case _ => // should never come here since illegal previous states are checked above } - info("Partition %s state changed from %s to OnlinePartition with leader %d".format(topicAndPartition, - partitionState(topicAndPartition), controllerContext.allLeaders(topicAndPartition).leaderAndIsr.leader)) partitionState.put(topicAndPartition, OnlinePartition) + val leader = controllerContext.allLeaders(topicAndPartition).leaderAndIsr.leader + stateChangeLogger.trace("Controller %d epoch %d changed partition %s from %s to OnlinePartition with leader %d" + .format(controllerId, controller.epoch, topicAndPartition, partitionState(topicAndPartition), leader)) // post: partition has a leader case OfflinePartition => - // pre: partition should be in Online state + // pre: partition should be in New or Online state assertValidPreviousStates(topicAndPartition, List(NewPartition, OnlinePartition), OfflinePartition) // should be called when the leader for a partition is no longer alive - info("Partition %s state changed from Online to Offline".format(topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed partition %s state from Online to Offline" + .format(controllerId, controller.epoch, topicAndPartition)) partitionState.put(topicAndPartition, OfflinePartition) // post: partition has no alive leader case NonExistentPartition => - // pre: partition could be in either of the above states + // pre: partition should be in Offline state assertValidPreviousStates(topicAndPartition, List(OfflinePartition), NonExistentPartition) - info("Partition %s state changed from Offline to NotExists".format(topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed partition %s state from Offline to NotExists" + .format(controllerId, controller.epoch, topicAndPartition)) partitionState.put(topicAndPartition, NonExistentPartition) // post: partition state is deleted from all brokers and zookeeper } } catch { - case t: Throwable => error("State change for partition %s ".format(topicAndPartition) + - "from %s to %s failed".format(currState, targetState), t) + case t: Throwable => + stateChangeLogger.error("Controller %d epoch %d initiated state change for partition %s from %s to %s failed" + .format(controllerId, controller.epoch, topicAndPartition, currState, targetState), t) } } @@ -225,9 +233,11 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { liveAssignedReplicas.size match { case 0 => ControllerStats.offlinePartitionRate.mark() - throw new StateChangeFailedException(("During state change of partition %s from NEW to ONLINE, assigned replicas are " + - "[%s], live brokers are [%s]. No assigned replica is alive").format(topicAndPartition, - replicaAssignment.mkString(","), controllerContext.liveBrokerIds)) + val failMsg = ("encountered error during state change of partition %s from New to Online, assigned replicas are [%s], " + + "live brokers are [%s]. No assigned replica is alive.") + .format(topicAndPartition, replicaAssignment.mkString(","), controllerContext.liveBrokerIds) + stateChangeLogger.error("Controller %d epoch %d ".format(controllerId, controller.epoch) + failMsg) + throw new StateChangeFailedException(failMsg) case _ => debug("Live assigned replicas for partition %s are: [%s]".format(topicAndPartition, liveAssignedReplicas)) // make the first replica in the list of assigned replicas, the leader @@ -251,9 +261,11 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { val leaderIsrAndEpoch = ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topicAndPartition.topic, topicAndPartition.partition).get ControllerStats.offlinePartitionRate.mark() - throw new StateChangeFailedException("Error while changing partition %s's state from New to Online" - .format(topicAndPartition) + " since Leader and isr path already exists with value " + - "%s and controller epoch %d".format(leaderIsrAndEpoch.leaderAndIsr.toString(), leaderIsrAndEpoch.controllerEpoch)) + val failMsg = ("encountered error while changing partition %s's state from New to Online since LeaderAndIsr path already " + + "exists with value %s and controller epoch %d") + .format(topicAndPartition, leaderIsrAndEpoch.leaderAndIsr.toString(), leaderIsrAndEpoch.controllerEpoch) + stateChangeLogger.error("Controller %d epoch %d ".format(controllerId, controller.epoch) + failMsg) + throw new StateChangeFailedException(failMsg) } } } @@ -268,7 +280,8 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { def electLeaderForPartition(topic: String, partition: Int, leaderSelector: PartitionLeaderSelector) { val topicAndPartition = TopicAndPartition(topic, partition) // handle leader election for the partitions whose leader is no longer alive - info("Electing leader for partition %s".format(topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d started leader election for partition %s" + .format(controllerId, controller.epoch, topicAndPartition)) try { var zookeeperPathUpdateSucceeded: Boolean = false var newLeaderAndIsr: LeaderAndIsr = null @@ -277,10 +290,14 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { val currentLeaderIsrAndEpoch = getLeaderIsrAndEpochOrThrowException(topic, partition) val currentLeaderAndIsr = currentLeaderIsrAndEpoch.leaderAndIsr val controllerEpoch = currentLeaderIsrAndEpoch.controllerEpoch - if(controllerEpoch > controller.epoch) - throw new StateChangeFailedException("Leader and isr path written by another controller. This probably" + - "means the current controller with epoch %d went through a soft failure and another ".format(controller.epoch) + - "controller was elected with epoch %d. Aborting state change by this controller".format(controllerEpoch)) + if (controllerEpoch > controller.epoch) { + val failMsg = ("aborted leader election for partition [%s,%d] since the LeaderAndIsr path was " + + "already written by another controller. This probably means that the current controller %d went through " + + "a soft failure and another controller was elected with epoch %d.") + .format(topic, partition, controllerId, controllerEpoch) + stateChangeLogger.error("Controller %d epoch %d ".format(controllerId, controller.epoch) + failMsg) + throw new StateChangeFailedException(failMsg) + } // elect new leader or throw exception val (leaderAndIsr, replicas) = leaderSelector.selectLeader(topicAndPartition, currentLeaderAndIsr) val (updateSucceeded, newVersion) = ZkUtils.conditionalUpdatePersistentPath(zkClient, @@ -294,7 +311,8 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { val newLeaderIsrAndControllerEpoch = new LeaderIsrAndControllerEpoch(newLeaderAndIsr, controller.epoch) // update the leader cache controllerContext.allLeaders.put(TopicAndPartition(topic, partition), newLeaderIsrAndControllerEpoch) - info("Elected leader %d for Offline partition %s".format(newLeaderAndIsr.leader, topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d elected leader %d for Offline partition %s" + .format(controllerId, controller.epoch, newLeaderAndIsr.leader, topicAndPartition)) // store new leader and isr info in cache brokerRequestBatch.addLeaderAndIsrRequestForBrokers(replicasForThisPartition, topic, partition, newLeaderIsrAndControllerEpoch, controllerContext.partitionReplicaAssignment(TopicAndPartition(topic, partition)).size) @@ -302,8 +320,10 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { case poe: PartitionOfflineException => throw new PartitionOfflineException("All replicas %s for partition %s are dead." .format(controllerContext.partitionReplicaAssignment(topicAndPartition).mkString(","), topicAndPartition) + " Marking this partition offline", poe) - case sce => throw new StateChangeFailedException(("Error while electing leader for partition " + - " %s due to: %s.").format(topicAndPartition, sce.getMessage), sce) + case sce => + val failMsg = "encountered error while electing leader for partition %s due to: %s.".format(topicAndPartition, sce.getMessage) + stateChangeLogger.error("Controller %d epoch %d ".format(controllerId, controller.epoch) + failMsg) + throw new StateChangeFailedException(failMsg, sce) } debug("After leader election, leader cache is updated to %s".format(controllerContext.allLeaders.map(l => (l._1, l._2)))) } @@ -321,8 +341,9 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition) match { case Some(currentLeaderIsrAndEpoch) => currentLeaderIsrAndEpoch case None => - throw new StateChangeFailedException("Leader and ISR information doesn't exist for partition " + - "%s in %s state".format(topicAndPartition, partitionState(topicAndPartition))) + val failMsg = "LeaderAndIsr information doesn't exist for partition %s in %s state" + .format(topicAndPartition, partitionState(topicAndPartition)) + throw new StateChangeFailedException(failMsg) } } @@ -362,7 +383,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { } class PartitionChangeListener(topic: String) extends IZkChildListener with Logging { - this.logIdent = "[Controller " + controller.config.brokerId + "], " + this.logIdent = "[Controller " + controller.config.brokerId + "]: " @throws(classOf[Exception]) def handleChildChange(parentPath : String, children : java.util.List[String]) { diff --git a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala index 20d9c4f..88058ec 100644 --- a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala +++ b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala @@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicBoolean import kafka.common.{TopicAndPartition, StateChangeFailedException} import kafka.utils.{ZkUtils, Logging} import org.I0Itec.zkclient.IZkChildListener +import org.apache.log4j.Logger /** * This class represents the state machine for replicas. It defines the states that a replica can be in, and @@ -37,12 +38,14 @@ import org.I0Itec.zkclient.IZkChildListener * 4. NonExistentReplica: If a replica is deleted, it is moved to this state. Valid previous state is OfflineReplica */ class ReplicaStateMachine(controller: KafkaController) extends Logging { - this.logIdent = "[Replica state machine on Controller " + controller.config.brokerId + "]: " private val controllerContext = controller.controllerContext + private val controllerId = controller.config.brokerId private val zkClient = controllerContext.zkClient var replicaState: mutable.Map[(String, Int, Int), ReplicaState] = mutable.Map.empty - val brokerRequestBatch = new ControllerBrokerRequestBatch(controller.sendRequest) + val brokerRequestBatch = new ControllerBrokerRequestBatch(controller.sendRequest, controller.config.brokerId) private val isShuttingDown = new AtomicBoolean(false) + this.logIdent = "[Replica state machine on controller " + controller.config.brokerId + "]: " + private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) /** * Invoked on successful controller election. First registers a broker change listener since that triggers all @@ -117,17 +120,18 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { case None => // new leader request will be sent to this replica when one gets elected } replicaState.put((topic, partition, replicaId), NewReplica) - info("Replica %d for partition %s state changed to NewReplica".format(replicaId, topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to NewReplica" + .format(controllerId, controller.epoch, replicaId, topicAndPartition)) case NonExistentReplica => assertValidPreviousStates(topic, partition, replicaId, List(OfflineReplica), targetState) // send stop replica command brokerRequestBatch.addStopReplicaRequestForBrokers(List(replicaId), topic, partition, deletePartition = true) // remove this replica from the assigned replicas list for its partition val currentAssignedReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition) - controllerContext.partitionReplicaAssignment.put(topicAndPartition, - currentAssignedReplicas.filterNot(_ == replicaId)) - info("Replica %d for partition %s state changed to NonExistentReplica".format(replicaId, topicAndPartition)) + controllerContext.partitionReplicaAssignment.put(topicAndPartition, currentAssignedReplicas.filterNot(_ == replicaId)) replicaState.remove((topic, partition, replicaId)) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to NonExistentReplica" + .format(controllerId, controller.epoch, replicaId, topicAndPartition)) case OnlineReplica => assertValidPreviousStates(topic, partition, replicaId, List(NewReplica, OnlineReplica, OfflineReplica), targetState) replicaState((topic, partition, replicaId)) match { @@ -135,7 +139,8 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { // add this replica to the assigned replicas list for its partition val currentAssignedReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition) controllerContext.partitionReplicaAssignment.put(topicAndPartition, currentAssignedReplicas :+ replicaId) - info("Replica %d for partition %s state changed to OnlineReplica".format(replicaId, topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to OnlineReplica" + .format(controllerId, controller.epoch, replicaId, topicAndPartition)) case _ => // check if the leader for this partition is alive or even exists controllerContext.allLeaders.get(topicAndPartition) match { @@ -146,7 +151,8 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { topic, partition, leaderIsrAndControllerEpoch, replicaAssignment.size) replicaState.put((topic, partition, replicaId), OnlineReplica) - info("Replica %d for partition %s state changed to OnlineReplica".format(replicaId, topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to OnlineReplica" + .format(controllerId, controller.epoch, replicaId, topicAndPartition)) case false => // ignore partitions whose leader is not alive } case None => // ignore partitions who don't have a leader yet @@ -167,8 +173,8 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { topic, partition, updatedLeaderIsrAndControllerEpoch, replicaAssignment.size) replicaState.put((topic, partition, replicaId), OfflineReplica) - info("Replica %d for partition %s state changed to OfflineReplica".format(replicaId, topicAndPartition)) - info("Removed offline replica %d from ISR for partition %s".format(replicaId, topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to OfflineReplica" + .format(controllerId, controller.epoch, replicaId, topicAndPartition)) false case None => true @@ -184,15 +190,16 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { } } catch { - case t: Throwable => error("Error while changing state of replica %d for partition ".format(replicaId) + - "[%s, %d] to %s".format(topic, partition, targetState), t) + case t: Throwable => + stateChangeLogger.error("Controller %d epoch %d initiated state change of replica %d for partition [%s,%d] to %s failed" + .format(controllerId, controller.epoch, replicaId, topic, partition, targetState), t) } } private def assertValidPreviousStates(topic: String, partition: Int, replicaId: Int, fromStates: Seq[ReplicaState], targetState: ReplicaState) { assert(fromStates.contains(replicaState((topic, partition, replicaId))), - "Replica %s for partition [%s, %d] should be in the %s states before moving to %s state" + "Replica %s for partition [%s,%d] should be in the %s states before moving to %s state" .format(replicaId, topic, partition, fromStates.mkString(","), targetState) + ". Instead it is in %s state".format(replicaState((topic, partition, replicaId)))) } diff --git a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala index 3d92569..5f80df7 100644 --- a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala @@ -21,9 +21,11 @@ import java.nio.ByteBuffer import scala.collection.JavaConversions class TopicMetadataRequest(val versionId: Short, - val correlationId: Int, + override val correlationId: Int, val clientId: String, - val topics: java.util.List[String]) extends RequestOrResponse(Some(kafka.api.RequestKeys.MetadataKey)) { + val topics: java.util.List[String]) + extends RequestOrResponse(Some(kafka.api.RequestKeys.MetadataKey), correlationId) { + val underlying: kafka.api.TopicMetadataRequest = new kafka.api.TopicMetadataRequest(versionId, correlationId, clientId, JavaConversions.asBuffer(topics)) @@ -36,4 +38,5 @@ class TopicMetadataRequest(val versionId: Short, def writeTo(buffer: ByteBuffer) = underlying.writeTo(buffer) def sizeInBytes: Int = underlying.sizeInBytes() + } diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 631953f..6ba31cb 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -211,14 +211,14 @@ class Log(val dir: File, * * @throws KafkaStorageException If the append fails due to an I/O error. * - * @return Information about the appended messages including the first and last offset + * @return the first and the last offset in the appended messages */ - def append(messages: ByteBufferMessageSet, assignOffsets: Boolean = true): LogAppendInfo = { + def append(messages: ByteBufferMessageSet, assignOffsets: Boolean = true): (Long, Long) = { val appendInfo = analyzeAndValidateMessageSet(messages) // if we have any valid messages, append them to the log if(appendInfo.count == 0) - return appendInfo + return (-1L, -1L) // trim any invalid bytes or partial messages before appending it to the on-disk log var validMessages = trimInvalidBytes(messages) @@ -226,24 +226,27 @@ class Log(val dir: File, try { // they are valid, insert them in the log lock synchronized { + val firstOffset = nextOffset.get + // maybe roll the log if this segment is full val segment = maybeRoll() - - if(assignOffsets) { - // assign offsets to the messageset - appendInfo.firstOffset = nextOffset.get - val offset = new AtomicLong(nextOffset.get) - try { - validMessages = validMessages.assignOffsets(offset, appendInfo.codec) - } catch { - case e: IOException => throw new KafkaException("Error in validating messages while appending to log '%s'".format(name), e) + + val lastOffset = + if(assignOffsets) { + // assign offsets to the messageset + val offset = new AtomicLong(nextOffset.get) + try { + validMessages = validMessages.assignOffsets(offset, appendInfo.codec) + } catch { + case e: IOException => throw new KafkaException("Error in validating messages while appending to log '%s'".format(name), e) + } + offset.get - 1 + } else { + // we are taking the offsets we are given + if(!appendInfo.offsetsMonotonic || appendInfo.firstOffset < nextOffset.get) + throw new IllegalArgumentException("Out of order offsets found in " + messages) + appendInfo.lastOffset } - appendInfo.lastOffset = offset.get - 1 - } else { - // we are taking the offsets we are given - if(!appendInfo.offsetsMonotonic || appendInfo.firstOffset < nextOffset.get) - throw new IllegalArgumentException("Out of order offsets found in " + messages) - } // Check if the message sizes are valid. This check is done after assigning offsets to ensure the comparison // happens with the new message size (after re-compression, if any) @@ -254,16 +257,19 @@ class Log(val dir: File, } // now append to the log - trace("Appending message set to %s with offsets %d to %d.".format(name, appendInfo.firstOffset, appendInfo.lastOffset)) - segment.append(appendInfo.firstOffset, validMessages) - + segment.append(firstOffset, validMessages) + // increment the log end offset - nextOffset.set(appendInfo.lastOffset + 1) - - // maybe flush the log and index - maybeFlush(appendInfo.count) - - appendInfo + nextOffset.set(lastOffset + 1) + + trace("Appended message set to log %s with first offset: %d, next offset: %d, and messages: %s" + .format(this.name, firstOffset, nextOffset.get(), validMessages)) + + val numAppendedMessages = (lastOffset - firstOffset + 1).toInt + maybeFlush(numAppendedMessages) + + // return the offset of the first and the last message in the messageset appended + (firstOffset, lastOffset) } } catch { case e: IOException => throw new KafkaStorageException("I/O exception in append to log '%s'".format(name), e) diff --git a/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala b/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala index 0ac21c3..a20ab90 100644 --- a/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala +++ b/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala @@ -45,7 +45,7 @@ trait KafkaMetricsGroup extends Logging { def newMeter(name: String, eventType: String, timeUnit: TimeUnit) = Metrics.defaultRegistry().newMeter(metricName(name), eventType, timeUnit) - def newHistogram(name: String, biased: Boolean = false) = + def newHistogram(name: String, biased: Boolean = true) = Metrics.defaultRegistry().newHistogram(metricName(name), biased) def newTimer(name: String, durationUnit: TimeUnit, rateUnit: TimeUnit) = diff --git a/core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala b/core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala index 617fc43..72597ef 100644 --- a/core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala +++ b/core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala @@ -80,12 +80,12 @@ class BrokerPartitionInfo(producerConfig: ProducerConfig, if(tmd.errorCode == ErrorMapping.NoError){ topicPartitionInfo.put(tmd.topic, tmd) } else - warn("Error while fetching metadata for topic [%s]: [%s]".format(tmd.topic, tmd), ErrorMapping.exceptionFor(tmd.errorCode)) + warn("Error while fetching metadata [%s] for topic [%s]: %s ".format(tmd, tmd.topic, ErrorMapping.exceptionFor(tmd.errorCode).getClass)) tmd.partitionsMetadata.foreach(pmd =>{ - if (pmd.errorCode != ErrorMapping.NoError){ - warn("Error while fetching metadata for topic partition [%s,%d]: [%s]".format(tmd.topic, pmd.partitionId, pmd), - ErrorMapping.exceptionFor(pmd.errorCode)) - } + if (pmd.errorCode != ErrorMapping.NoError && pmd.errorCode == ErrorMapping.LeaderNotAvailableCode) { + warn("Error while fetching metadata %s for topic partition [%s,%d]: [%s]".format(pmd, tmd.topic, pmd.partitionId, + ErrorMapping.exceptionFor(pmd.errorCode).getClass)) + } // any other error code (e.g. ReplicaNotAvailable) can be ignored since the producer does not need to access the replica and isr metadata }) }) producerPool.updateProducer(topicsMetadata) diff --git a/core/src/main/scala/kafka/producer/ProducerPool.scala b/core/src/main/scala/kafka/producer/ProducerPool.scala index 4970029..43df70b 100644 --- a/core/src/main/scala/kafka/producer/ProducerPool.scala +++ b/core/src/main/scala/kafka/producer/ProducerPool.scala @@ -43,9 +43,9 @@ class ProducerPool(val config: ProducerConfig) extends Logging { private val syncProducers = new HashMap[Int, SyncProducer] private val lock = new Object() - def updateProducer(topicMetadatas: Seq[TopicMetadata]) { + def updateProducer(topicMetadata: Seq[TopicMetadata]) { val newBrokers = new collection.mutable.HashSet[Broker] - topicMetadatas.foreach(tmd => { + topicMetadata.foreach(tmd => { tmd.partitionsMetadata.foreach(pmd => { if(pmd.leader.isDefined) newBrokers+=(pmd.leader.get) diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index a7d39b1..4ee23cd 100644 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -114,7 +114,8 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke case(topicAndPartition, partitionData) => val (topic, partitionId) = topicAndPartition.asTuple val currentOffset = partitionMap.get(topicAndPartition) - if (currentOffset.isDefined) { + // we append to the log if the current offset is defined and it is the same as the offset requested during fetch + if (currentOffset.isDefined && fetchRequest.requestInfo(topicAndPartition).offset == currentOffset.get) { partitionData.error match { case ErrorMapping.NoError => val messages = partitionData.messages.asInstanceOf[ByteBufferMessageSet] diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index f8faf96..c6a455e 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -196,22 +196,32 @@ class KafkaApis(val requestChannel: RequestChannel, try { val localReplica = replicaManager.getLeaderReplicaIfLocal(topicAndPartition.topic, topicAndPartition.partition) val log = localReplica.log.get - val info = log.append(messages.asInstanceOf[ByteBufferMessageSet], assignOffsets = true) - + val (firstOffset, lastOffset) = log.append(messages.asInstanceOf[ByteBufferMessageSet], assignOffsets = true) + val numAppendedMessages = if (firstOffset == -1L || lastOffset == -1L) 0 else (lastOffset - firstOffset + 1) + // update stats - BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).messagesInRate.mark(info.count) - BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(info.count) + BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).messagesInRate.mark(numAppendedMessages) + BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numAppendedMessages) // we may need to increment high watermark since ISR could be down to 1 localReplica.partition.maybeIncrementLeaderHW(localReplica) trace("%d bytes written to log %s-%d beginning at offset %d and ending at offset %d" - .format(messages.size, topicAndPartition.topic, topicAndPartition.partition, info.firstOffset, info.lastOffset)) - ProduceResult(topicAndPartition, info.firstOffset, info.lastOffset) + .format(messages.size, topicAndPartition.topic, topicAndPartition.partition, firstOffset, lastOffset)) + ProduceResult(topicAndPartition, firstOffset, lastOffset) } catch { + // NOTE: Failed produce requests is not incremented for UnknownTopicOrPartitionException and NotLeaderForPartitionException + // since failed produce requests metric is supposed to indicate failure of a broker in handling a produce request + // for a partition it is the leader for case e: KafkaStorageException => fatal("Halting due to unrecoverable I/O error while handling produce request: ", e) Runtime.getRuntime.halt(1) null + case utpe: UnknownTopicOrPartitionException => + warn(utpe.getMessage) + new ProduceResult(topicAndPartition, utpe) + case nle: NotLeaderForPartitionException => + warn(nle.getMessage) + new ProduceResult(topicAndPartition, nle) case e => BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats.failedProduceRequestRate.mark() @@ -289,7 +299,16 @@ class KafkaApis(val requestChannel: RequestChannel, new FetchResponsePartitionData(ErrorMapping.NoError, highWatermark, messages) } } catch { - case t: Throwable => + // NOTE: Failed fetch requests is not incremented for UnknownTopicOrPartitionException and NotLeaderForPartitionException + // since failed fetch requests metric is supposed to indicate failure of a broker in handling a fetch request + // for a partition it is the leader for + case utpe: UnknownTopicOrPartitionException => + warn(utpe.getMessage) + new FetchResponsePartitionData(ErrorMapping.codeFor(utpe.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty) + case nle: NotLeaderForPartitionException => + warn(nle.getMessage) + new FetchResponsePartitionData(ErrorMapping.codeFor(nle.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty) + case t => BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats.failedFetchRequestRate.mark() error("error when processing request " + (topic, partition, offset, fetchSize), t) @@ -358,6 +377,14 @@ class KafkaApis(val requestChannel: RequestChannel, } (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.NoError, offsets)) } catch { + // NOTE: UnknownTopicOrPartitionException and NotLeaderForPartitionException are special cased since these error messages + // are typically transient and there is no value in logging the entire stack trace for the same + case utpe: UnknownTopicOrPartitionException => + warn(utpe.getMessage) + (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(utpe.getClass.asInstanceOf[Class[Throwable]]), Nil) ) + case nle: NotLeaderForPartitionException => + warn(nle.getMessage) + (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(nle.getClass.asInstanceOf[Class[Throwable]]), Nil) ) case e => warn("Error while responding to offset request", e) (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), Nil) ) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 5e4c9ca..96cbd62 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -145,7 +145,7 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro val logIndexIntervalBytes = props.getIntInRange("log.index.interval.bytes", 4096, (0, Int.MaxValue)) /* the number of messages accumulated on a log partition before messages are flushed to disk */ - val logFlushIntervalMessages = props.getIntInRange("log.flush.interval.messages", 500, (1, Int.MaxValue)) + val logFlushIntervalMessages = props.getIntInRange("log.flush.interval.messages", 10000, (1, Int.MaxValue)) /* the amount of time to wait before deleting a file from the filesystem */ val logDeleteDelayMs = props.getLongInRange("log.segment.delete.delay.ms", 60000, (0, Long.MaxValue)) diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 29d0af7..99e6f4e 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -63,21 +63,38 @@ class ReplicaFetcherThread(name:String, } } - // handle a partition whose offset is out of range and return a new fetch offset + /** + * Handle a partition whose offset is out of range and return a new fetch offset. + */ def handleOffsetOutOfRange(topicAndPartition: TopicAndPartition): Long = { - // This means the local replica is out of date. Truncate the log and catch up from beginning. - val request = OffsetRequest( - replicaId = brokerConfig.brokerId, - requestInfo = Map(topicAndPartition -> PartitionOffsetRequestInfo(OffsetRequest.EarliestTime, 1)) - ) - val partitionErrorAndOffset = simpleConsumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition) - val offset = partitionErrorAndOffset.error match { - case ErrorMapping.NoError => partitionErrorAndOffset.offsets.head - case _ => throw ErrorMapping.exceptionFor(partitionErrorAndOffset.error) - } val replica = replicaMgr.getReplica(topicAndPartition.topic, topicAndPartition.partition).get - replica.log.get.truncateFullyAndStartAt(offset) - offset + val log = replica.log.get + + /** + * Unclean leader election: A follower goes down, in the meanwhile the leader keeps appending messages. The follower comes back up + * and before it has completely caught up with the leader's logs, all replicas in the ISR go down. The follower is now uncleanly + * elected as the new leader, and it starts appending messages from the client. The old leader comes back up, becomes a follower + * and it may discover that the current leader's end offset is behind its own end offset. + * + * In such a case, truncate the current follower's log to the current leader's end offset and continue fetching. + * + * There is a potential for a mismatch between the logs of the two replicas here. We don't fix this mismatch as of now. + */ + val leaderEndOffset = simpleConsumer.earliestOrLatestOffset(topicAndPartition, OffsetRequest.LatestTime, brokerConfig.brokerId) + if (leaderEndOffset < log.logEndOffset) { + log.truncateTo(leaderEndOffset) + leaderEndOffset + } else { + /** + * The follower could have been down for a long time and when it starts up, its end offset could be smaller than the leader's + * start offset because the leader has deleted old logs (log.logEndOffset < leaderStartOffset). + * + * Roll out a new log at the follower with the start offset equal to the current leader's start offset and continue fetching. + */ + val leaderStartOffset = simpleConsumer.earliestOrLatestOffset(topicAndPartition, OffsetRequest.EarliestTime, brokerConfig.brokerId) + log.truncateFullyAndStartAt(leaderStartOffset) + leaderStartOffset + } } // any logic for partitions whose leader has changed diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 765d3cb..477f60e 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -30,6 +30,7 @@ import java.util.concurrent.TimeUnit import kafka.common._ import kafka.api.{StopReplicaRequest, PartitionStateInfo, LeaderAndIsrRequest} import kafka.controller.KafkaController +import org.apache.log4j.Logger object ReplicaManager { @@ -44,19 +45,25 @@ class ReplicaManager(val config: KafkaConfig, val logManager: LogManager) extends Logging with KafkaMetricsGroup { /* epoch of the controller that last changed the leader */ @volatile var controllerEpoch: Int = KafkaController.InitialControllerEpoch - 1 + private val localBrokerId = config.brokerId private val allPartitions = new Pool[(String, Int), Partition] private var leaderPartitions = new mutable.HashSet[Partition]() private val leaderPartitionsLock = new Object val replicaFetcherManager = new ReplicaFetcherManager(config, this) - this.logIdent = "Replica Manager on Broker " + config.brokerId + ": " private val highWatermarkCheckPointThreadStarted = new AtomicBoolean(false) val highWatermarkCheckpoints = config.logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, ReplicaManager.HighWatermarkFilename)))).toMap private var hwThreadInitialized = false + this.logIdent = "[Replica Manager on Broker " + localBrokerId + "]: " + private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) newGauge( "LeaderCount", new Gauge[Int] { - def getValue = leaderPartitions.size + def getValue = { + leaderPartitionsLock synchronized { + leaderPartitions.size + } + } } ) newGauge( @@ -104,7 +111,7 @@ class ReplicaManager(val config: KafkaConfig, } def stopReplica(topic: String, partitionId: Int, deletePartition: Boolean): Short = { - trace("Handling stop replica for partition [%s, %d]".format(topic, partitionId)) + stateChangeLogger.trace("Broker %d handling stop replica for partition [%s,%d]".format(localBrokerId, topic, partitionId)) val errorCode = ErrorMapping.NoError getReplica(topic, partitionId) match { case Some(replica) => @@ -116,18 +123,19 @@ class ReplicaManager(val config: KafkaConfig, leaderPartitions -= replica.partition } allPartitions.remove((topic, partitionId)) - info("After removing partition (%s, %d), the rest of allReplicas is: [%s]".format(topic, partitionId, allPartitions)) + info("After removing partition [%s,%d], the rest of allReplicas is: [%s]".format(topic, partitionId, allPartitions)) case None => //do nothing if replica no longer exists } - trace("Finish handling stop replica [%s, %d]".format(topic, partitionId)) + stateChangeLogger.trace("Broker %d finished handling stop replica [%s,%d]".format(localBrokerId, topic, partitionId)) errorCode } def stopReplicas(stopReplicaRequest: StopReplicaRequest): (mutable.Map[(String, Int), Short], Short) = { val responseMap = new collection.mutable.HashMap[(String, Int), Short] if(stopReplicaRequest.controllerEpoch < controllerEpoch) { - error("Received stop replica request from an old controller epoch %d.".format(stopReplicaRequest.controllerEpoch) + - " Latest known controller epoch is %d " + controllerEpoch) + stateChangeLogger.error("Broker %d received stop replica request from an old controller epoch %d." + .format(localBrokerId, stopReplicaRequest.controllerEpoch) + + " Latest known controller epoch is %d " + controllerEpoch) (responseMap, ErrorMapping.StaleControllerEpochCode) } else { controllerEpoch = stopReplicaRequest.controllerEpoch @@ -162,7 +170,7 @@ class ReplicaManager(val config: KafkaConfig, if(replicaOpt.isDefined) return replicaOpt.get else - throw new ReplicaNotAvailableException("Replica %d is not available for partiton [%s, %d] yet".format(config.brokerId, topic, partition)) + throw new ReplicaNotAvailableException("Replica %d is not available for partiton [%s,%d] yet".format(config.brokerId, topic, partition)) } def getLeaderReplicaIfLocal(topic: String, partitionId: Int): Replica = { @@ -174,8 +182,8 @@ class ReplicaManager(val config: KafkaConfig, partition.leaderReplicaIfLocal match { case Some(leaderReplica) => leaderReplica case None => - throw new LeaderNotAvailableException("Leader not local for topic %s partition %d on broker %d" - .format(topic, partitionId, config.brokerId)) + throw new NotLeaderForPartitionException("Leader not local for topic %s partition %d on broker %d" + .format(topic, partitionId, config.brokerId)) } } } @@ -189,13 +197,19 @@ class ReplicaManager(val config: KafkaConfig, } def becomeLeaderOrFollower(leaderAndISRRequest: LeaderAndIsrRequest): (collection.Map[(String, Int), Short], Short) = { - info("Handling leader and isr request %s".format(leaderAndISRRequest)) + leaderAndISRRequest.partitionStateInfos.foreach(p => + stateChangeLogger.trace("Broker %d handling LeaderAndIsr request correlationId %d received from controller %d epoch %d for partition [%s,%d]" + .format(localBrokerId, leaderAndISRRequest.correlationId, leaderAndISRRequest.controllerId, + leaderAndISRRequest.controllerEpoch, p._1._1, p._1._2))) + info("Handling LeaderAndIsr request %s".format(leaderAndISRRequest)) + val responseMap = new collection.mutable.HashMap[(String, Int), Short] if(leaderAndISRRequest.controllerEpoch < controllerEpoch) { - error("Received leader and isr request from an old controller epoch %d.".format(leaderAndISRRequest.controllerEpoch) + - " Latest known controller epoch is %d " + controllerEpoch) + stateChangeLogger.error("Broker %d received LeaderAndIsr request correlationId %d with an old controllerEpoch %d, latest known controllerEpoch is %d" + .format(localBrokerId, leaderAndISRRequest.controllerEpoch, leaderAndISRRequest.correlationId, controllerEpoch)) (responseMap, ErrorMapping.StaleControllerEpochCode) }else { + val controllerId = leaderAndISRRequest.controllerId controllerEpoch = leaderAndISRRequest.controllerEpoch for((topicAndPartition, partitionStateInfo) <- leaderAndISRRequest.partitionStateInfos) { var errorCode = ErrorMapping.NoError @@ -205,17 +219,25 @@ class ReplicaManager(val config: KafkaConfig, val requestedLeaderId = partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader try { if(requestedLeaderId == config.brokerId) - makeLeader(topic, partitionId, partitionStateInfo) + makeLeader(controllerId, controllerEpoch, topic, partitionId, partitionStateInfo, leaderAndISRRequest.correlationId) else - makeFollower(topic, partitionId, partitionStateInfo, leaderAndISRRequest.leaders) + makeFollower(controllerId, controllerEpoch, topic, partitionId, partitionStateInfo, leaderAndISRRequest.leaders, + leaderAndISRRequest.correlationId) } catch { case e => - error("Error processing leaderAndISR request %s".format(leaderAndISRRequest), e) + val errorMsg = ("Error on broker %d while processing LeaderAndIsr request correlationId %d received from controller %d " + + "epoch %d for partition %s").format(localBrokerId, leaderAndISRRequest.correlationId, leaderAndISRRequest.controllerId, + leaderAndISRRequest.controllerEpoch, topicAndPartition) + stateChangeLogger.error(errorMsg, e) errorCode = ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) } responseMap.put(topicAndPartition, errorCode) + leaderAndISRRequest.partitionStateInfos.foreach(p => + stateChangeLogger.trace("Broker %d handled LeaderAndIsr request correlationId %d received from controller %d epoch %d for partition [%s,%d]" + .format(localBrokerId, leaderAndISRRequest.correlationId, leaderAndISRRequest.controllerId, + leaderAndISRRequest.controllerEpoch, p._1._1, p._1._2))) } - info("Completed leader and isr request %s".format(leaderAndISRRequest)) + info("Handled leader and isr request %s".format(leaderAndISRRequest)) // 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 if (!hwThreadInitialized) { @@ -227,33 +249,38 @@ class ReplicaManager(val config: KafkaConfig, } } - private def makeLeader(topic: String, partitionId: Int, partitionStateInfo: PartitionStateInfo) = { + private def makeLeader(controllerId: Int, epoch:Int, topic: String, partitionId: Int, + partitionStateInfo: PartitionStateInfo, correlationId: Int) = { val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch - info("Becoming Leader for topic [%s] partition [%d]".format(topic, partitionId)) + stateChangeLogger.trace(("Broker %d received LeaderAndIsr request correlationId %d from controller %d epoch %d " + + "starting the become-leader transition for partition [%s,%d]") + .format(localBrokerId, correlationId, controllerId, epoch, topic, partitionId)) val partition = getOrCreatePartition(topic, partitionId, partitionStateInfo.replicationFactor) - if (partition.makeLeader(topic, partitionId, leaderIsrAndControllerEpoch)) { + if (partition.makeLeader(controllerId, topic, partitionId, leaderIsrAndControllerEpoch, correlationId)) { // also add this partition to the list of partitions for which the leader is the current broker leaderPartitionsLock synchronized { leaderPartitions += partition } } - info("Completed the leader state transition for topic %s partition %d".format(topic, partitionId)) + stateChangeLogger.trace("Broker %d completed become-leader transition for partition [%s,%d]".format(localBrokerId, topic, partitionId)) } - private def makeFollower(topic: String, partitionId: Int, partitionStateInfo: PartitionStateInfo, - liveBrokers: Set[Broker]) { + private def makeFollower(controllerId: Int, epoch: Int, topic: String, partitionId: Int, + partitionStateInfo: PartitionStateInfo, liveBrokers: Set[Broker], correlationId: Int) { val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch val leaderBrokerId: Int = leaderIsrAndControllerEpoch.leaderAndIsr.leader - info("Starting the follower state transition to follow leader %d for topic %s partition %d" - .format(leaderBrokerId, topic, partitionId)) + stateChangeLogger.trace(("Broker %d received LeaderAndIsr request correlationId %d from controller %d epoch %d " + + "starting the become-follower transition for partition [%s,%d]") + .format(localBrokerId, correlationId, controllerId, epoch, topic, partitionId)) val partition = getOrCreatePartition(topic, partitionId, partitionStateInfo.replicationFactor) - if (partition.makeFollower(topic, partitionId, leaderIsrAndControllerEpoch, liveBrokers)) { + if (partition.makeFollower(controllerId, topic, partitionId, leaderIsrAndControllerEpoch, liveBrokers, correlationId)) { // remove this replica's partition from the ISR expiration queue leaderPartitionsLock synchronized { leaderPartitions -= partition } } + stateChangeLogger.trace("Broker %d completed the become-follower transition for partition [%s,%d]".format(localBrokerId, topic, partitionId)) } private def maybeShrinkIsr(): Unit = { @@ -268,7 +295,7 @@ class ReplicaManager(val config: KafkaConfig, if(partitionOpt.isDefined) { partitionOpt.get.updateLeaderHWAndMaybeExpandIsr(replicaId, offset) } else { - warn("While recording the follower position, the partition [%s, %d] hasn't been created, skip updating leader HW".format(topic, partitionId)) + warn("While recording the follower position, the partition [%s,%d] hasn't been created, skip updating leader HW".format(topic, partitionId)) } } diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index d9546ca..5231e7c 100644 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -30,21 +30,17 @@ object DumpLogSegments { def main(args: Array[String]) { val parser = new OptionParser val printOpt = parser.accepts("print-data-log", "if set, printing the messages content when dumping data logs") - .withOptionalArg - .describedAs("print data log content") - .ofType(classOf[java.lang.Boolean]) - .defaultsTo(false) - val verifyOpt = parser.accepts("verify-index-only", "if set, just verify the index log without printing its content") - .withOptionalArg - .describedAs("just verify the index log") - .ofType(classOf[java.lang.Boolean]) - .defaultsTo(false) - val filesOpt = parser.accepts("files", "REQUIRED: The comma separated list of data and index log files to be dumped") .withRequiredArg .describedAs("file1, file2, ...") .ofType(classOf[String]) + val maxMessageSizeOpt = parser.accepts("max-message-size", "Size of largest message.") + .withRequiredArg + .describedAs("size") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(5 * 1024 * 1024) + val deepIterationOpt = parser.accepts("deep-iteration", "if set, uses deep instead of shallow iteration") val options = parser.parse(args : _*) if(!options.has(filesOpt)) { @@ -56,6 +52,8 @@ object DumpLogSegments { val print = if(options.has(printOpt)) true else false val verifyOnly = if(options.has(verifyOpt)) true else false val files = options.valueOf(filesOpt).split(",") + val maxMessageSize = options.valueOf(maxMessageSizeOpt).intValue() + val isDeepIteration = if(options.has(deepIterationOpt)) true else false val misMatchesForIndexFilesMap = new mutable.HashMap[String, List[(Long, Long)]] val nonConsecutivePairsForLogFilesMap = new mutable.HashMap[String, List[(Long, Long)]] @@ -64,17 +62,17 @@ object DumpLogSegments { val file = new File(arg) if(file.getName.endsWith(Log.LogFileSuffix)) { println("Dumping " + file) - dumpLog(file, print, nonConsecutivePairsForLogFilesMap) + dumpLog(file, print, nonConsecutivePairsForLogFilesMap, isDeepIteration) } else if(file.getName.endsWith(Log.IndexFileSuffix)) { println("Dumping " + file) - dumpIndex(file, verifyOnly, misMatchesForIndexFilesMap) + dumpIndex(file, verifyOnly, misMatchesForIndexFilesMap, maxMessageSize) } } misMatchesForIndexFilesMap.foreach { case (fileName, listOfMismatches) => { System.err.println("Mismatches in :" + fileName) listOfMismatches.foreach(m => { - System.err.println(" Index position: %d, log position: %d".format(m._1, m._2)) + System.err.println(" Index offset: %d, log offset: %d".format(m._1, m._2)) }) } } @@ -89,7 +87,10 @@ object DumpLogSegments { } /* print out the contents of the index */ - private def dumpIndex(file: File, verifyOnly: Boolean, misMatchesForIndexFilesMap: mutable.HashMap[String, List[(Long, Long)]]) { + private def dumpIndex(file: File, + verifyOnly: Boolean, + misMatchesForIndexFilesMap: mutable.HashMap[String, List[(Long, Long)]], + maxMessageSize: Int) { val startOffset = file.getName().split("\\.")(0).toLong val logFileName = file.getAbsolutePath.split("\\.")(0) + Log.LogFileSuffix val logFile = new File(logFileName) @@ -97,8 +98,8 @@ object DumpLogSegments { val index = new OffsetIndex(file = file, baseOffset = startOffset) for(i <- 0 until index.entries) { val entry = index.entry(i) - val partialFileMessageSet: FileMessageSet = messageSet.read(entry.position, messageSet.sizeInBytes()) - val messageAndOffset = partialFileMessageSet.head + val partialFileMessageSet: FileMessageSet = messageSet.read(entry.position, maxMessageSize) + val messageAndOffset = getIterator(partialFileMessageSet.head, isDeepIteration = true).next() if(messageAndOffset.offset != entry.offset + index.baseOffset) { var misMatchesSeq = misMatchesForIndexFilesMap.getOrElse(file.getName, List[(Long, Long)]()) misMatchesSeq ::=(entry.offset + index.baseOffset, messageAndOffset.offset) @@ -113,41 +114,74 @@ object DumpLogSegments { } /* print out the contents of the log */ - private def dumpLog(file: File, printContents: Boolean, nonConsecutivePairsForLogFilesMap: mutable.HashMap[String, List[(Long, Long)]]) { + private def dumpLog(file: File, + printContents: Boolean, + nonConsecutivePairsForLogFilesMap: mutable.HashMap[String, List[(Long, Long)]], + isDeepIteration: Boolean) { val startOffset = file.getName().split("\\.")(0).toLong println("Starting offset: " + startOffset) val messageSet = new FileMessageSet(file) var validBytes = 0L var lastOffset = -1l - for(messageAndOffset <- messageSet) { - val msg = messageAndOffset.message + for(shallowMessageAndOffset <- messageSet) { // this only does shallow iteration + val itr = getIterator(shallowMessageAndOffset, isDeepIteration) + for (messageAndOffset <- itr) { + val msg = messageAndOffset.message - if(lastOffset == -1) + if(lastOffset == -1) + lastOffset = messageAndOffset.offset + // If we are iterating uncompressed messages, offsets must be consecutive + else if (msg.compressionCodec == NoCompressionCodec && messageAndOffset.offset != lastOffset +1) { + var nonConsecutivePairsSeq = nonConsecutivePairsForLogFilesMap.getOrElse(file.getName, List[(Long, Long)]()) + nonConsecutivePairsSeq ::=(lastOffset, messageAndOffset.offset) + nonConsecutivePairsForLogFilesMap.put(file.getName, nonConsecutivePairsSeq) + } lastOffset = messageAndOffset.offset - // If it's uncompressed message, its offset must be lastOffset + 1 no matter last message is compressed or uncompressed - else if (msg.compressionCodec == NoCompressionCodec && messageAndOffset.offset != lastOffset +1) { - var nonConsecutivePairsSeq = nonConsecutivePairsForLogFilesMap.getOrElse(file.getName, List[(Long, Long)]()) - nonConsecutivePairsSeq ::=(lastOffset, messageAndOffset.offset) - nonConsecutivePairsForLogFilesMap.put(file.getName, nonConsecutivePairsSeq) - } - lastOffset = messageAndOffset.offset - print("offset: " + messageAndOffset.offset + " position: " + validBytes + " isvalid: " + msg.isValid + - " payloadsize: " + msg.payloadSize + " magic: " + msg.magic + - " compresscodec: " + msg.compressionCodec + " crc: " + msg.checksum) - validBytes += MessageSet.entrySize(msg) - if(msg.hasKey) - print(" keysize: " + msg.keySize) - if(printContents) { + print("offset: " + messageAndOffset.offset + " position: " + validBytes + " isvalid: " + msg.isValid + + " payloadsize: " + msg.payloadSize + " magic: " + msg.magic + + " compresscodec: " + msg.compressionCodec + " crc: " + msg.checksum) if(msg.hasKey) - print(" key: " + Utils.readString(messageAndOffset.message.key, "UTF-8")) - val payload = if(messageAndOffset.message.isNull) null else Utils.readString(messageAndOffset.message.payload, "UTF-8") - print(" payload: " + payload) + print(" keysize: " + msg.keySize) + if(printContents) { + if(msg.hasKey) + print(" key: " + Utils.readString(messageAndOffset.message.key, "UTF-8")) + val payload = if(messageAndOffset.message.isNull) null else Utils.readString(messageAndOffset.message.payload, "UTF-8") + print(" payload: " + payload) + } + println() } - println() + validBytes += MessageSet.entrySize(shallowMessageAndOffset.message) } val trailingBytes = messageSet.sizeInBytes - validBytes if(trailingBytes > 0) println("Found %d invalid bytes at the end of %s".format(trailingBytes, file.getName)) } + + private def getIterator(messageAndOffset: MessageAndOffset, isDeepIteration: Boolean) = { + if (isDeepIteration) { + val message = messageAndOffset.message + message.compressionCodec match { + case NoCompressionCodec => + getSingleMessageIterator(messageAndOffset) + case _ => + ByteBufferMessageSet.decompress(message).iterator + } + } else + getSingleMessageIterator(messageAndOffset) + } + + private def getSingleMessageIterator(messageAndOffset: MessageAndOffset) = { + new IteratorTemplate[MessageAndOffset] { + var messageIterated = false + + override def makeNext(): MessageAndOffset = { + if (!messageIterated) { + messageIterated = true + messageAndOffset + } else + allDone() + } + } + } } diff --git a/core/src/main/scala/kafka/tools/JmxTool.scala b/core/src/main/scala/kafka/tools/JmxTool.scala index 4bfac53..7e424e7 100644 --- a/core/src/main/scala/kafka/tools/JmxTool.scala +++ b/core/src/main/scala/kafka/tools/JmxTool.scala @@ -43,7 +43,7 @@ object JmxTool extends Logging { val attributesOpt = parser.accepts("attributes", "The whitelist of attributes to query. This is a comma-separated list. If no " + "attributes are specified all objects will be queried.") - .withOptionalArg() + .withRequiredArg .describedAs("name") .ofType(classOf[String]) val reportingIntervalOpt = parser.accepts("reporting-interval", "Interval in MS with which to poll jmx stats.") @@ -54,7 +54,7 @@ object JmxTool extends Logging { val helpOpt = parser.accepts("help", "Print usage information.") val dateFormatOpt = parser.accepts("date-format", "The date format to use for formatting the time field. " + "See java.text.SimpleDateFormat for options.") - .withOptionalArg() + .withRequiredArg .describedAs("format") .ofType(classOf[String]) val jmxServiceUrlOpt = diff --git a/core/src/main/scala/kafka/tools/KafkaMigrationTool.java b/core/src/main/scala/kafka/tools/KafkaMigrationTool.java index 2804908..ad00db6 100644 --- a/core/src/main/scala/kafka/tools/KafkaMigrationTool.java +++ b/core/src/main/scala/kafka/tools/KafkaMigrationTool.java @@ -35,7 +35,9 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; import java.util.Properties; -import java.util.concurrent.*; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; @@ -133,6 +135,13 @@ public class KafkaMigrationTool .describedAs("Java regex (String)") .ofType(String.class); + ArgumentAcceptingOptionSpec queueSizeOpt + = parser.accepts("queue.size", "Number of messages that are buffered between the 0.7 consumer and 0.8 producer") + .withRequiredArg() + .describedAs("Queue size in terms of number of messages") + .ofType(Integer.class) + .defaultsTo(10000); + OptionSpecBuilder helpOpt = parser.accepts("help", "Print this message."); @@ -211,7 +220,8 @@ public class KafkaMigrationTool kafkaProducerProperties_08.load(new FileInputStream(producerConfigFile_08)); kafkaProducerProperties_08.setProperty("serializer.class", "kafka.serializer.DefaultEncoder"); // create a producer channel instead - ProducerDataChannel> producerDataChannel = new ProducerDataChannel>(numProducers); + int queueSize = options.valueOf(queueSizeOpt); + ProducerDataChannel> producerDataChannel = new ProducerDataChannel>(queueSize); int threadId = 0; Runtime.getRuntime().addShutdownHook(new Thread() { @@ -242,9 +252,11 @@ public class KafkaMigrationTool thread.start(); migrationThreads.add(thread); } + + String clientId = kafkaProducerProperties_08.getProperty("client.id"); // start producer threads for (int i = 0; i < numProducers; i++) { - kafkaProducerProperties_08.put("client.id", String.valueOf(i) + "-" + i); + kafkaProducerProperties_08.put("client.id", clientId + "-" + i); ProducerConfig producerConfig_08 = new ProducerConfig(kafkaProducerProperties_08); Producer producer = new Producer(producerConfig_08); ProducerThread producerThread = new ProducerThread(producerDataChannel, producer, i); diff --git a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala index aa5e661..7629329 100644 --- a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala +++ b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala @@ -24,6 +24,7 @@ import kafka.client.ClientUtils import kafka.api.{OffsetRequest, FetchRequestBuilder, Request} import kafka.cluster.Broker import scala.collection.JavaConversions._ +import kafka.common.TopicAndPartition /** * Command line program to dump out messages to standard out using the simple consumer @@ -54,12 +55,12 @@ object SimpleConsumerShell extends Logging { .ofType(classOf[java.lang.Integer]) .defaultsTo(UseLeaderReplica) val offsetOpt = parser.accepts("offset", "The offset id to consume from, default to -2 which means from beginning; while value -1 means from end") - .withOptionalArg() + .withRequiredArg .describedAs("consume offset") .ofType(classOf[java.lang.Long]) .defaultsTo(OffsetRequest.EarliestTime) val clientIdOpt = parser.accepts("clientId", "The ID of this client.") - .withOptionalArg + .withRequiredArg .describedAs("clientId") .ofType(classOf[String]) .defaultsTo("SimpleConsumerShell") @@ -78,15 +79,16 @@ object SimpleConsumerShell extends Logging { .describedAs("prop") .ofType(classOf[String]) val printOffsetOpt = parser.accepts("print-offsets", "Print the offsets returned by the iterator") - .withOptionalArg - .describedAs("print offsets") - .ofType(classOf[java.lang.Boolean]) - .defaultsTo(false) val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.") .withRequiredArg .describedAs("ms") .ofType(classOf[java.lang.Integer]) .defaultsTo(1000) + val maxMessagesOpt = parser.accepts("max-messages", "The number of messages to consume") + .withRequiredArg + .describedAs("max-messages") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(Integer.MAX_VALUE) val skipMessageOnErrorOpt = parser.accepts("skip-message-on-error", "If there is an error when processing a message, " + "skip it instead of halt.") val noWaitAtEndOfLogOpt = parser.accepts("no-wait-at-logend", @@ -108,6 +110,7 @@ object SimpleConsumerShell extends Logging { val fetchSize = options.valueOf(fetchSizeOpt).intValue val clientId = options.valueOf(clientIdOpt).toString val maxWaitMs = options.valueOf(maxWaitMsOpt).intValue() + val maxMessages = options.valueOf(maxMessagesOpt).intValue val skipMessageOnError = if (options.has(skipMessageOnErrorOpt)) true else false val printOffsets = if(options.has(printOffsetOpt)) true else false @@ -164,21 +167,36 @@ object SimpleConsumerShell extends Logging { System.err.println("Invalid starting offset: %d".format(startingOffset)) System.exit(1) } - if(startingOffset < 0) - startingOffset = SimpleConsumer.earliestOrLatestOffset(fetchTargetBroker, topic, partitionId, startingOffset, clientId, false) + if (startingOffset < 0) { + val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, fetchTargetBroker.port, ConsumerConfig.SocketTimeout, + ConsumerConfig.SocketBufferSize, clientId) + try { + startingOffset = simpleConsumer.earliestOrLatestOffset(TopicAndPartition(topic, partitionId), startingOffset, + Request.DebuggingConsumerId) + } catch { + case t: Throwable => + System.err.println("Error in getting earliest or latest offset due to: " + Utils.stackTrace(t)) + System.exit(1) + } finally { + if (simpleConsumer != null) + simpleConsumer.close() + } + } // initializing formatter val formatter: MessageFormatter = messageFormatterClass.newInstance().asInstanceOf[MessageFormatter] formatter.init(formatterArgs) - info("Starting simple consumer shell to partition [%s, %d], replica [%d], host and port: [%s, %d], from offset [%d]" - .format(topic, partitionId, replicaId, fetchTargetBroker.host, fetchTargetBroker.port, startingOffset)) + val replicaString = if(replicaId > 0) "leader" else "replica" + info("Starting simple consumer shell to partition [%s, %d], %s [%d], host and port: [%s, %d], from offset [%d]" + .format(topic, partitionId, replicaString, replicaId, fetchTargetBroker.host, fetchTargetBroker.port, startingOffset)) val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, fetchTargetBroker.port, 10000, 64*1024, clientId) val thread = Utils.newThread("kafka-simpleconsumer-shell", new Runnable() { def run() { var offset = startingOffset + var numMessagesConsumed = 0 try { - while(true) { + while(numMessagesConsumed < maxMessages) { val fetchRequest = fetchRequestBuilder .addFetch(topic, partitionId, offset, fetchSize) .build() @@ -189,7 +207,7 @@ object SimpleConsumerShell extends Logging { return } debug("multi fetched " + messageSet.sizeInBytes + " bytes from offset " + offset) - for(messageAndOffset <- messageSet) { + for(messageAndOffset <- messageSet if(numMessagesConsumed < maxMessages)) { try { offset = messageAndOffset.nextOffset if(printOffsets) @@ -197,6 +215,7 @@ object SimpleConsumerShell extends Logging { val message = messageAndOffset.message val key = if(message.hasKey) Utils.readBytes(message.key) else null formatter.writeTo(key, if(message.isNull) null else Utils.readBytes(message.payload), System.out) + numMessagesConsumed += 1 } catch { case e => if (skipMessageOnError) @@ -216,6 +235,8 @@ object SimpleConsumerShell extends Logging { } catch { case e: Throwable => error("Error consuming topic, partition, replica (%s, %d, %d) with offset [%d]".format(topic, partitionId, replicaId, offset), e) + }finally { + info("Consumed " + numMessagesConsumed + " messages") } } }, false) diff --git a/core/src/main/scala/kafka/tools/StateChangeLogMerger.scala b/core/src/main/scala/kafka/tools/StateChangeLogMerger.scala new file mode 100644 index 0000000..97970fb --- /dev/null +++ b/core/src/main/scala/kafka/tools/StateChangeLogMerger.scala @@ -0,0 +1,188 @@ +/** + * 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.tools + +import joptsimple._ +import scala.util.matching.Regex +import collection.mutable +import java.util.Date +import java.text.SimpleDateFormat +import kafka.utils.Logging +import kafka.common.Topic +import java.io.{BufferedOutputStream, OutputStream} + +/** + * A utility that merges the state change logs (possibly obtained from different brokers and over multiple days). + * + * This utility expects at least one of the following two arguments - + * 1. A list of state change log files + * 2. A regex to specify state change log file names. + * + * This utility optionally also accepts the following arguments - + * 1. The topic whose state change logs should be merged + * 2. A list of partitions whose state change logs should be merged (can be specified only when the topic argument + * is explicitly specified) + * 3. Start time from when the logs should be merged + * 4. End time until when the logs should be merged + */ + +object StateChangeLogMerger extends Logging { + + val dateFormatString = "yyyy-MM-dd HH:mm:ss,SSS" + val topicPartitionRegex = new Regex("\\[(" + Topic.legalChars + "+),( )*([0-9]+)\\]") + val dateRegex = new Regex("[0-9]{4}-[0-9]{2}-[0-9]{2} [0-9]{2}:[0-9]{2}:[0-9]{2},[0-9]{3}") + val dateFormat = new SimpleDateFormat(dateFormatString) + var files: List[String] = List() + var topic: String = null + var partitions: List[Int] = List() + var startDate: Date = null + var endDate: Date = null + + def main(args: Array[String]) { + + // Parse input arguments. + val parser = new OptionParser + val filesOpt = parser.accepts("logs", "Comma separated list of state change logs or a regex for the log file names") + .withRequiredArg + .describedAs("file1,file2,...") + .ofType(classOf[String]) + val regexOpt = parser.accepts("logs-regex", "Regex to match the state change log files to be merged") + .withRequiredArg + .describedAs("for example: /tmp/state-change.log*") + .ofType(classOf[String]) + val topicOpt = parser.accepts("topic", "The topic whose state change logs should be merged") + .withRequiredArg + .describedAs("topic") + .ofType(classOf[String]) + val partitionsOpt = parser.accepts("partitions", "Comma separated list of partition ids whose state change logs should be merged") + .withRequiredArg + .describedAs("0,1,2,...") + .ofType(classOf[String]) + val startTimeOpt = parser.accepts("start-time", "The earliest timestamp of state change log entries to be merged") + .withRequiredArg + .describedAs("start timestamp in the format " + dateFormat) + .ofType(classOf[String]) + .defaultsTo("0000-00-00 00:00:00,000") + val endTimeOpt = parser.accepts("end-time", "The latest timestamp of state change log entries to be merged") + .withRequiredArg + .describedAs("end timestamp in the format " + dateFormat) + .ofType(classOf[String]) + .defaultsTo("9999-12-31 23:59:59,999") + + + val options = parser.parse(args : _*) + if ((!options.has(filesOpt) && !options.has(regexOpt)) || (options.has(filesOpt) && options.has(regexOpt))) { + System.err.println("Provide arguments to exactly one of the two options \"" + filesOpt + "\" or \"" + regexOpt + "\"") + parser.printHelpOn(System.err) + System.exit(1) + } + if (options.has(partitionsOpt) && !options.has(topicOpt)) { + System.err.println("The option \"" + topicOpt + "\" needs to be provided an argument when specifying partition ids") + parser.printHelpOn(System.err) + System.exit(1) + } + + // Populate data structures. + if (options.has(filesOpt)) { + files :::= options.valueOf(filesOpt).split(",").toList + } else if (options.has(regexOpt)) { + val regex = options.valueOf(regexOpt) + val fileNameIndex = regex.lastIndexOf('/') + 1 + val dirName = if (fileNameIndex == 0) "." else regex.substring(0, fileNameIndex - 1) + val fileNameRegex = new Regex(regex.substring(fileNameIndex)) + files :::= new java.io.File(dirName).listFiles.filter(f => fileNameRegex.findFirstIn(f.getName) != None).map(dirName + "/" + _.getName).toList + } + if (options.has(topicOpt)) { + topic = options.valueOf(topicOpt) + } + if (options.has(partitionsOpt)) { + partitions = options.valueOf(partitionsOpt).split(",").toList.map(_.toInt) + } + startDate = dateFormat.parse(options.valueOf(startTimeOpt).replace('\"', ' ').trim) + endDate = dateFormat.parse(options.valueOf(endTimeOpt).replace('\"', ' ').trim) + + /** + * n-way merge from m input files: + * 1. Read a line that matches the specified topic/partitions and date range from every input file in a priority queue. + * 2. Take the line from the file with the earliest date and add it to a buffered output stream. + * 3. Add another line from the file selected in step 2 in the priority queue. + * 4. Flush the output buffer at the end. (The buffer will also be automatically flushed every K bytes.) + */ + val pqueue = new mutable.PriorityQueue[LineIterator]()(dateBasedOrdering) + val output: OutputStream = new BufferedOutputStream(System.out, 1024*1024) + val lineIterators = files.map(io.Source.fromFile(_).getLines) + var lines: List[LineIterator] = List() + + for (itr <- lineIterators) { + val lineItr = getNextLine(itr) + if (!lineItr.isEmpty) + lines ::= lineItr + } + if (!lines.isEmpty) pqueue.enqueue(lines:_*) + + while (!pqueue.isEmpty) { + val lineItr = pqueue.dequeue() + output.write((lineItr.line + "\n").getBytes) + val nextLineItr = getNextLine(lineItr.itr) + if (!nextLineItr.isEmpty) + pqueue.enqueue(nextLineItr) + } + + output.flush() + } + + /** + * Returns the next line that matches the specified topic/partitions from the file that has the earliest date + * from the specified date range. + * @param itr Line iterator of a file + * @return (line from a file, line iterator for the same file) + */ + def getNextLine(itr: Iterator[String]): LineIterator = { + while (itr != null && itr.hasNext) { + val nextLine = itr.next + dateRegex.findFirstIn(nextLine) match { + case Some(d) => + val date = dateFormat.parse(d) + if ((date.equals(startDate) || date.after(startDate)) && (date.equals(endDate) || date.before(endDate))) { + topicPartitionRegex.findFirstMatchIn(nextLine) match { + case Some(matcher) => + if ((topic == null || topic == matcher.group(1)) && (partitions.isEmpty || partitions.contains(matcher.group(3).toInt))) + return new LineIterator(nextLine, itr) + case None => + } + } + case None => + } + } + new LineIterator() + } + + class LineIterator(val line: String, val itr: Iterator[String]) { + def this() = this("", null) + def isEmpty = (line == "" && itr == null) + } + + implicit object dateBasedOrdering extends Ordering[LineIterator] { + def compare(first: LineIterator, second: LineIterator) = { + val firstDate = dateRegex.findFirstIn(first.line).get + val secondDate = dateRegex.findFirstIn(second.line).get + secondDate.compareTo(firstDate) + } + } + +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala index c8fdf4a..8692abc 100644 --- a/core/src/main/scala/kafka/utils/Utils.scala +++ b/core/src/main/scala/kafka/utils/Utils.scala @@ -24,6 +24,7 @@ import java.nio.channels._ import java.lang.management._ import javax.management._ import scala.collection._ +import mutable.ListBuffer import scala.collection.mutable import java.util.Properties import kafka.common.KafkaException @@ -444,27 +445,42 @@ object Utils extends Logging { def nullOrEmpty(s: String): Boolean = s == null || s.equals("") /** - * Format a Map[String, String] as JSON object. + * Merge JSON fields of the format "key" : value/object/array. */ - def mapToJson(jsonDataMap: Map[String, String], valueInQuotes: Boolean): String = { + def mergeJsonFields(objects: Seq[String]): String = { val builder = new StringBuilder builder.append("{ ") - var numElements = 0 - for ( (key, value) <- jsonDataMap.toList.sorted) { - if (numElements > 0) - builder.append(", ") + builder.append(objects.sorted.map(_.trim).mkString(", ")) + builder.append(" }") + builder.toString + } + + /** + * Format a Map[String, String] as JSON object. + */ + def mapToJsonFields(jsonDataMap: Map[String, String], valueInQuotes: Boolean): Seq[String] = { + val jsonFields: mutable.ListBuffer[String] = ListBuffer() + val builder = new StringBuilder + for ((key, value) <- jsonDataMap.toList.sorted) { builder.append("\"" + key + "\":") if (valueInQuotes) builder.append("\"" + value + "\"") else builder.append(value) - numElements += 1 + jsonFields += builder.toString + builder.clear() } - builder.append(" }") - builder.toString + jsonFields } /** + * Format a Map[String, String] as JSON object. + */ + def mapToJson(jsonDataMap: Map[String, String], valueInQuotes: Boolean): String = { + mergeJsonFields(mapToJsonFields(jsonDataMap, valueInQuotes)) + } + + /** * Format a Seq[String] as JSON array. */ def seqToJson(jsonData: Seq[String], valueInQuotes: Boolean): String = { @@ -483,45 +499,11 @@ object Utils extends Logging { */ def mapWithSeqValuesToJson(jsonDataMap: Map[String, Seq[Int]]): String = { - val builder = new StringBuilder - builder.append("{ ") - var numElements = 0 - for ((key, value) <- jsonDataMap.toList.sortBy(_._1)) { - if (numElements > 0) - builder.append(", ") - builder.append("\"" + key + "\": ") - builder.append(Utils.seqToJson(value.map(_.toString), valueInQuotes = false)) - numElements += 1 - } - builder.append(" }") - builder.toString - } - - - /** - * Merge arbitrary JSON objects. - */ - def mergeJsonObjects(objects: Seq[String]): String = { - val builder = new StringBuilder - builder.append("{ ") - var obs = List[String]() - objects.foreach(ob => obs = obs ::: getJsonContents(ob).split(',').toList) - obs = obs.sorted.map(_.trim) - builder.append(obs.mkString(", ")) - builder.append(" }") - builder.toString + mergeJsonFields(mapToJsonFields(jsonDataMap.map(e => (e._1 -> seqToJson(e._2.map(_.toString), valueInQuotes = false))), + valueInQuotes = false)) } /** - * Get the contents of a JSON object or array. - */ - def getJsonContents(str: String): String = { - str.trim().substring(1, str.length - 1) - } - - - - /** * Create a circular (looping) iterator over a collection. * @param coll An iterable over the underlying collection. * @return A circular iterator over the collection. diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index c6119d9..8c68821 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -24,6 +24,7 @@ import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException, import org.I0Itec.zkclient.serialize.ZkSerializer import collection._ import kafka.api.LeaderAndIsr +import mutable.ListBuffer import org.apache.zookeeper.data.Stat import java.util.concurrent.locks.{ReentrantLock, Condition} import kafka.admin._ @@ -190,8 +191,9 @@ object ZkUtils extends Logging { def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, port: Int, jmxPort: Int) { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id val brokerInfo = - Utils.mergeJsonObjects(Seq(Utils.mapToJson(Map("host" -> host), valueInQuotes = true), - Utils.mapToJson(Map("version" -> 1.toString, "jmx_port" -> jmxPort.toString, "port" -> port.toString), valueInQuotes = false))) + Utils.mergeJsonFields(Utils.mapToJsonFields(Map("host" -> host), valueInQuotes = true) ++ + Utils.mapToJsonFields(Map("version" -> 1.toString, "jmx_port" -> jmxPort.toString, "port" -> port.toString), + valueInQuotes = false)) try { createEphemeralPathExpectConflict(zkClient, brokerIdPath, brokerInfo) } catch { @@ -216,7 +218,7 @@ object ZkUtils extends Logging { * Get JSON partition to replica map from zookeeper. */ def replicaAssignmentZkdata(map: Map[String, Seq[Int]]): String = { - val jsonReplicaAssignmentMap = Utils.mapWithSeqValuesToJson(map.map(e => (e._1.toString -> e._2))) + val jsonReplicaAssignmentMap = Utils.mapWithSeqValuesToJson(map) Utils.mapToJson(Map("version" -> 1.toString, "partitions" -> jsonReplicaAssignmentMap), valueInQuotes = false) } @@ -564,26 +566,41 @@ object ZkUtils extends Logging { jsonPartitionMapOpt match { case Some(jsonPartitionMap) => val reassignedPartitions = parsePartitionReassignmentData(jsonPartitionMap) - reassignedPartitions.map { p => - val newReplicas = p._2 - (p._1 -> new ReassignedPartitionsContext(newReplicas)) - } + reassignedPartitions.map(p => (p._1 -> new ReassignedPartitionsContext(p._2))) case None => Map.empty[TopicAndPartition, ReassignedPartitionsContext] } } - def parsePartitionReassignmentData(jsonData: String):Map[TopicAndPartition, Seq[Int]] = { + def parsePartitionReassignmentData(jsonData: String): Map[TopicAndPartition, Seq[Int]] = { + val reassignedPartitions: mutable.Map[TopicAndPartition, Seq[Int]] = mutable.Map() Json.parseFull(jsonData) match { case Some(m) => - val replicaMap = m.asInstanceOf[Map[String, Seq[Int]]] - replicaMap.map { reassignedPartitions => - val topic = reassignedPartitions._1.split(",").head.trim - val partition = reassignedPartitions._1.split(",").last.trim.toInt - val newReplicas = reassignedPartitions._2 - TopicAndPartition(topic, partition) -> newReplicas + m.asInstanceOf[Map[String, Any]].get("partitions") match { + case Some(partitionsSeq) => + partitionsSeq.asInstanceOf[Seq[Map[String, Any]]].foreach(p => { + val topic = p.get("topic").get.asInstanceOf[String] + val partition = p.get("partition").get.asInstanceOf[Int] + val newReplicas = p.get("replicas").get.asInstanceOf[Seq[Int]] + reassignedPartitions += TopicAndPartition(topic, partition) -> newReplicas + }) + case None => } - case None => Map.empty[TopicAndPartition, Seq[Int]] + case None => + } + reassignedPartitions + } + + def getPartitionReassignmentZkData(partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]]): String = { + var jsonPartitionsData: mutable.ListBuffer[String] = ListBuffer[String]() + for (p <- partitionsToBeReassigned) { + val jsonReplicasData = Utils.seqToJson(p._2.map(_.toString), valueInQuotes = false) + val jsonTopicData = Utils.mapToJsonFields(Map("topic" -> p._1.topic), valueInQuotes = true) + val jsonPartitionData = Utils.mapToJsonFields(Map("partition" -> p._1.partition.toString, "replicas" -> jsonReplicasData), + valueInQuotes = false) + jsonPartitionsData += Utils.mergeJsonFields(jsonTopicData ++ jsonPartitionData) } + Utils.mapToJson(Map("version" -> 1.toString, "partitions" -> Utils.seqToJson(jsonPartitionsData.toSeq, valueInQuotes = false)), + valueInQuotes = false) } def updatePartitionReassignmentData(zkClient: ZkClient, partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]]) { @@ -593,11 +610,11 @@ object ZkUtils extends Logging { deletePath(zkClient, zkPath) info("No more partitions need to be reassigned. Deleting zk path %s".format(zkPath)) case _ => - val jsonData = Utils.mapWithSeqValuesToJson(partitionsToBeReassigned.map(p => ("%s,%s".format(p._1.topic, p._1.partition)) -> p._2)) + val jsonData = getPartitionReassignmentZkData(partitionsToBeReassigned) try { updatePersistentPath(zkClient, zkPath, jsonData) info("Updated partition reassignment path with %s".format(jsonData)) - }catch { + } catch { case nne: ZkNoNodeException => ZkUtils.createPersistentPath(zkClient, zkPath, jsonData) debug("Created path %s with %s for partition reassignment".format(zkPath, jsonData)) diff --git a/core/src/test/scala/other/kafka/StressTestLog.scala b/core/src/test/scala/other/kafka/StressTestLog.scala index c6e7a57..2ffcd65 100644 --- a/core/src/test/scala/other/kafka/StressTestLog.scala +++ b/core/src/test/scala/other/kafka/StressTestLog.scala @@ -79,8 +79,8 @@ object StressTestLog { class WriterThread(val log: Log) extends WorkerThread { @volatile var offset = 0 override def work() { - val logAppendInfo = log.append(TestUtils.singleMessageSet(offset.toString.getBytes)) - require(logAppendInfo.firstOffset == offset && logAppendInfo.lastOffset == offset) + val (firstOffset, lastOffset) = log.append(TestUtils.singleMessageSet(offset.toString.getBytes)) + require(firstOffset == offset && lastOffset == offset) offset += 1 if(offset % 1000 == 0) Thread.sleep(500) diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index 00acb43..517e9ac 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -89,7 +89,7 @@ object SerializationTestUtils{ val leaderAndIsr2 = new LeaderIsrAndControllerEpoch(new LeaderAndIsr(leader2, 1, isr2, 2), 1) val map = Map(((topic1, 0), PartitionStateInfo(leaderAndIsr1, 3)), ((topic2, 0), PartitionStateInfo(leaderAndIsr2, 3))) - new LeaderAndIsrRequest(map.toMap, collection.immutable.Set[Broker](), 1, 0) + new LeaderAndIsrRequest(map.toMap, collection.immutable.Set[Broker](), 0, 1, 0) } def createTestLeaderAndIsrResponse() : LeaderAndIsrResponse = { @@ -99,7 +99,8 @@ object SerializationTestUtils{ } def createTestStopReplicaRequest() : StopReplicaRequest = { - new StopReplicaRequest(controllerEpoch = 1, correlationId = 0, deletePartitions = true, partitions = collection.immutable.Set((topic1, 0), (topic2, 0))) + new StopReplicaRequest(controllerId = 0, controllerEpoch = 1, correlationId = 0, deletePartitions = true, + partitions = collection.immutable.Set((topic1, 0), (topic2, 0))) } def createTestStopReplicaResponse() : StopReplicaResponse = { diff --git a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala index fec17aa..b43b063 100644 --- a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala @@ -79,7 +79,6 @@ class ConsumerIteratorTest extends JUnit3Suite with KafkaServerTestHarness { consumerConfig.consumerTimeoutMs, new StringDecoder(), new StringDecoder(), - enableShallowIterator = false, clientId = "") val receivedMessages = (0 until 5).map(i => iter.next.message).toList diff --git a/core/src/test/scala/unit/kafka/log/CleanerTest.scala b/core/src/test/scala/unit/kafka/log/CleanerTest.scala index 4619d86..d861388 100644 --- a/core/src/test/scala/unit/kafka/log/CleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/CleanerTest.scala @@ -208,7 +208,7 @@ class CleanerTest extends JUnitSuite { def writeToLog(log: Log, seq: Iterable[(Int, Int)]): Iterable[Long] = { for((key, value) <- seq) - yield log.append(message(key, value)).firstOffset + yield log.append(message(key, value))._1 } def key(id: Int) = ByteBuffer.wrap(id.toString.getBytes) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 5a489f9..15e9b60 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -77,7 +77,7 @@ class LogCleanerIntegrationTest extends JUnitSuite { def writeDups(numKeys: Int, numDups: Int, log: Log): Seq[(Int, Int)] = { for(dup <- 0 until numDups; key <- 0 until numKeys) yield { val count = counter - val appendInfo = log.append(TestUtils.singleMessageSet(payload = counter.toString.getBytes, key = key.toString.getBytes), assignOffsets = true) + log.append(TestUtils.singleMessageSet(payload = counter.toString.getBytes, key = key.toString.getBytes), assignOffsets = true) counter += 1 (key, count) } diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 6916df4..6e9c1ec 100644 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -84,8 +84,8 @@ class LogManagerTest extends JUnit3Suite { var offset = 0L for(i <- 0 until 200) { var set = TestUtils.singleMessageSet("test".getBytes()) - val info = log.append(set) - offset = info.lastOffset + val (firstOffset, lastOffset) = log.append(set) + offset = lastOffset } assertTrue("There should be more than one segment now.", log.numberOfSegments > 1) @@ -127,8 +127,8 @@ class LogManagerTest extends JUnit3Suite { val numMessages = 200 for(i <- 0 until numMessages) { val set = TestUtils.singleMessageSet("test".getBytes()) - val info = log.append(set) - offset = info.firstOffset + val (firstOffset, lastOffset) = log.append(set) + offset = lastOffset } assertEquals("Check we have the expected number of segments.", numMessages * setSize / config.segmentSize, log.numberOfSegments) diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 5658ed4..cf5c608 100644 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -215,7 +215,7 @@ class LogTest extends JUnitSuite { val log = new Log(logDir, logConfig.copy(segmentSize = 100), needsRecovery = false, time.scheduler, time = time) val numMessages = 100 val messageSets = (0 until numMessages).map(i => TestUtils.singleMessageSet(i.toString.getBytes)) - val offsets = messageSets.map(log.append(_).firstOffset) + messageSets.foreach(log.append(_)) log.flush /* do successive reads to ensure all our messages are there */ @@ -275,7 +275,7 @@ class LogTest extends JUnitSuite { assertEquals("Still no change in the logEndOffset", currOffset, log.logEndOffset) assertEquals("Should still be able to append and should get the logEndOffset assigned to the new append", currOffset, - log.append(TestUtils.singleMessageSet("hello".toString.getBytes)).firstOffset) + log.append(TestUtils.singleMessageSet("hello".toString.getBytes))._1) // cleanup the log log.delete() diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index 176718e..8a42074 100644 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -23,7 +23,7 @@ import kafka.admin.AdminUtils import kafka.utils.TestUtils._ import junit.framework.Assert._ import kafka.utils.{ZkUtils, Utils, TestUtils} -import kafka.controller.{LeaderIsrAndControllerEpoch, ControllerChannelManager} +import kafka.controller.{ControllerContext, LeaderIsrAndControllerEpoch, ControllerChannelManager} import kafka.cluster.Broker import kafka.common.ErrorMapping import kafka.api._ @@ -120,18 +120,20 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { assertTrue("Leader could be broker 0 or broker 1", (leader1.getOrElse(-1) == 0) || (leader1.getOrElse(-1) == 1)) assertEquals("First epoch value should be 0", 0, leaderEpoch1) - // start another controller - val controllerConfig = new KafkaConfig(TestUtils.createBrokerConfig(2, TestUtils.choosePort())) + val controllerId = 2 + val controllerConfig = new KafkaConfig(TestUtils.createBrokerConfig(controllerId, TestUtils.choosePort())) val brokers = servers.map(s => new Broker(s.config.brokerId, "localhost", s.config.port)) - val controllerChannelManager = new ControllerChannelManager(brokers.toSet, controllerConfig) + val controllerContext = new ControllerContext(zkClient) + controllerContext.liveBrokers = brokers.toSet + val controllerChannelManager = new ControllerChannelManager(controllerContext, controllerConfig) controllerChannelManager.startup() val staleControllerEpoch = 0 val leaderAndIsr = new collection.mutable.HashMap[(String, Int), LeaderIsrAndControllerEpoch] leaderAndIsr.put((topic, partitionId), new LeaderIsrAndControllerEpoch(new LeaderAndIsr(brokerId2, List(brokerId1, brokerId2)), 2)) val partitionStateInfo = leaderAndIsr.mapValues(l => new PartitionStateInfo(l, 1)).toMap - val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfo, brokers.toSet, staleControllerEpoch, 0) + val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfo, brokers.toSet, controllerId, staleControllerEpoch, 0) controllerChannelManager.sendRequest(brokerId2, leaderAndIsrRequest, staleControllerEpochCallback) TestUtils.waitUntilTrue(() => staleControllerEpochDetected == true, 1000) diff --git a/core/src/test/scala/unit/kafka/utils/UtilsTest.scala b/core/src/test/scala/unit/kafka/utils/UtilsTest.scala index f7f734f..ce1679a 100644 --- a/core/src/test/scala/unit/kafka/utils/UtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/UtilsTest.scala @@ -24,8 +24,7 @@ import org.apache.log4j.Logger import org.scalatest.junit.JUnitSuite import org.junit.Assert._ import kafka.common.KafkaException -import org.junit.{Test} -import kafka.tools.KafkaMigrationTool +import org.junit.Test class UtilsTest extends JUnitSuite { diff --git a/perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala b/perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala index 9c9eead..c52ada0 100644 --- a/perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala +++ b/perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala @@ -134,7 +134,7 @@ object SimpleConsumerPerformance { .ofType(classOf[java.lang.Integer]) .defaultsTo(1024*1024) val clientIdOpt = parser.accepts("clientId", "The ID of this client.") - .withOptionalArg + .withRequiredArg .describedAs("clientId") .ofType(classOf[String]) .defaultsTo("SimpleConsumerPerformanceClient") diff --git a/system_test/migration_tool_testsuite/testcase_9001/testcase_9001_properties.json b/system_test/migration_tool_testsuite/testcase_9001/testcase_9001_properties.json index 47272a3..cf84caa 100644 --- a/system_test/migration_tool_testsuite/testcase_9001/testcase_9001_properties.json +++ b/system_test/migration_tool_testsuite/testcase_9001/testcase_9001_properties.json @@ -59,7 +59,7 @@ "entity_id": "4", "port": "9094", "broker.id": "4", - "log.segment.size": "51200", + "log.segment.bytes": "51200", "log.dir": "/tmp/kafka_server_4_logs", "log_filename": "kafka_server_4.log", "config_filename": "kafka_server_4.properties" @@ -68,7 +68,7 @@ "entity_id": "5", "port": "9095", "broker.id": "5", - "log.segment.size": "51200", + "log.segment.bytes": "51200", "log.dir": "/tmp/kafka_server_5_logs", "log_filename": "kafka_server_5.log", "config_filename": "kafka_server_5.properties" @@ -77,7 +77,7 @@ "entity_id": "6", "port": "9096", "broker.id": "6", - "log.segment.size": "51200", + "log.segment.bytes": "51200", "log.dir": "/tmp/kafka_server_6_logs", "log_filename": "kafka_server_6.log", "config_filename": "kafka_server_6.properties" diff --git a/system_test/migration_tool_testsuite/testcase_9003/testcase_9003_properties.json b/system_test/migration_tool_testsuite/testcase_9003/testcase_9003_properties.json index 0288eea..0b413c4 100644 --- a/system_test/migration_tool_testsuite/testcase_9003/testcase_9003_properties.json +++ b/system_test/migration_tool_testsuite/testcase_9003/testcase_9003_properties.json @@ -60,7 +60,7 @@ "entity_id": "4", "port": "9094", "broker.id": "4", - "log.segment.size": "51200", + "log.segment.bytes": "51200", "log.dir": "/tmp/kafka_server_4_logs", "log_filename": "kafka_server_4.log", "config_filename": "kafka_server_4.properties" @@ -69,7 +69,7 @@ "entity_id": "5", "port": "9095", "broker.id": "5", - "log.segment.size": "51200", + "log.segment.bytes": "51200", "log.dir": "/tmp/kafka_server_5_logs", "log_filename": "kafka_server_5.log", "config_filename": "kafka_server_5.properties" @@ -78,7 +78,7 @@ "entity_id": "6", "port": "9096", "broker.id": "6", - "log.segment.size": "51200", + "log.segment.bytes": "51200", "log.dir": "/tmp/kafka_server_6_logs", "log_filename": "kafka_server_6.log", "config_filename": "kafka_server_6.properties" diff --git a/system_test/migration_tool_testsuite/testcase_9004/testcase_9004_properties.json b/system_test/migration_tool_testsuite/testcase_9004/testcase_9004_properties.json index 1137696..5c6baaf 100644 --- a/system_test/migration_tool_testsuite/testcase_9004/testcase_9004_properties.json +++ b/system_test/migration_tool_testsuite/testcase_9004/testcase_9004_properties.json @@ -60,7 +60,7 @@ "entity_id": "4", "port": "9094", "broker.id": "4", - "log.segment.size": "51200", + "log.segment.bytes": "51200", "log.dir": "/tmp/kafka_server_4_logs", "log_filename": "kafka_server_4.log", "config_filename": "kafka_server_4.properties" @@ -69,7 +69,7 @@ "entity_id": "5", "port": "9095", "broker.id": "5", - "log.segment.size": "51200", + "log.segment.bytes": "51200", "log.dir": "/tmp/kafka_server_5_logs", "log_filename": "kafka_server_5.log", "config_filename": "kafka_server_5.properties" @@ -78,7 +78,7 @@ "entity_id": "6", "port": "9096", "broker.id": "6", - "log.segment.size": "51200", + "log.segment.bytes": "51200", "log.dir": "/tmp/kafka_server_6_logs", "log_filename": "kafka_server_6.log", "config_filename": "kafka_server_6.properties" diff --git a/system_test/migration_tool_testsuite/testcase_9005/testcase_9005_properties.json b/system_test/migration_tool_testsuite/testcase_9005/testcase_9005_properties.json index 8867aa5..8597e1a 100644 --- a/system_test/migration_tool_testsuite/testcase_9005/testcase_9005_properties.json +++ b/system_test/migration_tool_testsuite/testcase_9005/testcase_9005_properties.json @@ -71,7 +71,7 @@ "entity_id": "5", "port": "9094", "broker.id": "4", - "log.segment.size": "51200", + "log.segment.bytes": "51200", "log.dir": "/tmp/kafka_server_5_logs", "log_filename": "kafka_server_5.log", "config_filename": "kafka_server_5.properties" @@ -80,7 +80,7 @@ "entity_id": "6", "port": "9095", "broker.id": "5", - "log.segment.size": "51200", + "log.segment.bytes": "51200", "log.dir": "/tmp/kafka_server_6_logs", "log_filename": "kafka_server_6.log", "config_filename": "kafka_server_6.properties" @@ -89,7 +89,7 @@ "entity_id": "7", "port": "9096", "broker.id": "6", - "log.segment.size": "51200", + "log.segment.bytes": "51200", "log.dir": "/tmp/kafka_server_7_logs", "log_filename": "kafka_server_7.log", "config_filename": "kafka_server_7.properties" diff --git a/system_test/migration_tool_testsuite/testcase_9006/testcase_9006_properties.json b/system_test/migration_tool_testsuite/testcase_9006/testcase_9006_properties.json index 0ad5e7f..ec3290f 100644 --- a/system_test/migration_tool_testsuite/testcase_9006/testcase_9006_properties.json +++ b/system_test/migration_tool_testsuite/testcase_9006/testcase_9006_properties.json @@ -71,7 +71,7 @@ "entity_id": "5", "port": "9094", "broker.id": "4", - "log.segment.size": "51200", + "log.segment.bytes": "51200", "log.dir": "/tmp/kafka_server_5_logs", "log_filename": "kafka_server_5.log", "config_filename": "kafka_server_5.properties" @@ -80,7 +80,7 @@ "entity_id": "6", "port": "9095", "broker.id": "5", - "log.segment.size": "51200", + "log.segment.bytes": "51200", "log.dir": "/tmp/kafka_server_6_logs", "log_filename": "kafka_server_6.log", "config_filename": "kafka_server_6.properties" @@ -89,7 +89,7 @@ "entity_id": "7", "port": "9096", "broker.id": "6", - "log.segment.size": "51200", + "log.segment.bytes": "51200", "log.dir": "/tmp/kafka_server_7_logs", "log_filename": "kafka_server_7.log", "config_filename": "kafka_server_7.properties" diff --git a/system_test/mirror_maker_testsuite/testcase_5001/testcase_5001_properties.json b/system_test/mirror_maker_testsuite/testcase_5001/testcase_5001_properties.json index f520d6a..abb500b 100644 --- a/system_test/mirror_maker_testsuite/testcase_5001/testcase_5001_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5001/testcase_5001_properties.json @@ -54,7 +54,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_4_logs", "log_filename": "kafka_server_4.log", "config_filename": "kafka_server_4.properties" @@ -63,7 +63,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_5_logs", "log_filename": "kafka_server_5.log", "config_filename": "kafka_server_5.properties" @@ -72,7 +72,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_6_logs", "log_filename": "kafka_server_6.log", "config_filename": "kafka_server_6.properties" @@ -81,7 +81,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_7_logs", "log_filename": "kafka_server_7.log", "config_filename": "kafka_server_7.properties" @@ -90,7 +90,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_8_logs", "log_filename": "kafka_server_8.log", "config_filename": "kafka_server_8.properties" @@ -99,7 +99,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_9_logs", "log_filename": "kafka_server_9.log", "config_filename": "kafka_server_9.properties" diff --git a/system_test/mirror_maker_testsuite/testcase_5002/testcase_5002_properties.json b/system_test/mirror_maker_testsuite/testcase_5002/testcase_5002_properties.json index 86fd73d..8d25bad 100644 --- a/system_test/mirror_maker_testsuite/testcase_5002/testcase_5002_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5002/testcase_5002_properties.json @@ -54,7 +54,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_4_logs", "log_filename": "kafka_server_4.log", "config_filename": "kafka_server_4.properties" @@ -63,7 +63,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_5_logs", "log_filename": "kafka_server_5.log", "config_filename": "kafka_server_5.properties" @@ -72,7 +72,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_6_logs", "log_filename": "kafka_server_6.log", "config_filename": "kafka_server_6.properties" @@ -81,7 +81,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_7_logs", "log_filename": "kafka_server_7.log", "config_filename": "kafka_server_7.properties" @@ -90,7 +90,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_8_logs", "log_filename": "kafka_server_8.log", "config_filename": "kafka_server_8.properties" @@ -99,7 +99,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_9_logs", "log_filename": "kafka_server_9.log", "config_filename": "kafka_server_9.properties" diff --git a/system_test/mirror_maker_testsuite/testcase_5003/testcase_5003_properties.json b/system_test/mirror_maker_testsuite/testcase_5003/testcase_5003_properties.json index f186298..b9d72a1 100644 --- a/system_test/mirror_maker_testsuite/testcase_5003/testcase_5003_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5003/testcase_5003_properties.json @@ -55,7 +55,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_4_logs", "log_filename": "kafka_server_4.log", "config_filename": "kafka_server_4.properties" @@ -64,7 +64,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_5_logs", "log_filename": "kafka_server_5.log", "config_filename": "kafka_server_5.properties" @@ -73,7 +73,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_6_logs", "log_filename": "kafka_server_6.log", "config_filename": "kafka_server_6.properties" @@ -82,7 +82,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_7_logs", "log_filename": "kafka_server_7.log", "config_filename": "kafka_server_7.properties" @@ -91,7 +91,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_8_logs", "log_filename": "kafka_server_8.log", "config_filename": "kafka_server_8.properties" @@ -100,7 +100,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_9_logs", "log_filename": "kafka_server_9.log", "config_filename": "kafka_server_9.properties" diff --git a/system_test/mirror_maker_testsuite/testcase_5004/testcase_5004_properties.json b/system_test/mirror_maker_testsuite/testcase_5004/testcase_5004_properties.json index a887af2..b72f265 100644 --- a/system_test/mirror_maker_testsuite/testcase_5004/testcase_5004_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5004/testcase_5004_properties.json @@ -55,7 +55,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_4_logs", "log_filename": "kafka_server_4.log", "config_filename": "kafka_server_4.properties" @@ -64,7 +64,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_5_logs", "log_filename": "kafka_server_5.log", "config_filename": "kafka_server_5.properties" @@ -73,7 +73,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_6_logs", "log_filename": "kafka_server_6.log", "config_filename": "kafka_server_6.properties" @@ -82,7 +82,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_7_logs", "log_filename": "kafka_server_7.log", "config_filename": "kafka_server_7.properties" @@ -91,7 +91,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_8_logs", "log_filename": "kafka_server_8.log", "config_filename": "kafka_server_8.properties" @@ -100,7 +100,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_9_logs", "log_filename": "kafka_server_9.log", "config_filename": "kafka_server_9.properties" diff --git a/system_test/mirror_maker_testsuite/testcase_5005/testcase_5005_properties.json b/system_test/mirror_maker_testsuite/testcase_5005/testcase_5005_properties.json index 6dab54e..83ca969 100644 --- a/system_test/mirror_maker_testsuite/testcase_5005/testcase_5005_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5005/testcase_5005_properties.json @@ -55,7 +55,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_4_logs", "log_filename": "kafka_server_4.log", "config_filename": "kafka_server_4.properties" @@ -64,7 +64,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_5_logs", "log_filename": "kafka_server_5.log", "config_filename": "kafka_server_5.properties" @@ -73,7 +73,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_6_logs", "log_filename": "kafka_server_6.log", "config_filename": "kafka_server_6.properties" @@ -82,7 +82,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_7_logs", "log_filename": "kafka_server_7.log", "config_filename": "kafka_server_7.properties" @@ -91,7 +91,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_8_logs", "log_filename": "kafka_server_8.log", "config_filename": "kafka_server_8.properties" @@ -100,7 +100,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_9_logs", "log_filename": "kafka_server_9.log", "config_filename": "kafka_server_9.properties" diff --git a/system_test/mirror_maker_testsuite/testcase_5006/testcase_5006_properties.json b/system_test/mirror_maker_testsuite/testcase_5006/testcase_5006_properties.json index 8181a21..fe32712 100644 --- a/system_test/mirror_maker_testsuite/testcase_5006/testcase_5006_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5006/testcase_5006_properties.json @@ -55,7 +55,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_4_logs", "log_filename": "kafka_server_4.log", "config_filename": "kafka_server_4.properties" @@ -64,7 +64,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_5_logs", "log_filename": "kafka_server_5.log", "config_filename": "kafka_server_5.properties" @@ -73,7 +73,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_6_logs", "log_filename": "kafka_server_6.log", "config_filename": "kafka_server_6.properties" @@ -82,7 +82,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_7_logs", "log_filename": "kafka_server_7.log", "config_filename": "kafka_server_7.properties" @@ -91,7 +91,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_8_logs", "log_filename": "kafka_server_8.log", "config_filename": "kafka_server_8.properties" @@ -100,7 +100,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_9_logs", "log_filename": "kafka_server_9.log", "config_filename": "kafka_server_9.properties" diff --git a/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json b/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json index 10dc33b..a3e94cc 100644 --- a/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json +++ b/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json @@ -29,8 +29,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -38,8 +40,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -47,8 +51,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0002/testcase_0002_properties.json b/system_test/replication_testsuite/testcase_0002/testcase_0002_properties.json index f83ed71..721f14f 100644 --- a/system_test/replication_testsuite/testcase_0002/testcase_0002_properties.json +++ b/system_test/replication_testsuite/testcase_0002/testcase_0002_properties.json @@ -29,8 +29,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -38,8 +40,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -47,8 +51,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0003/testcase_0003_properties.json b/system_test/replication_testsuite/testcase_0003/testcase_0003_properties.json index 8ab7d9d..be2bbb1 100644 --- a/system_test/replication_testsuite/testcase_0003/testcase_0003_properties.json +++ b/system_test/replication_testsuite/testcase_0003/testcase_0003_properties.json @@ -29,8 +29,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -38,8 +40,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -47,8 +51,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0004/testcase_0004_properties.json b/system_test/replication_testsuite/testcase_0004/testcase_0004_properties.json index f5b2b0c..d51b988 100644 --- a/system_test/replication_testsuite/testcase_0004/testcase_0004_properties.json +++ b/system_test/replication_testsuite/testcase_0004/testcase_0004_properties.json @@ -29,8 +29,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -38,8 +40,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -47,8 +51,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0005/testcase_0005_properties.json b/system_test/replication_testsuite/testcase_0005/testcase_0005_properties.json index 1c884a7..ebf88c7 100644 --- a/system_test/replication_testsuite/testcase_0005/testcase_0005_properties.json +++ b/system_test/replication_testsuite/testcase_0005/testcase_0005_properties.json @@ -29,8 +29,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -38,8 +40,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -47,8 +51,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0006/testcase_0006_properties.json b/system_test/replication_testsuite/testcase_0006/testcase_0006_properties.json index 49aece9..b4044ad 100644 --- a/system_test/replication_testsuite/testcase_0006/testcase_0006_properties.json +++ b/system_test/replication_testsuite/testcase_0006/testcase_0006_properties.json @@ -29,8 +29,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -38,8 +40,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -47,8 +51,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0007/testcase_0007_properties.json b/system_test/replication_testsuite/testcase_0007/testcase_0007_properties.json index e21b9bd..3702a47 100644 --- a/system_test/replication_testsuite/testcase_0007/testcase_0007_properties.json +++ b/system_test/replication_testsuite/testcase_0007/testcase_0007_properties.json @@ -29,8 +29,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -38,8 +40,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -47,8 +51,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0008/testcase_0008_properties.json b/system_test/replication_testsuite/testcase_0008/testcase_0008_properties.json index 8c3b8c9..3f0cfa9 100644 --- a/system_test/replication_testsuite/testcase_0008/testcase_0008_properties.json +++ b/system_test/replication_testsuite/testcase_0008/testcase_0008_properties.json @@ -29,8 +29,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -38,8 +40,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -47,8 +51,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0009/testcase_0009_properties.json b/system_test/replication_testsuite/testcase_0009/testcase_0009_properties.json index 656f935..1f78162 100644 --- a/system_test/replication_testsuite/testcase_0009/testcase_0009_properties.json +++ b/system_test/replication_testsuite/testcase_0009/testcase_0009_properties.json @@ -29,8 +29,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -38,8 +40,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -47,8 +51,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0010/testcase_0010_properties.json b/system_test/replication_testsuite/testcase_0010/testcase_0010_properties.json index 218a7c6..5b00112 100644 --- a/system_test/replication_testsuite/testcase_0010/testcase_0010_properties.json +++ b/system_test/replication_testsuite/testcase_0010/testcase_0010_properties.json @@ -29,8 +29,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "1048576", + "log.segment.bytes": "1048576", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -38,8 +40,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "1048576", + "log.segment.bytes": "1048576", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -47,8 +51,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "1048576", + "log.segment.bytes": "1048576", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0021/testcase_0021_properties.json b/system_test/replication_testsuite/testcase_0021/testcase_0021_properties.json index 5b414d6..82787a7 100644 --- a/system_test/replication_testsuite/testcase_0021/testcase_0021_properties.json +++ b/system_test/replication_testsuite/testcase_0021/testcase_0021_properties.json @@ -28,8 +28,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -37,8 +39,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -46,8 +50,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0022/testcase_0022_properties.json b/system_test/replication_testsuite/testcase_0022/testcase_0022_properties.json index 9a60436..f8c884a 100644 --- a/system_test/replication_testsuite/testcase_0022/testcase_0022_properties.json +++ b/system_test/replication_testsuite/testcase_0022/testcase_0022_properties.json @@ -28,8 +28,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -37,8 +39,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -46,8 +50,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0023/testcase_0023_properties.json b/system_test/replication_testsuite/testcase_0023/testcase_0023_properties.json index 33abd17..9d71e30 100644 --- a/system_test/replication_testsuite/testcase_0023/testcase_0023_properties.json +++ b/system_test/replication_testsuite/testcase_0023/testcase_0023_properties.json @@ -28,8 +28,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "512000", + "log.segment.bytes": "512000", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -37,8 +39,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "512000", + "log.segment.bytes": "512000", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -46,8 +50,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "512000", + "log.segment.bytes": "512000", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0101/testcase_0101_properties.json b/system_test/replication_testsuite/testcase_0101/testcase_0101_properties.json index 07776ae..85e4b61 100644 --- a/system_test/replication_testsuite/testcase_0101/testcase_0101_properties.json +++ b/system_test/replication_testsuite/testcase_0101/testcase_0101_properties.json @@ -32,8 +32,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -41,8 +43,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -50,8 +54,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0102/testcase_0102_properties.json b/system_test/replication_testsuite/testcase_0102/testcase_0102_properties.json index 79211f1..0d2f59f 100644 --- a/system_test/replication_testsuite/testcase_0102/testcase_0102_properties.json +++ b/system_test/replication_testsuite/testcase_0102/testcase_0102_properties.json @@ -32,8 +32,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -41,8 +43,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -50,8 +54,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0103/testcase_0103_properties.json b/system_test/replication_testsuite/testcase_0103/testcase_0103_properties.json index 45d5578..34acfa9 100644 --- a/system_test/replication_testsuite/testcase_0103/testcase_0103_properties.json +++ b/system_test/replication_testsuite/testcase_0103/testcase_0103_properties.json @@ -32,8 +32,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -41,8 +43,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -50,8 +54,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0104/testcase_0104_properties.json b/system_test/replication_testsuite/testcase_0104/testcase_0104_properties.json index b793608..4145345 100644 --- a/system_test/replication_testsuite/testcase_0104/testcase_0104_properties.json +++ b/system_test/replication_testsuite/testcase_0104/testcase_0104_properties.json @@ -32,8 +32,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -41,8 +43,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -50,8 +54,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0105/testcase_0105_properties.json b/system_test/replication_testsuite/testcase_0105/testcase_0105_properties.json index 7171b42..2eecc76 100644 --- a/system_test/replication_testsuite/testcase_0105/testcase_0105_properties.json +++ b/system_test/replication_testsuite/testcase_0105/testcase_0105_properties.json @@ -32,8 +32,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -41,8 +43,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -50,8 +54,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "1", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0106/testcase_0106_properties.json b/system_test/replication_testsuite/testcase_0106/testcase_0106_properties.json index 1b909a3..744174e 100644 --- a/system_test/replication_testsuite/testcase_0106/testcase_0106_properties.json +++ b/system_test/replication_testsuite/testcase_0106/testcase_0106_properties.json @@ -32,8 +32,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -41,8 +43,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -50,8 +54,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0107/testcase_0107_properties.json b/system_test/replication_testsuite/testcase_0107/testcase_0107_properties.json index 8d13531..e881b13 100644 --- a/system_test/replication_testsuite/testcase_0107/testcase_0107_properties.json +++ b/system_test/replication_testsuite/testcase_0107/testcase_0107_properties.json @@ -32,8 +32,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -41,8 +43,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -50,8 +54,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0108/testcase_0108_properties.json b/system_test/replication_testsuite/testcase_0108/testcase_0108_properties.json index 9017e6b..7b48fdb 100644 --- a/system_test/replication_testsuite/testcase_0108/testcase_0108_properties.json +++ b/system_test/replication_testsuite/testcase_0108/testcase_0108_properties.json @@ -32,8 +32,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -41,8 +43,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -50,8 +54,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0109/testcase_0109_properties.json b/system_test/replication_testsuite/testcase_0109/testcase_0109_properties.json index 0654620..a98ae03 100644 --- a/system_test/replication_testsuite/testcase_0109/testcase_0109_properties.json +++ b/system_test/replication_testsuite/testcase_0109/testcase_0109_properties.json @@ -32,8 +32,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -41,8 +43,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -50,8 +54,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0110/testcase_0110_properties.json b/system_test/replication_testsuite/testcase_0110/testcase_0110_properties.json index 24b33b6..f51abc1 100644 --- a/system_test/replication_testsuite/testcase_0110/testcase_0110_properties.json +++ b/system_test/replication_testsuite/testcase_0110/testcase_0110_properties.json @@ -32,8 +32,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "1048576", + "log.segment.bytes": "1048576", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -41,8 +43,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "1048576", + "log.segment.bytes": "1048576", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -50,8 +54,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "1048576", + "log.segment.bytes": "1048576", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0111/testcase_0111_properties.json b/system_test/replication_testsuite/testcase_0111/testcase_0111_properties.json index 0f7700e..8d66c5d 100644 --- a/system_test/replication_testsuite/testcase_0111/testcase_0111_properties.json +++ b/system_test/replication_testsuite/testcase_0111/testcase_0111_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "replica_factor": "3", "num_partition": "3", @@ -32,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -41,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -50,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0112/testcase_0112_properties.json b/system_test/replication_testsuite/testcase_0112/testcase_0112_properties.json index 4e9c03c..3522f87 100644 --- a/system_test/replication_testsuite/testcase_0112/testcase_0112_properties.json +++ b/system_test/replication_testsuite/testcase_0112/testcase_0112_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "replica_factor": "3", "num_partition": "3", @@ -32,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -41,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -50,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0113/testcase_0113_properties.json b/system_test/replication_testsuite/testcase_0113/testcase_0113_properties.json index 358abbe..58a98d0 100644 --- a/system_test/replication_testsuite/testcase_0113/testcase_0113_properties.json +++ b/system_test/replication_testsuite/testcase_0113/testcase_0113_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "replica_factor": "3", "num_partition": "3", @@ -32,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -41,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -50,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0114/testcase_0114_properties.json b/system_test/replication_testsuite/testcase_0114/testcase_0114_properties.json index 053f17c..f3f9621 100644 --- a/system_test/replication_testsuite/testcase_0114/testcase_0114_properties.json +++ b/system_test/replication_testsuite/testcase_0114/testcase_0114_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "replica_factor": "3", "num_partition": "3", @@ -32,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -41,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -50,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0115/testcase_0115_properties.json b/system_test/replication_testsuite/testcase_0115/testcase_0115_properties.json index 0a30580..ba78605 100644 --- a/system_test/replication_testsuite/testcase_0115/testcase_0115_properties.json +++ b/system_test/replication_testsuite/testcase_0115/testcase_0115_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "replica_factor": "3", "num_partition": "3", @@ -32,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -41,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -50,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0116/testcase_0116_properties.json b/system_test/replication_testsuite/testcase_0116/testcase_0116_properties.json index b1e4eb2..db4aea2 100644 --- a/system_test/replication_testsuite/testcase_0116/testcase_0116_properties.json +++ b/system_test/replication_testsuite/testcase_0116/testcase_0116_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "replica_factor": "3", "num_partition": "3", @@ -32,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -41,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -50,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0117/testcase_0117_properties.json b/system_test/replication_testsuite/testcase_0117/testcase_0117_properties.json index f395fe7..b0353e6 100644 --- a/system_test/replication_testsuite/testcase_0117/testcase_0117_properties.json +++ b/system_test/replication_testsuite/testcase_0117/testcase_0117_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "replica_factor": "3", "num_partition": "3", @@ -32,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -41,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -50,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0118/testcase_0118_properties.json b/system_test/replication_testsuite/testcase_0118/testcase_0118_properties.json index 7d2ed9a..6c9b214 100644 --- a/system_test/replication_testsuite/testcase_0118/testcase_0118_properties.json +++ b/system_test/replication_testsuite/testcase_0118/testcase_0118_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "replica_factor": "3", "num_partition": "3", @@ -32,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -41,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -50,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0121/testcase_0121_properties.json b/system_test/replication_testsuite/testcase_0121/testcase_0121_properties.json index bdc17ce..d1dcbdb 100644 --- a/system_test/replication_testsuite/testcase_0121/testcase_0121_properties.json +++ b/system_test/replication_testsuite/testcase_0121/testcase_0121_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "replica_factor": "3", "num_partition": "3", @@ -31,8 +32,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -40,8 +43,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -49,8 +54,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0122/testcase_0122_properties.json b/system_test/replication_testsuite/testcase_0122/testcase_0122_properties.json index 8f00df6..52a6cd3 100644 --- a/system_test/replication_testsuite/testcase_0122/testcase_0122_properties.json +++ b/system_test/replication_testsuite/testcase_0122/testcase_0122_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "replica_factor": "3", "num_partition": "3", @@ -31,8 +32,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "512000", + "log.segment.bytes": "512000", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -40,8 +43,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "512000", + "log.segment.bytes": "512000", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -49,8 +54,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "512000", + "log.segment.bytes": "512000", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0123/testcase_0123_properties.json b/system_test/replication_testsuite/testcase_0123/testcase_0123_properties.json index 6de5d62..49d4b75 100644 --- a/system_test/replication_testsuite/testcase_0123/testcase_0123_properties.json +++ b/system_test/replication_testsuite/testcase_0123/testcase_0123_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "replica_factor": "3", "num_partition": "3", @@ -31,8 +32,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "512000", + "log.segment.bytes": "512000", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -40,8 +43,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "512000", + "log.segment.bytes": "512000", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -49,8 +54,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "512000", + "log.segment.bytes": "512000", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0124/testcase_0124_properties.json b/system_test/replication_testsuite/testcase_0124/testcase_0124_properties.json index fe9e545..e3e0b15 100644 --- a/system_test/replication_testsuite/testcase_0124/testcase_0124_properties.json +++ b/system_test/replication_testsuite/testcase_0124/testcase_0124_properties.json @@ -13,6 +13,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "replica_factor": "3", "num_partition": "3", @@ -32,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log.index.interval.bytes": "490", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" @@ -42,8 +45,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log.index.interval.bytes": "490", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" @@ -52,8 +57,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log.index.interval.bytes": "490", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" diff --git a/system_test/replication_testsuite/testcase_0125/testcase_0125_properties.json b/system_test/replication_testsuite/testcase_0125/testcase_0125_properties.json index 8085d7d..0489d8e 100644 --- a/system_test/replication_testsuite/testcase_0125/testcase_0125_properties.json +++ b/system_test/replication_testsuite/testcase_0125/testcase_0125_properties.json @@ -13,6 +13,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "replica_factor": "3", "num_partition": "3", @@ -32,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log.index.interval.bytes": "490", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" @@ -42,8 +45,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log.index.interval.bytes": "490", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" @@ -52,8 +57,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log.index.interval.bytes": "490", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" diff --git a/system_test/replication_testsuite/testcase_0126/testcase_0126_properties.json b/system_test/replication_testsuite/testcase_0126/testcase_0126_properties.json index 31afba3..676dd63 100644 --- a/system_test/replication_testsuite/testcase_0126/testcase_0126_properties.json +++ b/system_test/replication_testsuite/testcase_0126/testcase_0126_properties.json @@ -13,6 +13,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "replica_factor": "3", "num_partition": "3", @@ -32,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log.index.interval.bytes": "490", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" @@ -42,8 +45,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log.index.interval.bytes": "490", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" @@ -52,8 +57,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log.index.interval.bytes": "490", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" diff --git a/system_test/replication_testsuite/testcase_0127/testcase_0127_properties.json b/system_test/replication_testsuite/testcase_0127/testcase_0127_properties.json index 12daea9..8d8bb61 100644 --- a/system_test/replication_testsuite/testcase_0127/testcase_0127_properties.json +++ b/system_test/replication_testsuite/testcase_0127/testcase_0127_properties.json @@ -13,6 +13,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "replica_factor": "3", "num_partition": "3", @@ -32,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log.index.interval.bytes": "490", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" @@ -42,8 +45,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log.index.interval.bytes": "490", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" @@ -52,8 +57,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log.index.interval.bytes": "490", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" diff --git a/system_test/replication_testsuite/testcase_0131/testcase_0131_properties.json b/system_test/replication_testsuite/testcase_0131/testcase_0131_properties.json index 99b6bf8..7193294 100644 --- a/system_test/replication_testsuite/testcase_0131/testcase_0131_properties.json +++ b/system_test/replication_testsuite/testcase_0131/testcase_0131_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "replica_factor": "2", "num_partition": "3", @@ -31,8 +32,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "2", + "num.partitions": "3", "log.index.interval.bytes": "10", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" @@ -41,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "2", + "num.partitions": "3", "log.index.interval.bytes": "10", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" @@ -51,8 +56,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "2", + "num.partitions": "3", "log.index.interval.bytes": "10", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" diff --git a/system_test/replication_testsuite/testcase_0132/testcase_0132_properties.json b/system_test/replication_testsuite/testcase_0132/testcase_0132_properties.json index 93c1955..77d6c2f 100644 --- a/system_test/replication_testsuite/testcase_0132/testcase_0132_properties.json +++ b/system_test/replication_testsuite/testcase_0132/testcase_0132_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "replica_factor": "2", "num_partition": "3", @@ -31,8 +32,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "512000", + "log.segment.bytes": "512000", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "2", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -40,8 +43,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "512000", + "log.segment.bytes": "512000", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "2", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -49,8 +54,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "512000", + "log.segment.bytes": "512000", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "2", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0133/testcase_0133_properties.json b/system_test/replication_testsuite/testcase_0133/testcase_0133_properties.json index c930d86..482ca59 100644 --- a/system_test/replication_testsuite/testcase_0133/testcase_0133_properties.json +++ b/system_test/replication_testsuite/testcase_0133/testcase_0133_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "replica_factor": "2", "num_partition": "3", @@ -31,8 +32,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "512000", + "log.segment.bytes": "512000", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "2", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -40,8 +43,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "512000", + "log.segment.bytes": "512000", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "2", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -49,8 +54,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "512000", + "log.segment.bytes": "512000", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "2", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0151/testcase_0151_properties.json b/system_test/replication_testsuite/testcase_0151/testcase_0151_properties.json index a30443d..eebba4d 100644 --- a/system_test/replication_testsuite/testcase_0151/testcase_0151_properties.json +++ b/system_test/replication_testsuite/testcase_0151/testcase_0151_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "signal_type": "SIGKILL", "replica_factor": "3", @@ -33,8 +34,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +45,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +56,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0152/testcase_0152_properties.json b/system_test/replication_testsuite/testcase_0152/testcase_0152_properties.json index 1b9cca2..debf544 100644 --- a/system_test/replication_testsuite/testcase_0152/testcase_0152_properties.json +++ b/system_test/replication_testsuite/testcase_0152/testcase_0152_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "signal_type": "SIGKILL", "replica_factor": "3", @@ -33,8 +34,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +45,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +56,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0153/testcase_0153_properties.json b/system_test/replication_testsuite/testcase_0153/testcase_0153_properties.json index d45327f..57b7d98 100644 --- a/system_test/replication_testsuite/testcase_0153/testcase_0153_properties.json +++ b/system_test/replication_testsuite/testcase_0153/testcase_0153_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "signal_type": "SIGKILL", "replica_factor": "3", @@ -33,8 +34,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +45,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +56,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0154/testcase_0154_properties.json b/system_test/replication_testsuite/testcase_0154/testcase_0154_properties.json index 09678a7..c09fab7 100644 --- a/system_test/replication_testsuite/testcase_0154/testcase_0154_properties.json +++ b/system_test/replication_testsuite/testcase_0154/testcase_0154_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "signal_type": "SIGKILL", "replica_factor": "3", @@ -33,8 +34,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +45,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +56,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0155/testcase_0155_properties.json b/system_test/replication_testsuite/testcase_0155/testcase_0155_properties.json index 27f6083..dd5ac52 100644 --- a/system_test/replication_testsuite/testcase_0155/testcase_0155_properties.json +++ b/system_test/replication_testsuite/testcase_0155/testcase_0155_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "signal_type": "SIGKILL", "replica_factor": "3", @@ -33,8 +34,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +45,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +56,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0156/testcase_0156_properties.json b/system_test/replication_testsuite/testcase_0156/testcase_0156_properties.json index 645573a..8236ca5 100644 --- a/system_test/replication_testsuite/testcase_0156/testcase_0156_properties.json +++ b/system_test/replication_testsuite/testcase_0156/testcase_0156_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "signal_type": "SIGKILL", "replica_factor": "3", @@ -33,8 +34,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +45,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +56,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0157/testcase_0157_properties.json b/system_test/replication_testsuite/testcase_0157/testcase_0157_properties.json index 565372d..a28bf81 100644 --- a/system_test/replication_testsuite/testcase_0157/testcase_0157_properties.json +++ b/system_test/replication_testsuite/testcase_0157/testcase_0157_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "signal_type": "SIGKILL", "replica_factor": "3", @@ -33,8 +34,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +45,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +56,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0158/testcase_0158_properties.json b/system_test/replication_testsuite/testcase_0158/testcase_0158_properties.json index e172728..3d6edbd 100644 --- a/system_test/replication_testsuite/testcase_0158/testcase_0158_properties.json +++ b/system_test/replication_testsuite/testcase_0158/testcase_0158_properties.json @@ -12,6 +12,7 @@ }, "testcase_args": { "broker_type": "leader", + "auto_create_topic": "true", "bounce_broker": "true", "signal_type": "SIGKILL", "replica_factor": "3", @@ -33,8 +34,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +45,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +56,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0201/testcase_0201_properties.json b/system_test/replication_testsuite/testcase_0201/testcase_0201_properties.json index 637e860..095a7b5 100644 --- a/system_test/replication_testsuite/testcase_0201/testcase_0201_properties.json +++ b/system_test/replication_testsuite/testcase_0201/testcase_0201_properties.json @@ -33,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0202/testcase_0202_properties.json b/system_test/replication_testsuite/testcase_0202/testcase_0202_properties.json index 8057d23..557c081 100644 --- a/system_test/replication_testsuite/testcase_0202/testcase_0202_properties.json +++ b/system_test/replication_testsuite/testcase_0202/testcase_0202_properties.json @@ -33,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0203/testcase_0203_properties.json b/system_test/replication_testsuite/testcase_0203/testcase_0203_properties.json index 4a071e0..751efc5 100644 --- a/system_test/replication_testsuite/testcase_0203/testcase_0203_properties.json +++ b/system_test/replication_testsuite/testcase_0203/testcase_0203_properties.json @@ -33,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0204/testcase_0204_properties.json b/system_test/replication_testsuite/testcase_0204/testcase_0204_properties.json index 05b758b..761385f 100644 --- a/system_test/replication_testsuite/testcase_0204/testcase_0204_properties.json +++ b/system_test/replication_testsuite/testcase_0204/testcase_0204_properties.json @@ -33,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0205/testcase_0205_properties.json b/system_test/replication_testsuite/testcase_0205/testcase_0205_properties.json index 1b4b204..772b310 100644 --- a/system_test/replication_testsuite/testcase_0205/testcase_0205_properties.json +++ b/system_test/replication_testsuite/testcase_0205/testcase_0205_properties.json @@ -33,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0206/testcase_0206_properties.json b/system_test/replication_testsuite/testcase_0206/testcase_0206_properties.json index d5ba204..b7759fe 100644 --- a/system_test/replication_testsuite/testcase_0206/testcase_0206_properties.json +++ b/system_test/replication_testsuite/testcase_0206/testcase_0206_properties.json @@ -33,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0207/testcase_0207_properties.json b/system_test/replication_testsuite/testcase_0207/testcase_0207_properties.json index ac72c88..a4f414b 100644 --- a/system_test/replication_testsuite/testcase_0207/testcase_0207_properties.json +++ b/system_test/replication_testsuite/testcase_0207/testcase_0207_properties.json @@ -33,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0208/testcase_0208_properties.json b/system_test/replication_testsuite/testcase_0208/testcase_0208_properties.json index e840880..62f97d7 100644 --- a/system_test/replication_testsuite/testcase_0208/testcase_0208_properties.json +++ b/system_test/replication_testsuite/testcase_0208/testcase_0208_properties.json @@ -33,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0251/testcase_0251_properties.json b/system_test/replication_testsuite/testcase_0251/testcase_0251_properties.json index 0968cd8..47b857f 100644 --- a/system_test/replication_testsuite/testcase_0251/testcase_0251_properties.json +++ b/system_test/replication_testsuite/testcase_0251/testcase_0251_properties.json @@ -33,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0252/testcase_0252_properties.json b/system_test/replication_testsuite/testcase_0252/testcase_0252_properties.json index 49e44f5..37e96a6 100644 --- a/system_test/replication_testsuite/testcase_0252/testcase_0252_properties.json +++ b/system_test/replication_testsuite/testcase_0252/testcase_0252_properties.json @@ -33,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0253/testcase_0253_properties.json b/system_test/replication_testsuite/testcase_0253/testcase_0253_properties.json index eac08a4..bec4fc4 100644 --- a/system_test/replication_testsuite/testcase_0253/testcase_0253_properties.json +++ b/system_test/replication_testsuite/testcase_0253/testcase_0253_properties.json @@ -33,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0254/testcase_0254_properties.json b/system_test/replication_testsuite/testcase_0254/testcase_0254_properties.json index 338abcb..80fc064 100644 --- a/system_test/replication_testsuite/testcase_0254/testcase_0254_properties.json +++ b/system_test/replication_testsuite/testcase_0254/testcase_0254_properties.json @@ -33,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0255/testcase_0255_properties.json b/system_test/replication_testsuite/testcase_0255/testcase_0255_properties.json index bf06af9..5adfefb 100644 --- a/system_test/replication_testsuite/testcase_0255/testcase_0255_properties.json +++ b/system_test/replication_testsuite/testcase_0255/testcase_0255_properties.json @@ -33,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0256/testcase_0256_properties.json b/system_test/replication_testsuite/testcase_0256/testcase_0256_properties.json index 946667a..b8ef881 100644 --- a/system_test/replication_testsuite/testcase_0256/testcase_0256_properties.json +++ b/system_test/replication_testsuite/testcase_0256/testcase_0256_properties.json @@ -33,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0257/testcase_0257_properties.json b/system_test/replication_testsuite/testcase_0257/testcase_0257_properties.json index ee88b58..6ea8cbf 100644 --- a/system_test/replication_testsuite/testcase_0257/testcase_0257_properties.json +++ b/system_test/replication_testsuite/testcase_0257/testcase_0257_properties.json @@ -33,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0258/testcase_0258_properties.json b/system_test/replication_testsuite/testcase_0258/testcase_0258_properties.json index 2bb1af5..c1ca6b2 100644 --- a/system_test/replication_testsuite/testcase_0258/testcase_0258_properties.json +++ b/system_test/replication_testsuite/testcase_0258/testcase_0258_properties.json @@ -33,8 +33,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -42,8 +44,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -51,8 +55,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0301/testcase_0301_properties.json b/system_test/replication_testsuite/testcase_0301/testcase_0301_properties.json index f5099a3..f87074d 100644 --- a/system_test/replication_testsuite/testcase_0301/testcase_0301_properties.json +++ b/system_test/replication_testsuite/testcase_0301/testcase_0301_properties.json @@ -31,8 +31,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -40,8 +42,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -49,8 +53,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0302/testcase_0302_properties.json b/system_test/replication_testsuite/testcase_0302/testcase_0302_properties.json index b8f6fa0..cb186dd 100644 --- a/system_test/replication_testsuite/testcase_0302/testcase_0302_properties.json +++ b/system_test/replication_testsuite/testcase_0302/testcase_0302_properties.json @@ -31,8 +31,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -40,8 +42,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -49,8 +53,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0303/testcase_0303_properties.json b/system_test/replication_testsuite/testcase_0303/testcase_0303_properties.json index 1da345d..6a87d14 100644 --- a/system_test/replication_testsuite/testcase_0303/testcase_0303_properties.json +++ b/system_test/replication_testsuite/testcase_0303/testcase_0303_properties.json @@ -31,8 +31,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -40,8 +42,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -49,8 +53,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0304/testcase_0304_properties.json b/system_test/replication_testsuite/testcase_0304/testcase_0304_properties.json index d8cb0c1..fbe342e 100644 --- a/system_test/replication_testsuite/testcase_0304/testcase_0304_properties.json +++ b/system_test/replication_testsuite/testcase_0304/testcase_0304_properties.json @@ -31,8 +31,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -40,8 +42,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -49,8 +53,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0305/testcase_0305_properties.json b/system_test/replication_testsuite/testcase_0305/testcase_0305_properties.json index e3e9721..51a7af4 100644 --- a/system_test/replication_testsuite/testcase_0305/testcase_0305_properties.json +++ b/system_test/replication_testsuite/testcase_0305/testcase_0305_properties.json @@ -31,8 +31,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -40,8 +42,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -49,8 +53,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0306/testcase_0306_properties.json b/system_test/replication_testsuite/testcase_0306/testcase_0306_properties.json index 7040165..3a7a2ac 100644 --- a/system_test/replication_testsuite/testcase_0306/testcase_0306_properties.json +++ b/system_test/replication_testsuite/testcase_0306/testcase_0306_properties.json @@ -31,8 +31,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -40,8 +42,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -49,8 +53,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0307/testcase_0307_properties.json b/system_test/replication_testsuite/testcase_0307/testcase_0307_properties.json index cf87372..065cdb8 100644 --- a/system_test/replication_testsuite/testcase_0307/testcase_0307_properties.json +++ b/system_test/replication_testsuite/testcase_0307/testcase_0307_properties.json @@ -31,8 +31,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -40,8 +42,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -49,8 +53,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_0308/testcase_0308_properties.json b/system_test/replication_testsuite/testcase_0308/testcase_0308_properties.json index f61deaa..1d1fb96 100644 --- a/system_test/replication_testsuite/testcase_0308/testcase_0308_properties.json +++ b/system_test/replication_testsuite/testcase_0308/testcase_0308_properties.json @@ -31,8 +31,10 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -40,8 +42,10 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -49,8 +53,10 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_1/testcase_1_properties.json b/system_test/replication_testsuite/testcase_1/testcase_1_properties.json index 25dd2e3..7e1b3fb 100644 --- a/system_test/replication_testsuite/testcase_1/testcase_1_properties.json +++ b/system_test/replication_testsuite/testcase_1/testcase_1_properties.json @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_1_logs", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" @@ -41,7 +41,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_2_logs", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" @@ -50,7 +50,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "10240", + "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_3_logs", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" diff --git a/system_test/replication_testsuite/testcase_4001/testcase_4001_properties.json b/system_test/replication_testsuite/testcase_4001/testcase_4001_properties.json index a87ca15..3d0e8b7 100644 --- a/system_test/replication_testsuite/testcase_4001/testcase_4001_properties.json +++ b/system_test/replication_testsuite/testcase_4001/testcase_4001_properties.json @@ -33,9 +33,11 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -43,9 +45,11 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -53,9 +57,11 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_4002/testcase_4002_properties.json b/system_test/replication_testsuite/testcase_4002/testcase_4002_properties.json index 808e002..7fb20b7 100644 --- a/system_test/replication_testsuite/testcase_4002/testcase_4002_properties.json +++ b/system_test/replication_testsuite/testcase_4002/testcase_4002_properties.json @@ -33,9 +33,11 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -43,9 +45,11 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -53,9 +57,11 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_4003/testcase_4003_properties.json b/system_test/replication_testsuite/testcase_4003/testcase_4003_properties.json index b590644..d50cdcb 100644 --- a/system_test/replication_testsuite/testcase_4003/testcase_4003_properties.json +++ b/system_test/replication_testsuite/testcase_4003/testcase_4003_properties.json @@ -33,9 +33,11 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -43,9 +45,11 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -53,9 +57,11 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_4004/testcase_4004_properties.json b/system_test/replication_testsuite/testcase_4004/testcase_4004_properties.json index 3a3b1ca..c0423f1 100644 --- a/system_test/replication_testsuite/testcase_4004/testcase_4004_properties.json +++ b/system_test/replication_testsuite/testcase_4004/testcase_4004_properties.json @@ -33,9 +33,11 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -43,9 +45,11 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -53,9 +57,11 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_4005/testcase_4005_properties.json b/system_test/replication_testsuite/testcase_4005/testcase_4005_properties.json index 68ff605..a9155eb 100644 --- a/system_test/replication_testsuite/testcase_4005/testcase_4005_properties.json +++ b/system_test/replication_testsuite/testcase_4005/testcase_4005_properties.json @@ -33,9 +33,11 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -43,9 +45,11 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -53,9 +57,11 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_4006/testcase_4006_properties.json b/system_test/replication_testsuite/testcase_4006/testcase_4006_properties.json index 9edbace..a6c65b7 100644 --- a/system_test/replication_testsuite/testcase_4006/testcase_4006_properties.json +++ b/system_test/replication_testsuite/testcase_4006/testcase_4006_properties.json @@ -33,9 +33,11 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -43,9 +45,11 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -53,9 +57,11 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_4007/testcase_4007_properties.json b/system_test/replication_testsuite/testcase_4007/testcase_4007_properties.json index 3c01464..9d351ee 100644 --- a/system_test/replication_testsuite/testcase_4007/testcase_4007_properties.json +++ b/system_test/replication_testsuite/testcase_4007/testcase_4007_properties.json @@ -33,9 +33,11 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -43,9 +45,11 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -53,9 +57,11 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_4008/testcase_4008_properties.json b/system_test/replication_testsuite/testcase_4008/testcase_4008_properties.json index c333776..c3aa78d 100644 --- a/system_test/replication_testsuite/testcase_4008/testcase_4008_properties.json +++ b/system_test/replication_testsuite/testcase_4008/testcase_4008_properties.json @@ -33,9 +33,11 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -43,9 +45,11 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -53,9 +57,11 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "2", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_4011/testcase_4011_properties.json b/system_test/replication_testsuite/testcase_4011/testcase_4011_properties.json index b7716bb..5c5161f 100644 --- a/system_test/replication_testsuite/testcase_4011/testcase_4011_properties.json +++ b/system_test/replication_testsuite/testcase_4011/testcase_4011_properties.json @@ -33,9 +33,11 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -43,9 +45,11 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -53,9 +57,11 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_4012/testcase_4012_properties.json b/system_test/replication_testsuite/testcase_4012/testcase_4012_properties.json index 3f461b1..286d46e 100644 --- a/system_test/replication_testsuite/testcase_4012/testcase_4012_properties.json +++ b/system_test/replication_testsuite/testcase_4012/testcase_4012_properties.json @@ -33,9 +33,11 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -43,9 +45,11 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -53,9 +57,11 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_4013/testcase_4013_properties.json b/system_test/replication_testsuite/testcase_4013/testcase_4013_properties.json index c56019f..70b14d5 100644 --- a/system_test/replication_testsuite/testcase_4013/testcase_4013_properties.json +++ b/system_test/replication_testsuite/testcase_4013/testcase_4013_properties.json @@ -33,9 +33,11 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -43,9 +45,11 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -53,9 +57,11 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_4014/testcase_4014_properties.json b/system_test/replication_testsuite/testcase_4014/testcase_4014_properties.json index cabc8b9..5ea6368 100644 --- a/system_test/replication_testsuite/testcase_4014/testcase_4014_properties.json +++ b/system_test/replication_testsuite/testcase_4014/testcase_4014_properties.json @@ -33,9 +33,11 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -43,9 +45,11 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -53,9 +57,11 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_4015/testcase_4015_properties.json b/system_test/replication_testsuite/testcase_4015/testcase_4015_properties.json index ff96fde..6ac49c5 100644 --- a/system_test/replication_testsuite/testcase_4015/testcase_4015_properties.json +++ b/system_test/replication_testsuite/testcase_4015/testcase_4015_properties.json @@ -33,9 +33,11 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -43,9 +45,11 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -53,9 +57,11 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_4016/testcase_4016_properties.json b/system_test/replication_testsuite/testcase_4016/testcase_4016_properties.json index 2cce8cd..fe4ec5f 100644 --- a/system_test/replication_testsuite/testcase_4016/testcase_4016_properties.json +++ b/system_test/replication_testsuite/testcase_4016/testcase_4016_properties.json @@ -33,9 +33,11 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -43,9 +45,11 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -53,9 +57,11 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_4017/testcase_4017_properties.json b/system_test/replication_testsuite/testcase_4017/testcase_4017_properties.json index a6208ac..3b85302 100644 --- a/system_test/replication_testsuite/testcase_4017/testcase_4017_properties.json +++ b/system_test/replication_testsuite/testcase_4017/testcase_4017_properties.json @@ -33,9 +33,11 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -43,9 +45,11 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -53,9 +57,11 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_4018/testcase_4018_properties.json b/system_test/replication_testsuite/testcase_4018/testcase_4018_properties.json index b4c657b..8f6ee50 100644 --- a/system_test/replication_testsuite/testcase_4018/testcase_4018_properties.json +++ b/system_test/replication_testsuite/testcase_4018/testcase_4018_properties.json @@ -33,9 +33,11 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" }, @@ -43,9 +45,11 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" }, @@ -53,9 +57,11 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "102400", + "log.segment.bytes": "102400", "log.retention.size": "1048576", "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "2", + "num.partitions": "2", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, diff --git a/system_test/replication_testsuite/testcase_9051/testcase_9051_properties.json b/system_test/replication_testsuite/testcase_9051/testcase_9051_properties.json index 9f9ea1c..958eef7 100644 --- a/system_test/replication_testsuite/testcase_9051/testcase_9051_properties.json +++ b/system_test/replication_testsuite/testcase_9051/testcase_9051_properties.json @@ -35,7 +35,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.size": "1048576", + "log.segment.bytes": "1048576", "log.dir": "/tmp/kafka_server_1_logs", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" @@ -44,7 +44,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.size": "1048576", + "log.segment.bytes": "1048576", "log.dir": "/tmp/kafka_server_2_logs", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" @@ -53,7 +53,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.size": "1048576", + "log.segment.bytes": "1048576", "log.dir": "/tmp/kafka_server_3_logs", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties"