kafka-1240; Add ability to existing system tests to use the new producer client; patched by Jun Rao; reviewed by Neha Narkhede

This commit is contained in:
Jun Rao 2014-03-06 19:06:25 -08:00
parent 74c54c7eeb
commit c765d7bd4e
143 changed files with 3314 additions and 24 deletions

View File

@ -168,10 +168,10 @@ object MirrorMaker extends Logging {
val producerId = Utils.abs(java.util.Arrays.hashCode(producerRecord.key())) % producers.size
trace("Send message with key %s to producer %d.".format(java.util.Arrays.toString(producerRecord.key()), producerId))
val producer = producers(producerId)
producer.send(producerRecord)
producer.send(producerRecord, Utils.errorLoggingCallback(producerRecord.key(), producerRecord.value()))
} else {
val producerId = producerIndex.getAndSet((producerIndex.get() + 1) % producers.size)
producers(producerId).send(producerRecord)
producers(producerId).send(producerRecord, Utils.errorLoggingCallback(producerRecord.key(), producerRecord.value()))
trace("Sent message to producer " + producerId)
}
}

View File

@ -30,6 +30,7 @@ import scala.collection.mutable
import java.util.Properties
import kafka.common.KafkaException
import kafka.common.KafkaStorageException
import org.apache.kafka.clients.producer.{RecordMetadata, Callback}
/**
@ -540,5 +541,25 @@ object Utils extends Logging {
lock.unlock()
}
}
def errorLoggingCallback(key: Array[Byte], value: Array[Byte], logAsString: Boolean = false) = {
new Callback() {
def onCompletion(metadata: RecordMetadata, e: Exception) {
if (e != null) {
val keyString = if (key == null)
"null"
else {
if (logAsString) new String(key) else key.length + " bytes"
}
val valueString = if (value == null)
"null"
else {
if (logAsString) new String(value) else value.length + " bytes"
}
error("Error when sending message with key: " + keyString + ", value: " + valueString +
" with exception " + e.getMessage)
}
}
}
}
}

View File

@ -21,15 +21,14 @@ import java.util.concurrent.{ CountDownLatch, Executors }
import java.util.concurrent.atomic.AtomicLong
import kafka.producer._
import org.apache.log4j.Logger
import kafka.message.{ CompressionCodec, Message }
import kafka.message.CompressionCodec
import java.text.SimpleDateFormat
import kafka.serializer._
import java.util._
import collection.immutable.List
import kafka.utils.{ VerifiableProperties, Logging, Utils }
import kafka.metrics.KafkaMetricsReporter
import org.apache.kafka.clients.producer.KafkaProducer
import org.apache.kafka.clients.producer.ProducerRecord
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
/**
* Load test for the producer
@ -210,11 +209,18 @@ object ProducerPerformance extends Logging {
props.put("client.id", "perf-test")
props.put("request.required.acks", config.producerRequestRequiredAcks.toString)
props.put("request.timeout.ms", config.producerRequestTimeoutMs.toString)
props.put("request.retries", config.producerNumRetries.toString)
props.put("retry.backoff.ms", config.producerRetryBackoffMs.toString)
val producer = new KafkaProducer(props)
def send(topic: String, partition: Long, bytes: Array[Byte]) {
val part = partition % this.producer.partitionsFor(topic).size
this.producer.send(new ProducerRecord(topic, Utils.abs(part.toInt), null, bytes))
if (config.isSync) {
this.producer.send(new ProducerRecord(topic, Utils.abs(part.toInt), null, bytes)).get()
} else {
this.producer.send(new ProducerRecord(topic, Utils.abs(part.toInt), null, bytes),
Utils.errorLoggingCallback(null, bytes, if (config.seqIdMode) true else false))
}
}
def close() {
@ -280,21 +286,28 @@ object ProducerPerformance extends Logging {
var bytesSent = 0L
var nSends = 0
var j: Long = 0L
var message: Array[Byte] = null
while (j < messagesPerThread) {
try {
config.topics.foreach(
topic => {
producer.send(topic, j, generateProducerData(topic, j))
message = generateProducerData(topic, j)
producer.send(topic, j, message)
nSends += 1
if (config.messageSendGapMs > 0)
Thread.sleep(config.messageSendGapMs)
})
} catch {
case e: Exception => error("Error sending messages", e)
case e: Throwable => error("Error when sending message " + new String(message), e)
}
j += 1
}
producer.close()
try {
producer.close()
} catch {
case e: Throwable => error("Error when closing producer", e)
}
totalBytesSent.addAndGet(bytesSent)
totalMessagesSent.addAndGet(nSends)
allDone.countDown()

View File

@ -76,4 +76,7 @@ To create a new test case under "replication_testsuite", please do the following
2. Rename system_test/replication_testsuite/testcase_2/testcase_1_properties.json => system_test/replication_testsuite/testcase_2/testcase_2_properties.json
3. Update system_test/replication_testsuite/testcase_2/testcase_2_properties.json with the corresponding settings for testcase 2.
Note:
The following testcases are for the old producer and the old mirror maker. We can remove them once we phase out the old producer client.
replication_testsuite: testcase_{10101 - 10110} testcase_{10131 - 10134}
mirror_maker_testsuite: testcase_{15001 - 15006}

View File

@ -247,6 +247,7 @@ class MigrationToolTest(ReplicationUtils, SetupUtils):
str(self.testcaseEnv.userDefinedEnvVarDict["backgroundProducerStopped"]) + "]", extra=self.d)
if self.testcaseEnv.userDefinedEnvVarDict["backgroundProducerStopped"]:
time.sleep(1)
self.testcaseEnv.lock.release()
self.logger.info("all producer threads completed", extra=self.d)
break
time.sleep(1)

View File

@ -1,6 +1,5 @@
producer.type=async
queue.enqueue.timeout.ms=-1
block.on.buffer.full=true
metadata.broker.list=localhost:9094
compression.codec=0
message.send.max.retries=3
request.retries=3
request.required.acks=1

View File

@ -248,6 +248,7 @@ class MirrorMakerTest(ReplicationUtils, SetupUtils):
str(self.testcaseEnv.userDefinedEnvVarDict["backgroundProducerStopped"]) + "]", extra=self.d)
if self.testcaseEnv.userDefinedEnvVarDict["backgroundProducerStopped"]:
time.sleep(1)
self.testcaseEnv.lock.release()
self.logger.info("all producer threads completed", extra=self.d)
break
time.sleep(1)

View File

@ -0,0 +1,158 @@
{
"description": {"01":"To Test : 'Replication with Mirror Maker'",
"02":"Set up 2 clusters such as : SOURCE => MirrorMaker => TARGET",
"03":"Set up 2-node Zk cluster for both SOURCE & TARGET",
"04":"Produce and consume messages to a single topic - single partition.",
"05":"This test sends messages to 3 replicas",
"06":"At the end it verifies the log size and contents",
"07":"Use a consumer to verify no message loss in TARGET cluster.",
"08":"Producer dimensions : mode:sync, acks:-1, comp:0",
"09":"Log segment size : 10240"
},
"testcase_args": {
"bounce_leader": "false",
"bounce_mirror_maker": "false",
"replica_factor": "3",
"num_partition": "1",
"num_iteration": "1",
"sleep_seconds_between_producer_calls": "1",
"message_producing_free_time_sec": "15",
"num_messages_to_produce_per_producer_call": "50"
},
"entities": [
{
"entity_id": "0",
"clientPort": "2108",
"dataDir": "/tmp/zookeeper_0",
"log_filename": "zookeeper_0.log",
"config_filename": "zookeeper_0.properties"
},
{
"entity_id": "1",
"clientPort": "2118",
"dataDir": "/tmp/zookeeper_1",
"log_filename": "zookeeper_1.log",
"config_filename": "zookeeper_1.properties"
},
{
"entity_id": "2",
"clientPort": "2128",
"dataDir": "/tmp/zookeeper_2",
"log_filename": "zookeeper_2.log",
"config_filename": "zookeeper_2.properties"
},
{
"entity_id": "3",
"clientPort": "2138",
"dataDir": "/tmp/zookeeper_3",
"log_filename": "zookeeper_3.log",
"config_filename": "zookeeper_3.properties"
},
{
"entity_id": "4",
"port": "9091",
"broker.id": "1",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_4_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_4.log",
"config_filename": "kafka_server_4.properties"
},
{
"entity_id": "5",
"port": "9092",
"broker.id": "2",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_5_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_5.log",
"config_filename": "kafka_server_5.properties"
},
{
"entity_id": "6",
"port": "9093",
"broker.id": "3",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_6_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_6.log",
"config_filename": "kafka_server_6.properties"
},
{
"entity_id": "7",
"port": "9094",
"broker.id": "4",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_7_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_7.log",
"config_filename": "kafka_server_7.properties"
},
{
"entity_id": "8",
"port": "9095",
"broker.id": "5",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_8_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_8.log",
"config_filename": "kafka_server_8.properties"
},
{
"entity_id": "9",
"port": "9096",
"broker.id": "6",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_9_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_9.log",
"config_filename": "kafka_server_9.properties"
},
{
"entity_id": "10",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",
"message-size": "500",
"message": "500",
"request-num-acks": "-1",
"sync":"true",
"producer-num-retries":"5",
"log_filename": "producer_performance_10.log",
"config_filename": "producer_performance_10.properties"
},
{
"entity_id": "11",
"topic": "test_1",
"group.id": "mytestgroup",
"consumer-timeout-ms": "10000",
"log_filename": "console_consumer_11.log",
"config_filename": "console_consumer_11.properties"
},
{
"entity_id": "12",
"log_filename": "mirror_maker_12.log",
"mirror_consumer_config_filename": "mirror_consumer_12.properties",
"mirror_producer_config_filename": "mirror_producer_12.properties"
},
{
"entity_id": "13",
"topic": "test_1",
"group.id": "mytestgroup",
"consumer-timeout-ms": "10000",
"log_filename": "console_consumer_13.log",
"config_filename": "console_consumer_13.properties"
}
]
}

View File

@ -0,0 +1,158 @@
{
"description": {"01":"Replication with Mirror Maker => Bounce Mirror Maker",
"02":"Set up 2 clusters such as : SOURCE => MirrorMaker => TARGET",
"03":"Set up 2-node Zk cluster for both SOURCE & TARGET",
"04":"Produce and consume messages to a single topic - single partition.",
"05":"This test sends messages to 3 replicas",
"06":"At the end it verifies the log size and contents",
"07":"Use a consumer to verify no message loss in TARGET cluster.",
"08":"Producer dimensions : mode:sync, acks:-1, comp:0",
"09":"Log segment size : 10240"
},
"testcase_args": {
"bounce_leader": "false",
"bounce_mirror_maker": "true",
"replica_factor": "3",
"num_partition": "1",
"num_iteration": "1",
"sleep_seconds_between_producer_calls": "1",
"message_producing_free_time_sec": "15",
"num_messages_to_produce_per_producer_call": "50"
},
"entities": [
{
"entity_id": "0",
"clientPort": "2108",
"dataDir": "/tmp/zookeeper_0",
"log_filename": "zookeeper_0.log",
"config_filename": "zookeeper_0.properties"
},
{
"entity_id": "1",
"clientPort": "2118",
"dataDir": "/tmp/zookeeper_1",
"log_filename": "zookeeper_1.log",
"config_filename": "zookeeper_1.properties"
},
{
"entity_id": "2",
"clientPort": "2128",
"dataDir": "/tmp/zookeeper_2",
"log_filename": "zookeeper_2.log",
"config_filename": "zookeeper_2.properties"
},
{
"entity_id": "3",
"clientPort": "2138",
"dataDir": "/tmp/zookeeper_3",
"log_filename": "zookeeper_3.log",
"config_filename": "zookeeper_3.properties"
},
{
"entity_id": "4",
"port": "9091",
"broker.id": "1",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_4_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_4.log",
"config_filename": "kafka_server_4.properties"
},
{
"entity_id": "5",
"port": "9092",
"broker.id": "2",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_5_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_5.log",
"config_filename": "kafka_server_5.properties"
},
{
"entity_id": "6",
"port": "9093",
"broker.id": "3",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_6_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_6.log",
"config_filename": "kafka_server_6.properties"
},
{
"entity_id": "7",
"port": "9094",
"broker.id": "4",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_7_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_7.log",
"config_filename": "kafka_server_7.properties"
},
{
"entity_id": "8",
"port": "9095",
"broker.id": "5",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_8_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_8.log",
"config_filename": "kafka_server_8.properties"
},
{
"entity_id": "9",
"port": "9096",
"broker.id": "6",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_9_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_9.log",
"config_filename": "kafka_server_9.properties"
},
{
"entity_id": "10",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",
"message-size": "500",
"message": "100",
"request-num-acks": "-1",
"sync":"true",
"producer-num-retries":"5",
"log_filename": "producer_performance_10.log",
"config_filename": "producer_performance_10.properties"
},
{
"entity_id": "11",
"topic": "test_1",
"group.id": "mytestgroup",
"consumer-timeout-ms": "10000",
"log_filename": "console_consumer_11.log",
"config_filename": "console_consumer_11.properties"
},
{
"entity_id": "12",
"log_filename": "mirror_maker_12.log",
"mirror_consumer_config_filename": "mirror_consumer_12.properties",
"mirror_producer_config_filename": "mirror_producer_12.properties"
},
{
"entity_id": "13",
"topic": "test_1",
"group.id": "mytestgroup",
"consumer-timeout-ms": "10000",
"log_filename": "console_consumer_13.log",
"config_filename": "console_consumer_13.properties"
}
]
}

View File

@ -0,0 +1,135 @@
{
"cluster_config": [
{
"entity_id": "0",
"hostname": "localhost",
"role": "zookeeper",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9100"
},
{
"entity_id": "1",
"hostname": "localhost",
"role": "zookeeper",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9101"
},
{
"entity_id": "2",
"hostname": "localhost",
"role": "zookeeper",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9102"
},
{
"entity_id": "3",
"hostname": "localhost",
"role": "zookeeper",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9103"
},
{
"entity_id": "4",
"hostname": "localhost",
"role": "broker",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9104"
},
{
"entity_id": "5",
"hostname": "localhost",
"role": "broker",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9105"
},
{
"entity_id": "6",
"hostname": "localhost",
"role": "broker",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9106"
},
{
"entity_id": "7",
"hostname": "localhost",
"role": "broker",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9107"
},
{
"entity_id": "8",
"hostname": "localhost",
"role": "broker",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9108"
},
{
"entity_id": "9",
"hostname": "localhost",
"role": "broker",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9109"
},
{
"entity_id": "10",
"hostname": "localhost",
"role": "producer_performance",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9110"
},
{
"entity_id": "11",
"hostname": "localhost",
"role": "console_consumer",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9111"
},
{
"entity_id": "12",
"hostname": "localhost",
"role": "mirror_maker",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9112"
},
{
"entity_id": "13",
"hostname": "localhost",
"role": "mirror_maker",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9113"
}
]
}

View File

@ -0,0 +1,156 @@
{
"description": {"01":"Replication with Mirror Maker => Bounce Mirror Maker",
"02":"Set up 2 clusters such as : SOURCE => MirrorMaker => TARGET",
"03":"Set up 2-node Zk cluster for both SOURCE & TARGET",
"04":"Produce and consume messages to a single topic - single partition.",
"05":"This test sends messages to 3 replicas",
"06":"At the end it verifies the log size and contents",
"07":"Use a consumer to verify no message loss in TARGET cluster.",
"08":"Producer dimensions : mode:async, acks:-1, comp:1",
"09":"Log segment size : 10240"
},
"testcase_args": {
"bounce_leader": "false",
"bounce_mirror_maker": "true",
"bounced_entity_downtime_sec": "30",
"replica_factor": "3",
"num_partition": "1",
"num_iteration": "1",
"sleep_seconds_between_producer_calls": "1",
"message_producing_free_time_sec": "15",
"num_messages_to_produce_per_producer_call": "50"
},
"entities": [
{
"entity_id": "0",
"clientPort": "2108",
"dataDir": "/tmp/zookeeper_0",
"log_filename": "zookeeper_0.log",
"config_filename": "zookeeper_0.properties"
},
{
"entity_id": "1",
"clientPort": "2118",
"dataDir": "/tmp/zookeeper_1",
"log_filename": "zookeeper_1.log",
"config_filename": "zookeeper_1.properties"
},
{
"entity_id": "2",
"clientPort": "2128",
"dataDir": "/tmp/zookeeper_2",
"log_filename": "zookeeper_2.log",
"config_filename": "zookeeper_2.properties"
},
{
"entity_id": "3",
"clientPort": "2138",
"dataDir": "/tmp/zookeeper_3",
"log_filename": "zookeeper_3.log",
"config_filename": "zookeeper_3.properties"
},
{
"entity_id": "4",
"port": "9091",
"broker.id": "1",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_4_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_4.log",
"config_filename": "kafka_server_4.properties"
},
{
"entity_id": "5",
"port": "9092",
"broker.id": "2",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_5_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_5.log",
"config_filename": "kafka_server_5.properties"
},
{
"entity_id": "6",
"port": "9093",
"broker.id": "3",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_6_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_6.log",
"config_filename": "kafka_server_6.properties"
},
{
"entity_id": "7",
"port": "9094",
"broker.id": "4",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_7_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_7.log",
"config_filename": "kafka_server_7.properties"
},
{
"entity_id": "8",
"port": "9095",
"broker.id": "5",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_8_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_8.log",
"config_filename": "kafka_server_8.properties"
},
{
"entity_id": "9",
"port": "9096",
"broker.id": "6",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_9_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_9.log",
"config_filename": "kafka_server_9.properties"
},
{
"entity_id": "10",
"topic": "test_1",
"threads": "5",
"compression-codec": "2",
"message-size": "500",
"message": "100",
"request-num-acks": "-1",
"sync":"false",
"producer-num-retries":"5",
"log_filename": "producer_performance_10.log",
"config_filename": "producer_performance_10.properties"
},
{
"entity_id": "11",
"topic": "test_1",
"group.id": "mytestgroup",
"consumer-timeout-ms": "10000",
"log_filename": "console_consumer_11.log",
"config_filename": "console_consumer_11.properties"
},
{
"entity_id": "12",
"log_filename": "mirror_maker_12.log",
"mirror_consumer_config_filename": "mirror_consumer_12.properties",
"mirror_producer_config_filename": "mirror_producer_12.properties"
},
{
"entity_id": "13",
"log_filename": "mirror_maker_13.log",
"mirror_consumer_config_filename": "mirror_consumer_13.properties",
"mirror_producer_config_filename": "mirror_producer_13.properties"
}
]
}

View File

@ -0,0 +1,135 @@
{
"cluster_config": [
{
"entity_id": "0",
"hostname": "localhost",
"role": "zookeeper",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9100"
},
{
"entity_id": "1",
"hostname": "localhost",
"role": "zookeeper",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9101"
},
{
"entity_id": "2",
"hostname": "localhost",
"role": "zookeeper",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9102"
},
{
"entity_id": "3",
"hostname": "localhost",
"role": "zookeeper",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9103"
},
{
"entity_id": "4",
"hostname": "localhost",
"role": "broker",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9104"
},
{
"entity_id": "5",
"hostname": "localhost",
"role": "broker",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9105"
},
{
"entity_id": "6",
"hostname": "localhost",
"role": "broker",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9106"
},
{
"entity_id": "7",
"hostname": "localhost",
"role": "broker",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9107"
},
{
"entity_id": "8",
"hostname": "localhost",
"role": "broker",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9108"
},
{
"entity_id": "9",
"hostname": "localhost",
"role": "broker",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9109"
},
{
"entity_id": "10",
"hostname": "localhost",
"role": "producer_performance",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9110"
},
{
"entity_id": "11",
"hostname": "localhost",
"role": "console_consumer",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9111"
},
{
"entity_id": "12",
"hostname": "localhost",
"role": "mirror_maker",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9112"
},
{
"entity_id": "13",
"hostname": "localhost",
"role": "mirror_maker",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9113"
}
]
}

View File

@ -0,0 +1,156 @@
{
"description": {"01":"Replication with Mirror Maker => Bounce Mirror Maker",
"02":"Set up 2 clusters such as : SOURCE => MirrorMaker => TARGET",
"03":"Set up 2-node Zk cluster for both SOURCE & TARGET",
"04":"Produce and consume messages to a single topic - single partition.",
"05":"This test sends messages to 3 replicas",
"06":"At the end it verifies the log size and contents",
"07":"Use a consumer to verify no message loss in TARGET cluster.",
"08":"Producer dimensions : mode:async, acks:1, comp:1",
"09":"Log segment size : 10240"
},
"testcase_args": {
"bounce_leader": "false",
"bounce_mirror_maker": "true",
"bounced_entity_downtime_sec": "30",
"replica_factor": "3",
"num_partition": "1",
"num_iteration": "1",
"sleep_seconds_between_producer_calls": "1",
"message_producing_free_time_sec": "15",
"num_messages_to_produce_per_producer_call": "50"
},
"entities": [
{
"entity_id": "0",
"clientPort": "2108",
"dataDir": "/tmp/zookeeper_0",
"log_filename": "zookeeper_0.log",
"config_filename": "zookeeper_0.properties"
},
{
"entity_id": "1",
"clientPort": "2118",
"dataDir": "/tmp/zookeeper_1",
"log_filename": "zookeeper_1.log",
"config_filename": "zookeeper_1.properties"
},
{
"entity_id": "2",
"clientPort": "2128",
"dataDir": "/tmp/zookeeper_2",
"log_filename": "zookeeper_2.log",
"config_filename": "zookeeper_2.properties"
},
{
"entity_id": "3",
"clientPort": "2138",
"dataDir": "/tmp/zookeeper_3",
"log_filename": "zookeeper_3.log",
"config_filename": "zookeeper_3.properties"
},
{
"entity_id": "4",
"port": "9091",
"broker.id": "1",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_4_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_4.log",
"config_filename": "kafka_server_4.properties"
},
{
"entity_id": "5",
"port": "9092",
"broker.id": "2",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_5_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_5.log",
"config_filename": "kafka_server_5.properties"
},
{
"entity_id": "6",
"port": "9093",
"broker.id": "3",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_6_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_6.log",
"config_filename": "kafka_server_6.properties"
},
{
"entity_id": "7",
"port": "9094",
"broker.id": "4",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_7_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_7.log",
"config_filename": "kafka_server_7.properties"
},
{
"entity_id": "8",
"port": "9095",
"broker.id": "5",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_8_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_8.log",
"config_filename": "kafka_server_8.properties"
},
{
"entity_id": "9",
"port": "9096",
"broker.id": "6",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_9_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_9.log",
"config_filename": "kafka_server_9.properties"
},
{
"entity_id": "10",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",
"message-size": "500",
"message": "100",
"request-num-acks": "1",
"sync":"false",
"producer-num-retries":"5",
"log_filename": "producer_performance_10.log",
"config_filename": "producer_performance_10.properties"
},
{
"entity_id": "11",
"topic": "test_1",
"group.id": "mytestgroup",
"consumer-timeout-ms": "10000",
"log_filename": "console_consumer_11.log",
"config_filename": "console_consumer_11.properties"
},
{
"entity_id": "12",
"log_filename": "mirror_maker_12.log",
"mirror_consumer_config_filename": "mirror_consumer_12.properties",
"mirror_producer_config_filename": "mirror_producer_12.properties"
},
{
"entity_id": "13",
"log_filename": "mirror_maker_13.log",
"mirror_consumer_config_filename": "mirror_consumer_13.properties",
"mirror_producer_config_filename": "mirror_producer_13.properties"
}
]
}

View File

@ -0,0 +1,153 @@
{
"cluster_config": [
{
"entity_id": "0",
"hostname": "localhost",
"role": "zookeeper",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9100"
},
{
"entity_id": "1",
"hostname": "localhost",
"role": "zookeeper",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9101"
},
{
"entity_id": "2",
"hostname": "localhost",
"role": "zookeeper",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9102"
},
{
"entity_id": "3",
"hostname": "localhost",
"role": "zookeeper",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9103"
},
{
"entity_id": "4",
"hostname": "localhost",
"role": "broker",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9104"
},
{
"entity_id": "5",
"hostname": "localhost",
"role": "broker",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9105"
},
{
"entity_id": "6",
"hostname": "localhost",
"role": "broker",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9106"
},
{
"entity_id": "7",
"hostname": "localhost",
"role": "broker",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9107"
},
{
"entity_id": "8",
"hostname": "localhost",
"role": "broker",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9108"
},
{
"entity_id": "9",
"hostname": "localhost",
"role": "broker",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9109"
},
{
"entity_id": "10",
"hostname": "localhost",
"role": "producer_performance",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9110"
},
{
"entity_id": "11",
"hostname": "localhost",
"role": "producer_performance",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9111"
},
{
"entity_id": "12",
"hostname": "localhost",
"role": "console_consumer",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9112"
},
{
"entity_id": "13",
"hostname": "localhost",
"role": "console_consumer",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9113"
},
{
"entity_id": "14",
"hostname": "localhost",
"role": "mirror_maker",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9114"
},
{
"entity_id": "15",
"hostname": "localhost",
"role": "mirror_maker",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9115"
}
]
}

View File

@ -0,0 +1,178 @@
{
"description": {"01":"Replication with Mirror Maker => Bounce Mirror Maker",
"02":"Set up 2 clusters such as : SOURCE => MirrorMaker => TARGET",
"03":"Set up 2-node Zk cluster for both SOURCE & TARGET",
"04":"Produce and consume messages to 2 topics - 2 partitions.",
"05":"This test sends messages to 3 replicas",
"06":"At the end it verifies the log size and contents",
"07":"Use a consumer to verify no message loss in TARGET cluster.",
"08":"Producer dimensions : mode:async, acks:-1, comp:1",
"09":"Log segment size : 10240"
},
"testcase_args": {
"bounce_leader": "false",
"bounce_mirror_maker": "true",
"bounced_entity_downtime_sec": "30",
"replica_factor": "3",
"num_partition": "2",
"num_iteration": "1",
"sleep_seconds_between_producer_calls": "1",
"message_producing_free_time_sec": "15",
"num_messages_to_produce_per_producer_call": "50"
},
"entities": [
{
"entity_id": "0",
"clientPort": "2108",
"dataDir": "/tmp/zookeeper_0",
"log_filename": "zookeeper_0.log",
"config_filename": "zookeeper_0.properties"
},
{
"entity_id": "1",
"clientPort": "2118",
"dataDir": "/tmp/zookeeper_1",
"log_filename": "zookeeper_1.log",
"config_filename": "zookeeper_1.properties"
},
{
"entity_id": "2",
"clientPort": "2128",
"dataDir": "/tmp/zookeeper_2",
"log_filename": "zookeeper_2.log",
"config_filename": "zookeeper_2.properties"
},
{
"entity_id": "3",
"clientPort": "2138",
"dataDir": "/tmp/zookeeper_3",
"log_filename": "zookeeper_3.log",
"config_filename": "zookeeper_3.properties"
},
{
"entity_id": "4",
"port": "9091",
"broker.id": "1",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_4_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_4.log",
"config_filename": "kafka_server_4.properties"
},
{
"entity_id": "5",
"port": "9092",
"broker.id": "2",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_5_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_5.log",
"config_filename": "kafka_server_5.properties"
},
{
"entity_id": "6",
"port": "9093",
"broker.id": "3",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_6_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_6.log",
"config_filename": "kafka_server_6.properties"
},
{
"entity_id": "7",
"port": "9094",
"broker.id": "4",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_7_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_7.log",
"config_filename": "kafka_server_7.properties"
},
{
"entity_id": "8",
"port": "9095",
"broker.id": "5",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_8_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_8.log",
"config_filename": "kafka_server_8.properties"
},
{
"entity_id": "9",
"port": "9096",
"broker.id": "6",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_9_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_9.log",
"config_filename": "kafka_server_9.properties"
},
{
"entity_id": "10",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",
"message-size": "500",
"message": "100",
"request-num-acks": "-1",
"sync":"false",
"producer-num-retries":"5",
"log_filename": "producer_performance_10.log",
"config_filename": "producer_performance_10.properties"
},
{
"entity_id": "11",
"topic": "test_2",
"threads": "5",
"compression-codec": "1",
"message-size": "500",
"message": "100",
"request-num-acks": "-1",
"sync":"false",
"producer-num-retries":"5",
"log_filename": "producer_performance_11.log",
"config_filename": "producer_performance_11.properties"
},
{
"entity_id": "12",
"topic": "test_1",
"group.id": "mytestgroup",
"consumer-timeout-ms": "10000",
"log_filename": "console_consumer_12.log",
"config_filename": "console_consumer_12.properties"
},
{
"entity_id": "13",
"topic": "test_2",
"group.id": "mytestgroup",
"consumer-timeout-ms": "10000",
"log_filename": "console_consumer_13.log",
"config_filename": "console_consumer_13.properties"
},
{
"entity_id": "14",
"log_filename": "mirror_maker_14.log",
"mirror_consumer_config_filename": "mirror_consumer_14.properties",
"mirror_producer_config_filename": "mirror_producer_14.properties"
},
{
"entity_id": "15",
"log_filename": "mirror_maker_15.log",
"mirror_consumer_config_filename": "mirror_consumer_15.properties",
"mirror_producer_config_filename": "mirror_producer_15.properties"
}
]
}

View File

@ -0,0 +1,153 @@
{
"cluster_config": [
{
"entity_id": "0",
"hostname": "localhost",
"role": "zookeeper",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9100"
},
{
"entity_id": "1",
"hostname": "localhost",
"role": "zookeeper",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9101"
},
{
"entity_id": "2",
"hostname": "localhost",
"role": "zookeeper",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9102"
},
{
"entity_id": "3",
"hostname": "localhost",
"role": "zookeeper",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9103"
},
{
"entity_id": "4",
"hostname": "localhost",
"role": "broker",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9104"
},
{
"entity_id": "5",
"hostname": "localhost",
"role": "broker",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9105"
},
{
"entity_id": "6",
"hostname": "localhost",
"role": "broker",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9106"
},
{
"entity_id": "7",
"hostname": "localhost",
"role": "broker",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9107"
},
{
"entity_id": "8",
"hostname": "localhost",
"role": "broker",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9108"
},
{
"entity_id": "9",
"hostname": "localhost",
"role": "broker",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9109"
},
{
"entity_id": "10",
"hostname": "localhost",
"role": "producer_performance",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9110"
},
{
"entity_id": "11",
"hostname": "localhost",
"role": "producer_performance",
"cluster_name":"source",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9111"
},
{
"entity_id": "12",
"hostname": "localhost",
"role": "console_consumer",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9112"
},
{
"entity_id": "13",
"hostname": "localhost",
"role": "console_consumer",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9113"
},
{
"entity_id": "14",
"hostname": "localhost",
"role": "mirror_maker",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9114"
},
{
"entity_id": "15",
"hostname": "localhost",
"role": "mirror_maker",
"cluster_name":"target",
"kafka_home": "default",
"java_home": "default",
"jmx_port": "9115"
}
]
}

View File

@ -0,0 +1,178 @@
{
"description": {"01":"Replication with Mirror Maker => Bounce Mirror Maker",
"02":"Set up 2 clusters such as : SOURCE => MirrorMaker => TARGET",
"03":"Set up 2-node Zk cluster for both SOURCE & TARGET",
"04":"Produce and consume messages to 2 topics - 2 partitions.",
"05":"This test sends messages to 3 replicas",
"06":"At the end it verifies the log size and contents",
"07":"Use a consumer to verify no message loss in TARGET cluster.",
"08":"Producer dimensions : mode:async, acks:1, comp:1",
"09":"Log segment size : 10240"
},
"testcase_args": {
"bounce_leader": "false",
"bounce_mirror_maker": "true",
"bounced_entity_downtime_sec": "30",
"replica_factor": "3",
"num_partition": "2",
"num_iteration": "1",
"sleep_seconds_between_producer_calls": "1",
"message_producing_free_time_sec": "15",
"num_messages_to_produce_per_producer_call": "50"
},
"entities": [
{
"entity_id": "0",
"clientPort": "2108",
"dataDir": "/tmp/zookeeper_0",
"log_filename": "zookeeper_0.log",
"config_filename": "zookeeper_0.properties"
},
{
"entity_id": "1",
"clientPort": "2118",
"dataDir": "/tmp/zookeeper_1",
"log_filename": "zookeeper_1.log",
"config_filename": "zookeeper_1.properties"
},
{
"entity_id": "2",
"clientPort": "2128",
"dataDir": "/tmp/zookeeper_2",
"log_filename": "zookeeper_2.log",
"config_filename": "zookeeper_2.properties"
},
{
"entity_id": "3",
"clientPort": "2138",
"dataDir": "/tmp/zookeeper_3",
"log_filename": "zookeeper_3.log",
"config_filename": "zookeeper_3.properties"
},
{
"entity_id": "4",
"port": "9091",
"broker.id": "1",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_4_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_4.log",
"config_filename": "kafka_server_4.properties"
},
{
"entity_id": "5",
"port": "9092",
"broker.id": "2",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_5_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_5.log",
"config_filename": "kafka_server_5.properties"
},
{
"entity_id": "6",
"port": "9093",
"broker.id": "3",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_6_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_6.log",
"config_filename": "kafka_server_6.properties"
},
{
"entity_id": "7",
"port": "9094",
"broker.id": "4",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_7_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_7.log",
"config_filename": "kafka_server_7.properties"
},
{
"entity_id": "8",
"port": "9095",
"broker.id": "5",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_8_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_8.log",
"config_filename": "kafka_server_8.properties"
},
{
"entity_id": "9",
"port": "9096",
"broker.id": "6",
"log.segment.bytes": "10240",
"log.dir": "/tmp/kafka_server_9_logs",
"default.replication.factor": "3",
"num.partitions": "5",
"log_filename": "kafka_server_9.log",
"config_filename": "kafka_server_9.properties"
},
{
"entity_id": "10",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",
"message-size": "500",
"message": "100",
"request-num-acks": "1",
"sync":"false",
"producer-num-retries":"5",
"log_filename": "producer_performance_10.log",
"config_filename": "producer_performance_10.properties"
},
{
"entity_id": "11",
"topic": "test_2",
"threads": "5",
"compression-codec": "1",
"message-size": "500",
"message": "100",
"request-num-acks": "1",
"sync":"false",
"producer-num-retries":"5",
"log_filename": "producer_performance_11.log",
"config_filename": "producer_performance_11.properties"
},
{
"entity_id": "12",
"topic": "test_1",
"group.id": "mytestgroup",
"consumer-timeout-ms": "10000",
"log_filename": "console_consumer_12.log",
"config_filename": "console_consumer_12.properties"
},
{
"entity_id": "13",
"topic": "test_2",
"group.id": "mytestgroup",
"consumer-timeout-ms": "10000",
"log_filename": "console_consumer_13.log",
"config_filename": "console_consumer_13.properties"
},
{
"entity_id": "14",
"log_filename": "mirror_maker_14.log",
"mirror_consumer_config_filename": "mirror_consumer_14.properties",
"mirror_producer_config_filename": "mirror_producer_14.properties"
},
{
"entity_id": "15",
"log_filename": "mirror_maker_15.log",
"mirror_consumer_config_filename": "mirror_consumer_15.properties",
"mirror_producer_config_filename": "mirror_producer_15.properties"
}
]
}

View File

@ -119,6 +119,7 @@
{
"entity_id": "10",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",
@ -141,6 +142,7 @@
{
"entity_id": "12",
"new-producer":"true",
"log_filename": "mirror_maker_12.log",
"mirror_consumer_config_filename": "mirror_consumer_12.properties",
"mirror_producer_config_filename": "mirror_producer_12.properties"

View File

@ -119,6 +119,7 @@
{
"entity_id": "10",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",
@ -141,6 +142,7 @@
{
"entity_id": "12",
"new-producer":"true",
"log_filename": "mirror_maker_12.log",
"mirror_consumer_config_filename": "mirror_consumer_12.properties",
"mirror_producer_config_filename": "mirror_producer_12.properties"

View File

@ -120,6 +120,7 @@
{
"entity_id": "10",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "2",
@ -142,12 +143,14 @@
{
"entity_id": "12",
"new-producer":"true",
"log_filename": "mirror_maker_12.log",
"mirror_consumer_config_filename": "mirror_consumer_12.properties",
"mirror_producer_config_filename": "mirror_producer_12.properties"
},
{
"entity_id": "13",
"new-producer":"true",
"log_filename": "mirror_maker_13.log",
"mirror_consumer_config_filename": "mirror_consumer_13.properties",
"mirror_producer_config_filename": "mirror_producer_13.properties"

View File

@ -120,6 +120,7 @@
{
"entity_id": "10",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",
@ -142,12 +143,14 @@
{
"entity_id": "12",
"new-producer":"true",
"log_filename": "mirror_maker_12.log",
"mirror_consumer_config_filename": "mirror_consumer_12.properties",
"mirror_producer_config_filename": "mirror_producer_12.properties"
},
{
"entity_id": "13",
"new-producer":"true",
"log_filename": "mirror_maker_13.log",
"mirror_consumer_config_filename": "mirror_consumer_13.properties",
"mirror_producer_config_filename": "mirror_producer_13.properties"

View File

@ -120,6 +120,7 @@
{
"entity_id": "10",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",
@ -133,6 +134,7 @@
},
{
"entity_id": "11",
"new-producer":"true",
"topic": "test_2",
"threads": "5",
"compression-codec": "1",
@ -164,12 +166,14 @@
{
"entity_id": "14",
"new-producer":"true",
"log_filename": "mirror_maker_14.log",
"mirror_consumer_config_filename": "mirror_consumer_14.properties",
"mirror_producer_config_filename": "mirror_producer_14.properties"
},
{
"entity_id": "15",
"new-producer":"true",
"log_filename": "mirror_maker_15.log",
"mirror_consumer_config_filename": "mirror_consumer_15.properties",
"mirror_producer_config_filename": "mirror_producer_15.properties"

View File

@ -120,6 +120,7 @@
{
"entity_id": "10",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",
@ -133,6 +134,7 @@
},
{
"entity_id": "11",
"new-producer":"true",
"topic": "test_2",
"threads": "5",
"compression-codec": "1",
@ -164,12 +166,14 @@
{
"entity_id": "14",
"new-producer":"true",
"log_filename": "mirror_maker_14.log",
"mirror_consumer_config_filename": "mirror_consumer_14.properties",
"mirror_producer_config_filename": "mirror_producer_14.properties"
},
{
"entity_id": "15",
"new-producer":"true",
"log_filename": "mirror_maker_15.log",
"mirror_consumer_config_filename": "mirror_consumer_15.properties",
"mirror_producer_config_filename": "mirror_producer_15.properties"

View File

@ -363,6 +363,7 @@ class ReplicaBasicTest(ReplicationUtils, SetupUtils):
str(self.testcaseEnv.userDefinedEnvVarDict["backgroundProducerStopped"]) + "]", extra=self.d)
if self.testcaseEnv.userDefinedEnvVarDict["backgroundProducerStopped"]:
time.sleep(1)
self.testcaseEnv.lock.release()
self.logger.info("all producer threads completed", extra=self.d)
break
time.sleep(1)

View File

@ -60,6 +60,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

View File

@ -60,6 +60,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -60,6 +60,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -60,6 +60,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -60,6 +60,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -60,6 +60,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -60,6 +60,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -60,6 +60,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -60,6 +60,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -60,6 +60,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -61,6 +61,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -59,6 +59,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",
@ -72,6 +73,7 @@
},
{
"entity_id": "5",
"new-producer":"true",
"topic": "test_2",
"threads": "5",
"compression-codec": "0",

View File

@ -59,6 +59,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",
@ -72,6 +73,7 @@
},
{
"entity_id": "5",
"new-producer":"true",
"topic": "test_2",
"threads": "5",
"compression-codec": "0",

View File

@ -59,6 +59,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",
@ -72,6 +73,7 @@
},
{
"entity_id": "5",
"new-producer":"true",
"topic": "test_2",
"threads": "5",
"compression-codec": "1",

View File

@ -62,6 +62,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1,test_2",
"threads": "5",
"compression-codec": "1",

View File

@ -63,6 +63,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

View File

@ -63,6 +63,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -63,6 +63,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -63,6 +63,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -63,6 +63,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -63,6 +63,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -63,6 +63,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -63,6 +63,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -63,6 +63,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -63,6 +63,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

View File

@ -63,6 +63,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",
@ -76,6 +77,7 @@
},
{
"entity_id": "5",
"new-producer":"true",
"topic": "test_2",
"threads": "5",
"compression-codec": "0",

View File

@ -63,6 +63,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",
@ -76,6 +77,7 @@
},
{
"entity_id": "5",
"new-producer":"true",
"topic": "test_2",
"threads": "5",
"compression-codec": "0",

View File

@ -63,6 +63,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",
@ -76,6 +77,7 @@
},
{
"entity_id": "5",
"new-producer":"true",
"topic": "test_2",
"threads": "5",
"compression-codec": "1",

View File

@ -67,6 +67,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",
@ -80,6 +81,7 @@
},
{
"entity_id": "5",
"new-producer":"true",
"topic": "test_2",
"threads": "5",
"compression-codec": "0",

View File

@ -67,6 +67,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",
@ -80,6 +81,7 @@
},
{
"entity_id": "5",
"new-producer":"true",
"topic": "test_2",
"threads": "5",
"compression-codec": "0",

View File

@ -67,6 +67,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",
@ -80,6 +81,7 @@
},
{
"entity_id": "5",
"new-producer":"true",
"topic": "test_2",
"threads": "5",
"compression-codec": "1",

View File

@ -67,6 +67,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",
@ -80,6 +81,7 @@
},
{
"entity_id": "5",
"new-producer":"true",
"topic": "test_2",
"threads": "5",
"compression-codec": "1",

View File

@ -65,6 +65,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1,test_2",
"threads": "5",
"compression-codec": "1",

View File

@ -66,6 +66,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",
@ -79,6 +80,7 @@
},
{
"entity_id": "5",
"new-producer":"true",
"topic": "test_2",
"threads": "5",
"compression-codec": "0",

View File

@ -63,6 +63,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",
@ -76,6 +77,7 @@
},
{
"entity_id": "5",
"new-producer":"true",
"topic": "test_2",
"threads": "5",
"compression-codec": "0",

View File

@ -63,6 +63,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",
@ -76,6 +77,7 @@
},
{
"entity_id": "5",
"new-producer":"true",
"topic": "test_2",
"threads": "5",
"compression-codec": "1",

View File

@ -68,6 +68,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1,test_2",
"threads": "5",
"compression-codec": "1",

View File

@ -65,6 +65,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

View File

@ -65,6 +65,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

View File

@ -65,6 +65,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

View File

@ -65,6 +65,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

View File

@ -65,6 +65,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -65,6 +65,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -65,6 +65,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -65,6 +65,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -65,6 +65,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -65,6 +65,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -64,6 +64,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -65,6 +65,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "1",

View File

@ -62,6 +62,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

View File

@ -62,6 +62,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

View File

@ -62,6 +62,7 @@
},
{
"entity_id": "4",
"new-producer":"true",
"topic": "test_1",
"threads": "5",
"compression-codec": "0",

Some files were not shown because too many files have changed in this diff Show More