From 81e41562f3836e95e89e12f215c82b1b2d505381 Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Fri, 24 Apr 2015 09:32:54 +0800 Subject: [PATCH 01/41] Bootstrap Kafka system tests --- .gitignore | 5 + Vagrantfile | 51 +++++- tests/.gitignore | 11 ++ tests/README.md | 44 +++++ tests/__init__.py | 1 + tests/aws/aws-access-keys-commands | 3 + tests/aws/aws-example-Vagrantfile.local | 9 + tests/aws/aws-init.sh | 57 +++++++ tests/services/__init__.py | 0 tests/services/kafka_service.py | 212 ++++++++++++++++++++++++ tests/services/performance.py | 189 +++++++++++++++++++++ tests/services/zookeeper_service.py | 75 +++++++++ tests/templates/kafka.properties | 121 ++++++++++++++ tests/tests/__init__.py | 0 tests/tests/kafka_benchmark_test.py | 193 +++++++++++++++++++++ tests/tests/test.py | 51 ++++++ vagrant/base.sh | 9 + 17 files changed, 1028 insertions(+), 3 deletions(-) create mode 100644 tests/.gitignore create mode 100644 tests/README.md create mode 100644 tests/__init__.py create mode 100644 tests/aws/aws-access-keys-commands create mode 100644 tests/aws/aws-example-Vagrantfile.local create mode 100755 tests/aws/aws-init.sh create mode 100644 tests/services/__init__.py create mode 100644 tests/services/kafka_service.py create mode 100644 tests/services/performance.py create mode 100644 tests/services/zookeeper_service.py create mode 100644 tests/templates/kafka.properties create mode 100644 tests/tests/__init__.py create mode 100644 tests/tests/kafka_benchmark_test.py create mode 100644 tests/tests/test.py diff --git a/.gitignore b/.gitignore index 1f3ba7dbbf524..4aae6e76b96a2 100644 --- a/.gitignore +++ b/.gitignore @@ -29,3 +29,8 @@ config/server-* config/zookeeper-* core/data/* gradle/wrapper/* + +results +tests/results +.ducktape +tests/.ducktape diff --git a/Vagrantfile b/Vagrantfile index 55c67ddda4581..6cd6bc083f1a4 100644 --- a/Vagrantfile +++ b/Vagrantfile @@ -20,16 +20,20 @@ require 'socket' # Vagrantfile API/syntax version. Don't touch unless you know what you're doing! VAGRANTFILE_API_VERSION = "2" +# Mode +mode = "kafka_cluster" + # General config enable_dns = false +ram_megabytes = 1280 +num_workers = 0 # Generic workers that get the code, but don't start any services num_zookeepers = 1 num_brokers = 3 -num_workers = 0 # Generic workers that get the code, but don't start any services -ram_megabytes = 1280 # EC2 ec2_access_key = ENV['AWS_ACCESS_KEY'] ec2_secret_key = ENV['AWS_SECRET_KEY'] +ec2_session_token = ENV['AWS_SESSION_TOKEN'] ec2_keypair_name = nil ec2_keypair_file = nil @@ -49,6 +53,29 @@ if File.exists?(local_config_file) then eval(File.read(local_config_file), binding, "Vagrantfile.local") end +if mode == "test" + num_zookeepers = 0 + num_brokers = 0 +end + +# This is a horrible hack to work around bad interactions between +# vagrant-hostmanager and vagrant-aws/vagrant's implementation. Hostmanager +# wants to update the /etc/hosts entries, but tries to do so even on nodes that +# aren't up (e.g. even when all nodes are stopped and you run vagrant +# destroy). Because of the way the underlying code in vagrant works, it still +# tries to communicate with the node and has to wait for a very long +# timeout. This modifies the update to check for hosts that are not created or +# stopped, skipping the update in that case since it's impossible to update +# nodes in that state. +Object.const_get("VagrantPlugins").const_get("HostManager").const_get("HostsFile").class_eval do + alias_method :old_update_guest, :update_guest + def update_guest(machine) + state_id = machine.state.id + return if state_id == :not_created || state_id == :stopped + old_update_guest(machine) + end +end + # TODO(ksweeney): RAM requirements are not empirical and can probably be significantly lowered. Vagrant.configure(VAGRANTFILE_API_VERSION) do |config| config.hostmanager.enabled = true @@ -84,13 +111,31 @@ Vagrant.configure(VAGRANTFILE_API_VERSION) do |config| override.vm.box = "dummy" override.vm.box_url = "https://github.com/mitchellh/vagrant-aws/raw/master/dummy.box" - override.hostmanager.ignore_private_ip = true + cached_addresses = {} + # Use a custom resolver that SSH's into the machine and finds the IP address + # directly. This lets us get at the private IP address directly, avoiding + # some issues with using the default IP resolver, which uses the public IP + # address. + override.hostmanager.ip_resolver = proc do |vm, resolving_vm| + if !cached_addresses.has_key?(vm.name) + state_id = vm.state.id + if state_id != :not_created && state_id != :stopped && vm.communicate.ready? + vm.communicate.execute("/sbin/ifconfig eth0 | grep 'inet addr' | tail -n 1 | egrep -o '[0-9\.]+' | head -n 1 2>&1") do |type, contents| + cached_addresses[vm.name] = contents.split("\n").first[/(\d+\.\d+\.\d+\.\d+)/, 1] + end + else + cached_addresses[vm.name] = nil + end + end + cached_addresses[vm.name] + end override.ssh.username = ec2_user override.ssh.private_key_path = ec2_keypair_file aws.access_key_id = ec2_access_key aws.secret_access_key = ec2_secret_key + aws.session_token = ec2_session_token aws.keypair_name = ec2_keypair_name aws.region = ec2_region diff --git a/tests/.gitignore b/tests/.gitignore new file mode 100644 index 0000000000000..b218b83c4edee --- /dev/null +++ b/tests/.gitignore @@ -0,0 +1,11 @@ +Vagrantfile.local + +.idea/ + +*.pyc +*.ipynb + +.DS_Store + +.ducktape +results/ diff --git a/tests/README.md b/tests/README.md new file mode 100644 index 0000000000000..dc146b333641a --- /dev/null +++ b/tests/README.md @@ -0,0 +1,44 @@ +System Integration & Performance Testing +======================================== + +This directory contains Kafka system integration and performance tests. +[Ducktape](https://github.com/confluentinc/ducktape) is used to run the tests. + +Ducktape is a distributed testing framework which provides test runner, +result reporter and utilities to pull up and tear down services. It automatically +discovers tests from a directory and generate an HTML report for each run. + +To run the tests: + +1. Build a specific branch of Kafka + + $ cd kafka + $ git checkout $BRANCH + $ gradle + $ ./gradlew jar + +2. Setup a testing cluster. You can use Vagrant to create a cluster of local + VMs or on EC2. Configure your Vagrant setup by creating the file + `Vagrantfile.local` in the directory of your Kafka checkout. At a minimum + , you *MUST* set `mode = "test"` and the value of `num_workers` high enough for + the test you're trying to run. If you run on AWS, you also need to set + enable_dns = true. + +3. Bring up the cluster, making sure you have enough workers. For Vagrant, + use `vagrant up`. If you want to run on AWS, use `vagrant up + --provider=aws --no-parallel`. +4. Install ducktape: + + $ git clone https://github.com/confluentinc/ducktape + $ cd ducktape + $ pip install ducktape +5. Run the system tests using ducktape, you can view results in the `results` + directory. + + $ cd tests + $ ducktape tests +6. To iterate/run again if you made any changes: + + $ cd kafka + $ ./gradlew jar + $ vagrant rsync # Re-syncs build output to cluster diff --git a/tests/__init__.py b/tests/__init__.py new file mode 100644 index 0000000000000..8b137891791fe --- /dev/null +++ b/tests/__init__.py @@ -0,0 +1 @@ + diff --git a/tests/aws/aws-access-keys-commands b/tests/aws/aws-access-keys-commands new file mode 100644 index 0000000000000..d4aa263e89ad7 --- /dev/null +++ b/tests/aws/aws-access-keys-commands @@ -0,0 +1,3 @@ +export AWS_ACCESS_KEY=`curl -s http://169.254.169.254/latest/meta-data/iam/security-credentials/ducttape-master | grep AccessKeyId | awk -F\" '{ print $4 }'` +export AWS_SECRET_KEY=`curl -s http://169.254.169.254/latest/meta-data/iam/security-credentials/ducttape-master | grep SecretAccessKey | awk -F\" '{ print $4 }'` +export AWS_SESSION_TOKEN=`curl -s http://169.254.169.254/latest/meta-data/iam/security-credentials/ducttape-master | grep Token | awk -F\" '{ print $4 }'` diff --git a/tests/aws/aws-example-Vagrantfile.local b/tests/aws/aws-example-Vagrantfile.local new file mode 100644 index 0000000000000..7f9bbd4192dde --- /dev/null +++ b/tests/aws/aws-example-Vagrantfile.local @@ -0,0 +1,9 @@ +ec3_instance_type = "m3.medium" +enable_dns = true +mode = "test" +num_workers = 1 +ec2_keypair_name = +ec2_keypair_file = +ec2_security_groups = ['ducttape-insecure'] +ec2_region = 'us-west-2' +ec2_ami = "ami-29ebb519" diff --git a/tests/aws/aws-init.sh b/tests/aws/aws-init.sh new file mode 100755 index 0000000000000..027aa90748b79 --- /dev/null +++ b/tests/aws/aws-init.sh @@ -0,0 +1,57 @@ +#!/bin/bash + +# This script should be run once on your aws test driver machine before +# attempting to run any ducktape tests + +# Install dependencies +sudo apt-get install -y maven openjdk-6-jdk build-essential \ + ruby-dev zlib1g-dev realpath python-setuptools + +base_dir=`dirname $0`/.. + +if [ -z `which vagrant` ]; then + echo "Installing vagrant..." + wget https://dl.bintray.com/mitchellh/vagrant/vagrant_1.7.2_x86_64.deb + sudo dpkg -i vagrant_1.7.2_x86_64.deb + rm -f vagrant_1.7.2_x86_64.deb +fi + +# Install necessary vagrant plugins +# Note: Do NOT install vagrant-cachier since it doesn't work on AWS and only +# adds log noise +vagrant_plugins="vagrant-aws vagrant-hostmanager" +existing=`vagrant plugin list` +for plugin in $vagrant_plugins; do + echo $existing | grep $plugin > /dev/null + if [ $? != 0 ]; then + vagrant plugin install $plugin + fi +done + +# Create Vagrantfile.local as a convenience +if [ ! -e "$base_dir/Vagrantfile.local" ]; then + cp $base_dir/aws/aws-example-Vagrantfile.local $base_dir/Vagrantfile.local +fi + +gradle="gradle-2.2.1" +if [ -z `which gradle` ] && [ ! -d $base_dir/$gradle ]; then + if [ ! -e $gradle-bin.zip ]; then + wget https://services.gradle.org/distributions/$gradle-bin.zip + fi + unzip $gradle-bin.zip + rm -rf $gradle-bin.zip + mv $gradle $base_dir/$gradle +fi + +# Ensure aws access keys are in the environment when we use a EC2 driver machine +LOCAL_HOSTNAME=$(hostname -d) +if [[ ${LOCAL_HOSTNAME} =~ .*\.compute\.internal ]]; then + grep "AWS ACCESS KEYS" ~/.bashrc > /dev/null + if [ $? != 0 ]; then + echo "# --- AWS ACCESS KEYS ---" >> ~/.bashrc + echo ". `realpath $base_dir/aws/aws-access-keys-commands`" >> ~/.bashrc + echo "# -----------------------" >> ~/.bashrc + source ~/.bashrc + fi +fi + diff --git a/tests/services/__init__.py b/tests/services/__init__.py new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/tests/services/kafka_service.py b/tests/services/kafka_service.py new file mode 100644 index 0000000000000..dfcd5f8356850 --- /dev/null +++ b/tests/services/kafka_service.py @@ -0,0 +1,212 @@ +# Copyright 2014 Confluent Inc. +# +# Licensed 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. + +from ducktape.services.service import Service +import time, re, json + + +class KafkaService(Service): + def __init__(self, service_context, zk, topics=None): + """ + :type service_context ducktape.services.service.ServiceContext + :type zk: ZookeeperService + :type topics: dict + """ + super(KafkaService, self).__init__(service_context) + self.zk = zk + self.topics = topics + + def start(self): + super(KafkaService, self).start() + + # Start all nodes in this Kafka service + for idx, node in enumerate(self.nodes, 1): + self.logger.info("Starting Kafka node %d on %s", idx, node.account.hostname) + self._stop_and_clean(node, allow_fail=True) + self.start_node(node) + + # wait for start up + time.sleep(6) + + # Create topics if necessary + if self.topics is not None: + for topic, topic_cfg in self.topics.items(): + if topic_cfg is None: + topic_cfg = {} + + topic_cfg["topic"] = topic + self.create_topic(topic_cfg) + + def create_topic(self, topic_cfg): + node = self.nodes[0] # any node is fine here + self.logger.info("Creating topic %s with settings %s", topic_cfg["topic"], topic_cfg) + + cmd = "/opt/kafka/bin/kafka-topics.sh --zookeeper %(zk_connect)s --create "\ + "--topic %(topic)s --partitions %(partitions)d --replication-factor %(replication)d" % { + 'zk_connect': self.zk.connect_setting(), + 'topic': topic_cfg.get("topic"), + 'partitions': topic_cfg.get('partitions', 1), + 'replication': topic_cfg.get('replication-factor', 1) + } + + if "configs" in topic_cfg.keys() and topic_cfg["configs"] is not None: + for config_name, config_value in topic_cfg["configs"].items(): + cmd += " --config %s=%s" % (config_name, str(config_value)) + + self.logger.info("Running topic creation command...\n%s" % cmd) + node.account.ssh(cmd) + + time.sleep(1) + self.logger.info("Checking to see if topic was properly created...\n%s" % cmd) + for line in self.describe_topic(topic_cfg["topic"]).split("\n"): + self.logger.info(line) + + def describe_topic(self, topic): + node = self.nodes[0] + cmd = "/opt/kafka/bin/kafka-topics.sh --zookeeper %s --topic %s --describe" % \ + (self.zk.connect_setting(), topic) + output = "" + for line in node.account.ssh_capture(cmd): + output += line + return output + + def verify_reassign_partitions(self, reassignment): + """Run the reassign partitions admin tool in "verify" mode + """ + node = self.nodes[0] + json_file = "/tmp/" + str(time.time()) + "_reassign.json" + + # reassignment to json + json_str = json.dumps(reassignment) + json_str = json.dumps(json_str) + + # create command + cmd = "echo %s > %s && " % (json_str, json_file) + cmd += "/opt/kafka/bin/kafka-reassign-partitions.sh "\ + "--zookeeper %(zk_connect)s "\ + "--reassignment-json-file %(reassignment_file)s "\ + "--verify" % {'zk_connect': self.zk.connect_setting(), + 'reassignment_file': json_file} + cmd += " && sleep 1 && rm -f %s" % json_file + + # send command + self.logger.info("Verifying parition reassignment...") + self.logger.debug(cmd) + output = "" + for line in node.account.ssh_capture(cmd): + output += line + + self.logger.debug(output) + + if re.match(".*is in progress.*", output) is not None: + return False + + return True + + def execute_reassign_partitions(self, reassignment): + """Run the reassign partitions admin tool in "verify" mode + """ + node = self.nodes[0] + json_file = "/tmp/" + str(time.time()) + "_reassign.json" + + # reassignment to json + json_str = json.dumps(reassignment) + json_str = json.dumps(json_str) + + # create command + cmd = "echo %s > %s && " % (json_str, json_file) + cmd += "/opt/kafka/bin/kafka-reassign-partitions.sh "\ + "--zookeeper %(zk_connect)s "\ + "--reassignment-json-file %(reassignment_file)s "\ + "--execute" % {'zk_connect': self.zk.connect_setting(), + 'reassignment_file': json_file} + cmd += " && sleep 1 && rm -f %s" % json_file + + # send command + self.logger.info("Executing parition reassignment...") + self.logger.debug(cmd) + output = "" + for line in node.account.ssh_capture(cmd): + output += line + + self.logger.debug("Verify partition reassignment:") + self.logger.debug(output) + + def stop(self): + """If the service left any running processes or data, clean them up.""" + super(KafkaService, self).stop() + + for idx, node in enumerate(self.nodes, 1): + self.logger.info("Stopping %s node %d on %s" % (type(self).__name__, idx, node.account.hostname)) + self._stop_and_clean(node, allow_fail=True) + node.free() + + def _stop_and_clean(self, node, allow_fail=False): + node.account.ssh("/opt/kafka/bin/kafka-server-stop.sh", allow_fail=allow_fail) + time.sleep(5) # the stop script doesn't wait + node.account.ssh("rm -rf /mnt/kafka-logs /mnt/kafka.properties /mnt/kafka.log") + + def stop_node(self, node, clean_shutdown=True, allow_fail=True): + node.account.kill_process("kafka", clean_shutdown, allow_fail) + + def start_node(self, node, config=None): + if config is None: + template = open('templates/kafka.properties').read() + template_params = { + 'broker_id': self.idx(node), + 'hostname': node.account.hostname, + 'zk_connect': self.zk.connect_setting() + } + + config = template % template_params + + node.account.create_file("/mnt/kafka.properties", config) + cmd = "/opt/kafka/bin/kafka-server-start.sh /mnt/kafka.properties 1>> /mnt/kafka.log 2>> /mnt/kafka.log &" + self.logger.debug("Attempting to start KafkaService on %s with command: %s" % (str(node.account), cmd)) + node.account.ssh(cmd) + + def restart_node(self, node, wait_sec=0, clean_shutdown=True): + self.stop_node(node, clean_shutdown, allow_fail=True) + time.sleep(wait_sec) + self.start_node(node) + + def get_leader_node(self, topic, partition=0): + """ Get the leader replica for the given topic and partition. + """ + cmd = "/opt/kafka/bin/kafka-run-class.sh kafka.tools.ZooKeeperMainWrapper -server %s " \ + % self.zk.connect_setting() + cmd += "get /brokers/topics/%s/partitions/%d/state" % (topic, partition) + self.logger.debug(cmd) + + node = self.nodes[0] + self.logger.debug("Querying zookeeper to find leader replica for topic %s: \n%s" % (cmd, topic)) + partition_state = None + for line in node.account.ssh_capture(cmd): + match = re.match("^({.+})$", line) + if match is not None: + partition_state = match.groups()[0] + break + + if partition_state is None: + raise Exception("Error finding partition state for topic %s and partition %d." % (topic, partition)) + + partition_state = json.loads(partition_state) + self.logger.info(partition_state) + + leader_idx = int(partition_state["leader"]) + self.logger.info("Leader for topic %s and partition %d is now: %d" % (topic, partition, leader_idx)) + return self.get_node(leader_idx) + + def bootstrap_servers(self): + return ','.join([node.account.hostname + ":9092" for node in self.nodes]) \ No newline at end of file diff --git a/tests/services/performance.py b/tests/services/performance.py new file mode 100644 index 0000000000000..9478b5ef956af --- /dev/null +++ b/tests/services/performance.py @@ -0,0 +1,189 @@ +# Copyright 2014 Confluent Inc. +# +# Licensed 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. + +from ducktape.services.service import Service +import threading + + +class PerformanceService(Service): + def __init__(self, service_context): + super(PerformanceService, self).__init__(service_context) + + def start(self): + super(PerformanceService, self).start() + self.worker_threads = [] + self.results = [None] * len(self.nodes) + self.stats = [[] for x in range(len(self.nodes))] + for idx,node in enumerate(self.nodes,1): + self.logger.info("Running %s node %d on %s", self.__class__.__name__, idx, node.account.hostname) + worker = threading.Thread( + name=self.__class__.__name__ + "-worker-" + str(idx), + target=self._worker, + args=(idx,node) + ) + worker.daemon = True + worker.start() + self.worker_threads.append(worker) + + def wait(self): + super(PerformanceService, self).wait() + for idx,worker in enumerate(self.worker_threads,1): + self.logger.debug("Waiting for %s worker %d to finish", self.__class__.__name__, idx) + worker.join() + self.worker_threads = None + + def stop(self): + super(PerformanceService, self).stop() + assert self.worker_threads is None, "%s.stop should only be called after wait" % self.__class__.__name__ + for idx,node in enumerate(self.nodes,1): + self.logger.debug("Stopping %s node %d on %s", self.__class__.__name__, idx, node.account.hostname) + node.free() + + +class ProducerPerformanceService(PerformanceService): + def __init__(self, service_context, kafka, topic, num_records, record_size, throughput, settings={}, intermediate_stats=False): + super(ProducerPerformanceService, self).__init__(service_context) + self.kafka = kafka + self.args = { + 'topic': topic, + 'num_records': num_records, + 'record_size': record_size, + 'throughput': throughput + } + self.settings = settings + self.intermediate_stats = intermediate_stats + + def _worker(self, idx, node): + args = self.args.copy() + args.update({'bootstrap_servers': self.kafka.bootstrap_servers()}) + cmd = "/opt/kafka/bin/kafka-run-class.sh org.apache.kafka.clients.tools.ProducerPerformance "\ + "%(topic)s %(num_records)d %(record_size)d %(throughput)d bootstrap.servers=%(bootstrap_servers)s" % args + + for key,value in self.settings.items(): + cmd += " %s=%s" % (str(key), str(value)) + self.logger.debug("Producer performance %d command: %s", idx, cmd) + def parse_stats(line): + parts = line.split(',') + return { + 'records': int(parts[0].split()[0]), + 'records_per_sec': float(parts[1].split()[0]), + 'mbps': float(parts[1].split('(')[1].split()[0]), + 'latency_avg_ms': float(parts[2].split()[0]), + 'latency_max_ms': float(parts[3].split()[0]), + 'latency_50th_ms': float(parts[4].split()[0]), + 'latency_95th_ms': float(parts[5].split()[0]), + 'latency_99th_ms': float(parts[6].split()[0]), + 'latency_999th_ms': float(parts[7].split()[0]), + } + last = None + for line in node.account.ssh_capture(cmd): + self.logger.debug("Producer performance %d: %s", idx, line.strip()) + if self.intermediate_stats: + try: + self.stats[idx-1].append(parse_stats(line)) + except: + # Sometimes there are extraneous log messages + pass + last = line + try: + self.results[idx-1] = parse_stats(last) + except: + self.logger.error("Bad last line: %s", last) + + +class ConsumerPerformanceService(PerformanceService): + def __init__(self, service_context, kafka, topic, num_records, throughput, threads=1, settings={}): + super(ConsumerPerformanceService, self).__init__(service_context) + self.kafka = kafka + self.args = { + 'topic': topic, + 'num_records': num_records, + 'throughput': throughput, + 'threads': threads, + } + self.settings = settings + + def _worker(self, idx, node): + args = self.args.copy() + args.update({'zk_connect': self.kafka.zk.connect_setting()}) + cmd = "/opt/kafka/bin/kafka-consumer-perf-test.sh "\ + "--topic %(topic)s --messages %(num_records)d --zookeeper %(zk_connect)s" % args + for key,value in self.settings.items(): + cmd += " %s=%s" % (str(key), str(value)) + self.logger.debug("Consumer performance %d command: %s", idx, cmd) + last = None + for line in node.account.ssh_capture(cmd): + self.logger.debug("Consumer performance %d: %s", idx, line.strip()) + last = line + # Parse and save the last line's information + parts = last.split(',') + self.results[idx-1] = { + 'total_mb': float(parts[3]), + 'mbps': float(parts[4]), + 'records_per_sec': float(parts[6]), + } + + +class EndToEndLatencyService(PerformanceService): + def __init__(self, service_context, kafka, topic, num_records, consumer_fetch_max_wait=100, acks=1): + super(EndToEndLatencyService, self).__init__(service_context) + self.kafka = kafka + self.args = { + 'topic': topic, + 'num_records': num_records, + 'consumer_fetch_max_wait': consumer_fetch_max_wait, + 'acks': acks + } + + def _worker(self, idx, node): + args = self.args.copy() + args.update({ + 'zk_connect': self.kafka.zk.connect_setting(), + 'bootstrap_servers': self.kafka.bootstrap_servers(), + }) + cmd = "/opt/kafka/bin/kafka-run-class.sh kafka.tools.TestEndToEndLatency "\ + "%(bootstrap_servers)s %(zk_connect)s %(topic)s %(num_records)d "\ + "%(consumer_fetch_max_wait)d %(acks)d" % args + self.logger.debug("End-to-end latency %d command: %s", idx, cmd) + results = {} + for line in node.account.ssh_capture(cmd): + self.logger.debug("End-to-end latency %d: %s", idx, line.strip()) + if line.startswith("Avg latency:"): + results['latency_avg_ms'] = float(line.split()[2]) + if line.startswith("Percentiles"): + results['latency_50th_ms'] = float(line.split()[3][:-1]) + results['latency_99th_ms'] = float(line.split()[6][:-1]) + results['latency_999th_ms'] = float(line.split()[9]) + self.results[idx-1] = results + + +def parse_performance_output(summary): + parts = summary.split(',') + results = { + 'records': int(parts[0].split()[0]), + 'records_per_sec': float(parts[1].split()[0]), + 'mbps': float(parts[1].split('(')[1].split()[0]), + 'latency_avg_ms': float(parts[2].split()[0]), + 'latency_max_ms': float(parts[3].split()[0]), + 'latency_50th_ms': float(parts[4].split()[0]), + 'latency_95th_ms': float(parts[5].split()[0]), + 'latency_99th_ms': float(parts[6].split()[0]), + 'latency_999th_ms': float(parts[7].split()[0]), + } + # To provide compatibility with ConsumerPerformanceService + results['total_mb'] = results['mbps'] * (results['records'] / results['records_per_sec']) + results['rate_mbps'] = results['mbps'] + results['rate_mps'] = results['records_per_sec'] + + return results diff --git a/tests/services/zookeeper_service.py b/tests/services/zookeeper_service.py new file mode 100644 index 0000000000000..efebe848257fe --- /dev/null +++ b/tests/services/zookeeper_service.py @@ -0,0 +1,75 @@ +# Copyright 2014 Confluent Inc. +# +# Licensed 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. + +from ducktape.services.service import Service +import time + + +class ZookeeperService(Service): + def __init__(self, service_context): + """ + :type service_context ducktape.services.service.ServiceContext + """ + super(ZookeeperService, self).__init__(service_context) + self.logs = {"zk_log": "/mnt/zk.log"} + + def start(self): + super(ZookeeperService, self).start() + config = """ +dataDir=/mnt/zookeeper +clientPort=2181 +maxClientCnxns=0 +initLimit=5 +syncLimit=2 +quorumListenOnAllIPs=true +""" + for idx, node in enumerate(self.nodes, 1): + template_params = { 'idx': idx, 'host': node.account.hostname } + config += "server.%(idx)d=%(host)s:2888:3888\n" % template_params + + for idx, node in enumerate(self.nodes, 1): + self.logger.info("Starting ZK node %d on %s", idx, node.account.hostname) + self._stop_and_clean(node, allow_fail=True) + node.account.ssh("mkdir -p /mnt/zookeeper") + node.account.ssh("echo %d > /mnt/zookeeper/myid" % idx) + node.account.create_file("/mnt/zookeeper.properties", config) + node.account.ssh( + "/opt/kafka/bin/zookeeper-server-start.sh /mnt/zookeeper.properties 1>> %(zk_log)s 2>> %(zk_log)s &" + % self.logs) + time.sleep(5) # give it some time to start + + def stop_node(self, node, allow_fail=True): + idx = self.idx(node) + self.logger.info("Stopping %s node %d on %s" % (type(self).__name__, idx, node.account.hostname)) + node.account.ssh("ps ax | grep -i 'zookeeper' | grep -v grep | awk '{print $1}' | xargs kill -SIGTERM", + allow_fail=allow_fail) + + def clean_node(self, node, allow_fail=True): + node.account.ssh("rm -rf /mnt/zookeeper /mnt/zookeeper.properties /mnt/zk.log", allow_fail=allow_fail) + + def stop(self): + """If the service left any running processes or data, clean them up.""" + super(ZookeeperService, self).stop() + + for idx, node in enumerate(self.nodes, 1): + self.stop_node(node, allow_fail=False) + self.clean_node(node) + node.free() + + def _stop_and_clean(self, node, allow_fail=False): + self.stop_node(node, allow_fail) + self.clean_node(node, allow_fail) + + def connect_setting(self): + return ','.join([node.account.hostname + ':2181' for node in self.nodes]) diff --git a/tests/templates/kafka.properties b/tests/templates/kafka.properties new file mode 100644 index 0000000000000..fe0636e045e2a --- /dev/null +++ b/tests/templates/kafka.properties @@ -0,0 +1,121 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +############################# Server Basics ############################# + +# The id of the broker. This must be set to a unique integer for each broker. +broker.id=%(broker_id)d + +############################# Socket Server Settings ############################# + +# The port the socket server listens on +port=9092 + +# Hostname the broker will bind to. If not set, the server will bind to all interfaces +#host.name=localhost + +# Hostname the broker will advertise to producers and consumers. If not set, it uses the +# value for "host.name" if configured. Otherwise, it will use the value returned from +# java.net.InetAddress.getCanonicalHostName(). +advertised.host.name=%(hostname)s + +# The port to publish to ZooKeeper for clients to use. If this is not set, +# it will publish the same port that the broker binds to. +#advertised.port= + +# The number of threads handling network requests +num.network.threads=3 + +# The number of threads doing disk I/O +num.io.threads=8 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=102400 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=65536 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# A comma seperated list of directories under which to store log files +log.dirs=/mnt/kafka-logs + +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. +num.partitions=1 + +# The number of threads per data directory to be used for log recovery at startup and flushing at shutdown. +# This value is recommended to be increased for installations with data dirs located in RAID array. +num.recovery.threads.per.data.dir=1 + +############################# Log Flush Policy ############################# + +# Messages are immediately written to the filesystem but by default we only fsync() to sync +# the OS cache lazily. The following configurations control the flush of data to disk. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data may be lost if you are not using replication. +# 2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as there will be a lot of data to flush. +# 3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to exceessive seeks. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +#log.flush.interval.messages=10000 + +# The maximum amount of time a message can sit in a log before we force a flush +#log.flush.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining +# segments don't drop below log.retention.bytes. +#log.retention.bytes=1073741824 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +log.segment.bytes=1073741824 + +# The interval at which log segments are checked to see if they can be deleted according +# to the retention policies +log.retention.check.interval.ms=300000 + +# By default the log cleaner is disabled and the log retention policy will default to just delete segments after their retention expires. +# If log.cleaner.enable=true is set the cleaner will be enabled and individual logs can then be marked for log compaction. +log.cleaner.enable=false + +############################# Zookeeper ############################# + +# Zookeeper connection string (see zookeeper docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect=%(zk_connect)s + +# Timeout in ms for connecting to zookeeper +zookeeper.connection.timeout.ms=2000 diff --git a/tests/tests/__init__.py b/tests/tests/__init__.py new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/tests/tests/kafka_benchmark_test.py b/tests/tests/kafka_benchmark_test.py new file mode 100644 index 0000000000000..5ea0913c054e0 --- /dev/null +++ b/tests/tests/kafka_benchmark_test.py @@ -0,0 +1,193 @@ +# Copyright 2014 Confluent Inc. +# +# Licensed 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. + +from ducktape.services.service import Service + +from tests.test import KafkaTest +from services.performance import ProducerPerformanceService, ConsumerPerformanceService, \ + EndToEndLatencyService + + +class KafkaBenchmark(KafkaTest): + '''A benchmark of Kafka producer/consumer performance. This replicates the test + run here: + https://engineering.linkedin.com/kafka/benchmarking-apache-kafka-2-million-writes-second-three-cheap-machines + ''' + def __init__(self, test_context): + super(KafkaBenchmark, self).__init__(test_context, num_zk=1, num_brokers=3, topics={ + 'test-rep-one' : { 'partitions': 6, 'replication-factor': 1 }, + 'test-rep-three' : { 'partitions': 6, 'replication-factor': 3 } + }) + + def run(self): + msgs_default = 50000000 + msgs_large = 100000000 + msg_size_default = 100 + batch_size = 8*1024 + buffer_memory = 64*1024*1024 + msg_sizes = [10, 100, 1000, 10000, 100000] + target_data_size = 1024*1024*1024 + target_data_size_gb = target_data_size/float(1024*1024*1024) + # These settings will work in the default local Vagrant VMs, useful for testing + if False: + msgs_default = 1000000 + msgs_large = 10000000 + msg_size_default = 100 + batch_size = 8*1024 + buffer_memory = 64*1024*1024 + msg_sizes = [10, 100, 1000, 10000, 100000] + target_data_size = 128*1024*1024 + target_data_size_gb = target_data_size/float(1024*1024*1024) + + # PRODUCER TESTS + + self.logger.info("BENCHMARK: Single producer, no replication") + single_no_rep = ProducerPerformanceService( + self.service_context(1), self.kafka, + topic="test-rep-one", num_records=msgs_default, record_size=msg_size_default, throughput=-1, + settings={'acks':1, 'batch.size':batch_size, 'buffer.memory':buffer_memory} + ) + single_no_rep.run() + + self.logger.info("BENCHMARK: Single producer, async 3x replication") + single_rep_async = ProducerPerformanceService( + self.service_context(1), self.kafka, + topic="test-rep-three", num_records=msgs_default, record_size=msg_size_default, throughput=-1, + settings={'acks':1, 'batch.size':batch_size, 'buffer.memory':buffer_memory} + ) + single_rep_async.run() + + self.logger.info("BENCHMARK: Single producer, sync 3x replication") + single_rep_sync = ProducerPerformanceService( + self.service_context(1), self.kafka, + topic="test-rep-three", num_records=msgs_default, record_size=msg_size_default, throughput=-1, + settings={'acks':-1, 'batch.size':batch_size, 'buffer.memory':buffer_memory} + ) + single_rep_sync.run() + + self.logger.info("BENCHMARK: Three producers, async 3x replication") + three_rep_async = ProducerPerformanceService( + self.service_context(3), self.kafka, + topic="test-rep-three", num_records=msgs_default, record_size=msg_size_default, throughput=-1, + settings={'acks':1, 'batch.size':batch_size, 'buffer.memory':buffer_memory} + ) + three_rep_async.run() + + + msg_size_perf = {} + for msg_size in msg_sizes: + self.logger.info("BENCHMARK: Message size %d (%f GB total, single producer, async 3x replication)", msg_size, target_data_size_gb) + # Always generate the same total amount of data + nrecords = int(target_data_size / msg_size) + perf = ProducerPerformanceService( + self.service_context(1), self.kafka, + topic="test-rep-three", num_records=nrecords, record_size=msg_size, throughput=-1, + settings={'acks':1, 'batch.size':batch_size, 'buffer.memory':buffer_memory} + ) + perf.run() + msg_size_perf[msg_size] = perf + + # CONSUMER TESTS + + # All consumer tests use the messages from the first benchmark, so + # they'll get messages of the default message size + self.logger.info("BENCHMARK: Single consumer") + single_consumer = ConsumerPerformanceService( + self.service_context(1), self.kafka, + topic="test-rep-three", num_records=msgs_default, throughput=-1, threads=1 + ) + single_consumer.run() + + self.logger.info("BENCHMARK: Three consumers") + three_consumers = ConsumerPerformanceService( + self.service_context(3), self.kafka, + topic="test-rep-three", num_records=msgs_default, throughput=-1, threads=1 + ) + three_consumers.run() + + # PRODUCER + CONSUMER TEST + self.logger.info("BENCHMARK: Producer + Consumer") + pc_producer = ProducerPerformanceService( + self.service_context(1), self.kafka, + topic="test-rep-three", num_records=msgs_default, record_size=msg_size_default, throughput=-1, + settings={'acks':1, 'batch.size':batch_size, 'buffer.memory':buffer_memory} + ) + pc_consumer = ConsumerPerformanceService( + self.service_context(1), self.kafka, + topic="test-rep-three", num_records=msgs_default, throughput=-1, threads=1 + ) + Service.run_parallel(pc_producer, pc_consumer) + + # END TO END LATENCY TEST + self.logger.info("BENCHMARK: End to end latency") + e2e_latency = EndToEndLatencyService( + self.service_context(1), self.kafka, + topic="test-rep-three", num_records=10000 + ) + e2e_latency.run() + + + # LONG TERM THROUGHPUT TEST + + # Because of how much space this ends up using, we clear out the + # existing cluster to start from a clean slate. This also keeps us from + # running out of space with limited disk space. + self.tearDown() + self.setUp() + self.logger.info("BENCHMARK: Long production") + throughput_perf = ProducerPerformanceService( + self.service_context(1), self.kafka, + topic="test-rep-three", num_records=msgs_large, record_size=msg_size_default, throughput=-1, + settings={'acks':1, 'batch.size':batch_size, 'buffer.memory':buffer_memory}, + intermediate_stats=True + ) + throughput_perf.run() + + # Summarize, extracting just the key info. With multiple + # producers/consumers, we display the aggregate value + def throughput(perf): + aggregate_rate = sum([r['records_per_sec'] for r in perf.results]) + aggregate_mbps = sum([r['mbps'] for r in perf.results]) + return "%f rec/sec (%f MB/s)" % (aggregate_rate, aggregate_mbps) + self.logger.info("=================") + self.logger.info("BENCHMARK RESULTS") + self.logger.info("=================") + self.logger.info("Single producer, no replication: %s", throughput(single_no_rep)) + self.logger.info("Single producer, async 3x replication: %s", throughput(single_rep_async)) + self.logger.info("Single producer, sync 3x replication: %s", throughput(single_rep_sync)) + self.logger.info("Three producers, async 3x replication: %s", throughput(three_rep_async)) + self.logger.info("Message size:") + for msg_size in msg_sizes: + self.logger.info(" %d: %s", msg_size, throughput(msg_size_perf[msg_size])) + self.logger.info("Throughput over long run, data > memory:") + # FIXME we should be generating a graph too + # Try to break it into 5 blocks, but fall back to a smaller number if + # there aren't even 5 elements + block_size = max(len(throughput_perf.stats[0]) / 5, 1) + nblocks = len(throughput_perf.stats[0]) / block_size + for i in range(nblocks): + subset = throughput_perf.stats[0][i*block_size:min((i+1)*block_size,len(throughput_perf.stats[0]))] + if len(subset) == 0: + self.logger.info(" Time block %d: (empty)", i) + else: + self.logger.info(" Time block %d: %f rec/sec (%f MB/s)", i, + sum([stat['records_per_sec'] for stat in subset])/float(len(subset)), + sum([stat['mbps'] for stat in subset])/float(len(subset)) + ) + self.logger.info("Single consumer: %s", throughput(single_consumer)) + self.logger.info("Three consumers: %s", throughput(three_consumers)) + self.logger.info("Producer + consumer:") + self.logger.info(" Producer: %s", throughput(pc_producer)) + self.logger.info(" Consumer: %s", throughput(pc_producer)) + self.logger.info("End-to-end latency: median %f ms, 99%% %f ms, 99.9%% %f ms", e2e_latency.results[0]['latency_50th_ms'], e2e_latency.results[0]['latency_99th_ms'], e2e_latency.results[0]['latency_999th_ms']) diff --git a/tests/tests/test.py b/tests/tests/test.py new file mode 100644 index 0000000000000..3ac511f1e5880 --- /dev/null +++ b/tests/tests/test.py @@ -0,0 +1,51 @@ +# Copyright 2014 Confluent Inc. +# +# Licensed 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. + +from ducktape.tests.test import Test +from ducktape.services.service import ServiceContext + +from services.zookeeper_service import ZookeeperService +from services.kafka_service import KafkaService + + +class KafkaTest(Test): + """ + Helper class that managest setting up a Kafka cluster. Use this if the + default settings for Kafka are sufficient for your test; any customization + needs to be done manually. Your run() method should call tearDown and + setUp. The Zookeeper and Kafka services are available as the fields + KafkaTest.zk and KafkaTest.kafka. + + + """ + def __init__(self, test_context, num_zk, num_brokers, topics=None): + super(KafkaTest, self).__init__(test_context) + self.num_zk = num_zk + self.num_brokers = num_brokers + self.topics = topics + + def min_cluster_size(self): + return self.num_zk + self.num_brokers + + def setUp(self): + self.zk = ZookeeperService(ServiceContext(self.cluster, self.num_zk, self.logger)) + self.kafka = KafkaService( + ServiceContext(self.cluster, self.num_brokers, self.logger), + self.zk, topics=self.topics) + self.zk.start() + self.kafka.start() + + def tearDown(self): + self.kafka.stop() + self.zk.stop() diff --git a/vagrant/base.sh b/vagrant/base.sh index 6f28dfed67877..133f10a95622c 100644 --- a/vagrant/base.sh +++ b/vagrant/base.sh @@ -41,3 +41,12 @@ chmod a+rw /opt if [ ! -e /opt/kafka ]; then ln -s /vagrant /opt/kafka fi + +# For EC2 nodes, we want to use /mnt, which should have the local disk. On local +# VMs, we can just create it if it doesn't exist and use it like we'd use +# /tmp. Eventually, we'd like to also support more directories, e.g. when EC2 +# instances have multiple local disks. +if [ ! -e /mnt ]; then + mkdir /mnt +fi +chmod a+rwx /mnt From a2789885806f98dcd1fd58edc9a10a30e4bd314c Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Tue, 26 May 2015 15:21:23 -0700 Subject: [PATCH 02/41] fixed typos --- core/src/main/scala/kafka/tools/ProducerPerformance.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/tools/ProducerPerformance.scala b/core/src/main/scala/kafka/tools/ProducerPerformance.scala index bc25cd2f371c1..fb261a96eef62 100644 --- a/core/src/main/scala/kafka/tools/ProducerPerformance.scala +++ b/core/src/main/scala/kafka/tools/ProducerPerformance.scala @@ -115,9 +115,9 @@ object ProducerPerformance extends Logging { .defaultsTo(0) val csvMetricsReporterEnabledOpt = parser.accepts("csv-reporter-enabled", "If set, the CSV metrics reporter will be enabled") val metricsDirectoryOpt = parser.accepts("metrics-dir", "If csv-reporter-enable is set, and this parameter is" + - "set, the csv metrics will be outputed here") + "set, the csv metrics will be output here") .withRequiredArg - .describedAs("metrics dictory") + .describedAs("metrics directory") .ofType(classOf[java.lang.String]) val useNewProducerOpt = parser.accepts("new-producer", "Use the new producer implementation.") From 07cd1c66a952ee29fc3c8e85464acb43a6981b8a Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Tue, 26 May 2015 15:22:14 -0700 Subject: [PATCH 03/41] Added simple producer which prints status of produced messages to stdout. --- .../scala/kafka/tools/VerboseProducer.java | 166 ++++++++++++++++++ 1 file changed, 166 insertions(+) create mode 100644 core/src/main/scala/kafka/tools/VerboseProducer.java diff --git a/core/src/main/scala/kafka/tools/VerboseProducer.java b/core/src/main/scala/kafka/tools/VerboseProducer.java new file mode 100644 index 0000000000000..b6ceef2437ed6 --- /dev/null +++ b/core/src/main/scala/kafka/tools/VerboseProducer.java @@ -0,0 +1,166 @@ +package kafka.tools; + +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +import joptsimple.ArgumentAcceptingOptionSpec; +import joptsimple.OptionParser; +import joptsimple.OptionSet; +import joptsimple.OptionSpec; +import joptsimple.OptionSpecBuilder; +import kafka.utils.CommandLineUtils; + +public class VerboseProducer { + OptionParser commandLineParser; + Map> commandLineOptions = new HashMap>(); + + String topic; + String sync; + private Properties producerProps = new Properties(); + private Producer producer; + + public VerboseProducer(String[] args) throws IOException { + this.configureParser(); + this.configure(args); + this.producer = new KafkaProducer(producerProps); + } + + /** Set up the command-line options. */ + private void configureParser() { + this.commandLineParser = new OptionParser(); + ArgumentAcceptingOptionSpec topicOpt = commandLineParser.accepts("topic", "REQUIRED: The topic id to produce messages to.") + .withRequiredArg() + .describedAs("topic") + .ofType(String.class); + commandLineOptions.put("topic", topicOpt); + + ArgumentAcceptingOptionSpec brokerListOpt = commandLineParser.accepts("broker-list", "REQUIRED: The broker list string in the form HOST1:PORT1,HOST2:PORT2.") + .withRequiredArg() + .describedAs("broker-list") + .ofType(String.class); + commandLineOptions.put("broker-list", brokerListOpt); + + // syncOpt = parser.accepts("sync", "If set message send requests to the brokers are synchronously, one at a time as they arrive.") +// val producerRequestRequiredAcks = options.valueOf(producerRequestRequiredAcksOpt).intValue() + + + + OptionSpecBuilder helpOpt + = commandLineParser.accepts("help", "Print this message."); + commandLineOptions.put("help", helpOpt); + } + + /** Validate command-line arguments and parse into properties object. */ + public void configure(String[] args) throws IOException { + + OptionSpec[] requiredArgs = new OptionSpec[]{commandLineOptions.get("topic"), + commandLineOptions.get("broker-list")}; + if(args.length == 0) { + CommandLineUtils.printUsageAndDie( + commandLineParser, "Read data from standard input and publish it to Kafka."); + } + + // Parse and validate + OptionSet options = commandLineParser.parse(args); + if (options.has(commandLineOptions.get("help"))) { + commandLineParser.printHelpOn(System.out); + System.exit(0); + } + checkRequiredArgs(commandLineParser, options, requiredArgs); + + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, options.valueOf("broker-list")); + this.topic = (String) options.valueOf("topic"); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.StringSerializer"); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.StringSerializer"); + producerProps.put(ProducerConfig.ACKS_CONFIG, "-1"); + + // No producer retries + producerProps.put("retries", "0"); + } + + private static void checkRequiredArgs( + OptionParser parser, OptionSet options, OptionSpec[] required) throws IOException + { + for (OptionSpec arg : required) { + if (!options.has(arg)) { + System.err.println("Missing required argument \"" + arg + "\""); + parser.printHelpOn(System.err); + System.exit(1); + } + } + } + + /** + * Produce a message with given value and no key. + */ + public void send(String value) { + ProducerRecord record = new ProducerRecord(topic, value); + producer.send(record, new PrintInfoCallback(value)); + } + + /** Need to close the producer to flush any remaining messages if we're in async mode. */ + public void close() { + producer.close(); + } + + /** + * Return JSON string encapsulating basic information about the exception, as well + * as the value which triggered the exception. + */ + String errorString(Exception e, String value) { + return "{\"producer\": \"VerboseProducer\", " + + "\"exception\": \"" + e.getClass() + "\"," + + "\"message\": \"" + e.getMessage() + "\"," + + "\"value\": \"" + value + "\"" + + "}"; + + } + + String successString(String value) { + return "{\"producer\": \"VerboseProducer\", " + + "\"value\": \"" + value + "\"" + + "}"; + } + + /** + * Callback which prints errors to stdout when the producer fails to send. + */ + private class PrintInfoCallback implements Callback { + private String value; + + PrintInfoCallback(String value) { + this.value = value; + } + + public void onCompletion(RecordMetadata recordMetadata, Exception e) { + if (e == null) { + System.out.println(successString(this.value)); + } + else { + System.out.println(errorString(e, this.value)); + } + } + } + + public static void main(String[] args) throws IOException { + + VerboseProducer producer = new VerboseProducer(args); + + for (int i = 0; i < 20; i++) { + producer.send(String.format("%d", i)); + } + + producer.close(); + } +} \ No newline at end of file From da94b8cbe79e6634cc32fbe8f6deb25388923029 Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Wed, 27 May 2015 14:07:20 -0700 Subject: [PATCH 04/41] Added number of messages option. --- .../scala/kafka/tools/VerboseProducer.java | 27 ++++++++++++------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/kafka/tools/VerboseProducer.java b/core/src/main/scala/kafka/tools/VerboseProducer.java index b6ceef2437ed6..32cc3c32c72b4 100644 --- a/core/src/main/scala/kafka/tools/VerboseProducer.java +++ b/core/src/main/scala/kafka/tools/VerboseProducer.java @@ -27,10 +27,11 @@ public class VerboseProducer { String sync; private Properties producerProps = new Properties(); private Producer producer; + private int numMessages; public VerboseProducer(String[] args) throws IOException { this.configureParser(); - this.configure(args); + this.parseCommandLineArgs(args); this.producer = new KafkaProducer(producerProps); } @@ -49,36 +50,44 @@ private void configureParser() { .ofType(String.class); commandLineOptions.put("broker-list", brokerListOpt); + + ArgumentAcceptingOptionSpec numMessagesOpt = commandLineParser.accepts("num-messages", "REQUIRED: The number of messages to produce.") + .withRequiredArg() + .describedAs("num-messages") + .ofType(String.class); + commandLineOptions.put("num-messages", numMessagesOpt); + // syncOpt = parser.accepts("sync", "If set message send requests to the brokers are synchronously, one at a time as they arrive.") // val producerRequestRequiredAcks = options.valueOf(producerRequestRequiredAcksOpt).intValue() - - + OptionSpecBuilder helpOpt = commandLineParser.accepts("help", "Print this message."); commandLineOptions.put("help", helpOpt); } /** Validate command-line arguments and parse into properties object. */ - public void configure(String[] args) throws IOException { + public void parseCommandLineArgs(String[] args) throws IOException { OptionSpec[] requiredArgs = new OptionSpec[]{commandLineOptions.get("topic"), - commandLineOptions.get("broker-list")}; + commandLineOptions.get("broker-list"), + commandLineOptions.get("num-messages")}; if(args.length == 0) { CommandLineUtils.printUsageAndDie( commandLineParser, "Read data from standard input and publish it to Kafka."); } - // Parse and validate OptionSet options = commandLineParser.parse(args); if (options.has(commandLineOptions.get("help"))) { commandLineParser.printHelpOn(System.out); System.exit(0); } checkRequiredArgs(commandLineParser, options, requiredArgs); + + this.numMessages = Integer.parseInt((String) options.valueOf("num-messages")); + this.topic = (String) options.valueOf("topic"); producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, options.valueOf("broker-list")); - this.topic = (String) options.valueOf("topic"); producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer"); producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, @@ -154,10 +163,10 @@ public void onCompletion(RecordMetadata recordMetadata, Exception e) { } public static void main(String[] args) throws IOException { - + VerboseProducer producer = new VerboseProducer(args); - for (int i = 0; i < 20; i++) { + for (int i = 0; i < producer.numMessages; i++) { producer.send(String.format("%d", i)); } From 2777712b39a2d75027299fbb1b1008d463a82aab Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Wed, 27 May 2015 15:35:06 -0700 Subject: [PATCH 05/41] Added some metadata to producer output. --- core/src/main/scala/kafka/tools/VerboseProducer.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/tools/VerboseProducer.java b/core/src/main/scala/kafka/tools/VerboseProducer.java index 32cc3c32c72b4..2ef26fee9cb15 100644 --- a/core/src/main/scala/kafka/tools/VerboseProducer.java +++ b/core/src/main/scala/kafka/tools/VerboseProducer.java @@ -131,13 +131,17 @@ String errorString(Exception e, String value) { return "{\"producer\": \"VerboseProducer\", " + "\"exception\": \"" + e.getClass() + "\"," + "\"message\": \"" + e.getMessage() + "\"," + + "\"topic\": \"" + this.topic + "\"," + "\"value\": \"" + value + "\"" + "}"; } - String successString(String value) { + String successString(String value, RecordMetadata recordMetadata) { return "{\"producer\": \"VerboseProducer\", " + + "\"topic\": \"" + this.topic + "\"," + + "\"partition\": \"" + recordMetadata.partition() + "\"," + + "\"offset\": \"" + recordMetadata.offset() + "\"," + "\"value\": \"" + value + "\"" + "}"; } @@ -154,7 +158,7 @@ private class PrintInfoCallback implements Callback { public void onCompletion(RecordMetadata recordMetadata, Exception e) { if (e == null) { - System.out.println(successString(this.value)); + System.out.println(successString(this.value, recordMetadata)); } else { System.out.println(errorString(e, this.value)); From 8b4b1f2aa9681632ef65aa92dfd3066cd7d62851 Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Fri, 29 May 2015 16:38:32 -0700 Subject: [PATCH 06/41] Minor updates to VerboseProducer --- .../scala/kafka/tools/VerboseProducer.java | 26 ++++++++++++------- 1 file changed, 16 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/kafka/tools/VerboseProducer.java b/core/src/main/scala/kafka/tools/VerboseProducer.java index 2ef26fee9cb15..97214e14511b5 100644 --- a/core/src/main/scala/kafka/tools/VerboseProducer.java +++ b/core/src/main/scala/kafka/tools/VerboseProducer.java @@ -19,6 +19,11 @@ import joptsimple.OptionSpecBuilder; import kafka.utils.CommandLineUtils; +/** + * Primarily intended for use with system testing, this producer produces a fixed number of + * increasing integers and prints metadata in the form of JSON to stdout for each failed or + * successful produce attempt. + */ public class VerboseProducer { OptionParser commandLineParser; Map> commandLineOptions = new HashMap>(); @@ -51,15 +56,11 @@ private void configureParser() { commandLineOptions.put("broker-list", brokerListOpt); - ArgumentAcceptingOptionSpec numMessagesOpt = commandLineParser.accepts("num-messages", "REQUIRED: The number of messages to produce.") + ArgumentAcceptingOptionSpec numMessagesOpt = commandLineParser.accepts("num-messages", "REQUIRED: The number of messages to produce.") .withRequiredArg() .describedAs("num-messages") - .ofType(String.class); - commandLineOptions.put("num-messages", numMessagesOpt); - - // syncOpt = parser.accepts("sync", "If set message send requests to the brokers are synchronously, one at a time as they arrive.") -// val producerRequestRequiredAcks = options.valueOf(producerRequestRequiredAcksOpt).intValue() - + .ofType(Integer.class); + commandLineOptions.put("num-messages", numMessagesOpt); OptionSpecBuilder helpOpt = commandLineParser.accepts("help", "Print this message."); @@ -84,7 +85,7 @@ public void parseCommandLineArgs(String[] args) throws IOException { } checkRequiredArgs(commandLineParser, options, requiredArgs); - this.numMessages = Integer.parseInt((String) options.valueOf("num-messages")); + this.numMessages = (Integer) options.valueOf("num-messages"); this.topic = (String) options.valueOf("topic"); producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, options.valueOf("broker-list")); @@ -115,7 +116,12 @@ private static void checkRequiredArgs( */ public void send(String value) { ProducerRecord record = new ProducerRecord(topic, value); - producer.send(record, new PrintInfoCallback(value)); + try { + producer.send(record, new PrintInfoCallback(value)); + } + catch (Exception e) { + System.out.println(errorString(e, value)); + } } /** Need to close the producer to flush any remaining messages if we're in async mode. */ @@ -176,4 +182,4 @@ public static void main(String[] args) throws IOException { producer.close(); } -} \ No newline at end of file +} From c0526fe44cea739519a0889ebe9ead01b406b365 Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Sun, 31 May 2015 19:27:15 -0700 Subject: [PATCH 07/41] Updates per review comments. --- build.gradle | 2 + .../tools/MetadataToStdoutProducer.java | 195 ++++++++++++++++++ 2 files changed, 197 insertions(+) create mode 100644 clients/src/main/java/org/apache/kafka/clients/tools/MetadataToStdoutProducer.java diff --git a/build.gradle b/build.gradle index 4775ee46c480e..5941d02672125 100644 --- a/build.gradle +++ b/build.gradle @@ -352,6 +352,8 @@ project(':clients') { compile "org.slf4j:slf4j-api:1.7.6" compile 'org.xerial.snappy:snappy-java:1.1.1.6' compile 'net.jpountz.lz4:lz4:1.2.0' + compile 'net.sf.jopt-simple:jopt-simple:4.8' + compile 'com.googlecode.json-simple:json-simple:1.1.1' testCompile 'com.novocode:junit-interface:0.9' testRuntime "$slf4jlog4j" diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/MetadataToStdoutProducer.java b/clients/src/main/java/org/apache/kafka/clients/tools/MetadataToStdoutProducer.java new file mode 100644 index 0000000000000..25118de0480c1 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/tools/MetadataToStdoutProducer.java @@ -0,0 +1,195 @@ +package org.apache.kafka.clients.tools; + +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; + +import org.json.simple.JSONObject; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +import joptsimple.ArgumentAcceptingOptionSpec; +import joptsimple.OptionParser; +import joptsimple.OptionSet; +import joptsimple.OptionSpec; +import joptsimple.OptionSpecBuilder; + +/** + * Primarily intended for use with system testing, this producer prints metadata + * in the form of JSON to stdout on each "send" request. For example, this helps + * with end-to-end correctness tests by making externally visible which messages have been + * acked and which have not. + * + * When used as a command-line tool, it produces a fixed number of increasing integers. + * If logging is left enabled, log output on stdout can be easily ignored by checking + * whether a given line is valid JSON. + */ +public class MetadataToStdoutProducer { + OptionParser commandLineParser; + Map> commandLineOptions = new HashMap>(); + + String topic; + private Properties producerProps = new Properties(); + private Producer producer; + private int numMessages; + + public MetadataToStdoutProducer(String[] args) throws IOException { + this.configureParser(); + this.parseCommandLineArgs(args); + this.producer = new KafkaProducer(producerProps); + } + + /** Set up the command-line options. */ + private void configureParser() { + this.commandLineParser = new OptionParser(); + ArgumentAcceptingOptionSpec topicOpt = commandLineParser.accepts("topic", "REQUIRED: The topic id to produce messages to.") + .withRequiredArg() + .describedAs("topic") + .ofType(String.class); + commandLineOptions.put("topic", topicOpt); + + ArgumentAcceptingOptionSpec brokerListOpt = commandLineParser.accepts("broker-list", "REQUIRED: The broker list string in the form HOST1:PORT1,HOST2:PORT2.") + .withRequiredArg() + .describedAs("broker-list") + .ofType(String.class); + commandLineOptions.put("broker-list", brokerListOpt); + + + ArgumentAcceptingOptionSpec numMessagesOpt = commandLineParser.accepts("num-messages", "REQUIRED: The number of messages to produce.") + .withRequiredArg() + .describedAs("num-messages") + .ofType(Integer.class); + commandLineOptions.put("num-messages", numMessagesOpt); + + OptionSpecBuilder helpOpt + = commandLineParser.accepts("help", "Print this message."); + commandLineOptions.put("help", helpOpt); + } + + /** Validate command-line arguments and parse into properties object. */ + public void parseCommandLineArgs(String[] args) throws IOException { + + OptionSpec[] requiredArgs = new OptionSpec[]{commandLineOptions.get("topic"), + commandLineOptions.get("broker-list"), + commandLineOptions.get("num-messages")}; + + OptionSet options = commandLineParser.parse(args); + if (options.has(commandLineOptions.get("help"))) { + commandLineParser.printHelpOn(System.out); + System.exit(0); + } + checkRequiredArgs(commandLineParser, options, requiredArgs); + + this.numMessages = (Integer) options.valueOf("num-messages"); + this.topic = (String) options.valueOf("topic"); + + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, options.valueOf("broker-list")); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.StringSerializer"); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.StringSerializer"); + producerProps.put(ProducerConfig.ACKS_CONFIG, "-1"); + + // No producer retries + producerProps.put("retries", "0"); + } + + private static void checkRequiredArgs( + OptionParser parser, OptionSet options, OptionSpec[] required) throws IOException + { + for (OptionSpec arg : required) { + if (!options.has(arg)) { + System.err.println("Missing required argument \"" + arg + "\""); + parser.printHelpOn(System.err); + System.exit(1); + } + } + } + + /** + * Produce a message with given value and no key. + */ + public void send(String key, String value) { + ProducerRecord record = new ProducerRecord(topic, key, value); + try { + producer.send(record, new PrintInfoCallback(key, value)); + } + catch (Exception e) { + System.out.println(errorString(e, key, value)); + } + } + + /** Need to close the producer to flush any remaining messages if we're in async mode. */ + public void close() { + producer.close(); + } + + /** + * Return JSON string encapsulating basic information about the exception, as well + * as the key and value which triggered the exception. + */ + String errorString(Exception e, String key, String value) { + assert e != null: "Expected non-null exception."; + + JSONObject obj = new JSONObject(); + obj.put("producer", "MetadataToStdoutProducer"); + obj.put("exception", e.getClass()); + obj.put("message", e.getMessage()); + obj.put("topic", this.topic); + obj.put("key", key); + obj.put("value", value); + return obj.toString(); + } + + String successString(RecordMetadata recordMetadata, String key, String value) { + assert recordMetadata != null: "Expected non-null recordMetadata object."; + + JSONObject obj = new JSONObject(); + obj.put("producer", "MetadataToStdoutProducer"); + obj.put("topic", this.topic); + obj.put("partition", recordMetadata.partition()); + obj.put("offset", recordMetadata.offset()); + obj.put("key", key); + obj.put("value", value); + return obj.toString(); + } + + /** + * Callback which prints errors to stdout when the producer fails to send. + */ + private class PrintInfoCallback implements Callback { + private String key; + private String value; + + PrintInfoCallback(String key, String value) { + this.key = key; + this.value = value; + } + + public void onCompletion(RecordMetadata recordMetadata, Exception e) { + if (e == null) { + System.out.println(successString(recordMetadata, this.key, this.value)); + } + else { + System.out.println(errorString(e, this.key, this.value)); + } + } + } + + public static void main(String[] args) throws IOException { + + MetadataToStdoutProducer producer = new MetadataToStdoutProducer(args); + + for (int i = 0; i < producer.numMessages; i++) { + producer.send(null, String.format("%d", i)); + } + + producer.close(); + } +} From bc009f218e00241cbdd23931d01b52c442eef6b7 Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Sun, 31 May 2015 19:28:28 -0700 Subject: [PATCH 08/41] Got rid of VerboseProducer in core (moved to clients) --- .../scala/kafka/tools/VerboseProducer.java | 185 ------------------ 1 file changed, 185 deletions(-) delete mode 100644 core/src/main/scala/kafka/tools/VerboseProducer.java diff --git a/core/src/main/scala/kafka/tools/VerboseProducer.java b/core/src/main/scala/kafka/tools/VerboseProducer.java deleted file mode 100644 index 97214e14511b5..0000000000000 --- a/core/src/main/scala/kafka/tools/VerboseProducer.java +++ /dev/null @@ -1,185 +0,0 @@ -package kafka.tools; - -import org.apache.kafka.clients.producer.Callback; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.Producer; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.clients.producer.RecordMetadata; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; - -import joptsimple.ArgumentAcceptingOptionSpec; -import joptsimple.OptionParser; -import joptsimple.OptionSet; -import joptsimple.OptionSpec; -import joptsimple.OptionSpecBuilder; -import kafka.utils.CommandLineUtils; - -/** - * Primarily intended for use with system testing, this producer produces a fixed number of - * increasing integers and prints metadata in the form of JSON to stdout for each failed or - * successful produce attempt. - */ -public class VerboseProducer { - OptionParser commandLineParser; - Map> commandLineOptions = new HashMap>(); - - String topic; - String sync; - private Properties producerProps = new Properties(); - private Producer producer; - private int numMessages; - - public VerboseProducer(String[] args) throws IOException { - this.configureParser(); - this.parseCommandLineArgs(args); - this.producer = new KafkaProducer(producerProps); - } - - /** Set up the command-line options. */ - private void configureParser() { - this.commandLineParser = new OptionParser(); - ArgumentAcceptingOptionSpec topicOpt = commandLineParser.accepts("topic", "REQUIRED: The topic id to produce messages to.") - .withRequiredArg() - .describedAs("topic") - .ofType(String.class); - commandLineOptions.put("topic", topicOpt); - - ArgumentAcceptingOptionSpec brokerListOpt = commandLineParser.accepts("broker-list", "REQUIRED: The broker list string in the form HOST1:PORT1,HOST2:PORT2.") - .withRequiredArg() - .describedAs("broker-list") - .ofType(String.class); - commandLineOptions.put("broker-list", brokerListOpt); - - - ArgumentAcceptingOptionSpec numMessagesOpt = commandLineParser.accepts("num-messages", "REQUIRED: The number of messages to produce.") - .withRequiredArg() - .describedAs("num-messages") - .ofType(Integer.class); - commandLineOptions.put("num-messages", numMessagesOpt); - - OptionSpecBuilder helpOpt - = commandLineParser.accepts("help", "Print this message."); - commandLineOptions.put("help", helpOpt); - } - - /** Validate command-line arguments and parse into properties object. */ - public void parseCommandLineArgs(String[] args) throws IOException { - - OptionSpec[] requiredArgs = new OptionSpec[]{commandLineOptions.get("topic"), - commandLineOptions.get("broker-list"), - commandLineOptions.get("num-messages")}; - if(args.length == 0) { - CommandLineUtils.printUsageAndDie( - commandLineParser, "Read data from standard input and publish it to Kafka."); - } - - OptionSet options = commandLineParser.parse(args); - if (options.has(commandLineOptions.get("help"))) { - commandLineParser.printHelpOn(System.out); - System.exit(0); - } - checkRequiredArgs(commandLineParser, options, requiredArgs); - - this.numMessages = (Integer) options.valueOf("num-messages"); - this.topic = (String) options.valueOf("topic"); - - producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, options.valueOf("broker-list")); - producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, - "org.apache.kafka.common.serialization.StringSerializer"); - producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, - "org.apache.kafka.common.serialization.StringSerializer"); - producerProps.put(ProducerConfig.ACKS_CONFIG, "-1"); - - // No producer retries - producerProps.put("retries", "0"); - } - - private static void checkRequiredArgs( - OptionParser parser, OptionSet options, OptionSpec[] required) throws IOException - { - for (OptionSpec arg : required) { - if (!options.has(arg)) { - System.err.println("Missing required argument \"" + arg + "\""); - parser.printHelpOn(System.err); - System.exit(1); - } - } - } - - /** - * Produce a message with given value and no key. - */ - public void send(String value) { - ProducerRecord record = new ProducerRecord(topic, value); - try { - producer.send(record, new PrintInfoCallback(value)); - } - catch (Exception e) { - System.out.println(errorString(e, value)); - } - } - - /** Need to close the producer to flush any remaining messages if we're in async mode. */ - public void close() { - producer.close(); - } - - /** - * Return JSON string encapsulating basic information about the exception, as well - * as the value which triggered the exception. - */ - String errorString(Exception e, String value) { - return "{\"producer\": \"VerboseProducer\", " - + "\"exception\": \"" + e.getClass() + "\"," - + "\"message\": \"" + e.getMessage() + "\"," - + "\"topic\": \"" + this.topic + "\"," - + "\"value\": \"" + value + "\"" - + "}"; - - } - - String successString(String value, RecordMetadata recordMetadata) { - return "{\"producer\": \"VerboseProducer\", " - + "\"topic\": \"" + this.topic + "\"," - + "\"partition\": \"" + recordMetadata.partition() + "\"," - + "\"offset\": \"" + recordMetadata.offset() + "\"," - + "\"value\": \"" + value + "\"" - + "}"; - } - - /** - * Callback which prints errors to stdout when the producer fails to send. - */ - private class PrintInfoCallback implements Callback { - private String value; - - PrintInfoCallback(String value) { - this.value = value; - } - - public void onCompletion(RecordMetadata recordMetadata, Exception e) { - if (e == null) { - System.out.println(successString(this.value, recordMetadata)); - } - else { - System.out.println(errorString(e, this.value)); - } - } - } - - public static void main(String[] args) throws IOException { - - VerboseProducer producer = new VerboseProducer(args); - - for (int i = 0; i < producer.numMessages; i++) { - producer.send(String.format("%d", i)); - } - - producer.close(); - } -} From 475423bb642ac8f816e8080f891867a6362c17fa Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Sun, 31 May 2015 21:05:09 -0700 Subject: [PATCH 09/41] Convert class to string before adding to json object. --- .../apache/kafka/clients/tools/MetadataToStdoutProducer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/MetadataToStdoutProducer.java b/clients/src/main/java/org/apache/kafka/clients/tools/MetadataToStdoutProducer.java index 25118de0480c1..8dc4999c610b3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/tools/MetadataToStdoutProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/tools/MetadataToStdoutProducer.java @@ -139,7 +139,7 @@ String errorString(Exception e, String key, String value) { JSONObject obj = new JSONObject(); obj.put("producer", "MetadataToStdoutProducer"); - obj.put("exception", e.getClass()); + obj.put("exception", e.getClass().toString()); obj.put("message", e.getMessage()); obj.put("topic", this.topic); obj.put("key", key); From 0a5de8e0590e3a8dce1a91769ad41497b5e07d17 Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Tue, 2 Jun 2015 15:46:52 -0700 Subject: [PATCH 10/41] Fixed checkstyle errors. Changed name to VerifiableProducer. Added synchronization for thread safety on println statements. --- checkstyle/import-control.xml | 2 + .../tools/MetadataToStdoutProducer.java | 195 ------------ .../clients/tools/VerifiableProducer.java | 279 ++++++++++++++++++ 3 files changed, 281 insertions(+), 195 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/tools/MetadataToStdoutProducer.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index f2e6cec267e67..68f0058db3e25 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -92,6 +92,8 @@ + + diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/MetadataToStdoutProducer.java b/clients/src/main/java/org/apache/kafka/clients/tools/MetadataToStdoutProducer.java deleted file mode 100644 index 8dc4999c610b3..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/tools/MetadataToStdoutProducer.java +++ /dev/null @@ -1,195 +0,0 @@ -package org.apache.kafka.clients.tools; - -import org.apache.kafka.clients.producer.Callback; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.Producer; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.clients.producer.RecordMetadata; - -import org.json.simple.JSONObject; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; - -import joptsimple.ArgumentAcceptingOptionSpec; -import joptsimple.OptionParser; -import joptsimple.OptionSet; -import joptsimple.OptionSpec; -import joptsimple.OptionSpecBuilder; - -/** - * Primarily intended for use with system testing, this producer prints metadata - * in the form of JSON to stdout on each "send" request. For example, this helps - * with end-to-end correctness tests by making externally visible which messages have been - * acked and which have not. - * - * When used as a command-line tool, it produces a fixed number of increasing integers. - * If logging is left enabled, log output on stdout can be easily ignored by checking - * whether a given line is valid JSON. - */ -public class MetadataToStdoutProducer { - OptionParser commandLineParser; - Map> commandLineOptions = new HashMap>(); - - String topic; - private Properties producerProps = new Properties(); - private Producer producer; - private int numMessages; - - public MetadataToStdoutProducer(String[] args) throws IOException { - this.configureParser(); - this.parseCommandLineArgs(args); - this.producer = new KafkaProducer(producerProps); - } - - /** Set up the command-line options. */ - private void configureParser() { - this.commandLineParser = new OptionParser(); - ArgumentAcceptingOptionSpec topicOpt = commandLineParser.accepts("topic", "REQUIRED: The topic id to produce messages to.") - .withRequiredArg() - .describedAs("topic") - .ofType(String.class); - commandLineOptions.put("topic", topicOpt); - - ArgumentAcceptingOptionSpec brokerListOpt = commandLineParser.accepts("broker-list", "REQUIRED: The broker list string in the form HOST1:PORT1,HOST2:PORT2.") - .withRequiredArg() - .describedAs("broker-list") - .ofType(String.class); - commandLineOptions.put("broker-list", brokerListOpt); - - - ArgumentAcceptingOptionSpec numMessagesOpt = commandLineParser.accepts("num-messages", "REQUIRED: The number of messages to produce.") - .withRequiredArg() - .describedAs("num-messages") - .ofType(Integer.class); - commandLineOptions.put("num-messages", numMessagesOpt); - - OptionSpecBuilder helpOpt - = commandLineParser.accepts("help", "Print this message."); - commandLineOptions.put("help", helpOpt); - } - - /** Validate command-line arguments and parse into properties object. */ - public void parseCommandLineArgs(String[] args) throws IOException { - - OptionSpec[] requiredArgs = new OptionSpec[]{commandLineOptions.get("topic"), - commandLineOptions.get("broker-list"), - commandLineOptions.get("num-messages")}; - - OptionSet options = commandLineParser.parse(args); - if (options.has(commandLineOptions.get("help"))) { - commandLineParser.printHelpOn(System.out); - System.exit(0); - } - checkRequiredArgs(commandLineParser, options, requiredArgs); - - this.numMessages = (Integer) options.valueOf("num-messages"); - this.topic = (String) options.valueOf("topic"); - - producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, options.valueOf("broker-list")); - producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, - "org.apache.kafka.common.serialization.StringSerializer"); - producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, - "org.apache.kafka.common.serialization.StringSerializer"); - producerProps.put(ProducerConfig.ACKS_CONFIG, "-1"); - - // No producer retries - producerProps.put("retries", "0"); - } - - private static void checkRequiredArgs( - OptionParser parser, OptionSet options, OptionSpec[] required) throws IOException - { - for (OptionSpec arg : required) { - if (!options.has(arg)) { - System.err.println("Missing required argument \"" + arg + "\""); - parser.printHelpOn(System.err); - System.exit(1); - } - } - } - - /** - * Produce a message with given value and no key. - */ - public void send(String key, String value) { - ProducerRecord record = new ProducerRecord(topic, key, value); - try { - producer.send(record, new PrintInfoCallback(key, value)); - } - catch (Exception e) { - System.out.println(errorString(e, key, value)); - } - } - - /** Need to close the producer to flush any remaining messages if we're in async mode. */ - public void close() { - producer.close(); - } - - /** - * Return JSON string encapsulating basic information about the exception, as well - * as the key and value which triggered the exception. - */ - String errorString(Exception e, String key, String value) { - assert e != null: "Expected non-null exception."; - - JSONObject obj = new JSONObject(); - obj.put("producer", "MetadataToStdoutProducer"); - obj.put("exception", e.getClass().toString()); - obj.put("message", e.getMessage()); - obj.put("topic", this.topic); - obj.put("key", key); - obj.put("value", value); - return obj.toString(); - } - - String successString(RecordMetadata recordMetadata, String key, String value) { - assert recordMetadata != null: "Expected non-null recordMetadata object."; - - JSONObject obj = new JSONObject(); - obj.put("producer", "MetadataToStdoutProducer"); - obj.put("topic", this.topic); - obj.put("partition", recordMetadata.partition()); - obj.put("offset", recordMetadata.offset()); - obj.put("key", key); - obj.put("value", value); - return obj.toString(); - } - - /** - * Callback which prints errors to stdout when the producer fails to send. - */ - private class PrintInfoCallback implements Callback { - private String key; - private String value; - - PrintInfoCallback(String key, String value) { - this.key = key; - this.value = value; - } - - public void onCompletion(RecordMetadata recordMetadata, Exception e) { - if (e == null) { - System.out.println(successString(recordMetadata, this.key, this.value)); - } - else { - System.out.println(errorString(e, this.key, this.value)); - } - } - } - - public static void main(String[] args) throws IOException { - - MetadataToStdoutProducer producer = new MetadataToStdoutProducer(args); - - for (int i = 0; i < producer.numMessages; i++) { - producer.send(null, String.format("%d", i)); - } - - producer.close(); - } -} diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java b/clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java new file mode 100644 index 0000000000000..25da838c24a3b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java @@ -0,0 +1,279 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.tools; + +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; + +import org.json.simple.JSONObject; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +import joptsimple.ArgumentAcceptingOptionSpec; +import joptsimple.OptionParser; +import joptsimple.OptionSet; +import joptsimple.OptionSpec; +import joptsimple.OptionSpecBuilder; + +/** + * Primarily intended for use with system testing, this producer prints metadata + * in the form of JSON to stdout on each "send" request. For example, this helps + * with end-to-end correctness tests by making externally visible which messages have been + * acked and which have not. + * + * When used as a command-line tool, it produces a fixed number of increasing integers. + * If logging is left enabled, log output on stdout can be easily ignored by checking + * whether a given line is valid JSON. + */ +public class VerifiableProducer { + + private static final long NS_PER_MS = 1000000L; + private static final long NS_PER_SEC = 1000 * NS_PER_MS; + private static final long MIN_SLEEP_NS = 2 * NS_PER_MS; + + OptionParser commandLineParser; + Map> commandLineOptions = new HashMap>(); + + String topic; + private Properties producerProps = new Properties(); + private Producer producer; + private int numMessages; + private long throughput; + + /** Construct with command-line arguments */ + public VerifiableProducer(String[] args) throws IOException { + this.configureParser(); + this.parseCommandLineArgs(args); + this.producer = new KafkaProducer(producerProps); + } + + /** Set up the command-line options. */ + private void configureParser() { + this.commandLineParser = new OptionParser(); + ArgumentAcceptingOptionSpec topicOpt = commandLineParser.accepts("topic", "REQUIRED: The topic id to produce messages to.") + .withRequiredArg() + .describedAs("topic") + .ofType(String.class); + commandLineOptions.put("topic", topicOpt); + + ArgumentAcceptingOptionSpec brokerListOpt = commandLineParser.accepts("broker-list", "REQUIRED: The broker list string in the form HOST1:PORT1,HOST2:PORT2.") + .withRequiredArg() + .describedAs("broker-list") + .ofType(String.class); + commandLineOptions.put("broker-list", brokerListOpt); + + + ArgumentAcceptingOptionSpec numMessagesOpt = commandLineParser.accepts("num-messages", "REQUIRED: The number of messages to produce.") + .withRequiredArg() + .describedAs("num-messages") + .ofType(Integer.class); + commandLineOptions.put("num-messages", numMessagesOpt); + + ArgumentAcceptingOptionSpec throughputOpt = commandLineParser.accepts("throughput", "REQUIRED: Average message throughput, in messages/sec.") + .withRequiredArg() + .describedAs("throughput") + .ofType(Long.class); + commandLineOptions.put("throughput", throughputOpt); + + OptionSpecBuilder helpOpt = commandLineParser.accepts("help", "Print this message."); + commandLineOptions.put("help", helpOpt); + } + + /** Validate command-line arguments and parse into properties object. */ + public void parseCommandLineArgs(String[] args) throws IOException { + + OptionSpec[] requiredArgs = new OptionSpec[]{commandLineOptions.get("topic"), + commandLineOptions.get("broker-list"), + commandLineOptions.get("num-messages"), + commandLineOptions.get("throughput")}; + + OptionSet options = commandLineParser.parse(args); + if (options.has(commandLineOptions.get("help"))) { + commandLineParser.printHelpOn(System.out); + System.exit(0); + } + checkRequiredArgs(commandLineParser, options, requiredArgs); + + this.numMessages = (Integer) options.valueOf("num-messages"); + this.topic = (String) options.valueOf("topic"); + this.throughput = (Long) options.valueOf("throughput"); + + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, options.valueOf("broker-list")); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.StringSerializer"); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.StringSerializer"); + producerProps.put(ProducerConfig.ACKS_CONFIG, "-1"); + + // No producer retries + producerProps.put("retries", "0"); + } + + private static void checkRequiredArgs( + OptionParser parser, OptionSet options, OptionSpec[] required) throws IOException + { + for (OptionSpec arg : required) { + if (!options.has(arg)) { + System.err.println("Missing required argument \"" + arg + "\""); + parser.printHelpOn(System.err); + System.exit(1); + } + } + } + + /** + * Produce a message with given value and no key. + */ + public void send(String key, String value) { + ProducerRecord record = new ProducerRecord(topic, key, value); + try { + producer.send(record, new PrintInfoCallback(key, value)); + } catch (Exception e) { + + synchronized (System.out) { + System.out.println(errorString(e, key, value, System.currentTimeMillis())); + } + } + } + + /** Need to close the producer to flush any remaining messages if we're in async mode. */ + public void close() { + producer.close(); + } + + /** + * Return JSON string encapsulating basic information about the exception, as well + * as the key and value which triggered the exception. + */ + String errorString(Exception e, String key, String value, Long nowMs) { + assert e != null : "Expected non-null exception."; + + JSONObject obj = new JSONObject(); + obj.put("class", this.getClass().toString()); + obj.put("name", "producer_send_error"); + + obj.put("time_ms", nowMs); + obj.put("exception", e.getClass().toString()); + obj.put("message", e.getMessage()); + obj.put("topic", this.topic); + obj.put("key", key); + obj.put("value", value); + return obj.toJSONString(); + } + + String successString(RecordMetadata recordMetadata, String key, String value, Long nowMs) { + assert recordMetadata != null : "Expected non-null recordMetadata object."; + + JSONObject obj = new JSONObject(); + obj.put("class", this.getClass().toString()); + obj.put("name", "producer_send_success"); + + obj.put("time_ms", nowMs); + obj.put("topic", this.topic); + obj.put("partition", recordMetadata.partition()); + obj.put("offset", recordMetadata.offset()); + obj.put("key", key); + obj.put("value", value); + return obj.toJSONString(); + } + + /** + * Callback which prints errors to stdout when the producer fails to send. + */ + private class PrintInfoCallback implements Callback { + + private String key; + private String value; + + PrintInfoCallback(String key, String value) { + this.key = key; + this.value = value; + } + + public void onCompletion(RecordMetadata recordMetadata, Exception e) { + synchronized (System.out) { + if (e == null) { + System.out.println(successString(recordMetadata, this.key, this.value, System.currentTimeMillis())); + } else { + System.out.println(errorString(e, this.key, this.value, System.currentTimeMillis())); + } + } + } + } + + public static void main(String[] args) throws IOException { + + VerifiableProducer producer = new VerifiableProducer(args); + + long sleepTimeNs = NS_PER_SEC / producer.throughput; + long sleepDeficitNs = 0; + long startMs = System.currentTimeMillis(); + + for (int i = 0; i < producer.numMessages; i++) { + long sendStartMs = System.currentTimeMillis(); + producer.send(null, String.format("%d", i)); + + // throttle message throughput by sleeping, on average, + // (NS_PER_SEC / producer.throughput) nanoseconds between each sent message + if (producer.throughput > 0) { + float elapsedMs = (sendStartMs - startMs) / 1000.f; + if (elapsedMs > 0 && i / elapsedMs > producer.throughput) { + sleepDeficitNs += sleepTimeNs; + + // If enough sleep deficit has accumulated, sleep a little + if (sleepDeficitNs >= MIN_SLEEP_NS) { + long sleepMs = sleepDeficitNs / 1000000; + long sleepNs = sleepDeficitNs - sleepMs * 1000000; + + long sleepStartNs = System.nanoTime(); + try { + Thread.sleep(sleepMs, (int) sleepNs); + sleepDeficitNs = 0; + } catch (InterruptedException e) { + // If sleep is cut short, reduce deficit by the amount of + // time we actually spent sleeping + long sleepElapsedNs = System.nanoTime() - sleepStartNs; + if (sleepElapsedNs <= sleepDeficitNs) { + sleepDeficitNs -= sleepElapsedNs; + } + } + + } + } + } + } + producer.close(); + + long stopMs = System.currentTimeMillis(); + double avgThroughput = 1000 * ((producer.numMessages) / (double) (stopMs - startMs)); + + JSONObject obj = new JSONObject(); + obj.put("class", producer.getClass().toString()); + obj.put("name", "tool_data"); + obj.put("target_throughput", producer.throughput); + obj.put("avg_throughput", avgThroughput); + System.out.println(obj.toJSONString()); + } +} From 9100417ce0717a71c822c5a279fe7858bfe7a7ee Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Wed, 3 Jun 2015 12:50:11 -0700 Subject: [PATCH 11/41] Updated command-line options for VerifiableProducer. Extracted throughput logic to make it reusable. --- .../tools/MessageThroughputThrottler.java | 89 +++++++++++ .../clients/tools/ProducerPerformance.java | 32 +--- .../clients/tools/VerifiableProducer.java | 148 ++++++++---------- 3 files changed, 161 insertions(+), 108 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/tools/MessageThroughputThrottler.java diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/MessageThroughputThrottler.java b/clients/src/main/java/org/apache/kafka/clients/tools/MessageThroughputThrottler.java new file mode 100644 index 0000000000000..1fb75000ff906 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/tools/MessageThroughputThrottler.java @@ -0,0 +1,89 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.tools; + + +/** + * This class helps producers throttle their maximum message throughput. + * + * The resulting average throughput will be approximately + * min(targetThroughput, maximumPossibleThroughput) + * + * To use, do this between successive send attempts: + *
+ *     {@code     
+ *      if (throttler.shouldThrottle(...)) {
+ *          throttler.throttle();
+ *      } 
+ *     } 
+ * 
+ */ +public class MessageThroughputThrottler { + + private static final long NS_PER_MS = 1000000L; + private static final long NS_PER_SEC = 1000 * NS_PER_MS; + private static final long MIN_SLEEP_NS = 2 * NS_PER_MS; + + long sleepTimeNs; + long sleepDeficitNs = 0; + long targetThroughput = -1; + long startMs; + + public MessageThroughputThrottler(long targetThroughput, long startMs) { + this.startMs = startMs; + this.targetThroughput = targetThroughput; + this.sleepTimeNs = NS_PER_SEC / targetThroughput; + } + + public boolean shouldThrottle(long messageNum, long sendStartMs) { + if (this.targetThroughput <= 0) { + // No throttling in this case + return false; + } + + float elapsedMs = (sendStartMs - startMs) / 1000.f; + return elapsedMs > 0 && (messageNum / elapsedMs) > this.targetThroughput; + } + + public void throttle() { + // throttle message throughput by sleeping, on average, + // (1 / this.throughput) seconds between each sent message + sleepDeficitNs += sleepTimeNs; + + // If enough sleep deficit has accumulated, sleep a little + if (sleepDeficitNs >= MIN_SLEEP_NS) { + long sleepMs = sleepDeficitNs / 1000000; + long sleepNs = sleepDeficitNs - sleepMs * 1000000; + + long sleepStartNs = System.nanoTime(); + try { + Thread.sleep(sleepMs, (int) sleepNs); + sleepDeficitNs = 0; + } catch (InterruptedException e) { + // If sleep is cut short, reduce deficit by the amount of + // time we actually spent sleeping + long sleepElapsedNs = System.nanoTime() - sleepStartNs; + if (sleepElapsedNs <= sleepDeficitNs) { + sleepDeficitNs -= sleepElapsedNs; + } + } + } + } +} + + \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java b/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java index 13f4d5958052a..a6d9ee42566d4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java +++ b/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java @@ -19,10 +19,6 @@ public class ProducerPerformance { - private static final long NS_PER_MS = 1000000L; - private static final long NS_PER_SEC = 1000 * NS_PER_MS; - private static final long MIN_SLEEP_NS = 2 * NS_PER_MS; - public static void main(String[] args) throws Exception { if (args.length < 4) { System.err.println("USAGE: java " + ProducerPerformance.class.getName() + @@ -51,31 +47,17 @@ public static void main(String[] args) throws Exception { byte[] payload = new byte[recordSize]; Arrays.fill(payload, (byte) 1); ProducerRecord record = new ProducerRecord(topicName, payload); - long sleepTime = NS_PER_SEC / throughput; - long sleepDeficitNs = 0; Stats stats = new Stats(numRecords, 5000); - long start = System.currentTimeMillis(); + long startMs = System.currentTimeMillis(); + + MessageThroughputThrottler throttler = new MessageThroughputThrottler(throughput, startMs); for (int i = 0; i < numRecords; i++) { - long sendStart = System.currentTimeMillis(); - Callback cb = stats.nextCompletion(sendStart, payload.length, stats); + long sendStartMs = System.currentTimeMillis(); + Callback cb = stats.nextCompletion(sendStartMs, payload.length, stats); producer.send(record, cb); - /* - * Maybe sleep a little to control throughput. Sleep time can be a bit inaccurate for times < 1 ms so - * instead of sleeping each time instead wait until a minimum sleep time accumulates (the "sleep deficit") - * and then make up the whole deficit in one longer sleep. - */ - if (throughput > 0) { - float elapsed = (sendStart - start) / 1000.f; - if (elapsed > 0 && i / elapsed > throughput) { - sleepDeficitNs += sleepTime; - if (sleepDeficitNs >= MIN_SLEEP_NS) { - long sleepMs = sleepDeficitNs / 1000000; - long sleepNs = sleepDeficitNs - sleepMs * 1000000; - Thread.sleep(sleepMs, (int) sleepNs); - sleepDeficitNs = 0; - } - } + if (throttler.shouldThrottle(i, sendStartMs)) { + throttler.throttle(); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java b/clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java index 25da838c24a3b..f88490e29ae57 100644 --- a/clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java @@ -43,25 +43,31 @@ * with end-to-end correctness tests by making externally visible which messages have been * acked and which have not. * - * When used as a command-line tool, it produces a fixed number of increasing integers. + * When used as a command-line tool, it produces increasing integers. It will produce a + * fixed number of messages unless the default max-messages -1 is used, in which case + * it produces indefinitely. + * * If logging is left enabled, log output on stdout can be easily ignored by checking * whether a given line is valid JSON. */ public class VerifiableProducer { - private static final long NS_PER_MS = 1000000L; - private static final long NS_PER_SEC = 1000 * NS_PER_MS; - private static final long MIN_SLEEP_NS = 2 * NS_PER_MS; - OptionParser commandLineParser; Map> commandLineOptions = new HashMap>(); String topic; private Properties producerProps = new Properties(); private Producer producer; - private int numMessages; + // If maxMessages < 0, produce until the process is killed externally + private long maxMessages = -1; + + // Number of messages for which acks were received + private long numAcked = 0; + + // Number of send attempts + private long numSent = 0; private long throughput; - + /** Construct with command-line arguments */ public VerifiableProducer(String[] args) throws IOException { this.configureParser(); @@ -74,28 +80,39 @@ private void configureParser() { this.commandLineParser = new OptionParser(); ArgumentAcceptingOptionSpec topicOpt = commandLineParser.accepts("topic", "REQUIRED: The topic id to produce messages to.") .withRequiredArg() + .required() .describedAs("topic") .ofType(String.class); commandLineOptions.put("topic", topicOpt); ArgumentAcceptingOptionSpec brokerListOpt = commandLineParser.accepts("broker-list", "REQUIRED: The broker list string in the form HOST1:PORT1,HOST2:PORT2.") .withRequiredArg() + .required() .describedAs("broker-list") .ofType(String.class); commandLineOptions.put("broker-list", brokerListOpt); - ArgumentAcceptingOptionSpec numMessagesOpt = commandLineParser.accepts("num-messages", "REQUIRED: The number of messages to produce.") - .withRequiredArg() - .describedAs("num-messages") - .ofType(Integer.class); - commandLineOptions.put("num-messages", numMessagesOpt); + ArgumentAcceptingOptionSpec numMessagesOpt = commandLineParser.accepts("max-messages", "Produce this many messages. Default: -1, produces messages until the process is killed externally.") + .withOptionalArg() + .defaultsTo("-1") + .describedAs("max-messages") + .ofType(String.class); + commandLineOptions.put("max-messages", numMessagesOpt); - ArgumentAcceptingOptionSpec throughputOpt = commandLineParser.accepts("throughput", "REQUIRED: Average message throughput, in messages/sec.") - .withRequiredArg() + ArgumentAcceptingOptionSpec throughputOpt = commandLineParser.accepts("throughput", "Average message throughput, in messages/sec. Default: -1, results in no throttling.") + .withOptionalArg() + .defaultsTo("-1") .describedAs("throughput") - .ofType(Long.class); + .ofType(String.class); commandLineOptions.put("throughput", throughputOpt); + + ArgumentAcceptingOptionSpec acksOpt = commandLineParser.accepts("acks", "number of acks required. Default: -1") + .withOptionalArg() + .defaultsTo("-1") + .describedAs("acks") + .ofType(String.class); + commandLineOptions.put("acks", acksOpt); OptionSpecBuilder helpOpt = commandLineParser.accepts("help", "Print this message."); commandLineOptions.put("help", helpOpt); @@ -103,51 +120,34 @@ private void configureParser() { /** Validate command-line arguments and parse into properties object. */ public void parseCommandLineArgs(String[] args) throws IOException { - - OptionSpec[] requiredArgs = new OptionSpec[]{commandLineOptions.get("topic"), - commandLineOptions.get("broker-list"), - commandLineOptions.get("num-messages"), - commandLineOptions.get("throughput")}; - + OptionSet options = commandLineParser.parse(args); if (options.has(commandLineOptions.get("help"))) { commandLineParser.printHelpOn(System.out); System.exit(0); } - checkRequiredArgs(commandLineParser, options, requiredArgs); - - this.numMessages = (Integer) options.valueOf("num-messages"); + + this.maxMessages = Integer.parseInt((String) options.valueOf("max-messages")); this.topic = (String) options.valueOf("topic"); - this.throughput = (Long) options.valueOf("throughput"); + this.throughput = Long.parseLong((String) options.valueOf("throughput")); producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, options.valueOf("broker-list")); producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer"); producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer"); - producerProps.put(ProducerConfig.ACKS_CONFIG, "-1"); + producerProps.put(ProducerConfig.ACKS_CONFIG, options.valueOf("acks")); // No producer retries producerProps.put("retries", "0"); } - private static void checkRequiredArgs( - OptionParser parser, OptionSet options, OptionSpec[] required) throws IOException - { - for (OptionSpec arg : required) { - if (!options.has(arg)) { - System.err.println("Missing required argument \"" + arg + "\""); - parser.printHelpOn(System.err); - System.exit(1); - } - } - } - /** * Produce a message with given value and no key. */ public void send(String key, String value) { ProducerRecord record = new ProducerRecord(topic, key, value); + numSent++; try { producer.send(record, new PrintInfoCallback(key, value)); } catch (Exception e) { @@ -215,6 +215,7 @@ private class PrintInfoCallback implements Callback { public void onCompletion(RecordMetadata recordMetadata, Exception e) { synchronized (System.out) { if (e == null) { + VerifiableProducer.this.numAcked++; System.out.println(successString(recordMetadata, this.key, this.value, System.currentTimeMillis())); } else { System.out.println(errorString(e, this.key, this.value, System.currentTimeMillis())); @@ -224,56 +225,37 @@ public void onCompletion(RecordMetadata recordMetadata, Exception e) { } public static void main(String[] args) throws IOException { - - VerifiableProducer producer = new VerifiableProducer(args); + + final VerifiableProducer producer = new VerifiableProducer(args); + final long startMs = System.currentTimeMillis(); + boolean infinite = producer.maxMessages < 0; + + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + producer.close(); - long sleepTimeNs = NS_PER_SEC / producer.throughput; - long sleepDeficitNs = 0; - long startMs = System.currentTimeMillis(); + long stopMs = System.currentTimeMillis(); + double avgThroughput = 1000 * ((producer.numAcked) / (double) (stopMs - startMs)); + + JSONObject obj = new JSONObject(); + obj.put("class", producer.getClass().toString()); + obj.put("name", "tool_data"); + obj.put("sent", producer.numSent); + obj.put("acked", producer.numAcked); + obj.put("target_throughput", producer.throughput); + obj.put("avg_throughput", avgThroughput); + System.out.println(obj.toJSONString()); + } + }); - for (int i = 0; i < producer.numMessages; i++) { + MessageThroughputThrottler throttler = new MessageThroughputThrottler(producer.throughput, startMs); + for (int i = 0; i < producer.maxMessages || infinite; i++) { long sendStartMs = System.currentTimeMillis(); producer.send(null, String.format("%d", i)); - - // throttle message throughput by sleeping, on average, - // (NS_PER_SEC / producer.throughput) nanoseconds between each sent message - if (producer.throughput > 0) { - float elapsedMs = (sendStartMs - startMs) / 1000.f; - if (elapsedMs > 0 && i / elapsedMs > producer.throughput) { - sleepDeficitNs += sleepTimeNs; - - // If enough sleep deficit has accumulated, sleep a little - if (sleepDeficitNs >= MIN_SLEEP_NS) { - long sleepMs = sleepDeficitNs / 1000000; - long sleepNs = sleepDeficitNs - sleepMs * 1000000; - - long sleepStartNs = System.nanoTime(); - try { - Thread.sleep(sleepMs, (int) sleepNs); - sleepDeficitNs = 0; - } catch (InterruptedException e) { - // If sleep is cut short, reduce deficit by the amount of - // time we actually spent sleeping - long sleepElapsedNs = System.nanoTime() - sleepStartNs; - if (sleepElapsedNs <= sleepDeficitNs) { - sleepDeficitNs -= sleepElapsedNs; - } - } - - } - } + if (throttler.shouldThrottle(i, sendStartMs)) { + throttler.throttle(); } } - producer.close(); - - long stopMs = System.currentTimeMillis(); - double avgThroughput = 1000 * ((producer.numMessages) / (double) (stopMs - startMs)); - - JSONObject obj = new JSONObject(); - obj.put("class", producer.getClass().toString()); - obj.put("name", "tool_data"); - obj.put("target_throughput", producer.throughput); - obj.put("avg_throughput", avgThroughput); - System.out.println(obj.toJSONString()); } } From 1228eefc4e52b58c214b3ad45feab36a475d5a66 Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Wed, 3 Jun 2015 18:09:14 -0700 Subject: [PATCH 12/41] Renamed throttler --- .../org/apache/kafka/clients/tools/ProducerPerformance.java | 2 +- ...ssageThroughputThrottler.java => ThroughputThrottler.java} | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) rename clients/src/main/java/org/apache/kafka/clients/tools/{MessageThroughputThrottler.java => ThroughputThrottler.java} (96%) diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java b/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java index a6d9ee42566d4..fd31c1a9fb64a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java +++ b/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java @@ -50,7 +50,7 @@ public static void main(String[] args) throws Exception { Stats stats = new Stats(numRecords, 5000); long startMs = System.currentTimeMillis(); - MessageThroughputThrottler throttler = new MessageThroughputThrottler(throughput, startMs); + ThroughputThrottler throttler = new ThroughputThrottler(throughput, startMs); for (int i = 0; i < numRecords; i++) { long sendStartMs = System.currentTimeMillis(); Callback cb = stats.nextCompletion(sendStartMs, payload.length, stats); diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/MessageThroughputThrottler.java b/clients/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java similarity index 96% rename from clients/src/main/java/org/apache/kafka/clients/tools/MessageThroughputThrottler.java rename to clients/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java index 1fb75000ff906..3ccfd6865b307 100644 --- a/clients/src/main/java/org/apache/kafka/clients/tools/MessageThroughputThrottler.java +++ b/clients/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java @@ -33,7 +33,7 @@ * } * */ -public class MessageThroughputThrottler { +public class ThroughputThrottler { private static final long NS_PER_MS = 1000000L; private static final long NS_PER_SEC = 1000 * NS_PER_MS; @@ -44,7 +44,7 @@ public class MessageThroughputThrottler { long targetThroughput = -1; long startMs; - public MessageThroughputThrottler(long targetThroughput, long startMs) { + public ThroughputThrottler(long targetThroughput, long startMs) { this.startMs = startMs; this.targetThroughput = targetThroughput; this.sleepTimeNs = NS_PER_SEC / targetThroughput; From 6842ed1ffad62a84df67a0f0b6a651a6df085d12 Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Wed, 3 Jun 2015 18:12:11 -0700 Subject: [PATCH 13/41] left out a file from last commit --- .../java/org/apache/kafka/clients/tools/VerifiableProducer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java b/clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java index f88490e29ae57..81ffd06d5293d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java @@ -249,7 +249,7 @@ public void run() { } }); - MessageThroughputThrottler throttler = new MessageThroughputThrottler(producer.throughput, startMs); + ThroughputThrottler throttler = new ThroughputThrottler(producer.throughput, startMs); for (int i = 0; i < producer.maxMessages || infinite; i++) { long sendStartMs = System.currentTimeMillis(); producer.send(null, String.format("%d", i)); From d586fb0eb63409807c02f280fae786cec55fb348 Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Wed, 3 Jun 2015 18:22:34 -0700 Subject: [PATCH 14/41] Updated comments to reflect that throttler is not message-specific --- .../clients/tools/ThroughputThrottler.java | 24 ++++++++++++++----- 1 file changed, 18 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java b/clients/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java index 3ccfd6865b307..03fa7b6b1e930 100644 --- a/clients/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java +++ b/clients/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java @@ -19,7 +19,7 @@ /** - * This class helps producers throttle their maximum message throughput. + * This class helps producers throttle throughput. * * The resulting average throughput will be approximately * min(targetThroughput, maximumPossibleThroughput) @@ -31,7 +31,9 @@ * throttler.throttle(); * } * } - * + * + * + * Note that this can be used to throttle message throughput or data throughput. */ public class ThroughputThrottler { @@ -44,25 +46,35 @@ public class ThroughputThrottler { long targetThroughput = -1; long startMs; + /** + * @param targetThroughput Can be messages/sec or bytes/sec + * @param startMs When the very first message is sent + */ public ThroughputThrottler(long targetThroughput, long startMs) { this.startMs = startMs; this.targetThroughput = targetThroughput; this.sleepTimeNs = NS_PER_SEC / targetThroughput; } - public boolean shouldThrottle(long messageNum, long sendStartMs) { + /** + * @param amountSoFar bytes produced so far if you want to throttle data throughput, or + * messages produced so far if you want to throttle message throughput. + * @param sendStartMs timestamp of the most recently sent message + * @return + */ + public boolean shouldThrottle(long amountSoFar, long sendStartMs) { if (this.targetThroughput <= 0) { // No throttling in this case return false; } float elapsedMs = (sendStartMs - startMs) / 1000.f; - return elapsedMs > 0 && (messageNum / elapsedMs) > this.targetThroughput; + return elapsedMs > 0 && (amountSoFar / elapsedMs) > this.targetThroughput; } public void throttle() { - // throttle message throughput by sleeping, on average, - // (1 / this.throughput) seconds between each sent message + // throttle throughput by sleeping, on average, + // (1 / this.throughput) seconds between "things sent" sleepDeficitNs += sleepTimeNs; // If enough sleep deficit has accumulated, sleep a little From a80a4282ba9a288edba7cdf409d31f01ebf3d458 Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Thu, 4 Jun 2015 13:47:21 -0700 Subject: [PATCH 15/41] Added shell program for VerifiableProducer. --- bin/kafka-verifiable-producer.sh | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) create mode 100755 bin/kafka-verifiable-producer.sh diff --git a/bin/kafka-verifiable-producer.sh b/bin/kafka-verifiable-producer.sh new file mode 100755 index 0000000000000..d0aa6c5591de4 --- /dev/null +++ b/bin/kafka-verifiable-producer.sh @@ -0,0 +1,20 @@ +#!/bin/bash +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +if [ "x$KAFKA_HEAP_OPTS" = "x" ]; then + export KAFKA_HEAP_OPTS="-Xmx512M" +fi +exec $(dirname $0)/kafka-run-class.sh org.apache.kafka.clients.tools.VerifiableProducer $@ From 51a94fd6ece926bcdd864af353efcf4c4d1b8ad8 Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Thu, 4 Jun 2015 13:55:02 -0700 Subject: [PATCH 16/41] Use argparse4j instead of joptsimple. ThroughputThrottler now has more intuitive behavior when targetThroughput is 0. --- build.gradle | 2 +- checkstyle/import-control.xml | 2 +- .../clients/tools/ThroughputThrottler.java | 25 ++- .../clients/tools/VerifiableProducer.java | 203 ++++++++++-------- 4 files changed, 140 insertions(+), 92 deletions(-) diff --git a/build.gradle b/build.gradle index 5941d02672125..516a1393f868a 100644 --- a/build.gradle +++ b/build.gradle @@ -352,7 +352,7 @@ project(':clients') { compile "org.slf4j:slf4j-api:1.7.6" compile 'org.xerial.snappy:snappy-java:1.1.1.6' compile 'net.jpountz.lz4:lz4:1.2.0' - compile 'net.sf.jopt-simple:jopt-simple:4.8' + compile 'net.sourceforge.argparse4j:argparse4j:0.5.0' compile 'com.googlecode.json-simple:json-simple:1.1.1' testCompile 'com.novocode:junit-interface:0.9' diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 68f0058db3e25..f27da4d1bc69e 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -93,7 +93,7 @@ - + diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java b/clients/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java index 03fa7b6b1e930..06c443f576cf9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java +++ b/clients/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java @@ -21,8 +21,9 @@ /** * This class helps producers throttle throughput. * - * The resulting average throughput will be approximately - * min(targetThroughput, maximumPossibleThroughput) + * If targetThroughput >= 0, the resulting average throughput will be approximately + * min(targetThroughput, maximumPossibleThroughput). If targetThroughput < 0, + * no throttling will occur. * * To use, do this between successive send attempts: *
@@ -53,7 +54,9 @@ public class ThroughputThrottler {
     public ThroughputThrottler(long targetThroughput, long startMs) {
         this.startMs = startMs;
         this.targetThroughput = targetThroughput;
-        this.sleepTimeNs = NS_PER_SEC / targetThroughput;
+        this.sleepTimeNs = targetThroughput > 0 ?
+                           NS_PER_SEC / targetThroughput : 
+                           Long.MAX_VALUE;
     }
 
     /**
@@ -63,7 +66,7 @@ public ThroughputThrottler(long targetThroughput, long startMs) {
      * @return
      */
     public boolean shouldThrottle(long amountSoFar, long sendStartMs) {
-        if (this.targetThroughput <= 0) {
+        if (this.targetThroughput < 0) {
             // No throttling in this case
             return false;
         }
@@ -72,7 +75,21 @@ public boolean shouldThrottle(long amountSoFar, long sendStartMs) {
         return elapsedMs > 0 && (amountSoFar / elapsedMs) > this.targetThroughput;
     }
 
+    /**
+     * Occasionally blocks for small amounts of time to achieve targetThroughput.
+     * 
+     * Note that if targetThroughput is 0, this will block extremely aggressively.
+     */
     public void throttle() {
+        if (targetThroughput == 0) {
+            try {
+                Thread.sleep(Long.MAX_VALUE);
+            } catch (InterruptedException e) {
+                // do nothing
+            }
+            return;
+        }
+        
         // throttle throughput by sleeping, on average,
         // (1 / this.throughput) seconds between "things sent"
         sleepDeficitNs += sleepTimeNs;
diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java b/clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java
index 81ffd06d5293d..c12a61993a5af 100644
--- a/clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java
@@ -27,15 +27,14 @@
 import org.json.simple.JSONObject;
 
 import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
 import java.util.Properties;
 
-import joptsimple.ArgumentAcceptingOptionSpec;
-import joptsimple.OptionParser;
-import joptsimple.OptionSet;
-import joptsimple.OptionSpec;
-import joptsimple.OptionSpecBuilder;
+import static net.sourceforge.argparse4j.impl.Arguments.store;
+
+import net.sourceforge.argparse4j.ArgumentParsers;
+import net.sourceforge.argparse4j.inf.ArgumentParser;
+import net.sourceforge.argparse4j.inf.ArgumentParserException;
+import net.sourceforge.argparse4j.inf.Namespace;
 
 /**
  * Primarily intended for use with system testing, this producer prints metadata
@@ -51,12 +50,8 @@
  * whether a given line is valid JSON.
  */
 public class VerifiableProducer {
-
-    OptionParser commandLineParser;
-    Map> commandLineOptions = new HashMap>();
-  
+    
     String topic;
-    private Properties producerProps = new Properties();
     private Producer producer;
     // If maxMessages < 0, produce until the process is killed externally
     private long maxMessages = -1;
@@ -66,85 +61,114 @@ public class VerifiableProducer {
     
     // Number of send attempts
     private long numSent = 0;
+    
+    // Throttle message throughput if this is set >= 0
     private long throughput;
+    
+    // Hook to trigger producing thread to stop sending messages
+    private boolean stopProducing = false;
+
+    public VerifiableProducer(
+            Properties producerProps, String topic, int throughput, int maxMessages) {
 
-    /** Construct with command-line arguments */
-    public VerifiableProducer(String[] args) throws IOException {
-        this.configureParser();
-        this.parseCommandLineArgs(args);
+        this.topic = topic;
+        this.throughput = throughput;
+        this.maxMessages = maxMessages;
         this.producer = new KafkaProducer(producerProps);
     }
-  
-    /** Set up the command-line options. */
-    private void configureParser() {
-        this.commandLineParser = new OptionParser();
-        ArgumentAcceptingOptionSpec topicOpt = commandLineParser.accepts("topic", "REQUIRED: The topic id to produce messages to.")
-                .withRequiredArg()
-                .required()
-                .describedAs("topic")
-                .ofType(String.class);
-        commandLineOptions.put("topic", topicOpt);
-    
-        ArgumentAcceptingOptionSpec  brokerListOpt = commandLineParser.accepts("broker-list", "REQUIRED: The broker list string in the form HOST1:PORT1,HOST2:PORT2.")
-                .withRequiredArg()
-                .required()
-                .describedAs("broker-list")
-                .ofType(String.class);
-        commandLineOptions.put("broker-list", brokerListOpt);
-    
-    
-        ArgumentAcceptingOptionSpec  numMessagesOpt = commandLineParser.accepts("max-messages", "Produce this many messages. Default: -1, produces messages until the process is killed externally.")
-                .withOptionalArg()
-                .defaultsTo("-1")
-                .describedAs("max-messages")
-                .ofType(String.class);
-        commandLineOptions.put("max-messages", numMessagesOpt);
 
-        ArgumentAcceptingOptionSpec  throughputOpt = commandLineParser.accepts("throughput", "Average message throughput, in messages/sec. Default: -1, results in no throttling.")
-                .withOptionalArg()
-                .defaultsTo("-1")
-                .describedAs("throughput")
-                .ofType(String.class);
-        commandLineOptions.put("throughput", throughputOpt);
+    /** Get the command-line argument parser. */
+    private static ArgumentParser argParser() {
+        ArgumentParser parser = ArgumentParsers
+                .newArgumentParser("verifiable-producer")
+                .defaultHelp(true)
+                .description("This tool produces increasing integers to the specified topic and prints JSON metadata to stdout on each \"send\" request, making externally visible which messages have been acked and which have not.");
 
-        ArgumentAcceptingOptionSpec  acksOpt = commandLineParser.accepts("acks", "number of acks required. Default: -1")
-                .withOptionalArg()
-                .defaultsTo("-1")
-                .describedAs("acks")
-                .ofType(String.class);
-        commandLineOptions.put("acks", acksOpt);
-    
-        OptionSpecBuilder helpOpt = commandLineParser.accepts("help", "Print this message.");
-        commandLineOptions.put("help", helpOpt);
+        parser.addArgument("--topic")
+                .action(store())
+                .required(true)
+                .type(String.class)
+                .metavar("TOPIC")
+                .help("Produce messages to this topic.");
+
+        parser.addArgument("--broker-list")
+                .action(store())
+                .required(true)
+                .type(String.class)
+                .metavar("HOST1:PORT1[,HOST2:PORT2[...]]")
+                .dest("brokerList")
+                .help("Comma-separated list of Kafka brokers in the form HOST1:PORT1,HOST2:PORT2,...");
+        
+        parser.addArgument("--max-messages")
+                .action(store())
+                .required(false)
+                .setDefault(-1)
+                .type(Integer.class)
+                .metavar("MAX-MESSAGES")
+                .dest("maxMessages")
+                .help("Produce this many messages. If -1, produce messages until the process is killed externally.");
+
+        parser.addArgument("--throughput")
+                .action(store())
+                .required(false)
+                .setDefault(-1)
+                .type(Integer.class)
+                .metavar("THROUGHPUT")
+                .help("If set >= 0, throttle maximum message throughput to *approximately* THROUGHPUT messages/sec.");
+
+        parser.addArgument("--acks")
+                .action(store())
+                .required(false)
+                .setDefault(-1)
+                .type(Integer.class)
+                .choices(0, 1, -1)
+                .metavar("ACKS")
+                .help("Acks required on each produced message. See Kafka docs on request.required.acks for details.");
+
+        return parser;
     }
   
-    /** Validate command-line arguments and parse into properties object. */
-    public void parseCommandLineArgs(String[] args) throws IOException {
+    /** Construct a VerifiableProducer object from command-line arguments. */
+    public static VerifiableProducer createFromArgs(String[] args) {
+        ArgumentParser parser = argParser();
+        VerifiableProducer producer = null;
+        
+        try {
+            Namespace res;
+            res = parser.parseArgs(args);
+            System.out.println(res);
+            System.out.println(res.getString("brokerList"));
+            
+            
+            int maxMessages = res.getInt("maxMessages");
+            String topic = res.getString("topic");
+            int throughput = res.getInt("throughput");
 
-        OptionSet options = commandLineParser.parse(args);
-        if (options.has(commandLineOptions.get("help"))) {
-            commandLineParser.printHelpOn(System.out);
-            System.exit(0);
-        }
+            Properties producerProps = new Properties();
+            producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, res.getString("brokerList"));
+            producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG,
+                              "org.apache.kafka.common.serialization.StringSerializer");
+            producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG,
+                              "org.apache.kafka.common.serialization.StringSerializer");
+            producerProps.put(ProducerConfig.ACKS_CONFIG, Integer.toString(res.getInt("acks")));
+            // No producer retries
+            producerProps.put("retries", "0");
 
-        this.maxMessages = Integer.parseInt((String) options.valueOf("max-messages"));
-        this.topic = (String) options.valueOf("topic");
-        this.throughput = Long.parseLong((String) options.valueOf("throughput"));
-    
-        producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, options.valueOf("broker-list"));
-        producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG,
-                          "org.apache.kafka.common.serialization.StringSerializer");
-        producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG,
-                          "org.apache.kafka.common.serialization.StringSerializer");
-        producerProps.put(ProducerConfig.ACKS_CONFIG, options.valueOf("acks"));
-    
-        // No producer retries
-        producerProps.put("retries", "0");
+            producer = new VerifiableProducer(producerProps, topic, throughput, maxMessages);
+        } catch (ArgumentParserException e) {
+            if (args.length == 0) {
+                parser.printHelp();
+                System.exit(0);
+            } else {
+                parser.handleError(e);
+                System.exit(1);
+            }
+        }
+        
+        return producer;
     }
   
-    /**
-     * Produce a message with given value and no key.
-     */
+    /** Produce a message with given key and value. */
     public void send(String key, String value) {
         ProducerRecord record = new ProducerRecord(topic, key, value);
         numSent++;
@@ -158,7 +182,7 @@ public void send(String key, String value) {
         }
     }
   
-    /** Need to close the producer to flush any remaining messages if we're in async mode. */
+    /** Close the producer to flush any remaining messages. */
     public void close() {
         producer.close();
     }
@@ -199,9 +223,7 @@ String successString(RecordMetadata recordMetadata, String key, String value, Lo
         return obj.toJSONString();
     }
   
-    /**
-     * Callback which prints errors to stdout when the producer fails to send.
-     */
+    /** Callback which prints errors to stdout when the producer fails to send. */
     private class PrintInfoCallback implements Callback {
         
         private String key;
@@ -226,18 +248,22 @@ public void onCompletion(RecordMetadata recordMetadata, Exception e) {
   
     public static void main(String[] args) throws IOException {
         
-        final VerifiableProducer producer = new VerifiableProducer(args);
+        final VerifiableProducer producer = createFromArgs(args);
         final long startMs = System.currentTimeMillis();
         boolean infinite = producer.maxMessages < 0;
 
         Runtime.getRuntime().addShutdownHook(new Thread() {
             @Override
             public void run() {
+                // Trigger main thread to stop producing messages
+                producer.stopProducing = true;
+                
+                // Flush any remaining messages
                 producer.close();
 
+                // Print a summary
                 long stopMs = System.currentTimeMillis();
                 double avgThroughput = 1000 * ((producer.numAcked) / (double) (stopMs - startMs));
-
                 JSONObject obj = new JSONObject();
                 obj.put("class", producer.getClass().toString());
                 obj.put("name", "tool_data");
@@ -248,14 +274,19 @@ public void run() {
                 System.out.println(obj.toJSONString());
             }
         });
-        
+
         ThroughputThrottler throttler = new ThroughputThrottler(producer.throughput, startMs);
         for (int i = 0; i < producer.maxMessages || infinite; i++) {
+            if (producer.stopProducing) {
+                break;
+            }
             long sendStartMs = System.currentTimeMillis();
             producer.send(null, String.format("%d", i));
+            
             if (throttler.shouldThrottle(i, sendStartMs)) {
                 throttler.throttle();
             }
         }
     }
+        
 }

From fc7c81c1f6cce497c19da34f7c452ee44800ab6d Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Wed, 10 Jun 2015 18:01:39 -0700
Subject: [PATCH 17/41] added setup.py

---
 .../kafka/tools/TestEndToEndLatency.scala     |  92 +++++++
 tests/kafkatest/__init__.py                   |  13 +
 tests/kafkatest/services/__init__.py          |   0
 tests/kafkatest/services/console_consumer.py  | 142 ++++++++++
 tests/kafkatest/services/kafka.py             | 224 +++++++++++++++
 tests/kafkatest/services/performance.py       | 167 +++++++++++
 .../templates/console_consumer.properties     |   3 +
 .../services/templates/kafka.properties       | 121 ++++++++
 .../services/templates/zookeeper.properties   |   9 +
 .../kafkatest/services/verifiable_producer.py | 108 ++++++++
 tests/kafkatest/services/zookeeper.py         |  62 +++++
 tests/kafkatest/tests/__init__.py             |   0
 tests/kafkatest/tests/kafka_benchmark_test.py | 259 ++++++++++++++++++
 tests/kafkatest/tests/kafka_test.py           |  44 +++
 tests/kafkatest/tests/replication_test.py     | 159 +++++++++++
 tests/setup.py                                |  10 +
 16 files changed, 1413 insertions(+)
 create mode 100755 core/src/main/scala/kafka/tools/TestEndToEndLatency.scala
 create mode 100644 tests/kafkatest/__init__.py
 create mode 100644 tests/kafkatest/services/__init__.py
 create mode 100644 tests/kafkatest/services/console_consumer.py
 create mode 100644 tests/kafkatest/services/kafka.py
 create mode 100644 tests/kafkatest/services/performance.py
 create mode 100644 tests/kafkatest/services/templates/console_consumer.properties
 create mode 100644 tests/kafkatest/services/templates/kafka.properties
 create mode 100644 tests/kafkatest/services/templates/zookeeper.properties
 create mode 100644 tests/kafkatest/services/verifiable_producer.py
 create mode 100644 tests/kafkatest/services/zookeeper.py
 create mode 100644 tests/kafkatest/tests/__init__.py
 create mode 100644 tests/kafkatest/tests/kafka_benchmark_test.py
 create mode 100644 tests/kafkatest/tests/kafka_test.py
 create mode 100644 tests/kafkatest/tests/replication_test.py
 create mode 100644 tests/setup.py

diff --git a/core/src/main/scala/kafka/tools/TestEndToEndLatency.scala b/core/src/main/scala/kafka/tools/TestEndToEndLatency.scala
new file mode 100755
index 0000000000000..3d25e9c053af0
--- /dev/null
+++ b/core/src/main/scala/kafka/tools/TestEndToEndLatency.scala
@@ -0,0 +1,92 @@
+/**
+ * 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 java.util.{Arrays, Properties}
+
+import kafka.consumer._
+import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
+
+import scala.Option.option2Iterable
+
+object TestEndToEndLatency {
+  def main(args: Array[String]) {
+    if (args.length != 6) {
+      System.err.println("USAGE: java " + getClass().getName + " broker_list zookeeper_connect topic num_messages consumer_fetch_max_wait producer_acks")
+      System.exit(1)
+    }
+
+    val brokerList = args(0)
+    val zkConnect = args(1)
+    val topic = args(2)
+    val numMessages = args(3).toInt
+    val consumerFetchMaxWait = args(4).toInt
+    val producerAcks = args(5).toInt
+
+    val consumerProps = new Properties()
+    consumerProps.put("group.id", topic)
+    consumerProps.put("auto.commit.enable", "false")
+    consumerProps.put("auto.offset.reset", "largest")
+    consumerProps.put("zookeeper.connect", zkConnect)
+    consumerProps.put("fetch.wait.max.ms", consumerFetchMaxWait.toString)
+    consumerProps.put("socket.timeout.ms", 1201000.toString)
+
+    val config = new ConsumerConfig(consumerProps)
+    val connector = Consumer.create(config)
+    val stream = connector.createMessageStreams(Map(topic -> 1)).get(topic).head.head
+    val iter = stream.iterator
+
+    val producerProps = new Properties()
+    producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
+    producerProps.put(ProducerConfig.LINGER_MS_CONFIG, "0")
+    producerProps.put(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "true")
+    producerProps.put(ProducerConfig.ACKS_CONFIG, producerAcks.toString)
+    producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer")
+    producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer")
+    val producer = new KafkaProducer[Array[Byte],Array[Byte]](producerProps)
+
+    // make sure the consumer fetcher has started before sending data since otherwise
+    // the consumption from the tail will skip the first message and hence be blocked
+    Thread.sleep(5000)
+
+    val message = "hello there beautiful".getBytes
+    var totalTime = 0.0
+    val latencies = new Array[Long](numMessages)
+    for (i <- 0 until numMessages) {
+      val begin = System.nanoTime
+      producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic, message))
+      val received = iter.next
+      val elapsed = System.nanoTime - begin
+      // poor man's progress bar
+      if (i % 1000 == 0)
+        println(i + "\t" + elapsed / 1000.0 / 1000.0)
+      totalTime += elapsed
+      latencies(i) = (elapsed / 1000 / 1000)
+    }
+    println("Avg latency: %.4f ms\n".format(totalTime / numMessages / 1000.0 / 1000.0))
+    Arrays.sort(latencies)
+    val p50 = latencies((latencies.length * 0.5).toInt)
+    val p99 = latencies((latencies.length * 0.99).toInt) 
+    val p999 = latencies((latencies.length * 0.999).toInt)
+    println("Percentiles: 50th = %d, 99th = %d, 99.9th = %d".format(p50, p99, p999))
+    producer.close()
+    connector.commitOffsets(true)
+    connector.shutdown()
+    System.exit(0)
+  }
+}
\ No newline at end of file
diff --git a/tests/kafkatest/__init__.py b/tests/kafkatest/__init__.py
new file mode 100644
index 0000000000000..1896e9e3bf471
--- /dev/null
+++ b/tests/kafkatest/__init__.py
@@ -0,0 +1,13 @@
+# Copyright 2015 Confluent Inc.
+#
+# Licensed 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.
diff --git a/tests/kafkatest/services/__init__.py b/tests/kafkatest/services/__init__.py
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/tests/kafkatest/services/console_consumer.py b/tests/kafkatest/services/console_consumer.py
new file mode 100644
index 0000000000000..e910a520e6cc9
--- /dev/null
+++ b/tests/kafkatest/services/console_consumer.py
@@ -0,0 +1,142 @@
+# Copyright 2015 Confluent Inc.
+#
+# Licensed 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.
+
+from ducktape.services.background_thread import BackgroundThreadService
+
+
+def is_int(msg):
+    """Default method used to check whether text pulled from console consumer is a message.
+
+    return int or None
+    """
+    try:
+        return int(msg)
+    except:
+        return None
+
+
+"""
+0.8.2.1 ConsoleConsumer options
+
+The console consumer is a tool that reads data from Kafka and outputs it to standard output.
+Option                                  Description
+------                                  -----------
+--blacklist                  Blacklist of topics to exclude from
+                                          consumption.
+--consumer.config          Consumer config properties file.
+--csv-reporter-enabled                  If set, the CSV metrics reporter will
+                                          be enabled
+--delete-consumer-offsets               If specified, the consumer path in
+                                          zookeeper is deleted when starting up
+--formatter                      The name of a class to use for
+                                          formatting kafka messages for
+                                          display. (default: kafka.tools.
+                                          DefaultMessageFormatter)
+--from-beginning                        If the consumer does not already have
+                                          an established offset to consume
+                                          from, start with the earliest
+                                          message present in the log rather
+                                          than the latest message.
+--max-messages   The maximum number of messages to
+                                          consume before exiting. If not set,
+                                          consumption is continual.
+--metrics-dir          If csv-reporter-enable is set, and
+                                          this parameter isset, the csv
+                                          metrics will be outputed here
+--property 
+--skip-message-on-error                 If there is an error when processing a
+                                          message, skip it instead of halt.
+--topic                          The topic id to consume on.
+--whitelist                  Whitelist of topics to include for
+                                          consumption.
+--zookeeper                       REQUIRED: The connection string for
+                                          the zookeeper connection in the form
+                                          host:port. Multiple URLS can be
+                                          given to allow fail-over.
+"""
+
+
+class ConsoleConsumer(BackgroundThreadService):
+    logs = {
+        "consumer_log": {
+            "path": "/mnt/consumer.log",
+            "collect_default": True}
+        }
+
+    def __init__(self, context, num_nodes, kafka, topic, message_validator=is_int, from_beginning=True, consumer_timeout_ms=None):
+        """
+        Args:
+            context:                    standard context
+            num_nodes:                  number of nodes to use (this should be 1)
+            kafka:                      kafka service
+            topic:                      consume from this topic
+            message_validator:          function which returns message or None
+            from_beginning:             consume from beginning if True, else from the end
+            consumer_timeout_ms:        corresponds to consumer.timeout.ms. consumer process ends if time between
+                                        successively consumed messages exceeds this timeout. Setting this and
+                                        waiting for the consumer to stop is a pretty good way to consume all messages
+                                        in a topic.
+        """
+        super(ConsoleConsumer, self).__init__(context, num_nodes)
+        self.kafka = kafka
+        self.args = {
+            'topic': topic,
+        }
+
+        self.consumer_timeout_ms = consumer_timeout_ms
+
+        self.from_beginning = from_beginning
+        self.message_validator = message_validator
+        self.messages_consumed = {idx: [] for idx in range(1, num_nodes + 1)}
+
+    @property
+    def start_cmd(self):
+        args = self.args.copy()
+        args.update({'zk_connect': self.kafka.zk.connect_setting()})
+        cmd = "/opt/kafka/bin/kafka-console-consumer.sh --topic %(topic)s --zookeeper %(zk_connect)s" \
+              " --consumer.config /mnt/console_consumer.properties" % args
+
+        if self.from_beginning:
+            cmd += " --from-beginning"
+
+        cmd += " 2>> /mnt/consumer.log | tee -a /mnt/consumer.log &"
+        return cmd
+
+    def _worker(self, idx, node):
+        # form config file
+        if self.consumer_timeout_ms is not None:
+            prop_file = self.render('console_consumer.properties', consumer_timeout_ms=self.consumer_timeout_ms)
+        else:
+            prop_file = self.render('console_consumer.properties')
+        node.account.create_file("/mnt/console_consumer.properties", prop_file)
+
+        # Run and capture output
+        cmd = self.start_cmd
+        self.logger.debug("Console consumer %d command: %s", idx, cmd)
+        for line in node.account.ssh_capture(cmd):
+            msg = line.strip()
+            msg = self.message_validator(msg)
+            if msg is not None:
+                self.logger.debug("consumed a message: " + str(msg))
+                self.messages_consumed[idx].append(msg)
+
+    def start_node(self, node):
+        super(ConsoleConsumer, self).start_node(node)
+
+    def stop_node(self, node):
+        node.account.kill_process("java", allow_fail=True)
+
+    def clean_node(self, node):
+        node.account.ssh("rm -rf /mnt/console_consumer.properties /mnt/consumer.log", allow_fail=True)
+
diff --git a/tests/kafkatest/services/kafka.py b/tests/kafkatest/services/kafka.py
new file mode 100644
index 0000000000000..976fe39d4a5a5
--- /dev/null
+++ b/tests/kafkatest/services/kafka.py
@@ -0,0 +1,224 @@
+# Copyright 2014 Confluent Inc.
+#
+# Licensed 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.
+
+from ducktape.services.service import Service
+
+import json
+import re
+import signal
+import time
+
+
+class KafkaService(Service):
+
+    logs = {
+        "kafka_log": {
+            "path": "/mnt/kafka.log",
+            "collect_default": True},
+        "kafka_data": {
+            "path": "/mnt/kafka-logs",
+            "collect_default": False}
+    }
+
+    def __init__(self, context, num_nodes, zk, topics=None):
+        """
+        :type context
+        :type zk: ZookeeperService
+        :type topics: dict
+        """
+        super(KafkaService, self).__init__(context, num_nodes)
+        self.zk = zk
+        self.topics = topics
+
+    def start(self):
+        super(KafkaService, self).start()
+
+        # Create topics if necessary
+        if self.topics is not None:
+            for topic, topic_cfg in self.topics.items():
+                if topic_cfg is None:
+                    topic_cfg = {}
+
+                topic_cfg["topic"] = topic
+                self.create_topic(topic_cfg)
+
+    def start_node(self, node):
+        props_file = self.render('kafka.properties', node=node, broker_id=self.idx(node))
+        node.account.create_file("/mnt/kafka.properties", props_file)
+
+        cmd = "/opt/kafka/bin/kafka-server-start.sh /mnt/kafka.properties 1>> /mnt/kafka.log 2>> /mnt/kafka.log & echo $! > /mnt/kafka.pid"
+        self.logger.debug("Attempting to start KafkaService on %s with command: %s" % (str(node.account), cmd))
+        node.account.ssh(cmd)
+        time.sleep(5)
+        if len(self.pids(node)) == 0:
+            raise Exception("No process ids recorded on node %s" % str(node))
+
+    def pids(self, node):
+        """Return process ids associated with running processes on the given node."""
+        try:
+            return [pid for pid in node.account.ssh_capture("cat /mnt/kafka.pid", callback=int)]
+        except:
+            return []
+
+    def signal_node(self, node, sig=signal.SIGTERM):
+        pids = self.pids(node)
+        for pid in pids:
+            node.account.signal(pid, sig)
+
+    def signal_leader(self, topic, partition=0, sig=signal.SIGTERM):
+        leader = self.leader(topic, partition)
+        self.signal_node(leader, sig)
+
+    def stop_node(self, node, clean_shutdown=True, allow_fail=True):
+        pids = self.pids(node)
+        sig = signal.SIGTERM if clean_shutdown else signal.SIGKILL
+
+        for pid in pids:
+            node.account.signal(pid, sig, allow_fail=allow_fail)
+
+        node.account.ssh("rm -f /mnt/kafka.pid", allow_fail=True)
+
+    def clean_node(self, node):
+        node.account.ssh("rm -rf /mnt/kafka-logs /mnt/kafka.properties /mnt/kafka.log")
+
+    def create_topic(self, topic_cfg):
+        node = self.nodes[0] # any node is fine here
+        self.logger.info("Creating topic %s with settings %s", topic_cfg["topic"], topic_cfg)
+
+        cmd = "/opt/kafka/bin/kafka-topics.sh --zookeeper %(zk_connect)s --create "\
+            "--topic %(topic)s --partitions %(partitions)d --replication-factor %(replication)d" % {
+                'zk_connect': self.zk.connect_setting(),
+                'topic': topic_cfg.get("topic"),
+                'partitions': topic_cfg.get('partitions', 1),
+                'replication': topic_cfg.get('replication-factor', 1)
+            }
+
+        if "configs" in topic_cfg.keys() and topic_cfg["configs"] is not None:
+            for config_name, config_value in topic_cfg["configs"].items():
+                cmd += " --config %s=%s" % (config_name, str(config_value))
+
+        self.logger.info("Running topic creation command...\n%s" % cmd)
+        node.account.ssh(cmd)
+
+        time.sleep(1)
+        self.logger.info("Checking to see if topic was properly created...\n%s" % cmd)
+        for line in self.describe_topic(topic_cfg["topic"]).split("\n"):
+            self.logger.info(line)
+
+    def describe_topic(self, topic):
+        node = self.nodes[0]
+        cmd = "/opt/kafka/bin/kafka-topics.sh --zookeeper %s --topic %s --describe" % \
+              (self.zk.connect_setting(), topic)
+        output = ""
+        for line in node.account.ssh_capture(cmd):
+            output += line
+        return output
+
+    def verify_reassign_partitions(self, reassignment):
+        """Run the reassign partitions admin tool in "verify" mode
+        """
+        node = self.nodes[0]
+        json_file = "/tmp/" + str(time.time()) + "_reassign.json"
+
+        # reassignment to json
+        json_str = json.dumps(reassignment)
+        json_str = json.dumps(json_str)
+
+        # create command
+        cmd = "echo %s > %s && " % (json_str, json_file)
+        cmd += "/opt/kafka/bin/kafka-reassign-partitions.sh "\
+                "--zookeeper %(zk_connect)s "\
+                "--reassignment-json-file %(reassignment_file)s "\
+                "--verify" % {'zk_connect': self.zk.connect_setting(),
+                                'reassignment_file': json_file}
+        cmd += " && sleep 1 && rm -f %s" % json_file
+
+        # send command
+        self.logger.info("Verifying parition reassignment...")
+        self.logger.debug(cmd)
+        output = ""
+        for line in node.account.ssh_capture(cmd):
+            output += line
+
+        self.logger.debug(output)
+
+        if re.match(".*is in progress.*", output) is not None:
+            return False
+
+        return True
+
+    def execute_reassign_partitions(self, reassignment):
+        """Run the reassign partitions admin tool in "verify" mode
+        """
+        node = self.nodes[0]
+        json_file = "/tmp/" + str(time.time()) + "_reassign.json"
+
+        # reassignment to json
+        json_str = json.dumps(reassignment)
+        json_str = json.dumps(json_str)
+
+        # create command
+        cmd = "echo %s > %s && " % (json_str, json_file)
+        cmd += "/opt/kafka/bin/kafka-reassign-partitions.sh "\
+                "--zookeeper %(zk_connect)s "\
+                "--reassignment-json-file %(reassignment_file)s "\
+                "--execute" % {'zk_connect': self.zk.connect_setting(),
+                                'reassignment_file': json_file}
+        cmd += " && sleep 1 && rm -f %s" % json_file
+
+        # send command
+        self.logger.info("Executing parition reassignment...")
+        self.logger.debug(cmd)
+        output = ""
+        for line in node.account.ssh_capture(cmd):
+            output += line
+
+        self.logger.debug("Verify partition reassignment:")
+        self.logger.debug(output)
+
+    def restart_node(self, node, wait_sec=0, clean_shutdown=True):
+        """Restart the given node, waiting wait_sec in between stopping and starting up again."""
+        self.stop_node(node, clean_shutdown, allow_fail=True)
+        time.sleep(wait_sec)
+        self.start_node(node)
+
+    def leader(self, topic, partition=0):
+        """ Get the leader replica for the given topic and partition.
+        """
+        cmd = "/opt/kafka/bin/kafka-run-class.sh kafka.tools.ZooKeeperMainWrapper -server %s " \
+              % self.zk.connect_setting()
+        cmd += "get /brokers/topics/%s/partitions/%d/state" % (topic, partition)
+        self.logger.debug(cmd)
+
+        node = self.nodes[0]
+        self.logger.debug("Querying zookeeper to find leader replica for topic %s: \n%s" % (cmd, topic))
+        partition_state = None
+        for line in node.account.ssh_capture(cmd):
+            match = re.match("^({.+})$", line)
+            if match is not None:
+                partition_state = match.groups()[0]
+                break
+
+        if partition_state is None:
+            raise Exception("Error finding partition state for topic %s and partition %d." % (topic, partition))
+
+        partition_state = json.loads(partition_state)
+        self.logger.info(partition_state)
+
+        leader_idx = int(partition_state["leader"])
+        self.logger.info("Leader for topic %s and partition %d is now: %d" % (topic, partition, leader_idx))
+        return self.get_node(leader_idx)
+
+    def bootstrap_servers(self):
+        return ','.join([node.account.hostname + ":9092" for node in self.nodes])
\ No newline at end of file
diff --git a/tests/kafkatest/services/performance.py b/tests/kafkatest/services/performance.py
new file mode 100644
index 0000000000000..423d60a8e5b05
--- /dev/null
+++ b/tests/kafkatest/services/performance.py
@@ -0,0 +1,167 @@
+# Copyright 2014 Confluent Inc.
+#
+# Licensed 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.
+
+from ducktape.services.background_thread import BackgroundThreadService
+
+
+class PerformanceService(BackgroundThreadService):
+    def __init__(self, context, num_nodes):
+        super(PerformanceService, self).__init__(context, num_nodes)
+        self.results = [None] * self.num_nodes
+        self.stats = [[] for x in range(self.num_nodes)]
+
+
+class ProducerPerformanceService(PerformanceService):
+    def __init__(self, context, num_nodes, kafka, topic, num_records, record_size, throughput, settings={}, intermediate_stats=False):
+        super(ProducerPerformanceService, self).__init__(context, num_nodes)
+        self.kafka = kafka
+        self.args = {
+            'topic': topic,
+            'num_records': num_records,
+            'record_size': record_size,
+            'throughput': throughput
+        }
+        self.settings = settings
+        self.intermediate_stats = intermediate_stats
+
+    def _worker(self, idx, node):
+        args = self.args.copy()
+        args.update({'bootstrap_servers': self.kafka.bootstrap_servers()})
+        cmd = "/opt/kafka/bin/kafka-run-class.sh org.apache.kafka.clients.tools.ProducerPerformance "\
+              "%(topic)s %(num_records)d %(record_size)d %(throughput)d bootstrap.servers=%(bootstrap_servers)s" % args
+
+        for key,value in self.settings.items():
+            cmd += " %s=%s" % (str(key), str(value))
+        self.logger.debug("Producer performance %d command: %s", idx, cmd)
+
+        def parse_stats(line):
+            parts = line.split(',')
+            return {
+                'records': int(parts[0].split()[0]),
+                'records_per_sec': float(parts[1].split()[0]),
+                'mbps': float(parts[1].split('(')[1].split()[0]),
+                'latency_avg_ms': float(parts[2].split()[0]),
+                'latency_max_ms': float(parts[3].split()[0]),
+                'latency_50th_ms': float(parts[4].split()[0]),
+                'latency_95th_ms': float(parts[5].split()[0]),
+                'latency_99th_ms': float(parts[6].split()[0]),
+                'latency_999th_ms': float(parts[7].split()[0]),
+            }
+        last = None
+        for line in node.account.ssh_capture(cmd):
+            self.logger.debug("Producer performance %d: %s", idx, line.strip())
+            if self.intermediate_stats:
+                try:
+                    self.stats[idx-1].append(parse_stats(line))
+                except:
+                    # Sometimes there are extraneous log messages
+                    pass
+            last = line
+        try:
+            self.results[idx-1] = parse_stats(last)
+        except:
+            self.logger.error("Bad last line: %s", last)
+
+
+class ConsumerPerformanceService(PerformanceService):
+    def __init__(self, context, num_nodes, kafka, topic, num_records, throughput, threads=1, settings={}):
+        super(ConsumerPerformanceService, self).__init__(context, num_nodes)
+        self.kafka = kafka
+        self.args = {
+            'topic': topic,
+            'num_records': num_records,
+            'throughput': throughput,
+            'threads': threads,
+        }
+        self.settings = settings
+
+    def _worker(self, idx, node):
+        args = self.args.copy()
+        args.update({'zk_connect': self.kafka.zk.connect_setting()})
+        cmd = "/opt/kafka/bin/kafka-consumer-perf-test.sh "\
+              "--topic %(topic)s --messages %(num_records)d --zookeeper %(zk_connect)s" % args
+        for key,value in self.settings.items():
+            cmd += " %s=%s" % (str(key), str(value))
+        self.logger.debug("Consumer performance %d command: %s", idx, cmd)
+        last = None
+        for line in node.account.ssh_capture(cmd):
+            self.logger.debug("Consumer performance %d: %s", idx, line.strip())
+            last = line
+        # Parse and save the last line's information
+        parts = last.split(',')
+
+        print "=" * 20
+        print "ConsumerPerformanceService data:"
+        print parts
+        print "-" * 20
+
+        self.results[idx-1] = {
+            'total_mb': float(parts[3]),
+            'mbps': float(parts[4]),
+            'records_per_sec': float(parts[5]),
+        }
+
+
+class EndToEndLatencyService(PerformanceService):
+    def __init__(self, context, num_nodes, kafka, topic, num_records, consumer_fetch_max_wait=100, acks=1):
+        super(EndToEndLatencyService, self).__init__(context, num_nodes)
+        self.kafka = kafka
+        self.args = {
+            'topic': topic,
+            'num_records': num_records,
+            'consumer_fetch_max_wait': consumer_fetch_max_wait,
+            'acks': acks
+        }
+
+    def _worker(self, idx, node):
+        args = self.args.copy()
+        args.update({
+            'zk_connect': self.kafka.zk.connect_setting(),
+            'bootstrap_servers': self.kafka.bootstrap_servers(),
+        })
+        cmd = "/opt/kafka/bin/kafka-run-class.sh kafka.tools.TestEndToEndLatency "\
+              "%(bootstrap_servers)s %(zk_connect)s %(topic)s %(num_records)d "\
+              "%(consumer_fetch_max_wait)d %(acks)d" % args
+        self.logger.debug("End-to-end latency %d command: %s", idx, cmd)
+        results = {}
+        for line in node.account.ssh_capture(cmd):
+            self.logger.debug("End-to-end latency %d: %s", idx, line.strip())
+            if line.startswith("Avg latency:"):
+                results['latency_avg_ms'] = float(line.split()[2])
+            if line.startswith("Percentiles"):
+                results['latency_50th_ms'] = float(line.split()[3][:-1])
+                results['latency_99th_ms'] = float(line.split()[6][:-1])
+                results['latency_999th_ms'] = float(line.split()[9])
+        self.results[idx-1] = results
+
+
+def parse_performance_output(summary):
+        parts = summary.split(',')
+        results = {
+            'records': int(parts[0].split()[0]),
+            'records_per_sec': float(parts[1].split()[0]),
+            'mbps': float(parts[1].split('(')[1].split()[0]),
+            'latency_avg_ms': float(parts[2].split()[0]),
+            'latency_max_ms': float(parts[3].split()[0]),
+            'latency_50th_ms': float(parts[4].split()[0]),
+            'latency_95th_ms': float(parts[5].split()[0]),
+            'latency_99th_ms': float(parts[6].split()[0]),
+            'latency_999th_ms': float(parts[7].split()[0]),
+        }
+        # To provide compatibility with ConsumerPerformanceService
+        results['total_mb'] = results['mbps'] * (results['records'] / results['records_per_sec'])
+        results['rate_mbps'] = results['mbps']
+        results['rate_mps'] = results['records_per_sec']
+
+        return results
diff --git a/tests/kafkatest/services/templates/console_consumer.properties b/tests/kafkatest/services/templates/console_consumer.properties
new file mode 100644
index 0000000000000..9179a199fd496
--- /dev/null
+++ b/tests/kafkatest/services/templates/console_consumer.properties
@@ -0,0 +1,3 @@
+{% if consumer_timeout_ms is defined %}
+consumer.timeout.ms={{ consumer_timeout_ms }}
+{% endif %}
\ No newline at end of file
diff --git a/tests/kafkatest/services/templates/kafka.properties b/tests/kafkatest/services/templates/kafka.properties
new file mode 100644
index 0000000000000..db1077a4a4eb8
--- /dev/null
+++ b/tests/kafkatest/services/templates/kafka.properties
@@ -0,0 +1,121 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+# 
+#    http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+# see kafka.server.KafkaConfig for additional details and defaults
+
+############################# Server Basics #############################
+
+# The id of the broker. This must be set to a unique integer for each broker.
+broker.id={{ broker_id }}
+
+############################# Socket Server Settings #############################
+
+# The port the socket server listens on
+port=9092
+
+# Hostname the broker will bind to. If not set, the server will bind to all interfaces
+#host.name=localhost
+
+# Hostname the broker will advertise to producers and consumers. If not set, it uses the
+# value for "host.name" if configured.  Otherwise, it will use the value returned from
+# java.net.InetAddress.getCanonicalHostName().
+advertised.host.name={{ node.account.hostname }}
+
+# The port to publish to ZooKeeper for clients to use. If this is not set,
+# it will publish the same port that the broker binds to.
+#advertised.port=
+
+# The number of threads handling network requests
+num.network.threads=3
+ 
+# The number of threads doing disk I/O
+num.io.threads=8
+
+# The send buffer (SO_SNDBUF) used by the socket server
+socket.send.buffer.bytes=102400
+
+# The receive buffer (SO_RCVBUF) used by the socket server
+socket.receive.buffer.bytes=65536
+
+# The maximum size of a request that the socket server will accept (protection against OOM)
+socket.request.max.bytes=104857600
+
+
+############################# Log Basics #############################
+
+# A comma seperated list of directories under which to store log files
+log.dirs=/mnt/kafka-logs
+
+# The default number of log partitions per topic. More partitions allow greater
+# parallelism for consumption, but this will also result in more files across
+# the brokers.
+num.partitions=1
+
+# The number of threads per data directory to be used for log recovery at startup and flushing at shutdown.
+# This value is recommended to be increased for installations with data dirs located in RAID array.
+num.recovery.threads.per.data.dir=1
+
+############################# Log Flush Policy #############################
+
+# Messages are immediately written to the filesystem but by default we only fsync() to sync
+# the OS cache lazily. The following configurations control the flush of data to disk. 
+# There are a few important trade-offs here:
+#    1. Durability: Unflushed data may be lost if you are not using replication.
+#    2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as there will be a lot of data to flush.
+#    3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to exceessive seeks. 
+# The settings below allow one to configure the flush policy to flush data after a period of time or
+# every N messages (or both). This can be done globally and overridden on a per-topic basis.
+
+# The number of messages to accept before forcing a flush of data to disk
+#log.flush.interval.messages=10000
+
+# The maximum amount of time a message can sit in a log before we force a flush
+#log.flush.interval.ms=1000
+
+############################# Log Retention Policy #############################
+
+# The following configurations control the disposal of log segments. The policy can
+# be set to delete segments after a period of time, or after a given size has accumulated.
+# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens
+# from the end of the log.
+
+# The minimum age of a log file to be eligible for deletion
+log.retention.hours=168
+
+# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining
+# segments don't drop below log.retention.bytes.
+#log.retention.bytes=1073741824
+
+# The maximum size of a log segment file. When this size is reached a new log segment will be created.
+log.segment.bytes=1073741824
+
+# The interval at which log segments are checked to see if they can be deleted according 
+# to the retention policies
+log.retention.check.interval.ms=300000
+
+# By default the log cleaner is disabled and the log retention policy will default to just delete segments after their retention expires.
+# If log.cleaner.enable=true is set the cleaner will be enabled and individual logs can then be marked for log compaction.
+log.cleaner.enable=false
+
+############################# Zookeeper #############################
+
+# Zookeeper connection string (see zookeeper docs for details).
+# This is a comma separated host:port pairs, each corresponding to a zk
+# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002".
+# You can also append an optional chroot string to the urls to specify the
+# root directory for all kafka znodes.
+zookeeper.connect={{ zk.connect_setting() }}
+
+# Timeout in ms for connecting to zookeeper
+zookeeper.connection.timeout.ms=2000
diff --git a/tests/kafkatest/services/templates/zookeeper.properties b/tests/kafkatest/services/templates/zookeeper.properties
new file mode 100644
index 0000000000000..740c30aa32c49
--- /dev/null
+++ b/tests/kafkatest/services/templates/zookeeper.properties
@@ -0,0 +1,9 @@
+dataDir=/mnt/zookeeper
+clientPort=2181
+maxClientCnxns=0
+initLimit=5
+syncLimit=2
+quorumListenOnAllIPs=true
+{% for node in nodes %}
+server.{{ loop.index }}={{ node.account.hostname }}:2888:3888
+{% endfor %}
\ No newline at end of file
diff --git a/tests/kafkatest/services/verifiable_producer.py b/tests/kafkatest/services/verifiable_producer.py
new file mode 100644
index 0000000000000..88455601dfae7
--- /dev/null
+++ b/tests/kafkatest/services/verifiable_producer.py
@@ -0,0 +1,108 @@
+# Copyright 2015 Confluent Inc.
+#
+# Licensed 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.
+
+from ducktape.services.background_thread import BackgroundThreadService
+
+import json
+
+
+class VerifiableProducer(BackgroundThreadService):
+
+    logs = {
+        "producer_log": {
+            "path": "/mnt/producer.log",
+            "collect_default": True}
+    }
+
+    def __init__(self, context, num_nodes, kafka, topic, max_messages=-1, throughput=100000):
+        super(VerifiableProducer, self).__init__(context, num_nodes)
+
+        self.kafka = kafka
+        self.topic = topic
+        self.max_messages = max_messages
+        self.throughput = throughput
+
+        self.acked_values = []
+        self.not_acked_values = []
+
+    def _worker(self, idx, node):
+        cmd = self.start_cmd
+        self.logger.debug("Verbose producer %d command: %s" % (idx, cmd))
+
+        for line in node.account.ssh_capture(cmd):
+            line = line.strip()
+
+            data = self.try_parse_json(line)
+            if data is not None:
+
+                self.logger.debug("VerifiableProducer: " + str(data))
+
+                with self.lock:
+                    if data["name"] == "producer_send_error":
+                        data["node"] = idx
+                        self.not_acked_values.append(int(data["value"]))
+
+                    elif data["name"] == "producer_send_success":
+                        self.acked_values.append(int(data["value"]))
+
+    @property
+    def start_cmd(self):
+        cmd = "/opt/kafka/bin/kafka-verifiable-producer.sh" \
+              " --topic %s --broker-list %s" % (self.topic, self.kafka.bootstrap_servers())
+        if self.max_messages > 0:
+            cmd += " --max-messages %s" % str(self.max_messages)
+        if self.throughput > 0:
+            cmd += " --throughput %s" % str(self.throughput)
+
+        cmd += " 2>> /mnt/producer.log | tee -a /mnt/producer.log &"
+        return cmd
+
+    @property
+    def acked(self):
+        with self.lock:
+            return self.acked_values
+
+    @property
+    def not_acked(self):
+        with self.lock:
+            return self.not_acked_values
+
+    @property
+    def num_acked(self):
+        with self.lock:
+            return len(self.acked_values)
+
+    @property
+    def num_not_acked(self):
+        with self.lock:
+            return len(self.not_acked_values)
+
+    def stop_node(self, node):
+        node.account.kill_process("VerifiableProducer", allow_fail=False)
+        # block until the corresponding thread exits
+        if len(self.worker_threads) >= self.idx(node):
+            # Need to guard this because stop is preemptively called before the worker threads are added and started
+            self.worker_threads[self.idx(node) - 1].join()
+
+    def clean_node(self, node):
+        node.account.ssh("rm -rf /mnt/producer.log")
+
+    def try_parse_json(self, string):
+        """Try to parse a string as json. Return None if not parseable."""
+        try:
+            record = json.loads(string)
+            return record
+        except ValueError:
+            self.logger.debug("Could not parse as json: %s" % str(string))
+            return None
diff --git a/tests/kafkatest/services/zookeeper.py b/tests/kafkatest/services/zookeeper.py
new file mode 100644
index 0000000000000..4c7e5a78f5fe9
--- /dev/null
+++ b/tests/kafkatest/services/zookeeper.py
@@ -0,0 +1,62 @@
+# Copyright 2015 Confluent Inc.
+#
+# Licensed 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.
+
+from ducktape.services.service import Service
+
+import time
+
+
+class ZookeeperService(Service):
+
+    logs = {
+        "zk_log": {
+            "path": "/mnt/zk.log",
+            "collect_default": True}
+    }
+
+    def __init__(self, context, num_nodes):
+        """
+        :type context
+        """
+        super(ZookeeperService, self).__init__(context, num_nodes)
+
+    def start_node(self, node):
+        idx = self.idx(node)
+        self.logger.info("Starting ZK node %d on %s", idx, node.account.hostname)
+
+        node.account.ssh("mkdir -p /mnt/zookeeper")
+        node.account.ssh("echo %d > /mnt/zookeeper/myid" % idx)
+        node.account.create_file("/mnt/zookeeper.properties", self.render('zookeeper.properties'))
+
+        node.account.ssh(
+            "/opt/kafka/bin/zookeeper-server-start.sh /mnt/zookeeper.properties 1>> %(path)s 2>> %(path)s &"
+            % self.logs["zk_log"])
+
+        time.sleep(5)  # give it some time to start
+
+    def stop_node(self, node, allow_fail=True):
+        # This uses Kafka-REST's stop service script because it's better behaved
+        # (knows how to wait) and sends SIGTERM instead of
+        # zookeeper-stop-server.sh's SIGINT. We don't actually care about clean
+        # shutdown here, so it's ok to use the bigger hammer
+        idx = self.idx(node)
+        self.logger.info("Stopping %s node %d on %s" % (type(self).__name__, idx, node.account.hostname))
+        node.account.ssh("/opt/kafka-rest/bin/kafka-rest-stop-service zookeeper", allow_fail=allow_fail)
+
+    def clean_node(self, node, allow_fail=True):
+        self.logger.info("Cleaning ZK node %d on %s", self.idx(node), node.account.hostname)
+        node.account.ssh("rm -rf /mnt/zookeeper /mnt/zookeeper.properties /mnt/zk.log", allow_fail=allow_fail)
+
+    def connect_setting(self):
+        return ','.join([node.account.hostname + ':2181' for node in self.nodes])
diff --git a/tests/kafkatest/tests/__init__.py b/tests/kafkatest/tests/__init__.py
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/tests/kafkatest/tests/kafka_benchmark_test.py b/tests/kafkatest/tests/kafka_benchmark_test.py
new file mode 100644
index 0000000000000..e05e57e44bce3
--- /dev/null
+++ b/tests/kafkatest/tests/kafka_benchmark_test.py
@@ -0,0 +1,259 @@
+# Copyright 2014 Confluent Inc.
+#
+# Licensed 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.
+
+from ducktape.services.service import Service
+
+from kafkatest.tests.kafka_test import KafkaTest
+from kafkatest.services.performance import ProducerPerformanceService, ConsumerPerformanceService, EndToEndLatencyService
+
+
+class KafkaBenchmark(KafkaTest):
+    '''A benchmark of Kafka producer/consumer performance. This replicates the test
+    run here:
+    https://engineering.linkedin.com/kafka/benchmarking-apache-kafka-2-million-writes-second-three-cheap-machines
+    '''
+    def __init__(self, test_context):
+        super(KafkaBenchmark, self).__init__(test_context, num_zk=1, num_brokers=3, topics={
+            'test-rep-one' : { 'partitions': 6, 'replication-factor': 1 },
+            'test-rep-three' : { 'partitions': 6, 'replication-factor': 3 }
+        })
+
+        if True:
+            # Works on both aws and local
+            self.msgs = 1000000
+            self.msgs_default = 1000000
+        else:
+            # Can use locally on Vagrant VMs, but may use too much memory for aws
+            self.msgs = 50000000
+            self.msgs_default = 50000000
+
+        self.msgs_large = 10000000
+        self.msg_size_default = 100
+        self.batch_size = 8*1024
+        self.buffer_memory = 64*1024*1024
+        self.msg_sizes = [10, 100, 1000, 10000, 100000]
+        self.target_data_size = 128*1024*1024
+        self.target_data_size_gb = self.target_data_size/float(1024*1024*1024)
+
+    def test_single_producer_no_replication(self):
+        self.logger.info("BENCHMARK: Single producer, no replication")
+        self.perf = ProducerPerformanceService(
+            self.test_context, 1, self.kafka,
+            topic="test-rep-one", num_records=self.msgs_default, record_size=self.msg_size_default, throughput=-1,
+            settings={'acks':1, 'batch.size':self.batch_size, 'buffer.memory':self.buffer_memory}
+        )
+        self.perf.run()
+        data = compute_throughput(self.perf)
+        self.logger.info("Single producer, no replication: %s", str(data))
+        return data
+
+    def test_single_producer_replication(self):
+        self.logger.info("BENCHMARK: Single producer, async 3x replication")
+        self.perf = ProducerPerformanceService(
+            self.test_context, 1, self.kafka,
+            topic="test-rep-three", num_records=self.msgs_default, record_size=self.msg_size_default, throughput=-1,
+            settings={'acks':1, 'batch.size':self.batch_size, 'buffer.memory':self.buffer_memory}
+        )
+        self.perf.run()
+        data = compute_throughput(self.perf)
+        self.logger.info("Single producer, async 3x replication: %s" % str(data))
+        return data
+
+    def test_single_producer_sync(self):
+        self.logger.info("BENCHMARK: Single producer, sync 3x replication")
+        self.perf = ProducerPerformanceService(
+            self.test_context, 1, self.kafka,
+            topic="test-rep-three", num_records=self.msgs_default, record_size=self.msg_size_default, throughput=-1,
+            settings={'acks':-1, 'batch.size':self.batch_size, 'buffer.memory':self.buffer_memory}
+        )
+        self.perf.run()
+
+        data = compute_throughput(self.perf)
+        self.logger.info("Single producer, sync 3x replication: %s" % data)
+        return data
+
+    def test_three_producers_async(self):
+        self.logger.info("BENCHMARK: Three producers, async 3x replication")
+        self.perf = ProducerPerformanceService(
+            self.test_context, 3, self.kafka,
+            topic="test-rep-three", num_records=self.msgs_default, record_size=self.msg_size_default, throughput=-1,
+            settings={'acks':1, 'batch.size':self.batch_size, 'buffer.memory':self.buffer_memory}
+        )
+        self.perf.run()
+
+        data = compute_throughput(self.perf)
+        self.logger.info("Three producers, async 3x replication: %s" % data)
+        return data
+
+    def test_multiple_message_size(self):
+        # TODO this would be a great place to use parametrization
+        self.perfs = {}
+        for msg_size in self.msg_sizes:
+            self.logger.info("BENCHMARK: Message size %d (%f GB total, single producer, async 3x replication)", msg_size, self.target_data_size_gb)
+            # Always generate the same total amount of data
+            nrecords = int(self.target_data_size / msg_size)
+            self.perfs["perf-" + str(msg_size)] = ProducerPerformanceService(
+                self.test_context, 1, self.kafka,
+                topic="test-rep-three", num_records=nrecords, record_size=msg_size, throughput=-1,
+                settings={'acks': 1, 'batch.size': self.batch_size, 'buffer.memory': self.buffer_memory}
+            )
+
+        self.msg_size_perf = {}
+        for msg_size in self.msg_sizes:
+            perf = self.perfs["perf-" + str(msg_size)]
+            perf.run()
+            self.msg_size_perf[msg_size] = perf
+
+        summary = ["Message size:"]
+        data = {}
+        for msg_size in self.msg_sizes:
+            datum = compute_throughput(self.msg_size_perf[msg_size])
+            summary.append(" %d: %s" % (msg_size, datum))
+            data[msg_size] = datum
+        self.logger.info("\n".join(summary))
+        return data
+
+    def test_long_term_throughput(self):
+        self.logger.info("BENCHMARK: Long production")
+        self.perf = ProducerPerformanceService(
+            self.test_context, 1, self.kafka,
+            topic="test-rep-three", num_records=self.msgs_large, record_size=self.msg_size_default, throughput=-1,
+            settings={'acks':1, 'batch.size':self.batch_size, 'buffer.memory':self.buffer_memory},
+            intermediate_stats=True
+        )
+        self.perf.run()
+
+        summary = ["Throughput over long run, data > memory:"]
+        data = {}
+        # FIXME we should be generating a graph too
+        # Try to break it into 5 blocks, but fall back to a smaller number if
+        # there aren't even 5 elements
+        block_size = max(len(self.perf.stats[0]) / 5, 1)
+        nblocks = len(self.perf.stats[0]) / block_size
+        for i in range(nblocks):
+            subset = self.perf.stats[0][i*block_size:min((i+1)*block_size, len(self.perf.stats[0]))]
+            if len(subset) == 0:
+                summary.append(" Time block %d: (empty)" % i)
+                data[i] = None
+            else:
+                records_per_sec = sum([stat['records_per_sec'] for stat in subset])/float(len(subset))
+                mb_per_sec = sum([stat['mbps'] for stat in subset])/float(len(subset))
+
+                summary.append(" Time block %d: %f rec/sec (%f MB/s)" % (i, records_per_sec, mb_per_sec))
+                data[i] = throughput(records_per_sec, mb_per_sec)
+
+        self.logger.info("\n".join(summary))
+        return data
+
+    def test_end_to_end_latency(self):
+        self.logger.info("BENCHMARK: End to end latency")
+        self.perf = EndToEndLatencyService(
+            self.test_context, 1, self.kafka,
+            topic="test-rep-three", num_records=10000
+        )
+        self.perf.run()
+
+        data = latency(self.perf.results[0]['latency_50th_ms'],  self.perf.results[0]['latency_99th_ms'], self.perf.results[0]['latency_999th_ms'])
+        self.logger.info("End-to-end latency: %s" % str(data))
+        return data
+
+    def test_producer_and_consumer(self):
+        self.logger.info("BENCHMARK: Producer + Consumer")
+        self.producer = ProducerPerformanceService(
+            self.test_context, 1, self.kafka,
+            topic="test-rep-three", num_records=self.msgs_default, record_size=self.msg_size_default, throughput=-1,
+            settings={'acks':1, 'batch.size':self.batch_size, 'buffer.memory':self.buffer_memory}
+        )
+
+        self.consumer = ConsumerPerformanceService(
+            self.test_context, 1, self.kafka,
+            topic="test-rep-three", num_records=self.msgs_default, throughput=-1, threads=1
+        )
+
+        Service.run_parallel(self.producer, self.consumer)
+
+        data = {
+            "producer": compute_throughput(self.producer),
+            "consumer": compute_throughput(self.consumer)
+        }
+        summary = [
+            "Producer + consumer:",
+            str(data)]
+        self.logger.info("\n".join(summary))
+        return data
+
+    def test_single_consumer(self):
+        # All consumer tests use the messages from the first benchmark, so
+        # they'll get messages of the default message size
+        self.logger.info("BENCHMARK: Single consumer")
+        self.perf = ConsumerPerformanceService(
+            self.test_context, 1, self.kafka,
+            topic="test-rep-three", num_records=self.msgs_default, throughput=-1, threads=1
+        )
+        self.perf.run()
+
+        data = compute_throughput(self.perf)
+        self.logger.info("Single consumer: %s" % data)
+        return data
+
+    def test_three_consumers(self):
+        self.logger.info("BENCHMARK: Three consumers")
+        self.perf = ConsumerPerformanceService(
+            self.test_context, 3, self.kafka,
+            topic="test-rep-three", num_records=self.msgs_default, throughput=-1, threads=1
+        )
+        self.perf.run()
+
+        data = compute_throughput(self.perf)
+        self.logger.info("Three consumers: %s", data)
+        return data
+
+
+def throughput(records_per_sec, mb_per_sec):
+    """Helper method to ensure uniform representation of throughput data"""
+    return {
+        "records_per_sec": records_per_sec,
+        "mb_per_sec": mb_per_sec
+    }
+
+
+def latency(latency_50th_ms, latency_99th_ms, latency_999th_ms):
+    """Helper method to ensure uniform representation of latency data"""
+    return {
+        "latency_50th_ms": latency_50th_ms,
+        "latency_99th_ms": latency_99th_ms,
+        "latency_999th_ms": latency_999th_ms
+    }
+
+
+def compute_throughput(perf):
+    print "=" * 20
+    print perf.results
+    print "-" * 20
+    
+    """Helper method for computing throughput after running a performance service."""
+    aggregate_rate = sum([r['records_per_sec'] for r in perf.results])
+    aggregate_mbps = sum([r['mb_per_sec'] for r in perf.results])
+
+    return throughput(aggregate_rate, aggregate_mbps)
+
+
+
+
+
+
+
+
+
+
diff --git a/tests/kafkatest/tests/kafka_test.py b/tests/kafkatest/tests/kafka_test.py
new file mode 100644
index 0000000000000..3cbdf2f622808
--- /dev/null
+++ b/tests/kafkatest/tests/kafka_test.py
@@ -0,0 +1,44 @@
+# Copyright 2014 Confluent Inc.
+#
+# Licensed 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.
+
+from ducktape.tests.test import Test
+
+
+from kafkatest.services.zookeeper import ZookeeperService
+from kafkatest.services.kafka import KafkaService
+
+
+class KafkaTest(Test):
+    """
+    Helper class that manages setting up a Kafka cluster. Use this if the
+    default settings for Kafka are sufficient for your test; any customization
+    needs to be done manually. Your run() method should call tearDown and
+    setUp. The Zookeeper and Kafka services are available as the fields
+    KafkaTest.zk and KafkaTest.kafka.
+    """
+    def __init__(self, test_context, num_zk, num_brokers, topics=None):
+        super(KafkaTest, self).__init__(test_context)
+        self.num_zk = num_zk
+        self.num_brokers = num_brokers
+        self.topics = topics
+
+        self.zk = ZookeeperService(test_context, self.num_zk)
+
+        self.kafka = KafkaService(
+            test_context, self.num_brokers,
+            self.zk, topics=self.topics)
+
+    def setUp(self):
+        self.zk.start()
+        self.kafka.start()
\ No newline at end of file
diff --git a/tests/kafkatest/tests/replication_test.py b/tests/kafkatest/tests/replication_test.py
new file mode 100644
index 0000000000000..91c031e2ca0ba
--- /dev/null
+++ b/tests/kafkatest/tests/replication_test.py
@@ -0,0 +1,159 @@
+# Copyright 2015 Confluent Inc.
+#
+# Licensed 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.
+
+from ducktape.tests.test import Test
+from ducktape.utils.util import wait_until
+
+from kafkatest.services.zookeeper import ZookeeperService
+from kafkatest.services.kafka import KafkaService
+from kafkatest.services.verifiable_producer import VerifiableProducer
+from kafkatest.services.console_consumer import ConsoleConsumer
+
+import signal
+import time
+
+
+class ReplicationTest(Test):
+    """Replication tests.
+    These tests verify that replication provides simple durability guarantees by checking that data acked by
+    brokers is still available for consumption in the face of various failure scenarios."""
+
+    def __init__(self, test_context):
+        """:type test_context: ducktape.tests.test.TestContext"""
+        super(ReplicationTest, self).__init__(test_context=test_context)
+
+        self.topic = "test_topic"
+        self.zk = ZookeeperService(test_context, num_nodes=1)
+        self.kafka = KafkaService(test_context, num_nodes=3, zk=self.zk, topics={self.topic: {
+                                                                    "partitions": 3,
+                                                                    "replication-factor": 3,
+                                                                    "min.insync.replicas": 2}
+                                                                })
+        self.producer_throughput = 10000
+        self.num_producers = 1
+        self.num_consumers = 1
+
+    def setUp(self):
+        self.zk.start()
+        self.kafka.start()
+
+    def min_cluster_size(self):
+        """Override this since we're adding services outside of the constructor"""
+        return super(ReplicationTest, self).min_cluster_size() + self.num_producers + self.num_consumers
+
+    def run_with_failure(self, failure):
+        """This is the top-level test template.
+
+        The steps are:
+            Produce messages in the background while driving some failure condition
+            When done driving failures, immediately stop producing
+            Consume all messages
+            Validate that messages acked by brokers were consumed
+
+        Note that consuming is a bit tricky, at least with console consumer. The goal is to consume all messages
+        (foreach partition) in the topic. In this case, waiting for the last message may cause the consumer to stop
+        too soon since console consumer is consuming multiple partitions from a single thread and therefore we lose
+        ordering guarantees.
+
+        Waiting on a count of consumed messages can be unreliable: if we stop consuming when num_consumed == num_acked,
+        we might exit early if some messages are duplicated (though not an issue here since producer retries==0)
+
+        Therefore rely here on the consumer.timeout.ms setting which times out on the interval between successively
+        consumed messages. Since we run the producer to completion before running the consumer, this is a reliable
+        indicator that nothing is left to consume.
+
+        """
+        self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic, throughput=self.producer_throughput)
+        self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic, consumer_timeout_ms=3000)
+
+        # Produce in a background thread while driving broker failures
+        self.producer.start()
+        if not wait_until(lambda: self.producer.num_acked > 5, timeout_sec=5):
+            raise RuntimeError("Producer failed to start in a reasonable amount of time.")
+        failure()
+        self.producer.stop()
+
+        self.acked = self.producer.acked
+        self.not_acked = self.producer.not_acked
+        self.logger.info("num not acked: %d" % self.producer.num_not_acked)
+        self.logger.info("num acked:     %d" % self.producer.num_acked)
+
+        # Consume all messages
+        self.consumer.start()
+        self.consumer.wait()
+        self.consumed = self.consumer.messages_consumed[1]
+        self.logger.info("num consumed:  %d" % len(self.consumed))
+
+        # Check produced vs consumed
+        self.validate()
+
+    def clean_shutdown(self):
+        """Discover leader node for our topic and shut it down cleanly."""
+        self.kafka.signal_leader(self.topic, partition=0, sig=signal.SIGTERM)
+
+    def hard_shutdown(self):
+        """Discover leader node for our topic and shut it down with a hard kill."""
+        self.kafka.signal_leader(self.topic, partition=0, sig=signal.SIGKILL)
+
+    def clean_bounce(self):
+        """Chase the leader of one partition and restart it cleanly."""
+        for i in range(5):
+            prev_leader_node = self.kafka.leader(topic=self.topic, partition=0)
+            self.kafka.restart_node(prev_leader_node, wait_sec=5, clean_shutdown=True)
+
+    def hard_bounce(self):
+        """Chase the leader and restart it cleanly."""
+        for i in range(5):
+            prev_leader_node = self.kafka.leader(topic=self.topic, partition=0)
+            self.kafka.restart_node(prev_leader_node, wait_sec=5, clean_shutdown=False)
+
+            # Wait long enough for previous leader to probably be awake again
+            time.sleep(6)
+
+    def validate(self):
+        """Check that produced messages were consumed."""
+
+        success = True
+        msg = ""
+
+        if len(set(self.consumed)) != len(self.consumed):
+            # There are duplicates. This is ok, so report it but don't fail the test
+            msg += "There are duplicate messages in the log\n"
+
+        if not set(self.consumed).issuperset(set(self.acked)):
+            # Every acked message must appear in the logs. I.e. consumed messages must be superset of acked messages.
+            acked_minus_consumed = set(self.producer.acked) - set(self.consumed)
+            success = False
+            msg += "At least one acked message did not appear in the consumed messages. acked_minus_consumed: " + str(acked_minus_consumed)
+
+        if not success:
+            # Collect all the data logs if there was a failure
+            self.mark_for_collect(self.kafka)
+
+        assert success, msg
+
+    def test_clean_shutdown(self):
+        self.run_with_failure(self.clean_shutdown)
+
+    def test_hard_shutdown(self):
+        self.run_with_failure(self.hard_shutdown)
+
+    def test_clean_bounce(self):
+        self.run_with_failure(self.clean_bounce)
+
+    def test_hard_bounce(self):
+        self.run_with_failure(self.hard_bounce)
+
+
+
diff --git a/tests/setup.py b/tests/setup.py
new file mode 100644
index 0000000000000..8cf40b2a5186b
--- /dev/null
+++ b/tests/setup.py
@@ -0,0 +1,10 @@
+from setuptools import find_packages, setup
+
+setup(name="kafkatest",
+      version="0.1",
+      description="System tests for Apache Kafka",
+      author="Ewen Cheslack-Postava , Geoff Anderson ",
+      platforms=["any"], 
+      license="apache2.0",
+      packages=find_packages(),
+      )

From 884b20e3a7ce7a94f22594782322e4366b51f7eb Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Wed, 10 Jun 2015 18:02:11 -0700
Subject: [PATCH 18/41] Moved a bunch of files to kafkatest directory

---
 .../kafka/tools/TestEndToEndLatency.scala     |  91 --------
 tests/README.md                               |   8 +-
 tests/__init__.py                             |   1 -
 tests/kafkatest/services/console_consumer.py  |   3 +
 tests/kafkatest/services/kafka.py             |   2 +
 tests/kafkatest/services/performance.py       |   5 -
 tests/kafkatest/services/zookeeper.py         |   6 +-
 tests/kafkatest/tests/kafka_benchmark_test.py |   6 +-
 tests/services/__init__.py                    |   0
 tests/services/kafka_service.py               | 212 ------------------
 tests/services/performance.py                 | 189 ----------------
 tests/services/zookeeper_service.py           |  75 -------
 tests/templates/kafka.properties              | 121 ----------
 tests/tests/__init__.py                       |   0
 tests/tests/kafka_benchmark_test.py           | 193 ----------------
 tests/tests/test.py                           |  51 -----
 16 files changed, 18 insertions(+), 945 deletions(-)
 delete mode 100755 core/src/test/scala/kafka/tools/TestEndToEndLatency.scala
 delete mode 100644 tests/__init__.py
 delete mode 100644 tests/services/__init__.py
 delete mode 100644 tests/services/kafka_service.py
 delete mode 100644 tests/services/performance.py
 delete mode 100644 tests/services/zookeeper_service.py
 delete mode 100644 tests/templates/kafka.properties
 delete mode 100644 tests/tests/__init__.py
 delete mode 100644 tests/tests/kafka_benchmark_test.py
 delete mode 100644 tests/tests/test.py

diff --git a/core/src/test/scala/kafka/tools/TestEndToEndLatency.scala b/core/src/test/scala/kafka/tools/TestEndToEndLatency.scala
deleted file mode 100755
index 99b77a1ffbd8b..0000000000000
--- a/core/src/test/scala/kafka/tools/TestEndToEndLatency.scala
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package kafka.tools
-
-import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord, KafkaProducer}
-import kafka.consumer._
-import java.util.Properties
-import java.util.Arrays
-import scala.Option.option2Iterable
-
-object TestEndToEndLatency {
-  def main(args: Array[String]) {
-    if (args.length != 6) {
-      System.err.println("USAGE: java " + getClass().getName + " broker_list zookeeper_connect topic num_messages consumer_fetch_max_wait producer_acks")
-      System.exit(1)
-    }
-
-    val brokerList = args(0)
-    val zkConnect = args(1)
-    val topic = args(2)
-    val numMessages = args(3).toInt
-    val consumerFetchMaxWait = args(4).toInt
-    val producerAcks = args(5).toInt
-
-    val consumerProps = new Properties()
-    consumerProps.put("group.id", topic)
-    consumerProps.put("auto.commit.enable", "false")
-    consumerProps.put("auto.offset.reset", "largest")
-    consumerProps.put("zookeeper.connect", zkConnect)
-    consumerProps.put("fetch.wait.max.ms", consumerFetchMaxWait.toString)
-    consumerProps.put("socket.timeout.ms", 1201000.toString)
-
-    val config = new ConsumerConfig(consumerProps)
-    val connector = Consumer.create(config)
-    val stream = connector.createMessageStreams(Map(topic -> 1)).get(topic).head.head
-    val iter = stream.iterator
-
-    val producerProps = new Properties()
-    producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
-    producerProps.put(ProducerConfig.LINGER_MS_CONFIG, "0")
-    producerProps.put(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "true")
-    producerProps.put(ProducerConfig.ACKS_CONFIG, producerAcks.toString)
-    producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer")
-    producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer")
-    val producer = new KafkaProducer[Array[Byte],Array[Byte]](producerProps)
-
-    // make sure the consumer fetcher has started before sending data since otherwise
-    // the consumption from the tail will skip the first message and hence be blocked
-    Thread.sleep(5000)
-
-    val message = "hello there beautiful".getBytes
-    var totalTime = 0.0
-    val latencies = new Array[Long](numMessages)
-    for (i <- 0 until numMessages) {
-      val begin = System.nanoTime
-      producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic, message))
-      val received = iter.next
-      val elapsed = System.nanoTime - begin
-      // poor man's progress bar
-      if (i % 1000 == 0)
-        println(i + "\t" + elapsed / 1000.0 / 1000.0)
-      totalTime += elapsed
-      latencies(i) = (elapsed / 1000 / 1000)
-    }
-    println("Avg latency: %.4f ms\n".format(totalTime / numMessages / 1000.0 / 1000.0))
-    Arrays.sort(latencies)
-    val p50 = latencies((latencies.length * 0.5).toInt)
-    val p99 = latencies((latencies.length * 0.99).toInt) 
-    val p999 = latencies((latencies.length * 0.999).toInt)
-    println("Percentiles: 50th = %d, 99th = %d, 99.9th = %d".format(p50, p99, p999))
-    producer.close()
-    connector.commitOffsets(true)
-    connector.shutdown()
-    System.exit(0)
-  }
-}
\ No newline at end of file
diff --git a/tests/README.md b/tests/README.md
index dc146b333641a..6e1f33509f5e1 100644
--- a/tests/README.md
+++ b/tests/README.md
@@ -27,16 +27,22 @@ To run the tests:
 3. Bring up the cluster, making sure you have enough workers. For Vagrant, 
    use `vagrant up`. If you want to run on AWS, use `vagrant up
    --provider=aws --no-parallel`.
+
+Note that the initial provisioning process can be quite slow since it involves
+installing dependencies and updates on every vm.
+
 4. Install ducktape:
        
         $ git clone https://github.com/confluentinc/ducktape
         $ cd ducktape
-        $ pip install ducktape
+        $ python setup.py install
+
 5. Run the system tests using ducktape, you can view results in the `results`
    directory.
         
         $ cd tests
         $ ducktape tests
+
 6. To iterate/run again if you made any changes:
 
         $ cd kafka
diff --git a/tests/__init__.py b/tests/__init__.py
deleted file mode 100644
index 8b137891791fe..0000000000000
--- a/tests/__init__.py
+++ /dev/null
@@ -1 +0,0 @@
-
diff --git a/tests/kafkatest/services/console_consumer.py b/tests/kafkatest/services/console_consumer.py
index e910a520e6cc9..43268894178dc 100644
--- a/tests/kafkatest/services/console_consumer.py
+++ b/tests/kafkatest/services/console_consumer.py
@@ -119,6 +119,9 @@ def _worker(self, idx, node):
             prop_file = self.render('console_consumer.properties', consumer_timeout_ms=self.consumer_timeout_ms)
         else:
             prop_file = self.render('console_consumer.properties')
+
+        self.logger.info("console_consumer.properties:")
+        self.logger.info(prop_file)
         node.account.create_file("/mnt/console_consumer.properties", prop_file)
 
         # Run and capture output
diff --git a/tests/kafkatest/services/kafka.py b/tests/kafkatest/services/kafka.py
index 976fe39d4a5a5..da8c803e7126e 100644
--- a/tests/kafkatest/services/kafka.py
+++ b/tests/kafkatest/services/kafka.py
@@ -55,6 +55,8 @@ def start(self):
 
     def start_node(self, node):
         props_file = self.render('kafka.properties', node=node, broker_id=self.idx(node))
+        self.logger.info("kafka.properties:")
+        self.logger.info(props_file)
         node.account.create_file("/mnt/kafka.properties", props_file)
 
         cmd = "/opt/kafka/bin/kafka-server-start.sh /mnt/kafka.properties 1>> /mnt/kafka.log 2>> /mnt/kafka.log & echo $! > /mnt/kafka.pid"
diff --git a/tests/kafkatest/services/performance.py b/tests/kafkatest/services/performance.py
index 423d60a8e5b05..8026c5a5d11a8 100644
--- a/tests/kafkatest/services/performance.py
+++ b/tests/kafkatest/services/performance.py
@@ -101,11 +101,6 @@ def _worker(self, idx, node):
         # Parse and save the last line's information
         parts = last.split(',')
 
-        print "=" * 20
-        print "ConsumerPerformanceService data:"
-        print parts
-        print "-" * 20
-
         self.results[idx-1] = {
             'total_mb': float(parts[3]),
             'mbps': float(parts[4]),
diff --git a/tests/kafkatest/services/zookeeper.py b/tests/kafkatest/services/zookeeper.py
index 4c7e5a78f5fe9..3f35c11dd8659 100644
--- a/tests/kafkatest/services/zookeeper.py
+++ b/tests/kafkatest/services/zookeeper.py
@@ -37,7 +37,11 @@ def start_node(self, node):
 
         node.account.ssh("mkdir -p /mnt/zookeeper")
         node.account.ssh("echo %d > /mnt/zookeeper/myid" % idx)
-        node.account.create_file("/mnt/zookeeper.properties", self.render('zookeeper.properties'))
+
+        config_file = self.render('zookeeper.properties')
+        self.logger.info("zookeeper.properties:")
+        self.logger.info(config_file)
+        node.account.create_file("/mnt/zookeeper.properties", config_file)
 
         node.account.ssh(
             "/opt/kafka/bin/zookeeper-server-start.sh /mnt/zookeeper.properties 1>> %(path)s 2>> %(path)s &"
diff --git a/tests/kafkatest/tests/kafka_benchmark_test.py b/tests/kafkatest/tests/kafka_benchmark_test.py
index e05e57e44bce3..75e26e34b617f 100644
--- a/tests/kafkatest/tests/kafka_benchmark_test.py
+++ b/tests/kafkatest/tests/kafka_benchmark_test.py
@@ -238,13 +238,9 @@ def latency(latency_50th_ms, latency_99th_ms, latency_999th_ms):
 
 
 def compute_throughput(perf):
-    print "=" * 20
-    print perf.results
-    print "-" * 20
-    
     """Helper method for computing throughput after running a performance service."""
     aggregate_rate = sum([r['records_per_sec'] for r in perf.results])
-    aggregate_mbps = sum([r['mb_per_sec'] for r in perf.results])
+    aggregate_mbps = sum([r['mbps'] for r in perf.results])
 
     return throughput(aggregate_rate, aggregate_mbps)
 
diff --git a/tests/services/__init__.py b/tests/services/__init__.py
deleted file mode 100644
index e69de29bb2d1d..0000000000000
diff --git a/tests/services/kafka_service.py b/tests/services/kafka_service.py
deleted file mode 100644
index dfcd5f8356850..0000000000000
--- a/tests/services/kafka_service.py
+++ /dev/null
@@ -1,212 +0,0 @@
-# Copyright 2014 Confluent Inc.
-#
-# Licensed 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.
-
-from ducktape.services.service import Service
-import time, re, json
-
-
-class KafkaService(Service):
-    def __init__(self, service_context, zk, topics=None):
-        """
-        :type service_context ducktape.services.service.ServiceContext
-        :type zk: ZookeeperService
-        :type topics: dict
-        """
-        super(KafkaService, self).__init__(service_context)
-        self.zk = zk
-        self.topics = topics
-
-    def start(self):
-        super(KafkaService, self).start()
-
-        # Start all nodes in this Kafka service
-        for idx, node in enumerate(self.nodes, 1):
-            self.logger.info("Starting Kafka node %d on %s", idx, node.account.hostname)
-            self._stop_and_clean(node, allow_fail=True)
-            self.start_node(node)
-
-            # wait for start up
-            time.sleep(6)
-
-        # Create topics if necessary
-        if self.topics is not None:
-            for topic, topic_cfg in self.topics.items():
-                if topic_cfg is None:
-                    topic_cfg = {}
-
-                topic_cfg["topic"] = topic
-                self.create_topic(topic_cfg)
-
-    def create_topic(self, topic_cfg):
-        node = self.nodes[0] # any node is fine here
-        self.logger.info("Creating topic %s with settings %s", topic_cfg["topic"], topic_cfg)
-
-        cmd = "/opt/kafka/bin/kafka-topics.sh --zookeeper %(zk_connect)s --create "\
-            "--topic %(topic)s --partitions %(partitions)d --replication-factor %(replication)d" % {
-                'zk_connect': self.zk.connect_setting(),
-                'topic': topic_cfg.get("topic"),
-                'partitions': topic_cfg.get('partitions', 1),
-                'replication': topic_cfg.get('replication-factor', 1)
-            }
-
-        if "configs" in topic_cfg.keys() and topic_cfg["configs"] is not None:
-            for config_name, config_value in topic_cfg["configs"].items():
-                cmd += " --config %s=%s" % (config_name, str(config_value))
-
-        self.logger.info("Running topic creation command...\n%s" % cmd)
-        node.account.ssh(cmd)
-
-        time.sleep(1)
-        self.logger.info("Checking to see if topic was properly created...\n%s" % cmd)
-        for line in self.describe_topic(topic_cfg["topic"]).split("\n"):
-            self.logger.info(line)
-
-    def describe_topic(self, topic):
-        node = self.nodes[0]
-        cmd = "/opt/kafka/bin/kafka-topics.sh --zookeeper %s --topic %s --describe" % \
-              (self.zk.connect_setting(), topic)
-        output = ""
-        for line in node.account.ssh_capture(cmd):
-            output += line
-        return output
-
-    def verify_reassign_partitions(self, reassignment):
-        """Run the reassign partitions admin tool in "verify" mode
-        """
-        node = self.nodes[0]
-        json_file = "/tmp/" + str(time.time()) + "_reassign.json"
-
-        # reassignment to json
-        json_str = json.dumps(reassignment)
-        json_str = json.dumps(json_str)
-
-        # create command
-        cmd = "echo %s > %s && " % (json_str, json_file)
-        cmd += "/opt/kafka/bin/kafka-reassign-partitions.sh "\
-                "--zookeeper %(zk_connect)s "\
-                "--reassignment-json-file %(reassignment_file)s "\
-                "--verify" % {'zk_connect': self.zk.connect_setting(),
-                                'reassignment_file': json_file}
-        cmd += " && sleep 1 && rm -f %s" % json_file
-
-        # send command
-        self.logger.info("Verifying parition reassignment...")
-        self.logger.debug(cmd)
-        output = ""
-        for line in node.account.ssh_capture(cmd):
-            output += line
-
-        self.logger.debug(output)
-
-        if re.match(".*is in progress.*", output) is not None:
-            return False
-
-        return True
-
-    def execute_reassign_partitions(self, reassignment):
-        """Run the reassign partitions admin tool in "verify" mode
-        """
-        node = self.nodes[0]
-        json_file = "/tmp/" + str(time.time()) + "_reassign.json"
-
-        # reassignment to json
-        json_str = json.dumps(reassignment)
-        json_str = json.dumps(json_str)
-
-        # create command
-        cmd = "echo %s > %s && " % (json_str, json_file)
-        cmd += "/opt/kafka/bin/kafka-reassign-partitions.sh "\
-                "--zookeeper %(zk_connect)s "\
-                "--reassignment-json-file %(reassignment_file)s "\
-                "--execute" % {'zk_connect': self.zk.connect_setting(),
-                                'reassignment_file': json_file}
-        cmd += " && sleep 1 && rm -f %s" % json_file
-
-        # send command
-        self.logger.info("Executing parition reassignment...")
-        self.logger.debug(cmd)
-        output = ""
-        for line in node.account.ssh_capture(cmd):
-            output += line
-
-        self.logger.debug("Verify partition reassignment:")
-        self.logger.debug(output)
-
-    def stop(self):
-        """If the service left any running processes or data, clean them up."""
-        super(KafkaService, self).stop()
-
-        for idx, node in enumerate(self.nodes, 1):
-            self.logger.info("Stopping %s node %d on %s" % (type(self).__name__, idx, node.account.hostname))
-            self._stop_and_clean(node, allow_fail=True)
-            node.free()
-
-    def _stop_and_clean(self, node, allow_fail=False):
-        node.account.ssh("/opt/kafka/bin/kafka-server-stop.sh", allow_fail=allow_fail)
-        time.sleep(5)  # the stop script doesn't wait
-        node.account.ssh("rm -rf /mnt/kafka-logs /mnt/kafka.properties /mnt/kafka.log")
-
-    def stop_node(self, node, clean_shutdown=True, allow_fail=True):
-        node.account.kill_process("kafka", clean_shutdown, allow_fail)
-
-    def start_node(self, node, config=None):
-        if config is None:
-            template = open('templates/kafka.properties').read()
-            template_params = {
-                'broker_id': self.idx(node),
-                'hostname': node.account.hostname,
-                'zk_connect': self.zk.connect_setting()
-            }
-
-            config = template % template_params
-
-        node.account.create_file("/mnt/kafka.properties", config)
-        cmd = "/opt/kafka/bin/kafka-server-start.sh /mnt/kafka.properties 1>> /mnt/kafka.log 2>> /mnt/kafka.log &"
-        self.logger.debug("Attempting to start KafkaService on %s with command: %s" % (str(node.account), cmd))
-        node.account.ssh(cmd)
-
-    def restart_node(self, node, wait_sec=0, clean_shutdown=True):
-        self.stop_node(node, clean_shutdown, allow_fail=True)
-        time.sleep(wait_sec)
-        self.start_node(node)
-
-    def get_leader_node(self, topic, partition=0):
-        """ Get the leader replica for the given topic and partition.
-        """
-        cmd = "/opt/kafka/bin/kafka-run-class.sh kafka.tools.ZooKeeperMainWrapper -server %s " \
-              % self.zk.connect_setting()
-        cmd += "get /brokers/topics/%s/partitions/%d/state" % (topic, partition)
-        self.logger.debug(cmd)
-
-        node = self.nodes[0]
-        self.logger.debug("Querying zookeeper to find leader replica for topic %s: \n%s" % (cmd, topic))
-        partition_state = None
-        for line in node.account.ssh_capture(cmd):
-            match = re.match("^({.+})$", line)
-            if match is not None:
-                partition_state = match.groups()[0]
-                break
-
-        if partition_state is None:
-            raise Exception("Error finding partition state for topic %s and partition %d." % (topic, partition))
-
-        partition_state = json.loads(partition_state)
-        self.logger.info(partition_state)
-
-        leader_idx = int(partition_state["leader"])
-        self.logger.info("Leader for topic %s and partition %d is now: %d" % (topic, partition, leader_idx))
-        return self.get_node(leader_idx)
-
-    def bootstrap_servers(self):
-        return ','.join([node.account.hostname + ":9092" for node in self.nodes])
\ No newline at end of file
diff --git a/tests/services/performance.py b/tests/services/performance.py
deleted file mode 100644
index 9478b5ef956af..0000000000000
--- a/tests/services/performance.py
+++ /dev/null
@@ -1,189 +0,0 @@
-# Copyright 2014 Confluent Inc.
-#
-# Licensed 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.
-
-from ducktape.services.service import Service
-import threading
-
-
-class PerformanceService(Service):
-    def __init__(self, service_context):
-        super(PerformanceService, self).__init__(service_context)
-
-    def start(self):
-        super(PerformanceService, self).start()
-        self.worker_threads = []
-        self.results = [None] * len(self.nodes)
-        self.stats = [[] for x in range(len(self.nodes))]
-        for idx,node in enumerate(self.nodes,1):
-            self.logger.info("Running %s node %d on %s", self.__class__.__name__, idx, node.account.hostname)
-            worker = threading.Thread(
-                name=self.__class__.__name__ + "-worker-" + str(idx),
-                target=self._worker,
-                args=(idx,node)
-            )
-            worker.daemon = True
-            worker.start()
-            self.worker_threads.append(worker)
-
-    def wait(self):
-        super(PerformanceService, self).wait()
-        for idx,worker in enumerate(self.worker_threads,1):
-            self.logger.debug("Waiting for %s worker %d to finish", self.__class__.__name__, idx)
-            worker.join()
-        self.worker_threads = None
-
-    def stop(self):
-        super(PerformanceService, self).stop()
-        assert self.worker_threads is None, "%s.stop should only be called after wait" % self.__class__.__name__
-        for idx,node in enumerate(self.nodes,1):
-            self.logger.debug("Stopping %s node %d on %s", self.__class__.__name__, idx, node.account.hostname)
-            node.free()
-
-
-class ProducerPerformanceService(PerformanceService):
-    def __init__(self, service_context, kafka, topic, num_records, record_size, throughput, settings={}, intermediate_stats=False):
-        super(ProducerPerformanceService, self).__init__(service_context)
-        self.kafka = kafka
-        self.args = {
-            'topic': topic,
-            'num_records': num_records,
-            'record_size': record_size,
-            'throughput': throughput
-        }
-        self.settings = settings
-        self.intermediate_stats = intermediate_stats
-
-    def _worker(self, idx, node):
-        args = self.args.copy()
-        args.update({'bootstrap_servers': self.kafka.bootstrap_servers()})
-        cmd = "/opt/kafka/bin/kafka-run-class.sh org.apache.kafka.clients.tools.ProducerPerformance "\
-              "%(topic)s %(num_records)d %(record_size)d %(throughput)d bootstrap.servers=%(bootstrap_servers)s" % args
-
-        for key,value in self.settings.items():
-            cmd += " %s=%s" % (str(key), str(value))
-        self.logger.debug("Producer performance %d command: %s", idx, cmd)
-        def parse_stats(line):
-            parts = line.split(',')
-            return {
-                'records': int(parts[0].split()[0]),
-                'records_per_sec': float(parts[1].split()[0]),
-                'mbps': float(parts[1].split('(')[1].split()[0]),
-                'latency_avg_ms': float(parts[2].split()[0]),
-                'latency_max_ms': float(parts[3].split()[0]),
-                'latency_50th_ms': float(parts[4].split()[0]),
-                'latency_95th_ms': float(parts[5].split()[0]),
-                'latency_99th_ms': float(parts[6].split()[0]),
-                'latency_999th_ms': float(parts[7].split()[0]),
-            }
-        last = None
-        for line in node.account.ssh_capture(cmd):
-            self.logger.debug("Producer performance %d: %s", idx, line.strip())
-            if self.intermediate_stats:
-                try:
-                    self.stats[idx-1].append(parse_stats(line))
-                except:
-                    # Sometimes there are extraneous log messages
-                    pass
-            last = line
-        try:
-            self.results[idx-1] = parse_stats(last)
-        except:
-            self.logger.error("Bad last line: %s", last)
-
-
-class ConsumerPerformanceService(PerformanceService):
-    def __init__(self, service_context, kafka, topic, num_records, throughput, threads=1, settings={}):
-        super(ConsumerPerformanceService, self).__init__(service_context)
-        self.kafka = kafka
-        self.args = {
-            'topic': topic,
-            'num_records': num_records,
-            'throughput': throughput,
-            'threads': threads,
-        }
-        self.settings = settings
-
-    def _worker(self, idx, node):
-        args = self.args.copy()
-        args.update({'zk_connect': self.kafka.zk.connect_setting()})
-        cmd = "/opt/kafka/bin/kafka-consumer-perf-test.sh "\
-              "--topic %(topic)s --messages %(num_records)d --zookeeper %(zk_connect)s" % args
-        for key,value in self.settings.items():
-            cmd += " %s=%s" % (str(key), str(value))
-        self.logger.debug("Consumer performance %d command: %s", idx, cmd)
-        last = None
-        for line in node.account.ssh_capture(cmd):
-            self.logger.debug("Consumer performance %d: %s", idx, line.strip())
-            last = line
-        # Parse and save the last line's information
-        parts = last.split(',')
-        self.results[idx-1] = {
-            'total_mb': float(parts[3]),
-            'mbps': float(parts[4]),
-            'records_per_sec': float(parts[6]),
-        }
-
-
-class EndToEndLatencyService(PerformanceService):
-    def __init__(self, service_context, kafka, topic, num_records, consumer_fetch_max_wait=100, acks=1):
-        super(EndToEndLatencyService, self).__init__(service_context)
-        self.kafka = kafka
-        self.args = {
-            'topic': topic,
-            'num_records': num_records,
-            'consumer_fetch_max_wait': consumer_fetch_max_wait,
-            'acks': acks
-        }
-
-    def _worker(self, idx, node):
-        args = self.args.copy()
-        args.update({
-            'zk_connect': self.kafka.zk.connect_setting(),
-            'bootstrap_servers': self.kafka.bootstrap_servers(),
-        })
-        cmd = "/opt/kafka/bin/kafka-run-class.sh kafka.tools.TestEndToEndLatency "\
-              "%(bootstrap_servers)s %(zk_connect)s %(topic)s %(num_records)d "\
-              "%(consumer_fetch_max_wait)d %(acks)d" % args
-        self.logger.debug("End-to-end latency %d command: %s", idx, cmd)
-        results = {}
-        for line in node.account.ssh_capture(cmd):
-            self.logger.debug("End-to-end latency %d: %s", idx, line.strip())
-            if line.startswith("Avg latency:"):
-                results['latency_avg_ms'] = float(line.split()[2])
-            if line.startswith("Percentiles"):
-                results['latency_50th_ms'] = float(line.split()[3][:-1])
-                results['latency_99th_ms'] = float(line.split()[6][:-1])
-                results['latency_999th_ms'] = float(line.split()[9])
-        self.results[idx-1] = results
-
-
-def parse_performance_output(summary):
-        parts = summary.split(',')
-        results = {
-            'records': int(parts[0].split()[0]),
-            'records_per_sec': float(parts[1].split()[0]),
-            'mbps': float(parts[1].split('(')[1].split()[0]),
-            'latency_avg_ms': float(parts[2].split()[0]),
-            'latency_max_ms': float(parts[3].split()[0]),
-            'latency_50th_ms': float(parts[4].split()[0]),
-            'latency_95th_ms': float(parts[5].split()[0]),
-            'latency_99th_ms': float(parts[6].split()[0]),
-            'latency_999th_ms': float(parts[7].split()[0]),
-        }
-        # To provide compatibility with ConsumerPerformanceService
-        results['total_mb'] = results['mbps'] * (results['records'] / results['records_per_sec'])
-        results['rate_mbps'] = results['mbps']
-        results['rate_mps'] = results['records_per_sec']
-
-        return results
diff --git a/tests/services/zookeeper_service.py b/tests/services/zookeeper_service.py
deleted file mode 100644
index efebe848257fe..0000000000000
--- a/tests/services/zookeeper_service.py
+++ /dev/null
@@ -1,75 +0,0 @@
-# Copyright 2014 Confluent Inc.
-#
-# Licensed 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.
-
-from ducktape.services.service import Service
-import time
-
-
-class ZookeeperService(Service):
-    def __init__(self, service_context):
-        """
-        :type service_context ducktape.services.service.ServiceContext
-        """
-        super(ZookeeperService, self).__init__(service_context)
-        self.logs = {"zk_log": "/mnt/zk.log"}
-
-    def start(self):
-        super(ZookeeperService, self).start()
-        config = """
-dataDir=/mnt/zookeeper
-clientPort=2181
-maxClientCnxns=0
-initLimit=5
-syncLimit=2
-quorumListenOnAllIPs=true
-"""
-        for idx, node in enumerate(self.nodes, 1):
-            template_params = { 'idx': idx, 'host': node.account.hostname }
-            config += "server.%(idx)d=%(host)s:2888:3888\n" % template_params
-
-        for idx, node in enumerate(self.nodes, 1):
-            self.logger.info("Starting ZK node %d on %s", idx, node.account.hostname)
-            self._stop_and_clean(node, allow_fail=True)
-            node.account.ssh("mkdir -p /mnt/zookeeper")
-            node.account.ssh("echo %d > /mnt/zookeeper/myid" % idx)
-            node.account.create_file("/mnt/zookeeper.properties", config)
-            node.account.ssh(
-                "/opt/kafka/bin/zookeeper-server-start.sh /mnt/zookeeper.properties 1>> %(zk_log)s 2>> %(zk_log)s &"
-                % self.logs)
-            time.sleep(5)  # give it some time to start
-
-    def stop_node(self, node, allow_fail=True):
-        idx = self.idx(node)
-        self.logger.info("Stopping %s node %d on %s" % (type(self).__name__, idx, node.account.hostname))
-        node.account.ssh("ps ax | grep -i 'zookeeper' | grep -v grep | awk '{print $1}' | xargs kill -SIGTERM",
-                         allow_fail=allow_fail)
-
-    def clean_node(self, node, allow_fail=True):
-        node.account.ssh("rm -rf /mnt/zookeeper /mnt/zookeeper.properties /mnt/zk.log", allow_fail=allow_fail)
-
-    def stop(self):
-        """If the service left any running processes or data, clean them up."""
-        super(ZookeeperService, self).stop()
-
-        for idx, node in enumerate(self.nodes, 1):
-            self.stop_node(node, allow_fail=False)
-            self.clean_node(node)
-            node.free()
-
-    def _stop_and_clean(self, node, allow_fail=False):
-        self.stop_node(node, allow_fail)
-        self.clean_node(node, allow_fail)
-
-    def connect_setting(self):
-        return ','.join([node.account.hostname + ':2181' for node in self.nodes])
diff --git a/tests/templates/kafka.properties b/tests/templates/kafka.properties
deleted file mode 100644
index fe0636e045e2a..0000000000000
--- a/tests/templates/kafka.properties
+++ /dev/null
@@ -1,121 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-# 
-#    http://www.apache.org/licenses/LICENSE-2.0
-# 
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-# see kafka.server.KafkaConfig for additional details and defaults
-
-############################# Server Basics #############################
-
-# The id of the broker. This must be set to a unique integer for each broker.
-broker.id=%(broker_id)d
-
-############################# Socket Server Settings #############################
-
-# The port the socket server listens on
-port=9092
-
-# Hostname the broker will bind to. If not set, the server will bind to all interfaces
-#host.name=localhost
-
-# Hostname the broker will advertise to producers and consumers. If not set, it uses the
-# value for "host.name" if configured.  Otherwise, it will use the value returned from
-# java.net.InetAddress.getCanonicalHostName().
-advertised.host.name=%(hostname)s
-
-# The port to publish to ZooKeeper for clients to use. If this is not set,
-# it will publish the same port that the broker binds to.
-#advertised.port=
-
-# The number of threads handling network requests
-num.network.threads=3
- 
-# The number of threads doing disk I/O
-num.io.threads=8
-
-# The send buffer (SO_SNDBUF) used by the socket server
-socket.send.buffer.bytes=102400
-
-# The receive buffer (SO_RCVBUF) used by the socket server
-socket.receive.buffer.bytes=65536
-
-# The maximum size of a request that the socket server will accept (protection against OOM)
-socket.request.max.bytes=104857600
-
-
-############################# Log Basics #############################
-
-# A comma seperated list of directories under which to store log files
-log.dirs=/mnt/kafka-logs
-
-# The default number of log partitions per topic. More partitions allow greater
-# parallelism for consumption, but this will also result in more files across
-# the brokers.
-num.partitions=1
-
-# The number of threads per data directory to be used for log recovery at startup and flushing at shutdown.
-# This value is recommended to be increased for installations with data dirs located in RAID array.
-num.recovery.threads.per.data.dir=1
-
-############################# Log Flush Policy #############################
-
-# Messages are immediately written to the filesystem but by default we only fsync() to sync
-# the OS cache lazily. The following configurations control the flush of data to disk. 
-# There are a few important trade-offs here:
-#    1. Durability: Unflushed data may be lost if you are not using replication.
-#    2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as there will be a lot of data to flush.
-#    3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to exceessive seeks. 
-# The settings below allow one to configure the flush policy to flush data after a period of time or
-# every N messages (or both). This can be done globally and overridden on a per-topic basis.
-
-# The number of messages to accept before forcing a flush of data to disk
-#log.flush.interval.messages=10000
-
-# The maximum amount of time a message can sit in a log before we force a flush
-#log.flush.interval.ms=1000
-
-############################# Log Retention Policy #############################
-
-# The following configurations control the disposal of log segments. The policy can
-# be set to delete segments after a period of time, or after a given size has accumulated.
-# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens
-# from the end of the log.
-
-# The minimum age of a log file to be eligible for deletion
-log.retention.hours=168
-
-# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining
-# segments don't drop below log.retention.bytes.
-#log.retention.bytes=1073741824
-
-# The maximum size of a log segment file. When this size is reached a new log segment will be created.
-log.segment.bytes=1073741824
-
-# The interval at which log segments are checked to see if they can be deleted according 
-# to the retention policies
-log.retention.check.interval.ms=300000
-
-# By default the log cleaner is disabled and the log retention policy will default to just delete segments after their retention expires.
-# If log.cleaner.enable=true is set the cleaner will be enabled and individual logs can then be marked for log compaction.
-log.cleaner.enable=false
-
-############################# Zookeeper #############################
-
-# Zookeeper connection string (see zookeeper docs for details).
-# This is a comma separated host:port pairs, each corresponding to a zk
-# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002".
-# You can also append an optional chroot string to the urls to specify the
-# root directory for all kafka znodes.
-zookeeper.connect=%(zk_connect)s
-
-# Timeout in ms for connecting to zookeeper
-zookeeper.connection.timeout.ms=2000
diff --git a/tests/tests/__init__.py b/tests/tests/__init__.py
deleted file mode 100644
index e69de29bb2d1d..0000000000000
diff --git a/tests/tests/kafka_benchmark_test.py b/tests/tests/kafka_benchmark_test.py
deleted file mode 100644
index 5ea0913c054e0..0000000000000
--- a/tests/tests/kafka_benchmark_test.py
+++ /dev/null
@@ -1,193 +0,0 @@
-# Copyright 2014 Confluent Inc.
-#
-# Licensed 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.
-
-from ducktape.services.service import Service
-
-from tests.test import KafkaTest
-from services.performance import ProducerPerformanceService, ConsumerPerformanceService, \
-    EndToEndLatencyService
-
-
-class KafkaBenchmark(KafkaTest):
-    '''A benchmark of Kafka producer/consumer performance. This replicates the test
-    run here:
-    https://engineering.linkedin.com/kafka/benchmarking-apache-kafka-2-million-writes-second-three-cheap-machines
-    '''
-    def __init__(self, test_context):
-        super(KafkaBenchmark, self).__init__(test_context, num_zk=1, num_brokers=3, topics={
-            'test-rep-one' : { 'partitions': 6, 'replication-factor': 1 },
-            'test-rep-three' : { 'partitions': 6, 'replication-factor': 3 }
-        })
-
-    def run(self):
-        msgs_default = 50000000
-        msgs_large = 100000000
-        msg_size_default = 100
-        batch_size = 8*1024
-        buffer_memory = 64*1024*1024
-        msg_sizes = [10, 100, 1000, 10000, 100000]
-        target_data_size = 1024*1024*1024
-        target_data_size_gb = target_data_size/float(1024*1024*1024)
-        # These settings will work in the default local Vagrant VMs, useful for testing
-        if False:
-            msgs_default = 1000000
-            msgs_large = 10000000
-            msg_size_default = 100
-            batch_size = 8*1024
-            buffer_memory = 64*1024*1024
-            msg_sizes = [10, 100, 1000, 10000, 100000]
-            target_data_size = 128*1024*1024
-            target_data_size_gb = target_data_size/float(1024*1024*1024)
-
-        # PRODUCER TESTS
-
-        self.logger.info("BENCHMARK: Single producer, no replication")
-        single_no_rep = ProducerPerformanceService(
-            self.service_context(1), self.kafka,
-            topic="test-rep-one", num_records=msgs_default, record_size=msg_size_default, throughput=-1,
-            settings={'acks':1, 'batch.size':batch_size, 'buffer.memory':buffer_memory}
-        )
-        single_no_rep.run()
-
-        self.logger.info("BENCHMARK: Single producer, async 3x replication")
-        single_rep_async = ProducerPerformanceService(
-            self.service_context(1), self.kafka,
-            topic="test-rep-three", num_records=msgs_default, record_size=msg_size_default, throughput=-1,
-            settings={'acks':1, 'batch.size':batch_size, 'buffer.memory':buffer_memory}
-        )
-        single_rep_async.run()
-
-        self.logger.info("BENCHMARK: Single producer, sync 3x replication")
-        single_rep_sync = ProducerPerformanceService(
-            self.service_context(1), self.kafka,
-            topic="test-rep-three", num_records=msgs_default, record_size=msg_size_default, throughput=-1,
-            settings={'acks':-1, 'batch.size':batch_size, 'buffer.memory':buffer_memory}
-        )
-        single_rep_sync.run()
-
-        self.logger.info("BENCHMARK: Three producers, async 3x replication")
-        three_rep_async = ProducerPerformanceService(
-            self.service_context(3), self.kafka,
-            topic="test-rep-three", num_records=msgs_default, record_size=msg_size_default, throughput=-1,
-            settings={'acks':1, 'batch.size':batch_size, 'buffer.memory':buffer_memory}
-        )
-        three_rep_async.run()
-
-
-        msg_size_perf = {}
-        for msg_size in msg_sizes:
-            self.logger.info("BENCHMARK: Message size %d (%f GB total, single producer, async 3x replication)", msg_size, target_data_size_gb)
-            # Always generate the same total amount of data
-            nrecords = int(target_data_size / msg_size)
-            perf = ProducerPerformanceService(
-                self.service_context(1), self.kafka,
-                topic="test-rep-three", num_records=nrecords, record_size=msg_size, throughput=-1,
-                settings={'acks':1, 'batch.size':batch_size, 'buffer.memory':buffer_memory}
-            )
-            perf.run()
-            msg_size_perf[msg_size] = perf
-
-        # CONSUMER TESTS
-
-        # All consumer tests use the messages from the first benchmark, so
-        # they'll get messages of the default message size
-        self.logger.info("BENCHMARK: Single consumer")
-        single_consumer = ConsumerPerformanceService(
-            self.service_context(1), self.kafka,
-            topic="test-rep-three", num_records=msgs_default, throughput=-1, threads=1
-        )
-        single_consumer.run()
-
-        self.logger.info("BENCHMARK: Three consumers")
-        three_consumers = ConsumerPerformanceService(
-            self.service_context(3), self.kafka,
-            topic="test-rep-three", num_records=msgs_default, throughput=-1, threads=1
-        )
-        three_consumers.run()
-
-        # PRODUCER + CONSUMER TEST
-        self.logger.info("BENCHMARK: Producer + Consumer")
-        pc_producer = ProducerPerformanceService(
-            self.service_context(1), self.kafka,
-            topic="test-rep-three", num_records=msgs_default, record_size=msg_size_default, throughput=-1,
-            settings={'acks':1, 'batch.size':batch_size, 'buffer.memory':buffer_memory}
-        )
-        pc_consumer = ConsumerPerformanceService(
-            self.service_context(1), self.kafka,
-            topic="test-rep-three", num_records=msgs_default, throughput=-1, threads=1
-        )
-        Service.run_parallel(pc_producer, pc_consumer)
-
-        # END TO END LATENCY TEST
-        self.logger.info("BENCHMARK: End to end latency")
-        e2e_latency = EndToEndLatencyService(
-            self.service_context(1), self.kafka,
-            topic="test-rep-three", num_records=10000
-        )
-        e2e_latency.run()
-
-
-        # LONG TERM THROUGHPUT TEST
-
-        # Because of how much space this ends up using, we clear out the
-        # existing cluster to start from a clean slate. This also keeps us from
-        # running out of space with limited disk space.
-        self.tearDown()
-        self.setUp()
-        self.logger.info("BENCHMARK: Long production")
-        throughput_perf = ProducerPerformanceService(
-            self.service_context(1), self.kafka,
-            topic="test-rep-three", num_records=msgs_large, record_size=msg_size_default, throughput=-1,
-            settings={'acks':1, 'batch.size':batch_size, 'buffer.memory':buffer_memory},
-            intermediate_stats=True
-        )
-        throughput_perf.run()
-
-        # Summarize, extracting just the key info. With multiple
-        # producers/consumers, we display the aggregate value
-        def throughput(perf):
-            aggregate_rate = sum([r['records_per_sec'] for r in perf.results])
-            aggregate_mbps = sum([r['mbps'] for r in perf.results])
-            return "%f rec/sec (%f MB/s)" % (aggregate_rate, aggregate_mbps)
-        self.logger.info("=================")
-        self.logger.info("BENCHMARK RESULTS")
-        self.logger.info("=================")
-        self.logger.info("Single producer, no replication: %s", throughput(single_no_rep))
-        self.logger.info("Single producer, async 3x replication: %s", throughput(single_rep_async))
-        self.logger.info("Single producer, sync 3x replication: %s", throughput(single_rep_sync))
-        self.logger.info("Three producers, async 3x replication: %s", throughput(three_rep_async))
-        self.logger.info("Message size:")
-        for msg_size in msg_sizes:
-            self.logger.info(" %d: %s", msg_size, throughput(msg_size_perf[msg_size]))
-        self.logger.info("Throughput over long run, data > memory:")
-        # FIXME we should be generating a graph too
-        # Try to break it into 5 blocks, but fall back to a smaller number if
-        # there aren't even 5 elements
-        block_size = max(len(throughput_perf.stats[0]) / 5, 1)
-        nblocks = len(throughput_perf.stats[0]) / block_size
-        for i in range(nblocks):
-            subset = throughput_perf.stats[0][i*block_size:min((i+1)*block_size,len(throughput_perf.stats[0]))]
-            if len(subset) == 0:
-                self.logger.info(" Time block %d: (empty)", i)
-            else:
-                self.logger.info(" Time block %d: %f rec/sec (%f MB/s)", i,
-                                 sum([stat['records_per_sec'] for stat in subset])/float(len(subset)),
-                                 sum([stat['mbps'] for stat in subset])/float(len(subset))
-                             )
-        self.logger.info("Single consumer: %s", throughput(single_consumer))
-        self.logger.info("Three consumers: %s", throughput(three_consumers))
-        self.logger.info("Producer + consumer:")
-        self.logger.info(" Producer: %s", throughput(pc_producer))
-        self.logger.info(" Consumer: %s", throughput(pc_producer))
-        self.logger.info("End-to-end latency: median %f ms, 99%% %f ms, 99.9%% %f ms", e2e_latency.results[0]['latency_50th_ms'], e2e_latency.results[0]['latency_99th_ms'], e2e_latency.results[0]['latency_999th_ms'])
diff --git a/tests/tests/test.py b/tests/tests/test.py
deleted file mode 100644
index 3ac511f1e5880..0000000000000
--- a/tests/tests/test.py
+++ /dev/null
@@ -1,51 +0,0 @@
-# Copyright 2014 Confluent Inc.
-#
-# Licensed 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.
-
-from ducktape.tests.test import Test
-from ducktape.services.service import ServiceContext
-
-from services.zookeeper_service import ZookeeperService
-from services.kafka_service import KafkaService
-
-
-class KafkaTest(Test):
-    """
-    Helper class that managest setting up a Kafka cluster. Use this if the
-    default settings for Kafka are sufficient for your test; any customization
-    needs to be done manually. Your run() method should call tearDown and
-    setUp. The Zookeeper and Kafka services are available as the fields
-    KafkaTest.zk and KafkaTest.kafka.
-
-
-    """
-    def __init__(self, test_context, num_zk, num_brokers, topics=None):
-        super(KafkaTest, self).__init__(test_context)
-        self.num_zk = num_zk
-        self.num_brokers = num_brokers
-        self.topics = topics
-
-    def min_cluster_size(self):
-        return self.num_zk + self.num_brokers
-
-    def setUp(self):
-        self.zk = ZookeeperService(ServiceContext(self.cluster, self.num_zk, self.logger))
-        self.kafka = KafkaService(
-            ServiceContext(self.cluster, self.num_brokers, self.logger),
-            self.zk, topics=self.topics)
-        self.zk.start()
-        self.kafka.start()
-
-    def tearDown(self):
-        self.kafka.stop()
-        self.zk.stop()

From 25a413d6ae3333938e9773eb2b20509760bab464 Mon Sep 17 00:00:00 2001
From: Geoff 
Date: Thu, 11 Jun 2015 13:29:21 -0700
Subject: [PATCH 19/41] Update aws-example-Vagrantfile.local

---
 tests/aws/aws-example-Vagrantfile.local | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)

diff --git a/tests/aws/aws-example-Vagrantfile.local b/tests/aws/aws-example-Vagrantfile.local
index 7f9bbd4192dde..8a1a4c2700d04 100644
--- a/tests/aws/aws-example-Vagrantfile.local
+++ b/tests/aws/aws-example-Vagrantfile.local
@@ -1,9 +1,9 @@
 ec3_instance_type = "m3.medium"
 enable_dns = true
 mode = "test"
-num_workers = 1
+num_workers = 
 ec2_keypair_name =
 ec2_keypair_file =
-ec2_security_groups = ['ducttape-insecure']
-ec2_region = 'us-west-2'
-ec2_ami = "ami-29ebb519"
+ec2_security_groups = []
+ec2_region = 
+ec2_ami = "ami-5189a661"

From 96533c3718a9285d78393fb453b951592c72a490 Mon Sep 17 00:00:00 2001
From: Geoff 
Date: Thu, 11 Jun 2015 13:36:33 -0700
Subject: [PATCH 20/41] Update aws-access-keys-commands

---
 tests/aws/aws-access-keys-commands | 16 +++++++++++++---
 1 file changed, 13 insertions(+), 3 deletions(-)

diff --git a/tests/aws/aws-access-keys-commands b/tests/aws/aws-access-keys-commands
index d4aa263e89ad7..c11a10a96b156 100644
--- a/tests/aws/aws-access-keys-commands
+++ b/tests/aws/aws-access-keys-commands
@@ -1,3 +1,13 @@
-export AWS_ACCESS_KEY=`curl -s http://169.254.169.254/latest/meta-data/iam/security-credentials/ducttape-master | grep AccessKeyId | awk -F\" '{ print $4 }'`
-export AWS_SECRET_KEY=`curl -s http://169.254.169.254/latest/meta-data/iam/security-credentials/ducttape-master | grep SecretAccessKey | awk -F\" '{ print $4 }'`
-export AWS_SESSION_TOKEN=`curl -s http://169.254.169.254/latest/meta-data/iam/security-credentials/ducttape-master | grep Token | awk -F\" '{ print $4 }'`
+if [ -z "$AWS_IAM" ];then
+  echo "Warning: AWS_IAM is not set"
+fi
+
+export AWS_ACCESS_KEY=`curl -s http://169.254.169.254/latest/meta-data/iam/security-credentials/$AWS_IAM | grep AccessKeyId | awk -F\" '{ print $4 }'`
+export AWS_SECRET_KEY=`curl -s http://169.254.169.254/latest/meta-data/iam/security-credentials/$AWS_IAM | grep SecretAccessKey | awk -F\" '{ print $4 }'`
+export AWS_SESSION_TOKEN=`curl -s http://169.254.169.254/latest/meta-data/iam/security-credentials/$AWS_IAM | grep Token | awk -F\" '{ print $4 }'`
+
+if [ -z "$AWS_ACCESS_KEY" ]; then
+    echo "Failed to populate environment variables AWS_ACCESS_KEY, AWS_SECRET_KEY, and AWS_SESSION_TOKEN."
+    echo "AWS_IAM is currently $AWS_IAM. Double-check that this is correct. If not set, add this command to your .bashrc file:"
+    echo "export AWS_IAM=  # put this into your ~/.bashrc"
+fi

From e5edf031aeb99b9176a6ae8375963f2aedaaa6d7 Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Thu, 11 Jun 2015 17:27:49 -0700
Subject: [PATCH 21/41] Updated example aws Vagrantfile.local

---
 tests/aws/aws-example-Vagrantfile.local | 9 ++++-----
 1 file changed, 4 insertions(+), 5 deletions(-)

diff --git a/tests/aws/aws-example-Vagrantfile.local b/tests/aws/aws-example-Vagrantfile.local
index 7f9bbd4192dde..615cde9c17176 100644
--- a/tests/aws/aws-example-Vagrantfile.local
+++ b/tests/aws/aws-example-Vagrantfile.local
@@ -1,9 +1,8 @@
 ec3_instance_type = "m3.medium"
-enable_dns = true
 mode = "test"
-num_workers = 1
-ec2_keypair_name =
-ec2_keypair_file =
-ec2_security_groups = ['ducttape-insecure']
+num_workers = 9
+ec2_keypair_name = kafkatest
+ec2_keypair_file = ../kafkatest.pem
+ec2_security_groups = ['kafkatest-insecure']
 ec2_region = 'us-west-2'
 ec2_ami = "ami-29ebb519"

From 5af88fc1d9fc357c191a7c5fdbca60e37f42e9fc Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Thu, 11 Jun 2015 17:28:19 -0700
Subject: [PATCH 22/41] Updated README to include aws quickstart

---
 tests/README.md | 131 +++++++++++++++++++++++++++++++++++++++---------
 1 file changed, 107 insertions(+), 24 deletions(-)

diff --git a/tests/README.md b/tests/README.md
index 6e1f33509f5e1..96d91f626d7bf 100644
--- a/tests/README.md
+++ b/tests/README.md
@@ -2,49 +2,132 @@ System Integration & Performance Testing
 ========================================
 
 This directory contains Kafka system integration and performance tests. 
-[Ducktape](https://github.com/confluentinc/ducktape) is used to run the tests.  
+[ducktape](https://github.com/confluentinc/ducktape) is used to run the tests.  
+(ducktape is a distributed testing framework which provides test runner, 
+result reporter and utilities to pull up and tear down services.) 
 
-Ducktape is a distributed testing framework which provides test runner, 
-result reporter and utilities to pull up and tear down services. It automatically
-discovers tests from a directory and generate an HTML report for each run.
+Local Quickstart
+----------------
+This quickstart will help you run the Kafka system tests on your local machine.
 
-To run the tests: 
+* Install Virtual Box from [https://www.virtualbox.org/](https://www.virtualbox.org/) (run `$ vboxmanage --version` to check if it's installed).
+* Install Vagrant >= 1.6.4 from [http://www.vagrantup.com/](http://www.vagrantup.com/) (run `vagrant --version` to check if it's installed).
+* Install Vagrant Plugins:
 
-1. Build a specific branch of Kafka
+        # Required
+        $ vagrant plugin install vagrant-hostmanager
+
+* Build a specific branch of Kafka
        
         $ cd kafka
         $ git checkout $BRANCH
         $ gradle
         $ ./gradlew jar
       
-2. Setup a testing cluster. You can use Vagrant to create a cluster of local 
-   VMs or on EC2. Configure your Vagrant setup by creating the file 
-   `Vagrantfile.local` in the directory of your Kafka checkout. At a minimum
-   , you *MUST* set `mode = "test"` and the value of `num_workers` high enough for 
-   the test you're trying to run. If you run on AWS, you also need to set 
-   enable_dns = true.
-        
-3. Bring up the cluster, making sure you have enough workers. For Vagrant, 
-   use `vagrant up`. If you want to run on AWS, use `vagrant up
-   --provider=aws --no-parallel`.
+* Setup a testing cluster with Vagrant. Configure your Vagrant setup by creating the file 
+   `Vagrantfile.local` in the directory of your Kafka checkout. At a minimum, you *MUST* 
+   set `mode = "test"` and the value of `num_workers` high enough for the test(s) you're trying to run.
+
+        # Example Vagrantfile.local for use on local machine
+        mode = "test"
+        num_workers = 9
+
+* Bring up the cluster (note that the initial provisioning process can be slow since it involves
+installing dependencies and updates on every vm.):
 
-Note that the initial provisioning process can be quite slow since it involves
-installing dependencies and updates on every vm.
+        $ vagrant up
 
-4. Install ducktape:
+* Install ducktape:
        
         $ git clone https://github.com/confluentinc/ducktape
         $ cd ducktape
         $ python setup.py install
 
-5. Run the system tests using ducktape, you can view results in the `results`
-   directory.
-        
+* Run the system tests using ducktape:
+
         $ cd tests
-        $ ducktape tests
+        $ ducktape kafkatest/tests
 
-6. To iterate/run again if you made any changes:
+* If you make changes to your Kafka checkout, you'll need to rebuild and resync to your Vagrant cluster:
 
         $ cd kafka
         $ ./gradlew jar
         $ vagrant rsync # Re-syncs build output to cluster
+        
+EC2 Quickstart
+--------------
+This quickstart will help you run the Kafka system tests using Amazon EC2. As a convention, we'll use "kafkatest" 
+in most names, but you can use whatever you want.
+
+* [Create an IAM role](http://docs.aws.amazon.com/IAM/latest/UserGuide/Using_SettingUpUser.html#Using_CreateUser_console). We'll give this role the ability to launch or kill additional EC2 machines.
+ - Create role "kafkatest-master"
+ - Role type: Amazon EC2
+ - Attach policy: AmazonEC2FullAccess
+ 
+* If you haven't already, [set up a keypair to use for SSH access](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/ec2-key-pairs.html). For the purpose
+of this quickstart, let's say the keypair name is kafkatest, and you've saved the private key in kafktest.pem
+
+* Next, create a security group called "kafkatest-insecure". 
+ - After creating the group, inbound rules: allow SSH on port 22 from anywhere; also, allow access on all ports (0-65535) from other machines in the kafkatest-insecure group.
+
+* Launch a new test driver machine 
+ - OS: Ubuntu server is recommended
+ - Instance type: t2.medium is easily enough since this machine is just a driver
+ - Instance details: Most defaults are fine.
+ - IAM role -> kafkatest-master
+ - Tagging the instance with a useful name is recommended. 
+ - Security group -> 'kafkatest-insecure'
+  
+* Once the machine is started, upload the SSH key:
+
+        $ scp -i /path/to/kafkatest.pem \
+            /path/to/kafkatest.pem ubuntu@public.hostname.amazonaws.com:kafkatest.pem
+
+* Grab the public hostname/IP and SSH into the host:
+
+        $ ssh -i /path/to/kafkatest.pem ubuntu@public.hostname.amazonaws.com
+        
+* The following steps assume you are logged into
+the test driver machine.
+
+* Start by making sure you're up to date, and install git and ducktape:
+
+        $ sudo apt-get update && sudo apt-get -y upgrade && sudo apt-get install -y git
+        $ git clone https://github.com/confluentinc/ducktape.git
+        $ cd ducktape
+        $ python setup.py install
+
+* Get Kafka:
+
+        $ git clone https://git-wip-us.apache.org/repos/asf/kafka.git kafka
+        
+* Install some dependencies:
+
+        $ cd kafka
+        $ aws/aws-init.sh
+        $ . ~/.bashrc
+
+* An example Vagrantfile.local has been created by aws-init.sh which looks something like:
+
+        # Vagrantfile.local
+        ec2_instance_type = "..." # Pick something appropriate for your
+                                  # test. Note that the default m3.medium has
+                                  # a small disk.
+        mode = "test"
+        num_workers = 9
+        ec2_keypair_name = 'kafkatest'
+        ec2_keypair_file = '/home/ubuntu/kafkatest.pem'
+        ec2_security_groups = ['kafkatest-insecure']
+        ec2_region = 'us-west-2'
+        ec2_ami = "ami-29ebb519"
+
+* Start up the instances:
+
+        $ vagrant up --provider=aws --no-provision --no-parallel && vagrant provision
+
+* Now you should be able to run tests:
+
+        $ cd kafka/tests
+        $ ducktape kafkatest/tests
+
+

From 4f476fec65e92ff5bf940dc4928e2fb64d424c0e Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Fri, 12 Jun 2015 15:17:26 -0700
Subject: [PATCH 23/41] Moved aws scripts to vagrant directory

---
 tests/README.md                                     | 10 +++-------
 tests/aws/aws-access-keys-commands                  |  3 ---
 vagrant/aws/aws-access-keys-commands                | 13 +++++++++++++
 .../aws/aws-example-Vagrantfile.local               |  0
 {tests => vagrant}/aws/aws-init.sh                  |  8 ++++----
 5 files changed, 20 insertions(+), 14 deletions(-)
 delete mode 100644 tests/aws/aws-access-keys-commands
 create mode 100644 vagrant/aws/aws-access-keys-commands
 rename {tests => vagrant}/aws/aws-example-Vagrantfile.local (100%)
 rename {tests => vagrant}/aws/aws-init.sh (87%)

diff --git a/tests/README.md b/tests/README.md
index 96d91f626d7bf..91fd8b6145817 100644
--- a/tests/README.md
+++ b/tests/README.md
@@ -39,9 +39,7 @@ installing dependencies and updates on every vm.):
 
 * Install ducktape:
        
-        $ git clone https://github.com/confluentinc/ducktape
-        $ cd ducktape
-        $ python setup.py install
+        $ pip install ducktape
 
 * Run the system tests using ducktape:
 
@@ -93,9 +91,7 @@ the test driver machine.
 * Start by making sure you're up to date, and install git and ducktape:
 
         $ sudo apt-get update && sudo apt-get -y upgrade && sudo apt-get install -y git
-        $ git clone https://github.com/confluentinc/ducktape.git
-        $ cd ducktape
-        $ python setup.py install
+        $ pip install ducktape
 
 * Get Kafka:
 
@@ -104,7 +100,7 @@ the test driver machine.
 * Install some dependencies:
 
         $ cd kafka
-        $ aws/aws-init.sh
+        $ kafka/vagrant/aws/aws-init.sh
         $ . ~/.bashrc
 
 * An example Vagrantfile.local has been created by aws-init.sh which looks something like:
diff --git a/tests/aws/aws-access-keys-commands b/tests/aws/aws-access-keys-commands
deleted file mode 100644
index d4aa263e89ad7..0000000000000
--- a/tests/aws/aws-access-keys-commands
+++ /dev/null
@@ -1,3 +0,0 @@
-export AWS_ACCESS_KEY=`curl -s http://169.254.169.254/latest/meta-data/iam/security-credentials/ducttape-master | grep AccessKeyId | awk -F\" '{ print $4 }'`
-export AWS_SECRET_KEY=`curl -s http://169.254.169.254/latest/meta-data/iam/security-credentials/ducttape-master | grep SecretAccessKey | awk -F\" '{ print $4 }'`
-export AWS_SESSION_TOKEN=`curl -s http://169.254.169.254/latest/meta-data/iam/security-credentials/ducttape-master | grep Token | awk -F\" '{ print $4 }'`
diff --git a/vagrant/aws/aws-access-keys-commands b/vagrant/aws/aws-access-keys-commands
new file mode 100644
index 0000000000000..4607ec00ff747
--- /dev/null
+++ b/vagrant/aws/aws-access-keys-commands
@@ -0,0 +1,13 @@
+if [ -z "$AWS_IAM" ];then
+    echo "Warning: AWS_IAM is not set"
+fi
+
+export AWS_ACCESS_KEY=`curl -s http://169.254.169.254/latest/meta-data/iam/security-credentials/$AWS_IAM | grep AccessKeyId | awk -F\" '{ print $4 }'`
+export AWS_SECRET_KEY=`curl -s http://169.254.169.254/latest/meta-data/iam/security-credentials/$AWS_IAM | grep SecretAccessKey | awk -F\" '{ print $4 }'`
+export AWS_SESSION_TOKEN=`curl -s http://169.254.169.254/latest/meta-data/iam/security-credentials/$AWS_IAM | grep Token | awk -F\" '{ print $4 }'`
+
+if [ -z "$AWS_ACCESS_KEY" ]; then
+    echo "Failed to populate environment variables AWS_ACCESS_KEY, AWS_SECRET_KEY, and AWS_SESSION_TOKEN."
+    echo "AWS_IAM is currently $AWS_IAM. Double-check that this is correct. If not set, add this command to your .bashrc file:"
+    echo "export AWS_IAM=  # put this into your ~/.bashrc"
+fi
diff --git a/tests/aws/aws-example-Vagrantfile.local b/vagrant/aws/aws-example-Vagrantfile.local
similarity index 100%
rename from tests/aws/aws-example-Vagrantfile.local
rename to vagrant/aws/aws-example-Vagrantfile.local
diff --git a/tests/aws/aws-init.sh b/vagrant/aws/aws-init.sh
similarity index 87%
rename from tests/aws/aws-init.sh
rename to vagrant/aws/aws-init.sh
index 027aa90748b79..f3c32c5495592 100755
--- a/tests/aws/aws-init.sh
+++ b/vagrant/aws/aws-init.sh
@@ -1,13 +1,13 @@
 #!/bin/bash
 
-# This script should be run once on your aws test driver machine before
-# attempting to run any ducktape tests
+# This script can be used to set up a driver machine on aws from which you will run tests
+# or bring up your mini Kafka cluster.
 
 # Install dependencies
-sudo apt-get install -y maven openjdk-6-jdk build-essential \
+sudo apt-get install -y maven openjdk-7-jdk build-essential \
             ruby-dev zlib1g-dev realpath python-setuptools
 
-base_dir=`dirname $0`/..
+base_dir=`dirname $0`/../..
 
 if [ -z `which vagrant` ]; then
     echo "Installing vagrant..."

From c60125cf4b983de958685cdcf10e7bab9813b119 Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Fri, 12 Jun 2015 15:18:59 -0700
Subject: [PATCH 24/41] TestEndToEndLatency -> EndToEndLatency

---
 .../tools/{TestEndToEndLatency.scala => EndToEndLatency.scala}  | 2 +-
 tests/kafkatest/services/performance.py                         | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
 rename core/src/main/scala/kafka/tools/{TestEndToEndLatency.scala => EndToEndLatency.scala} (99%)

diff --git a/core/src/main/scala/kafka/tools/TestEndToEndLatency.scala b/core/src/main/scala/kafka/tools/EndToEndLatency.scala
similarity index 99%
rename from core/src/main/scala/kafka/tools/TestEndToEndLatency.scala
rename to core/src/main/scala/kafka/tools/EndToEndLatency.scala
index 3d25e9c053af0..7bb69b71e9b66 100755
--- a/core/src/main/scala/kafka/tools/TestEndToEndLatency.scala
+++ b/core/src/main/scala/kafka/tools/EndToEndLatency.scala
@@ -24,7 +24,7 @@ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, Produce
 
 import scala.Option.option2Iterable
 
-object TestEndToEndLatency {
+object EndToEndLatency {
   def main(args: Array[String]) {
     if (args.length != 6) {
       System.err.println("USAGE: java " + getClass().getName + " broker_list zookeeper_connect topic num_messages consumer_fetch_max_wait producer_acks")
diff --git a/tests/kafkatest/services/performance.py b/tests/kafkatest/services/performance.py
index 8026c5a5d11a8..e0152ca2d8dc2 100644
--- a/tests/kafkatest/services/performance.py
+++ b/tests/kafkatest/services/performance.py
@@ -125,7 +125,7 @@ def _worker(self, idx, node):
             'zk_connect': self.kafka.zk.connect_setting(),
             'bootstrap_servers': self.kafka.bootstrap_servers(),
         })
-        cmd = "/opt/kafka/bin/kafka-run-class.sh kafka.tools.TestEndToEndLatency "\
+        cmd = "/opt/kafka/bin/kafka-run-class.sh kafka.tools.EndToEndLatency "\
               "%(bootstrap_servers)s %(zk_connect)s %(topic)s %(num_records)d "\
               "%(consumer_fetch_max_wait)d %(acks)d" % args
         self.logger.debug("End-to-end latency %d command: %s", idx, cmd)

From 7f7c3e0e68d9c3c50fb4b836b90887f39b43c466 Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Fri, 12 Jun 2015 15:20:25 -0700
Subject: [PATCH 25/41] Updated setup.py for kafkatest

---
 tests/setup.py | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)

diff --git a/tests/setup.py b/tests/setup.py
index 8cf40b2a5186b..caf94883d09fe 100644
--- a/tests/setup.py
+++ b/tests/setup.py
@@ -1,10 +1,11 @@
 from setuptools import find_packages, setup
 
 setup(name="kafkatest",
-      version="0.1",
-      description="System tests for Apache Kafka",
-      author="Ewen Cheslack-Postava , Geoff Anderson ",
+      version="0.8.3-SNAPSHOT",
+      description="Apache Kafka System Tests",
+      author="Apache Kafka",
       platforms=["any"], 
       license="apache2.0",
       packages=find_packages(),
+      requires=["ducktape(>=0.2.0)"]
       )

From 42dcdb1d66704bf512ddadc3020c493e74c5ba25 Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Fri, 12 Jun 2015 15:21:07 -0700
Subject: [PATCH 26/41] Tweaked behavior of stop_node, clean_node to generally
 fail fast

---
 tests/kafkatest/services/console_consumer.py    |  4 ++--
 tests/kafkatest/services/kafka.py               |  8 ++++----
 tests/kafkatest/services/verifiable_producer.py |  2 +-
 tests/kafkatest/services/zookeeper.py           | 12 ++++--------
 4 files changed, 11 insertions(+), 15 deletions(-)

diff --git a/tests/kafkatest/services/console_consumer.py b/tests/kafkatest/services/console_consumer.py
index 43268894178dc..990cba8035fa7 100644
--- a/tests/kafkatest/services/console_consumer.py
+++ b/tests/kafkatest/services/console_consumer.py
@@ -138,8 +138,8 @@ def start_node(self, node):
         super(ConsoleConsumer, self).start_node(node)
 
     def stop_node(self, node):
-        node.account.kill_process("java", allow_fail=True)
+        node.account.kill_process("java", allow_fail=False)
 
     def clean_node(self, node):
-        node.account.ssh("rm -rf /mnt/console_consumer.properties /mnt/consumer.log", allow_fail=True)
+        node.account.ssh("rm -rf /mnt/console_consumer.properties /mnt/consumer.log", allow_fail=False)
 
diff --git a/tests/kafkatest/services/kafka.py b/tests/kafkatest/services/kafka.py
index da8c803e7126e..5e3e5a0736a6d 100644
--- a/tests/kafkatest/services/kafka.py
+++ b/tests/kafkatest/services/kafka.py
@@ -82,17 +82,17 @@ def signal_leader(self, topic, partition=0, sig=signal.SIGTERM):
         leader = self.leader(topic, partition)
         self.signal_node(leader, sig)
 
-    def stop_node(self, node, clean_shutdown=True, allow_fail=True):
+    def stop_node(self, node, clean_shutdown=True):
         pids = self.pids(node)
         sig = signal.SIGTERM if clean_shutdown else signal.SIGKILL
 
         for pid in pids:
-            node.account.signal(pid, sig, allow_fail=allow_fail)
+            node.account.signal(pid, sig, allow_fail=False)
 
-        node.account.ssh("rm -f /mnt/kafka.pid", allow_fail=True)
+        node.account.ssh("rm -f /mnt/kafka.pid", allow_fail=False)
 
     def clean_node(self, node):
-        node.account.ssh("rm -rf /mnt/kafka-logs /mnt/kafka.properties /mnt/kafka.log")
+        node.account.ssh("rm -rf /mnt/kafka-logs /mnt/kafka.properties /mnt/kafka.log", allow_fail=False)
 
     def create_topic(self, topic_cfg):
         node = self.nodes[0] # any node is fine here
diff --git a/tests/kafkatest/services/verifiable_producer.py b/tests/kafkatest/services/verifiable_producer.py
index 88455601dfae7..95e91cf96882b 100644
--- a/tests/kafkatest/services/verifiable_producer.py
+++ b/tests/kafkatest/services/verifiable_producer.py
@@ -96,7 +96,7 @@ def stop_node(self, node):
             self.worker_threads[self.idx(node) - 1].join()
 
     def clean_node(self, node):
-        node.account.ssh("rm -rf /mnt/producer.log")
+        node.account.ssh("rm -rf /mnt/producer.log", allow_fail=False)
 
     def try_parse_json(self, string):
         """Try to parse a string as json. Return None if not parseable."""
diff --git a/tests/kafkatest/services/zookeeper.py b/tests/kafkatest/services/zookeeper.py
index 3f35c11dd8659..95aaa268a0a22 100644
--- a/tests/kafkatest/services/zookeeper.py
+++ b/tests/kafkatest/services/zookeeper.py
@@ -49,18 +49,14 @@ def start_node(self, node):
 
         time.sleep(5)  # give it some time to start
 
-    def stop_node(self, node, allow_fail=True):
-        # This uses Kafka-REST's stop service script because it's better behaved
-        # (knows how to wait) and sends SIGTERM instead of
-        # zookeeper-stop-server.sh's SIGINT. We don't actually care about clean
-        # shutdown here, so it's ok to use the bigger hammer
+    def stop_node(self, node):
         idx = self.idx(node)
         self.logger.info("Stopping %s node %d on %s" % (type(self).__name__, idx, node.account.hostname))
-        node.account.ssh("/opt/kafka-rest/bin/kafka-rest-stop-service zookeeper", allow_fail=allow_fail)
+        node.account.kill_process("zookeeper", allow_fail=False)
 
-    def clean_node(self, node, allow_fail=True):
+    def clean_node(self, node):
         self.logger.info("Cleaning ZK node %d on %s", self.idx(node), node.account.hostname)
-        node.account.ssh("rm -rf /mnt/zookeeper /mnt/zookeeper.properties /mnt/zk.log", allow_fail=allow_fail)
+        node.account.ssh("rm -rf /mnt/zookeeper /mnt/zookeeper.properties /mnt/zk.log", allow_fail=False)
 
     def connect_setting(self):
         return ','.join([node.account.hostname + ':2181' for node in self.nodes])

From f469f84e6d827a580c26a62ef01f0e53f002c3e3 Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Mon, 15 Jun 2015 13:13:17 -0700
Subject: [PATCH 27/41] Tweaked readme, added example Vagrantfile.local

---
 tests/README.md                       | 2 ++
 vagrant/system-test-Vagrantfile.local | 3 +++
 2 files changed, 5 insertions(+)
 create mode 100644 vagrant/system-test-Vagrantfile.local

diff --git a/tests/README.md b/tests/README.md
index 91fd8b6145817..982525cbea21b 100644
--- a/tests/README.md
+++ b/tests/README.md
@@ -27,8 +27,10 @@ This quickstart will help you run the Kafka system tests on your local machine.
 * Setup a testing cluster with Vagrant. Configure your Vagrant setup by creating the file 
    `Vagrantfile.local` in the directory of your Kafka checkout. At a minimum, you *MUST* 
    set `mode = "test"` and the value of `num_workers` high enough for the test(s) you're trying to run.
+    An example resides in kafka/vagrant/system-test-Vagrantfile.local
 
         # Example Vagrantfile.local for use on local machine
+        # Vagrantfile.local should reside in the base Kafka directory
         mode = "test"
         num_workers = 9
 
diff --git a/vagrant/system-test-Vagrantfile.local b/vagrant/system-test-Vagrantfile.local
new file mode 100644
index 0000000000000..7598b7579bc1d
--- /dev/null
+++ b/vagrant/system-test-Vagrantfile.local
@@ -0,0 +1,3 @@
+# Use this Vagrantfile.local for running system tests
+mode = "test"
+num_workers = 9

From 0f75187458ff38e8ce9e980469ade2b687c23507 Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Mon, 15 Jun 2015 17:11:49 -0700
Subject: [PATCH 28/41] Rmoved stray allow_fail. kafka_benchmark_test ->
 benchmark_test

---
 tests/kafkatest/services/kafka.py                             | 4 ++--
 .../tests/{kafka_benchmark_test.py => benchmark_test.py}      | 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)
 rename tests/kafkatest/tests/{kafka_benchmark_test.py => benchmark_test.py} (99%)

diff --git a/tests/kafkatest/services/kafka.py b/tests/kafkatest/services/kafka.py
index 5e3e5a0736a6d..beb88f8fb7586 100644
--- a/tests/kafkatest/services/kafka.py
+++ b/tests/kafkatest/services/kafka.py
@@ -92,7 +92,7 @@ def stop_node(self, node, clean_shutdown=True):
         node.account.ssh("rm -f /mnt/kafka.pid", allow_fail=False)
 
     def clean_node(self, node):
-        node.account.ssh("rm -rf /mnt/kafka-logs /mnt/kafka.properties /mnt/kafka.log", allow_fail=False)
+        node.account.ssh("rm -rf /mnt/kafka-logs /mnt/kafka.properties /mnt/kafka.log /mnt/kafka.pid", allow_fail=False)
 
     def create_topic(self, topic_cfg):
         node = self.nodes[0] # any node is fine here
@@ -191,7 +191,7 @@ def execute_reassign_partitions(self, reassignment):
 
     def restart_node(self, node, wait_sec=0, clean_shutdown=True):
         """Restart the given node, waiting wait_sec in between stopping and starting up again."""
-        self.stop_node(node, clean_shutdown, allow_fail=True)
+        self.stop_node(node, clean_shutdown)
         time.sleep(wait_sec)
         self.start_node(node)
 
diff --git a/tests/kafkatest/tests/kafka_benchmark_test.py b/tests/kafkatest/tests/benchmark_test.py
similarity index 99%
rename from tests/kafkatest/tests/kafka_benchmark_test.py
rename to tests/kafkatest/tests/benchmark_test.py
index 75e26e34b617f..19ae6f392db71 100644
--- a/tests/kafkatest/tests/kafka_benchmark_test.py
+++ b/tests/kafkatest/tests/benchmark_test.py
@@ -18,7 +18,7 @@
 from kafkatest.services.performance import ProducerPerformanceService, ConsumerPerformanceService, EndToEndLatencyService
 
 
-class KafkaBenchmark(KafkaTest):
+class Benchmark(KafkaTest):
     '''A benchmark of Kafka producer/consumer performance. This replicates the test
     run here:
     https://engineering.linkedin.com/kafka/benchmarking-apache-kafka-2-million-writes-second-three-cheap-machines

From b15b24f177684b8ec8139d323e1e37cbc44cd9ec Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Mon, 15 Jun 2015 17:36:33 -0700
Subject: [PATCH 29/41] leftover KafkaBenchmark in super call

---
 tests/kafkatest/tests/benchmark_test.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/tests/kafkatest/tests/benchmark_test.py b/tests/kafkatest/tests/benchmark_test.py
index 19ae6f392db71..2be0bed796d9b 100644
--- a/tests/kafkatest/tests/benchmark_test.py
+++ b/tests/kafkatest/tests/benchmark_test.py
@@ -24,7 +24,7 @@ class Benchmark(KafkaTest):
     https://engineering.linkedin.com/kafka/benchmarking-apache-kafka-2-million-writes-second-three-cheap-machines
     '''
     def __init__(self, test_context):
-        super(KafkaBenchmark, self).__init__(test_context, num_zk=1, num_brokers=3, topics={
+        super(Benchmark, self).__init__(test_context, num_zk=1, num_brokers=3, topics={
             'test-rep-one' : { 'partitions': 6, 'replication-factor': 1 },
             'test-rep-three' : { 'partitions': 6, 'replication-factor': 3 }
         })

From e6a41f1c9a373975662b9d97f5c7323ec0982f3d Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Mon, 15 Jun 2015 18:51:58 -0700
Subject: [PATCH 30/41] removed stray println

---
 .../org/apache/kafka/clients/tools/VerifiableProducer.java   | 5 +----
 tests/kafkatest/services/verifiable_producer.py              | 2 +-
 2 files changed, 2 insertions(+), 5 deletions(-)

diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java b/clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java
index c12a61993a5af..7a174289aacba 100644
--- a/clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java
@@ -136,10 +136,7 @@ public static VerifiableProducer createFromArgs(String[] args) {
         try {
             Namespace res;
             res = parser.parseArgs(args);
-            System.out.println(res);
-            System.out.println(res.getString("brokerList"));
-            
-            
+
             int maxMessages = res.getInt("maxMessages");
             String topic = res.getString("topic");
             int throughput = res.getInt("throughput");
diff --git a/tests/kafkatest/services/verifiable_producer.py b/tests/kafkatest/services/verifiable_producer.py
index 95e91cf96882b..656097696036c 100644
--- a/tests/kafkatest/services/verifiable_producer.py
+++ b/tests/kafkatest/services/verifiable_producer.py
@@ -38,7 +38,7 @@ def __init__(self, context, num_nodes, kafka, topic, max_messages=-1, throughput
 
     def _worker(self, idx, node):
         cmd = self.start_cmd
-        self.logger.debug("Verbose producer %d command: %s" % (idx, cmd))
+        self.logger.debug("VerifiableProducer %d command: %s" % (idx, cmd))
 
         for line in node.account.ssh_capture(cmd):
             line = line.strip()

From 98b725374c3310fec8c72efd61869e092ce1b158 Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Mon, 15 Jun 2015 19:18:28 -0700
Subject: [PATCH 31/41] Updated consumer tests to pre-populate kafka logs

---
 tests/kafkatest/services/performance.py |  4 ++--
 tests/kafkatest/tests/benchmark_test.py | 20 +++++++++++++++++++-
 2 files changed, 21 insertions(+), 3 deletions(-)

diff --git a/tests/kafkatest/services/performance.py b/tests/kafkatest/services/performance.py
index e0152ca2d8dc2..a0f97b6dab06d 100644
--- a/tests/kafkatest/services/performance.py
+++ b/tests/kafkatest/services/performance.py
@@ -102,8 +102,8 @@ def _worker(self, idx, node):
         parts = last.split(',')
 
         self.results[idx-1] = {
-            'total_mb': float(parts[3]),
-            'mbps': float(parts[4]),
+            'total_mb': float(parts[2]),
+            'mbps': float(parts[3]),
             'records_per_sec': float(parts[5]),
         }
 
diff --git a/tests/kafkatest/tests/benchmark_test.py b/tests/kafkatest/tests/benchmark_test.py
index 2be0bed796d9b..85b4ea08886f0 100644
--- a/tests/kafkatest/tests/benchmark_test.py
+++ b/tests/kafkatest/tests/benchmark_test.py
@@ -194,12 +194,21 @@ def test_producer_and_consumer(self):
         return data
 
     def test_single_consumer(self):
+        topic = "test-rep-three"
+
+        self.producer = ProducerPerformanceService(
+            self.test_context, 1, self.kafka,
+            topic=topic, num_records=self.msgs_default, record_size=self.msg_size_default, throughput=-1,
+            settings={'acks':1, 'batch.size':self.batch_size, 'buffer.memory':self.buffer_memory}
+        )
+        self.producer.run()
+
         # All consumer tests use the messages from the first benchmark, so
         # they'll get messages of the default message size
         self.logger.info("BENCHMARK: Single consumer")
         self.perf = ConsumerPerformanceService(
             self.test_context, 1, self.kafka,
-            topic="test-rep-three", num_records=self.msgs_default, throughput=-1, threads=1
+            topic=topic, num_records=self.msgs_default, throughput=-1, threads=1
         )
         self.perf.run()
 
@@ -208,6 +217,15 @@ def test_single_consumer(self):
         return data
 
     def test_three_consumers(self):
+        topic = "test-rep-three"
+
+        self.producer = ProducerPerformanceService(
+            self.test_context, 1, self.kafka,
+            topic=topic, num_records=self.msgs_default, record_size=self.msg_size_default, throughput=-1,
+            settings={'acks':1, 'batch.size':self.batch_size, 'buffer.memory':self.buffer_memory}
+        )
+        self.producer.run()
+
         self.logger.info("BENCHMARK: Three consumers")
         self.perf = ConsumerPerformanceService(
             self.test_context, 3, self.kafka,

From f14c50792a101b93efe6b7719e30af56a19298c1 Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Tue, 16 Jun 2015 09:47:33 -0700
Subject: [PATCH 32/41] Removed mode = "test" from Vagrantfile and
 Vagrantfile.local examples. Updated testing README to clarify aws setup.

---
 Vagrantfile                               |  5 ---
 tests/README.md                           | 48 +++++++++++++++--------
 vagrant/aws/aws-example-Vagrantfile.local |  8 +++-
 vagrant/system-test-Vagrantfile.local     |  7 +++-
 4 files changed, 43 insertions(+), 25 deletions(-)

diff --git a/Vagrantfile b/Vagrantfile
index 6cd6bc083f1a4..4a3b773218737 100644
--- a/Vagrantfile
+++ b/Vagrantfile
@@ -53,11 +53,6 @@ if File.exists?(local_config_file) then
   eval(File.read(local_config_file), binding, "Vagrantfile.local")
 end
 
-if mode == "test"
-  num_zookeepers = 0
-  num_brokers = 0
-end
-
 # This is a horrible hack to work around bad interactions between
 # vagrant-hostmanager and vagrant-aws/vagrant's implementation. Hostmanager
 # wants to update the /etc/hosts entries, but tries to do so even on nodes that
diff --git a/tests/README.md b/tests/README.md
index 982525cbea21b..83bf5c2cfea56 100644
--- a/tests/README.md
+++ b/tests/README.md
@@ -25,13 +25,14 @@ This quickstart will help you run the Kafka system tests on your local machine.
         $ ./gradlew jar
       
 * Setup a testing cluster with Vagrant. Configure your Vagrant setup by creating the file 
-   `Vagrantfile.local` in the directory of your Kafka checkout. At a minimum, you *MUST* 
-   set `mode = "test"` and the value of `num_workers` high enough for the test(s) you're trying to run.
-    An example resides in kafka/vagrant/system-test-Vagrantfile.local
+   `Vagrantfile.local` in the directory of your Kafka checkout. For testing purposes,
+  `num_brokers` and `num_kafka` should be 0, and `num_workers` should be set high enough
+  to run all of you tests. An example resides in kafka/vagrant/system-test-Vagrantfile.local
 
         # Example Vagrantfile.local for use on local machine
         # Vagrantfile.local should reside in the base Kafka directory
-        mode = "test"
+        num_zookeepers = 0
+        num_kafka = 0
         num_workers = 9
 
 * Bring up the cluster (note that the initial provisioning process can be slow since it involves
@@ -56,38 +57,51 @@ installing dependencies and updates on every vm.):
         
 EC2 Quickstart
 --------------
-This quickstart will help you run the Kafka system tests using Amazon EC2. As a convention, we'll use "kafkatest" 
-in most names, but you can use whatever you want.
+This quickstart will help you run the Kafka system tests using Amazon EC2. As a convention, we'll use "kafkatest" in most names, but you can use whatever you want. 
+
+There are a lot of steps here, but the basic goals are to create one distinguished EC2 instance that
+will be our "test driver", and to set up the security groups and iam role so that the test driver
+can create, destroy, and run ssh commands on any number of "workers".
+
+Preparation
+-----------
+In these steps, we will create an IAM role which has permission to create and destroy EC2 instances, 
+set up a keypair used for ssh access to the test driver and worker machines, and create a security group to allow the test driver and workers to all communicate via TCP.
 
 * [Create an IAM role](http://docs.aws.amazon.com/IAM/latest/UserGuide/Using_SettingUpUser.html#Using_CreateUser_console). We'll give this role the ability to launch or kill additional EC2 machines.
  - Create role "kafkatest-master"
  - Role type: Amazon EC2
- - Attach policy: AmazonEC2FullAccess
+ - Attach policy: AmazonEC2FullAccess (this will allow our test-driver to create and destroy EC2 instances)
  
 * If you haven't already, [set up a keypair to use for SSH access](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/ec2-key-pairs.html). For the purpose
 of this quickstart, let's say the keypair name is kafkatest, and you've saved the private key in kafktest.pem
 
-* Next, create a security group called "kafkatest-insecure". 
- - After creating the group, inbound rules: allow SSH on port 22 from anywhere; also, allow access on all ports (0-65535) from other machines in the kafkatest-insecure group.
+* Next, create a security group called "kafkatest". 
+ - After creating the group, inbound rules: allow SSH on port 22 from anywhere; also, allow access on all ports (0-65535) from other machines in the kafkatest group.
 
+Create the Test Driver
+----------------------
 * Launch a new test driver machine 
  - OS: Ubuntu server is recommended
  - Instance type: t2.medium is easily enough since this machine is just a driver
  - Instance details: Most defaults are fine.
  - IAM role -> kafkatest-master
  - Tagging the instance with a useful name is recommended. 
- - Security group -> 'kafkatest-insecure'
+ - Security group -> 'kafkatest'
   
-* Once the machine is started, upload the SSH key:
+
+* Once the machine is started, upload the SSH key to your test driver:
 
         $ scp -i /path/to/kafkatest.pem \
             /path/to/kafkatest.pem ubuntu@public.hostname.amazonaws.com:kafkatest.pem
 
-* Grab the public hostname/IP and SSH into the host:
+* Grab the public hostname/IP (available for example by navigating to your EC2 dashboard and viewing running instances) of your test driver and SSH into it:
 
         $ ssh -i /path/to/kafkatest.pem ubuntu@public.hostname.amazonaws.com
         
-* The following steps assume you are logged into
+Set Up the Test Driver
+----------------------
+The following steps assume you have ssh'd into
 the test driver machine.
 
 * Start by making sure you're up to date, and install git and ducktape:
@@ -111,15 +125,16 @@ the test driver machine.
         ec2_instance_type = "..." # Pick something appropriate for your
                                   # test. Note that the default m3.medium has
                                   # a small disk.
-        mode = "test"
+        num_zookeepers = 0
+        num_kafka = 0
         num_workers = 9
         ec2_keypair_name = 'kafkatest'
         ec2_keypair_file = '/home/ubuntu/kafkatest.pem'
-        ec2_security_groups = ['kafkatest-insecure']
+        ec2_security_groups = ['kafkatest']
         ec2_region = 'us-west-2'
         ec2_ami = "ami-29ebb519"
 
-* Start up the instances:
+* Start up the instances (note we have found bringing up machines in parallel can cause errors on aws):
 
         $ vagrant up --provider=aws --no-provision --no-parallel && vagrant provision
 
@@ -128,4 +143,5 @@ the test driver machine.
         $ cd kafka/tests
         $ ducktape kafkatest/tests
 
+* To halt your workers without destroying persistent state, run `vagrant halt`. Run `vagrant destroy -f` to destroy all traces of your workers.
 
diff --git a/vagrant/aws/aws-example-Vagrantfile.local b/vagrant/aws/aws-example-Vagrantfile.local
index 615cde9c17176..e46acc48df21a 100644
--- a/vagrant/aws/aws-example-Vagrantfile.local
+++ b/vagrant/aws/aws-example-Vagrantfile.local
@@ -1,8 +1,12 @@
+# Use this template Vagrantfile.local for running system tests on aws 
+# To use it, move it to the base kafka directory and rename
+# it to Vagrantfile.local, and adjust variables as needed.
 ec3_instance_type = "m3.medium"
-mode = "test"
+num_zookeepers = 0
+num_brokers = 0
 num_workers = 9
 ec2_keypair_name = kafkatest
 ec2_keypair_file = ../kafkatest.pem
-ec2_security_groups = ['kafkatest-insecure']
+ec2_security_groups = ['kafkatest']
 ec2_region = 'us-west-2'
 ec2_ami = "ami-29ebb519"
diff --git a/vagrant/system-test-Vagrantfile.local b/vagrant/system-test-Vagrantfile.local
index 7598b7579bc1d..ea4fd93a2e9a8 100644
--- a/vagrant/system-test-Vagrantfile.local
+++ b/vagrant/system-test-Vagrantfile.local
@@ -1,3 +1,6 @@
-# Use this Vagrantfile.local for running system tests
-mode = "test"
+# Use this example Vagrantfile.local for running system tests
+# To use it, move it to the base kafka directory and rename
+# it to Vagrantfile.local
+num_zookeepers = 0
+num_brokers = 0
 num_workers = 9

From 8c61e2d6fc03fb82a0095161b731828302bc240a Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Tue, 16 Jun 2015 09:56:15 -0700
Subject: [PATCH 33/41] Reverted jdk back to 6

---
 vagrant/aws/aws-init.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/vagrant/aws/aws-init.sh b/vagrant/aws/aws-init.sh
index f3c32c5495592..45fe7f40ffbb4 100755
--- a/vagrant/aws/aws-init.sh
+++ b/vagrant/aws/aws-init.sh
@@ -4,7 +4,7 @@
 # or bring up your mini Kafka cluster.
 
 # Install dependencies
-sudo apt-get install -y maven openjdk-7-jdk build-essential \
+sudo apt-get install -y maven openjdk-6-jdk build-essential \
             ruby-dev zlib1g-dev realpath python-setuptools
 
 base_dir=`dirname $0`/../..

From e6d532f0a6e330f1734a4006e868efc1e760d661 Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Tue, 16 Jun 2015 10:00:15 -0700
Subject: [PATCH 34/41] java 7 -> java 6

---
 vagrant/base.sh | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)

diff --git a/vagrant/base.sh b/vagrant/base.sh
index 133f10a95622c..c78839b566de3 100644
--- a/vagrant/base.sh
+++ b/vagrant/base.sh
@@ -24,16 +24,16 @@ if [ -z `which javac` ]; then
     apt-get -y update
 
     # Try to share cache. See Vagrantfile for details
-    mkdir -p /var/cache/oracle-jdk7-installer
-    if [ -e "/tmp/oracle-jdk7-installer-cache/" ]; then
-        find /tmp/oracle-jdk7-installer-cache/ -not -empty -exec cp '{}' /var/cache/oracle-jdk7-installer/ \;
+    mkdir -p /var/cache/oracle-jdk6-installer
+    if [ -e "/tmp/oracle-jdk6-installer-cache/" ]; then
+        find /tmp/oracle-jdk6-installer-cache/ -not -empty -exec cp '{}' /var/cache/oracle-jdk6-installer/ \;
     fi
 
     /bin/echo debconf shared/accepted-oracle-license-v1-1 select true | /usr/bin/debconf-set-selections
-    apt-get -y install oracle-java7-installer oracle-java7-set-default
+    apt-get -y install oracle-java6-installer oracle-java6-set-default
 
-    if [ -e "/tmp/oracle-jdk7-installer-cache/" ]; then
-        cp -R /var/cache/oracle-jdk7-installer/* /tmp/oracle-jdk7-installer-cache
+    if [ -e "/tmp/oracle-jdk6-installer-cache/" ]; then
+        cp -R /var/cache/oracle-jdk6-installer/* /tmp/oracle-jdk6-installer-cache
     fi
 fi
 

From 7751545f6245f50ac1701e28d658a9b16d8d930b Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Tue, 16 Jun 2015 10:01:02 -0700
Subject: [PATCH 35/41] Corrected license headers

---
 tests/kafkatest/__init__.py                     | 14 +-------------
 tests/kafkatest/services/console_consumer.py    | 13 +++++++------
 tests/kafkatest/services/kafka.py               | 13 +++++++------
 tests/kafkatest/services/performance.py         | 13 +++++++------
 tests/kafkatest/services/verifiable_producer.py | 13 +++++++------
 tests/kafkatest/services/zookeeper.py           | 14 ++++++++------
 tests/kafkatest/tests/benchmark_test.py         | 13 +++++++------
 tests/kafkatest/tests/kafka_test.py             | 13 +++++++------
 tests/kafkatest/tests/replication_test.py       | 13 +++++++------
 9 files changed, 58 insertions(+), 61 deletions(-)

diff --git a/tests/kafkatest/__init__.py b/tests/kafkatest/__init__.py
index 1896e9e3bf471..8b137891791fe 100644
--- a/tests/kafkatest/__init__.py
+++ b/tests/kafkatest/__init__.py
@@ -1,13 +1 @@
-# Copyright 2015 Confluent Inc.
-#
-# Licensed 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.
+
diff --git a/tests/kafkatest/services/console_consumer.py b/tests/kafkatest/services/console_consumer.py
index 990cba8035fa7..33ef4eaeb2e81 100644
--- a/tests/kafkatest/services/console_consumer.py
+++ b/tests/kafkatest/services/console_consumer.py
@@ -1,10 +1,11 @@
-# Copyright 2015 Confluent Inc.
+# 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
 #
-# Licensed 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
+#    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,
diff --git a/tests/kafkatest/services/kafka.py b/tests/kafkatest/services/kafka.py
index beb88f8fb7586..34ec5ef95de9a 100644
--- a/tests/kafkatest/services/kafka.py
+++ b/tests/kafkatest/services/kafka.py
@@ -1,10 +1,11 @@
-# Copyright 2014 Confluent Inc.
+# 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
 #
-# Licensed 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
+#    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,
diff --git a/tests/kafkatest/services/performance.py b/tests/kafkatest/services/performance.py
index a0f97b6dab06d..65c1a4d6ea078 100644
--- a/tests/kafkatest/services/performance.py
+++ b/tests/kafkatest/services/performance.py
@@ -1,10 +1,11 @@
-# Copyright 2014 Confluent Inc.
+# 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
 #
-# Licensed 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
+#    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,
diff --git a/tests/kafkatest/services/verifiable_producer.py b/tests/kafkatest/services/verifiable_producer.py
index 656097696036c..4062bbeaa30d9 100644
--- a/tests/kafkatest/services/verifiable_producer.py
+++ b/tests/kafkatest/services/verifiable_producer.py
@@ -1,10 +1,11 @@
-# Copyright 2015 Confluent Inc.
+# 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
 #
-# Licensed 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
+#    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,
diff --git a/tests/kafkatest/services/zookeeper.py b/tests/kafkatest/services/zookeeper.py
index 95aaa268a0a22..56f46068791dc 100644
--- a/tests/kafkatest/services/zookeeper.py
+++ b/tests/kafkatest/services/zookeeper.py
@@ -1,10 +1,11 @@
-# Copyright 2015 Confluent Inc.
+# 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
 #
-# Licensed 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
+#    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,
@@ -12,6 +13,7 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+
 from ducktape.services.service import Service
 
 import time
diff --git a/tests/kafkatest/tests/benchmark_test.py b/tests/kafkatest/tests/benchmark_test.py
index 85b4ea08886f0..b01f27bc20cce 100644
--- a/tests/kafkatest/tests/benchmark_test.py
+++ b/tests/kafkatest/tests/benchmark_test.py
@@ -1,10 +1,11 @@
-# Copyright 2014 Confluent Inc.
+# 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
 #
-# Licensed 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
+#    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,
diff --git a/tests/kafkatest/tests/kafka_test.py b/tests/kafkatest/tests/kafka_test.py
index 3cbdf2f622808..7118721b5ded2 100644
--- a/tests/kafkatest/tests/kafka_test.py
+++ b/tests/kafkatest/tests/kafka_test.py
@@ -1,10 +1,11 @@
-# Copyright 2014 Confluent Inc.
+# 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
 #
-# Licensed 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
+#    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,
diff --git a/tests/kafkatest/tests/replication_test.py b/tests/kafkatest/tests/replication_test.py
index 91c031e2ca0ba..75b63bc818ae8 100644
--- a/tests/kafkatest/tests/replication_test.py
+++ b/tests/kafkatest/tests/replication_test.py
@@ -1,10 +1,11 @@
-# Copyright 2015 Confluent Inc.
+# 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
 #
-# Licensed 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
+#    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,

From 2ea4e297059b68a6dddd032390452e6c48fc558d Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Tue, 7 Jul 2015 11:42:19 -0700
Subject: [PATCH 36/41] Tweaked README, changed default log collection behavior
 on VerifiableProducer

---
 tests/README.md                                 | 7 +++++--
 tests/kafkatest/services/verifiable_producer.py | 4 +---
 tests/kafkatest/tests/replication_test.py       | 9 +++++++--
 3 files changed, 13 insertions(+), 7 deletions(-)

diff --git a/tests/README.md b/tests/README.md
index 83bf5c2cfea56..ffbc0d57d7e10 100644
--- a/tests/README.md
+++ b/tests/README.md
@@ -15,7 +15,7 @@ This quickstart will help you run the Kafka system tests on your local machine.
 * Install Vagrant Plugins:
 
         # Required
-        $ vagrant plugin install vagrant-hostmanager
+        $ vagrant plugin install vagrant-hostmanager vagrant-cachier
 
 * Build a specific branch of Kafka
        
@@ -57,12 +57,15 @@ installing dependencies and updates on every vm.):
         
 EC2 Quickstart
 --------------
-This quickstart will help you run the Kafka system tests using Amazon EC2. As a convention, we'll use "kafkatest" in most names, but you can use whatever you want. 
+This quickstart will help you run the Kafka system tests on EC2. In this setup, all logic is run
+on EC2 and none on your local machine. 
 
 There are a lot of steps here, but the basic goals are to create one distinguished EC2 instance that
 will be our "test driver", and to set up the security groups and iam role so that the test driver
 can create, destroy, and run ssh commands on any number of "workers".
 
+As a convention, we'll use "kafkatest" in most names, but you can use whatever name you want. 
+
 Preparation
 -----------
 In these steps, we will create an IAM role which has permission to create and destroy EC2 instances, 
diff --git a/tests/kafkatest/services/verifiable_producer.py b/tests/kafkatest/services/verifiable_producer.py
index 4062bbeaa30d9..cca8227702200 100644
--- a/tests/kafkatest/services/verifiable_producer.py
+++ b/tests/kafkatest/services/verifiable_producer.py
@@ -23,7 +23,7 @@ class VerifiableProducer(BackgroundThreadService):
     logs = {
         "producer_log": {
             "path": "/mnt/producer.log",
-            "collect_default": True}
+            "collect_default": False}
     }
 
     def __init__(self, context, num_nodes, kafka, topic, max_messages=-1, throughput=100000):
@@ -47,8 +47,6 @@ def _worker(self, idx, node):
             data = self.try_parse_json(line)
             if data is not None:
 
-                self.logger.debug("VerifiableProducer: " + str(data))
-
                 with self.lock:
                     if data["name"] == "producer_send_error":
                         data["node"] = idx
diff --git a/tests/kafkatest/tests/replication_test.py b/tests/kafkatest/tests/replication_test.py
index 75b63bc818ae8..fed1ea1f8e2f3 100644
--- a/tests/kafkatest/tests/replication_test.py
+++ b/tests/kafkatest/tests/replication_test.py
@@ -97,7 +97,12 @@ def run_with_failure(self, failure):
         self.logger.info("num consumed:  %d" % len(self.consumed))
 
         # Check produced vs consumed
-        self.validate()
+        success, msg = self.validate()
+
+        if not success:
+            self.mark_for_collect(self.producer)
+
+        assert success, msg
 
     def clean_shutdown(self):
         """Discover leader node for our topic and shut it down cleanly."""
@@ -142,7 +147,7 @@ def validate(self):
             # Collect all the data logs if there was a failure
             self.mark_for_collect(self.kafka)
 
-        assert success, msg
+        return success, msg
 
     def test_clean_shutdown(self):
         self.run_with_failure(self.clean_shutdown)

From 1d93f06c4a9e0ca95d3bd3cba6665273af66cb0c Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Wed, 8 Jul 2015 13:51:30 -0700
Subject: [PATCH 37/41] Updated provisioning to use java 7 in light of
 KAFKA-2316

---
 vagrant/base.sh | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)

diff --git a/vagrant/base.sh b/vagrant/base.sh
index c78839b566de3..133f10a95622c 100644
--- a/vagrant/base.sh
+++ b/vagrant/base.sh
@@ -24,16 +24,16 @@ if [ -z `which javac` ]; then
     apt-get -y update
 
     # Try to share cache. See Vagrantfile for details
-    mkdir -p /var/cache/oracle-jdk6-installer
-    if [ -e "/tmp/oracle-jdk6-installer-cache/" ]; then
-        find /tmp/oracle-jdk6-installer-cache/ -not -empty -exec cp '{}' /var/cache/oracle-jdk6-installer/ \;
+    mkdir -p /var/cache/oracle-jdk7-installer
+    if [ -e "/tmp/oracle-jdk7-installer-cache/" ]; then
+        find /tmp/oracle-jdk7-installer-cache/ -not -empty -exec cp '{}' /var/cache/oracle-jdk7-installer/ \;
     fi
 
     /bin/echo debconf shared/accepted-oracle-license-v1-1 select true | /usr/bin/debconf-set-selections
-    apt-get -y install oracle-java6-installer oracle-java6-set-default
+    apt-get -y install oracle-java7-installer oracle-java7-set-default
 
-    if [ -e "/tmp/oracle-jdk6-installer-cache/" ]; then
-        cp -R /var/cache/oracle-jdk6-installer/* /tmp/oracle-jdk6-installer-cache
+    if [ -e "/tmp/oracle-jdk7-installer-cache/" ]; then
+        cp -R /var/cache/oracle-jdk7-installer/* /tmp/oracle-jdk7-installer-cache
     fi
 fi
 

From 321fdf87f7ffeb4f972f64a15b522de2f4252ded Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Wed, 8 Jul 2015 15:14:49 -0700
Subject: [PATCH 38/41] Ignore tests/ and vagrant/ directories when running rat
 build task

---
 build.gradle | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/build.gradle b/build.gradle
index 4fd277e874faa..580c686547340 100644
--- a/build.gradle
+++ b/build.gradle
@@ -65,6 +65,8 @@ rat {
     '**/README.md',
     '.reviewboardrc',
     'system_test/**',
+    'tests/**',
+    'vagrant/**',
   ])
 }
 

From d18db7bcab68c84b325d5cdb517c007118884893 Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Wed, 8 Jul 2015 15:41:44 -0700
Subject: [PATCH 39/41] fixed :rat errors (needed to add licenses)

---
 build.gradle                                     |  2 --
 tests/kafkatest/__init__.py                      | 15 +++++++++++++++
 tests/kafkatest/services/__init__.py             | 15 +++++++++++++++
 .../templates/console_consumer.properties        | 16 ++++++++++++++++
 .../services/templates/zookeeper.properties      | 16 ++++++++++++++++
 tests/kafkatest/tests/__init__.py                | 15 +++++++++++++++
 tests/setup.py                                   | 16 ++++++++++++++++
 vagrant/aws/aws-access-keys-commands             | 16 ++++++++++++++++
 vagrant/aws/aws-example-Vagrantfile.local        | 16 ++++++++++++++++
 vagrant/aws/aws-init.sh                          | 16 ++++++++++++++++
 vagrant/system-test-Vagrantfile.local            | 16 ++++++++++++++++
 11 files changed, 157 insertions(+), 2 deletions(-)

diff --git a/build.gradle b/build.gradle
index 580c686547340..4fd277e874faa 100644
--- a/build.gradle
+++ b/build.gradle
@@ -65,8 +65,6 @@ rat {
     '**/README.md',
     '.reviewboardrc',
     'system_test/**',
-    'tests/**',
-    'vagrant/**',
   ])
 }
 
diff --git a/tests/kafkatest/__init__.py b/tests/kafkatest/__init__.py
index 8b137891791fe..28d269bc4de87 100644
--- a/tests/kafkatest/__init__.py
+++ b/tests/kafkatest/__init__.py
@@ -1 +1,16 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+# see kafka.server.KafkaConfig for additional details and defaults
 
diff --git a/tests/kafkatest/services/__init__.py b/tests/kafkatest/services/__init__.py
index e69de29bb2d1d..ebc9bb3a9a03f 100644
--- a/tests/kafkatest/services/__init__.py
+++ b/tests/kafkatest/services/__init__.py
@@ -0,0 +1,15 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+# see kafka.server.KafkaConfig for additional details and defaults
diff --git a/tests/kafkatest/services/templates/console_consumer.properties b/tests/kafkatest/services/templates/console_consumer.properties
index 9179a199fd496..63782fc2e951e 100644
--- a/tests/kafkatest/services/templates/console_consumer.properties
+++ b/tests/kafkatest/services/templates/console_consumer.properties
@@ -1,3 +1,19 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+# see kafka.server.KafkaConfig for additional details and defaults
+
 {% if consumer_timeout_ms is defined %}
 consumer.timeout.ms={{ consumer_timeout_ms }}
 {% endif %}
\ No newline at end of file
diff --git a/tests/kafkatest/services/templates/zookeeper.properties b/tests/kafkatest/services/templates/zookeeper.properties
index 740c30aa32c49..e66c53f728875 100644
--- a/tests/kafkatest/services/templates/zookeeper.properties
+++ b/tests/kafkatest/services/templates/zookeeper.properties
@@ -1,3 +1,19 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+# see kafka.server.KafkaConfig for additional details and defaults
+
 dataDir=/mnt/zookeeper
 clientPort=2181
 maxClientCnxns=0
diff --git a/tests/kafkatest/tests/__init__.py b/tests/kafkatest/tests/__init__.py
index e69de29bb2d1d..ebc9bb3a9a03f 100644
--- a/tests/kafkatest/tests/__init__.py
+++ b/tests/kafkatest/tests/__init__.py
@@ -0,0 +1,15 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+# see kafka.server.KafkaConfig for additional details and defaults
diff --git a/tests/setup.py b/tests/setup.py
index caf94883d09fe..5ce4bb797aae5 100644
--- a/tests/setup.py
+++ b/tests/setup.py
@@ -1,3 +1,19 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+# see kafka.server.KafkaConfig for additional details and defaults
+
 from setuptools import find_packages, setup
 
 setup(name="kafkatest",
diff --git a/vagrant/aws/aws-access-keys-commands b/vagrant/aws/aws-access-keys-commands
index 4607ec00ff747..9c923f86f3f8f 100644
--- a/vagrant/aws/aws-access-keys-commands
+++ b/vagrant/aws/aws-access-keys-commands
@@ -1,3 +1,19 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+# see kafka.server.KafkaConfig for additional details and defaults
+
 if [ -z "$AWS_IAM" ];then
     echo "Warning: AWS_IAM is not set"
 fi
diff --git a/vagrant/aws/aws-example-Vagrantfile.local b/vagrant/aws/aws-example-Vagrantfile.local
index e46acc48df21a..c3b075b9420d4 100644
--- a/vagrant/aws/aws-example-Vagrantfile.local
+++ b/vagrant/aws/aws-example-Vagrantfile.local
@@ -1,3 +1,19 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+# see kafka.server.KafkaConfig for additional details and defaults
+
 # Use this template Vagrantfile.local for running system tests on aws 
 # To use it, move it to the base kafka directory and rename
 # it to Vagrantfile.local, and adjust variables as needed.
diff --git a/vagrant/aws/aws-init.sh b/vagrant/aws/aws-init.sh
index 45fe7f40ffbb4..61519280db3e2 100755
--- a/vagrant/aws/aws-init.sh
+++ b/vagrant/aws/aws-init.sh
@@ -1,3 +1,19 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+# see kafka.server.KafkaConfig for additional details and defaults
+
 #!/bin/bash
 
 # This script can be used to set up a driver machine on aws from which you will run tests
diff --git a/vagrant/system-test-Vagrantfile.local b/vagrant/system-test-Vagrantfile.local
index ea4fd93a2e9a8..7f280a4653c6b 100644
--- a/vagrant/system-test-Vagrantfile.local
+++ b/vagrant/system-test-Vagrantfile.local
@@ -1,3 +1,19 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+# see kafka.server.KafkaConfig for additional details and defaults
+
 # Use this example Vagrantfile.local for running system tests
 # To use it, move it to the base kafka directory and rename
 # it to Vagrantfile.local

From 47b7b64e0fa1ec7e1227a4def8eb01fe9be4202d Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Mon, 20 Jul 2015 16:33:14 -0700
Subject: [PATCH 40/41] Created separate tools jar so that the clients package
 does not pull in dependencies on the Jackson JSON tools or argparse4j.

---
 bin/kafka-run-class.sh                        | 10 +++
 build.gradle                                  | 62 ++++++++++++--
 checkstyle/import-control.xml                 |  2 +-
 settings.gradle                               |  3 +-
 .../clients/tools/ProducerPerformance.java    |  0
 .../clients/tools/ThroughputThrottler.java    |  0
 .../clients/tools/VerifiableProducer.java     | 80 ++++++++++++-------
 7 files changed, 117 insertions(+), 40 deletions(-)
 rename {clients => tools}/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java (100%)
 rename {clients => tools}/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java (100%)
 rename {clients => tools}/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java (83%)

diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh
index 8c3fa28614534..ebe74092a23dd 100755
--- a/bin/kafka-run-class.sh
+++ b/bin/kafka-run-class.sh
@@ -65,6 +65,16 @@ do
   CLASSPATH=$CLASSPATH:$file
 done
 
+for file in $base_dir/tools/build/libs/kafka-tools*.jar;
+do
+  CLASSPATH=$CLASSPATH:$file
+done
+
+for file in $base_dir/tools/build/dependant-libs-${SCALA_VERSION}*/*.jar;
+do
+  CLASSPATH=$CLASSPATH:$file
+done
+
 # classpath addition for release
 for file in $base_dir/libs/*.jar;
 do
diff --git a/build.gradle b/build.gradle
index 4fd277e874faa..6a8fc9295ddc2 100644
--- a/build.gradle
+++ b/build.gradle
@@ -203,20 +203,20 @@ for ( sv in ['2_9_1', '2_9_2', '2_10_5', '2_11_6'] ) {
   }
 }
 
-tasks.create(name: "jarAll", dependsOn: ['jar_core_2_9_1', 'jar_core_2_9_2', 'jar_core_2_10_5', 'jar_core_2_11_6', 'clients:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar', 'log4j-appender:jar']) {
+tasks.create(name: "jarAll", dependsOn: ['jar_core_2_9_1', 'jar_core_2_9_2', 'jar_core_2_10_5', 'jar_core_2_11_6', 'clients:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar', 'log4j-appender:jar', 'tools:jar']) {
 }
 
-tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_9_1', 'srcJar_2_9_2', 'srcJar_2_10_5', 'srcJar_2_11_6', 'clients:srcJar', 'examples:srcJar', 'contrib:hadoop-consumer:srcJar', 'contrib:hadoop-producer:srcJar', 'log4j-appender:srcJar']) { }
+tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_9_1', 'srcJar_2_9_2', 'srcJar_2_10_5', 'srcJar_2_11_6', 'clients:srcJar', 'examples:srcJar', 'contrib:hadoop-consumer:srcJar', 'contrib:hadoop-producer:srcJar', 'log4j-appender:srcJar', 'tools:srcJar']) { }
 
-tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_9_1', 'docsJar_2_9_2', 'docsJar_2_10_5', 'docsJar_2_11_6', 'clients:docsJar', 'examples:docsJar', 'contrib:hadoop-consumer:docsJar', 'contrib:hadoop-producer:docsJar', 'log4j-appender:docsJar']) { }
+tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_9_1', 'docsJar_2_9_2', 'docsJar_2_10_5', 'docsJar_2_11_6', 'clients:docsJar', 'examples:docsJar', 'contrib:hadoop-consumer:docsJar', 'contrib:hadoop-producer:docsJar', 'log4j-appender:docsJar', 'tools:docsJar']) { }
 
-tasks.create(name: "testAll", dependsOn: ['test_core_2_9_1', 'test_core_2_9_2', 'test_core_2_10_5', 'test_core_2_11_6', 'clients:test', 'log4j-appender:test']) {
+tasks.create(name: "testAll", dependsOn: ['test_core_2_9_1', 'test_core_2_9_2', 'test_core_2_10_5', 'test_core_2_11_6', 'clients:test', 'log4j-appender:test', 'tools:test']) {
 }
 
 tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_9_1', 'releaseTarGz_2_9_2', 'releaseTarGz_2_10_5', 'releaseTarGz_2_11_6']) {
 }
 
-tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_9_1', 'uploadCoreArchives_2_9_2', 'uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_6', 'clients:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives', 'log4j-appender:uploadArchives']) {
+tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_9_1', 'uploadCoreArchives_2_9_2', 'uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_6', 'clients:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives', 'log4j-appender:uploadArchives', 'tools:uploadArchives']) {
 }
 
 project(':core') {
@@ -378,8 +378,6 @@ project(':clients') {
     compile "org.slf4j:slf4j-api:1.7.6"
     compile 'org.xerial.snappy:snappy-java:1.1.1.7'
     compile 'net.jpountz.lz4:lz4:1.2.0'
-    compile 'net.sourceforge.argparse4j:argparse4j:0.5.0'
-    compile 'com.googlecode.json-simple:json-simple:1.1.1'
 
     testCompile 'junit:junit:4.6'
     testRuntime "$slf4jlog4j"
@@ -419,6 +417,56 @@ project(':clients') {
   test.dependsOn('checkstyleMain', 'checkstyleTest')
 }
 
+project(':tools') {
+    apply plugin: 'checkstyle'
+    archivesBaseName = "kafka-tools"
+
+    dependencies {
+        compile project(':clients')
+        compile 'net.sourceforge.argparse4j:argparse4j:0.5.0'
+        compile 'com.fasterxml.jackson.core:jackson-databind:2.5.4'
+        compile "$slf4jlog4j"
+
+        testCompile 'junit:junit:4.6'
+        testCompile project(path: ':clients', configuration: 'archives')
+    }
+
+    task testJar(type: Jar) {
+        classifier = 'test'
+        from sourceSets.test.output
+    }
+
+    test {
+        testLogging {
+            events "passed", "skipped", "failed"
+            exceptionFormat = 'full'
+        }
+    }
+
+    javadoc {
+        include "**/org/apache/kafka/tools/*"
+    }
+
+    tasks.create(name: "copyDependantLibs", type: Copy) {
+        from (configurations.testRuntime) {
+            include('slf4j-log4j12*')
+        }
+        from (configurations.runtime) {
+            exclude('kafka-clients*')
+        }
+        into "$buildDir/dependant-libs-${scalaVersion}"
+    }
+
+    jar {
+        dependsOn 'copyDependantLibs'
+    }
+
+    checkstyle {
+        configFile = new File(rootDir, "checkstyle/checkstyle.xml")
+    }
+    test.dependsOn('checkstyleMain', 'checkstyleTest')
+}
+
 project(':log4j-appender') {
   apply plugin: 'checkstyle'
   archivesBaseName = "kafka-log4j-appender"
diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index 90d84c2b00dcf..a562eef2d0e93 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -92,7 +92,7 @@
 		
 			
 			
-            
+            
             
 		
 	
diff --git a/settings.gradle b/settings.gradle
index 3b6a952cff2b3..1944917408792 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -14,4 +14,5 @@
 // limitations under the License.
 
 apply from: file('scala.gradle')
-include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients', 'log4j-appender'
+include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients', 'tools', 'log4j-appender'
+
diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java b/tools/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java
similarity index 100%
rename from clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java
rename to tools/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java
diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java b/tools/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java
similarity index 100%
rename from clients/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java
rename to tools/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java
diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java b/tools/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java
similarity index 83%
rename from clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java
rename to tools/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java
index 7a174289aacba..204166a25e740 100644
--- a/clients/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java
+++ b/tools/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java
@@ -24,9 +24,12 @@
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.clients.producer.RecordMetadata;
 
-import org.json.simple.JSONObject;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.Properties;
 
 import static net.sourceforge.argparse4j.impl.Arguments.store;
@@ -190,34 +193,47 @@ public void close() {
      */
     String errorString(Exception e, String key, String value, Long nowMs) {
         assert e != null : "Expected non-null exception.";
-    
-        JSONObject obj = new JSONObject();
-        obj.put("class", this.getClass().toString());
-        obj.put("name", "producer_send_error");
+
+        Map errorData = new HashMap<>();
+        errorData.put("class", this.getClass().toString());
+        errorData.put("name", "producer_send_error");
+
+        errorData.put("time_ms", nowMs);
+        errorData.put("exception", e.getClass().toString());
+        errorData.put("message", e.getMessage());
+        errorData.put("topic", this.topic);
+        errorData.put("key", key);
+        errorData.put("value", value);
         
-        obj.put("time_ms", nowMs);
-        obj.put("exception", e.getClass().toString());
-        obj.put("message", e.getMessage());
-        obj.put("topic", this.topic);
-        obj.put("key", key);
-        obj.put("value", value);
-        return obj.toJSONString();
+        return toJsonString(errorData);
     }
   
     String successString(RecordMetadata recordMetadata, String key, String value, Long nowMs) {
         assert recordMetadata != null : "Expected non-null recordMetadata object.";
-    
-        JSONObject obj = new JSONObject();
-        obj.put("class", this.getClass().toString());
-        obj.put("name", "producer_send_success");
+
+        Map successData = new HashMap<>();
+        successData.put("class", this.getClass().toString());
+        successData.put("name", "producer_send_success");
+
+        successData.put("time_ms", nowMs);
+        successData.put("topic", this.topic);
+        successData.put("partition", recordMetadata.partition());
+        successData.put("offset", recordMetadata.offset());
+        successData.put("key", key);
+        successData.put("value", value);
         
-        obj.put("time_ms", nowMs);
-        obj.put("topic", this.topic);
-        obj.put("partition", recordMetadata.partition());
-        obj.put("offset", recordMetadata.offset());
-        obj.put("key", key);
-        obj.put("value", value);
-        return obj.toJSONString();
+        return toJsonString(successData);
+    }
+    
+    private String toJsonString(Map data) {
+        String json;
+        try {
+            ObjectMapper mapper = new ObjectMapper();
+            json = mapper.writeValueAsString(data);
+        } catch(JsonProcessingException e) {
+            json = "Bad data can't be written as json: " + e.getMessage();
+        }
+        return json;
     }
   
     /** Callback which prints errors to stdout when the producer fails to send. */
@@ -261,14 +277,16 @@ public void run() {
                 // Print a summary
                 long stopMs = System.currentTimeMillis();
                 double avgThroughput = 1000 * ((producer.numAcked) / (double) (stopMs - startMs));
-                JSONObject obj = new JSONObject();
-                obj.put("class", producer.getClass().toString());
-                obj.put("name", "tool_data");
-                obj.put("sent", producer.numSent);
-                obj.put("acked", producer.numAcked);
-                obj.put("target_throughput", producer.throughput);
-                obj.put("avg_throughput", avgThroughput);
-                System.out.println(obj.toJSONString());
+                
+                Map data = new HashMap<>();
+                data.put("class", producer.getClass().toString());
+                data.put("name", "tool_data");
+                data.put("sent", producer.numSent);
+                data.put("acked", producer.numAcked);
+                data.put("target_throughput", producer.throughput);
+                data.put("avg_throughput", avgThroughput);
+                
+                System.out.println(producer.toJsonString(data));
             }
         });
 

From a62fb6c6458ef69b2f820342e77584d367f86e0d Mon Sep 17 00:00:00 2001
From: Geoff Anderson 
Date: Mon, 27 Jul 2015 15:19:35 -0700
Subject: [PATCH 41/41] fixed checkstyle errors

---
 checkstyle/import-control.xml                                   | 2 +-
 .../java/org/apache/kafka/clients/tools/VerifiableProducer.java | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index a562eef2d0e93..18be1bb1421f9 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -92,7 +92,7 @@
 		
 			
 			
-            
+            
             
 		
 	
diff --git a/tools/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java b/tools/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java
index 204166a25e740..b04876f8fc7d5 100644
--- a/tools/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java
+++ b/tools/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java
@@ -230,7 +230,7 @@ private String toJsonString(Map data) {
         try {
             ObjectMapper mapper = new ObjectMapper();
             json = mapper.writeValueAsString(data);
-        } catch(JsonProcessingException e) {
+        } catch (JsonProcessingException e) {
             json = "Bad data can't be written as json: " + e.getMessage();
         }
         return json;