Add km module kafka

This commit is contained in:
leewei
2023-02-14 16:27:47 +08:00
parent 229140f067
commit 0b8160a714
4039 changed files with 718112 additions and 46204 deletions

View File

@@ -0,0 +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.
from performance import PerformanceService, throughput, latency, compute_aggregate_throughput
from end_to_end_latency import EndToEndLatencyService
from producer_performance import ProducerPerformanceService
from consumer_performance import ConsumerPerformanceService

View File

@@ -0,0 +1,187 @@
# 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.
import os
from kafkatest.services.performance import PerformanceService
from kafkatest.services.security.security_config import SecurityConfig
from kafkatest.version import DEV_BRANCH, V_0_9_0_0, V_2_0_0, LATEST_0_10_0
class ConsumerPerformanceService(PerformanceService):
"""
See ConsumerPerformance.scala as the source of truth on these settings, but for reference:
"zookeeper" "The connection string for the zookeeper connection in the form host:port. Multiple URLS can
be given to allow fail-over. This option is only used with the old consumer."
"broker-list", "A broker list to use for connecting if using the new consumer."
"topic", "REQUIRED: The topic to consume from."
"group", "The group id to consume on."
"fetch-size", "The amount of data to fetch in a single request."
"from-latest", "If the consumer does not already have an establishedoffset to consume from,
start with the latest message present in the log rather than the earliest message."
"socket-buffer-size", "The size of the tcp RECV size."
"threads", "Number of processing threads."
"num-fetch-threads", "Number of fetcher threads. Defaults to 1"
"new-consumer", "Use the new consumer implementation."
"consumer.config", "Consumer config properties file."
"""
# Root directory for persistent output
PERSISTENT_ROOT = "/mnt/consumer_performance"
LOG_DIR = os.path.join(PERSISTENT_ROOT, "logs")
STDOUT_CAPTURE = os.path.join(PERSISTENT_ROOT, "consumer_performance.stdout")
STDERR_CAPTURE = os.path.join(PERSISTENT_ROOT, "consumer_performance.stderr")
LOG_FILE = os.path.join(LOG_DIR, "consumer_performance.log")
LOG4J_CONFIG = os.path.join(PERSISTENT_ROOT, "tools-log4j.properties")
CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "consumer.properties")
logs = {
"consumer_performance_output": {
"path": STDOUT_CAPTURE,
"collect_default": True},
"consumer_performance_stderr": {
"path": STDERR_CAPTURE,
"collect_default": True},
"consumer_performance_log": {
"path": LOG_FILE,
"collect_default": True}
}
def __init__(self, context, num_nodes, kafka, topic, messages, version=DEV_BRANCH, new_consumer=True, settings={}):
super(ConsumerPerformanceService, self).__init__(context, num_nodes)
self.kafka = kafka
self.security_config = kafka.security_config.client_config()
self.topic = topic
self.messages = messages
self.new_consumer = new_consumer
self.settings = settings
assert version >= V_0_9_0_0 or (not new_consumer), \
"new_consumer is only supported if version >= 0.9.0.0, version %s" % str(version)
assert version < V_2_0_0 or new_consumer, \
"new_consumer==false is only supported if version < 2.0.0, version %s" % str(version)
security_protocol = self.security_config.security_protocol
assert version >= V_0_9_0_0 or security_protocol == SecurityConfig.PLAINTEXT, \
"Security protocol %s is only supported if version >= 0.9.0.0, version %s" % (self.security_config, str(version))
# These less-frequently used settings can be updated manually after instantiation
self.fetch_size = None
self.socket_buffer_size = None
self.threads = None
self.num_fetch_threads = None
self.group = None
self.from_latest = None
for node in self.nodes:
node.version = version
def args(self, version):
"""Dictionary of arguments used to start the Consumer Performance script."""
args = {
'topic': self.topic,
'messages': self.messages,
}
if self.new_consumer:
if version <= LATEST_0_10_0:
args['new-consumer'] = ""
args['broker-list'] = self.kafka.bootstrap_servers(self.security_config.security_protocol)
else:
args['zookeeper'] = self.kafka.zk_connect_setting()
if self.fetch_size is not None:
args['fetch-size'] = self.fetch_size
if self.socket_buffer_size is not None:
args['socket-buffer-size'] = self.socket_buffer_size
if self.threads is not None:
args['threads'] = self.threads
if self.num_fetch_threads is not None:
args['num-fetch-threads'] = self.num_fetch_threads
if self.group is not None:
args['group'] = self.group
if self.from_latest:
args['from-latest'] = ""
return args
def start_cmd(self, node):
cmd = "export LOG_DIR=%s;" % ConsumerPerformanceService.LOG_DIR
cmd += " export KAFKA_OPTS=%s;" % self.security_config.kafka_opts
cmd += " export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\";" % ConsumerPerformanceService.LOG4J_CONFIG
cmd += " %s" % self.path.script("kafka-consumer-perf-test.sh", node)
for key, value in self.args(node.version).items():
cmd += " --%s %s" % (key, value)
if node.version >= V_0_9_0_0:
# This is only used for security settings
cmd += " --consumer.config %s" % ConsumerPerformanceService.CONFIG_FILE
for key, value in self.settings.items():
cmd += " %s=%s" % (str(key), str(value))
cmd += " 2>> %(stderr)s | tee -a %(stdout)s" % {'stdout': ConsumerPerformanceService.STDOUT_CAPTURE,
'stderr': ConsumerPerformanceService.STDERR_CAPTURE}
return cmd
def parse_results(self, line, version):
parts = line.split(',')
if version >= V_0_9_0_0:
result = {
'total_mb': float(parts[2]),
'mbps': float(parts[3]),
'records_per_sec': float(parts[5]),
}
else:
result = {
'total_mb': float(parts[3]),
'mbps': float(parts[4]),
'records_per_sec': float(parts[6]),
}
return result
def _worker(self, idx, node):
node.account.ssh("mkdir -p %s" % ConsumerPerformanceService.PERSISTENT_ROOT, allow_fail=False)
log_config = self.render('tools_log4j.properties', log_file=ConsumerPerformanceService.LOG_FILE)
node.account.create_file(ConsumerPerformanceService.LOG4J_CONFIG, log_config)
node.account.create_file(ConsumerPerformanceService.CONFIG_FILE, str(self.security_config))
self.security_config.setup_node(node)
cmd = self.start_cmd(node)
self.logger.debug("Consumer performance %d command: %s", idx, cmd)
last = None
for line in node.account.ssh_capture(cmd):
last = line
# Parse and save the last line's information
self.results[idx-1] = self.parse_results(last, node.version)

View File

@@ -0,0 +1,124 @@
# 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.
import os
from kafkatest.services.performance import PerformanceService
from kafkatest.services.security.security_config import SecurityConfig
from kafkatest.version import DEV_BRANCH, V_0_9_0_0
class EndToEndLatencyService(PerformanceService):
MESSAGE_BYTES = 21 # 0.8.X messages are fixed at 21 bytes, so we'll match that for other versions
# Root directory for persistent output
PERSISTENT_ROOT = "/mnt/end_to_end_latency"
LOG_DIR = os.path.join(PERSISTENT_ROOT, "logs")
STDOUT_CAPTURE = os.path.join(PERSISTENT_ROOT, "end_to_end_latency.stdout")
STDERR_CAPTURE = os.path.join(PERSISTENT_ROOT, "end_to_end_latency.stderr")
LOG_FILE = os.path.join(LOG_DIR, "end_to_end_latency.log")
LOG4J_CONFIG = os.path.join(PERSISTENT_ROOT, "tools-log4j.properties")
CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "client.properties")
logs = {
"end_to_end_latency_output": {
"path": STDOUT_CAPTURE,
"collect_default": True},
"end_to_end_latency_stderr": {
"path": STDERR_CAPTURE,
"collect_default": True},
"end_to_end_latency_log": {
"path": LOG_FILE,
"collect_default": True}
}
def __init__(self, context, num_nodes, kafka, topic, num_records, compression_type="none", version=DEV_BRANCH, acks=1):
super(EndToEndLatencyService, self).__init__(context, num_nodes,
root=EndToEndLatencyService.PERSISTENT_ROOT)
self.kafka = kafka
self.security_config = kafka.security_config.client_config()
security_protocol = self.security_config.security_protocol
if version < V_0_9_0_0:
assert security_protocol == SecurityConfig.PLAINTEXT, \
"Security protocol %s is only supported if version >= 0.9.0.0, version %s" % (self.security_config, str(version))
assert compression_type == "none", \
"Compression type %s is only supported if version >= 0.9.0.0, version %s" % (compression_type, str(version))
self.args = {
'topic': topic,
'num_records': num_records,
'acks': acks,
'compression_type': compression_type,
'kafka_opts': self.security_config.kafka_opts,
'message_bytes': EndToEndLatencyService.MESSAGE_BYTES
}
for node in self.nodes:
node.version = version
def start_cmd(self, node):
args = self.args.copy()
args.update({
'zk_connect': self.kafka.zk_connect_setting(),
'bootstrap_servers': self.kafka.bootstrap_servers(self.security_config.security_protocol),
'config_file': EndToEndLatencyService.CONFIG_FILE,
'kafka_run_class': self.path.script("kafka-run-class.sh", node),
'java_class_name': self.java_class_name()
})
cmd = "export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\"; " % EndToEndLatencyService.LOG4J_CONFIG
if node.version >= V_0_9_0_0:
cmd += "KAFKA_OPTS=%(kafka_opts)s %(kafka_run_class)s %(java_class_name)s " % args
cmd += "%(bootstrap_servers)s %(topic)s %(num_records)d %(acks)d %(message_bytes)d %(config_file)s" % args
else:
# Set fetch max wait to 0 to match behavior in later versions
cmd += "KAFKA_OPTS=%(kafka_opts)s %(kafka_run_class)s kafka.tools.TestEndToEndLatency " % args
cmd += "%(bootstrap_servers)s %(zk_connect)s %(topic)s %(num_records)d 0 %(acks)d" % args
cmd += " 2>> %(stderr)s | tee -a %(stdout)s" % {'stdout': EndToEndLatencyService.STDOUT_CAPTURE,
'stderr': EndToEndLatencyService.STDERR_CAPTURE}
return cmd
def _worker(self, idx, node):
node.account.ssh("mkdir -p %s" % EndToEndLatencyService.PERSISTENT_ROOT, allow_fail=False)
log_config = self.render('tools_log4j.properties', log_file=EndToEndLatencyService.LOG_FILE)
node.account.create_file(EndToEndLatencyService.LOG4J_CONFIG, log_config)
client_config = str(self.security_config)
if node.version >= V_0_9_0_0:
client_config += "compression_type=%(compression_type)s" % self.args
node.account.create_file(EndToEndLatencyService.CONFIG_FILE, client_config)
self.security_config.setup_node(node)
cmd = self.start_cmd(node)
self.logger.debug("End-to-end latency %d command: %s", idx, cmd)
results = {}
for line in node.account.ssh_capture(cmd):
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 java_class_name(self):
return "kafka.tools.EndToEndLatency"

View File

@@ -0,0 +1,72 @@
# 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.
from ducktape.services.background_thread import BackgroundThreadService
from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin
class PerformanceService(KafkaPathResolverMixin, BackgroundThreadService):
def __init__(self, context=None, num_nodes=0, root="/mnt/*", stop_timeout_sec=30):
super(PerformanceService, self).__init__(context, num_nodes)
self.results = [None] * self.num_nodes
self.stats = [[] for x in range(self.num_nodes)]
self.stop_timeout_sec = stop_timeout_sec
self.root = root
def java_class_name(self):
"""
Returns the name of the Java class which this service creates. Subclasses should override
this method, so that we know the name of the java process to stop. If it is not
overridden, we will kill all java processes in PerformanceService#stop_node (for backwards
compatibility.)
"""
return ""
def stop_node(self, node):
node.account.kill_java_processes(self.java_class_name(), clean_shutdown=True, allow_fail=True)
stopped = self.wait_node(node, timeout_sec=self.stop_timeout_sec)
assert stopped, "Node %s: did not stop within the specified timeout of %s seconds" % \
(str(node.account), str(self.stop_timeout_sec))
def clean_node(self, node):
node.account.kill_java_processes(self.java_class_name(), clean_shutdown=False, allow_fail=True)
node.account.ssh("rm -rf -- %s" % self.root, allow_fail=False)
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_aggregate_throughput(perf):
"""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['mbps'] for r in perf.results])
return throughput(aggregate_rate, aggregate_mbps)

View File

@@ -0,0 +1,174 @@
# 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.
import os
import time
from ducktape.utils.util import wait_until
from ducktape.cluster.remoteaccount import RemoteCommandError
from kafkatest.directory_layout.kafka_path import TOOLS_JAR_NAME, TOOLS_DEPENDANT_TEST_LIBS_JAR_NAME
from kafkatest.services.monitor.http import HttpMetricsCollector
from kafkatest.services.performance import PerformanceService
from kafkatest.services.security.security_config import SecurityConfig
from kafkatest.version import DEV_BRANCH, V_0_9_0_0
class ProducerPerformanceService(HttpMetricsCollector, PerformanceService):
PERSISTENT_ROOT = "/mnt/producer_performance"
STDOUT_CAPTURE = os.path.join(PERSISTENT_ROOT, "producer_performance.stdout")
STDERR_CAPTURE = os.path.join(PERSISTENT_ROOT, "producer_performance.stderr")
LOG_DIR = os.path.join(PERSISTENT_ROOT, "logs")
LOG_FILE = os.path.join(LOG_DIR, "producer_performance.log")
LOG4J_CONFIG = os.path.join(PERSISTENT_ROOT, "tools-log4j.properties")
def __init__(self, context, num_nodes, kafka, topic, num_records, record_size, throughput, version=DEV_BRANCH, settings=None,
intermediate_stats=False, client_id="producer-performance"):
super(ProducerPerformanceService, self).__init__(context=context, num_nodes=num_nodes)
self.logs = {
"producer_performance_stdout": {
"path": ProducerPerformanceService.STDOUT_CAPTURE,
"collect_default": True},
"producer_performance_stderr": {
"path": ProducerPerformanceService.STDERR_CAPTURE,
"collect_default": True},
"producer_performance_log": {
"path": ProducerPerformanceService.LOG_FILE,
"collect_default": True}
}
self.kafka = kafka
self.security_config = kafka.security_config.client_config()
security_protocol = self.security_config.security_protocol
assert version >= V_0_9_0_0 or security_protocol == SecurityConfig.PLAINTEXT, \
"Security protocol %s is only supported if version >= 0.9.0.0, version %s" % (self.security_config, str(version))
self.args = {
'topic': topic,
'kafka_opts': self.security_config.kafka_opts,
'num_records': num_records,
'record_size': record_size,
'throughput': throughput
}
self.settings = settings or {}
self.intermediate_stats = intermediate_stats
self.client_id = client_id
for node in self.nodes:
node.version = version
def start_cmd(self, node):
args = self.args.copy()
args.update({
'bootstrap_servers': self.kafka.bootstrap_servers(self.security_config.security_protocol),
'client_id': self.client_id,
'kafka_run_class': self.path.script("kafka-run-class.sh", node),
'metrics_props': ' '.join(["%s=%s" % (k, v) for k, v in self.http_metrics_client_configs.iteritems()])
})
cmd = ""
if node.version < DEV_BRANCH:
# In order to ensure more consistent configuration between versions, always use the ProducerPerformance
# tool from the development branch
tools_jar = self.path.jar(TOOLS_JAR_NAME, DEV_BRANCH)
tools_dependant_libs_jar = self.path.jar(TOOLS_DEPENDANT_TEST_LIBS_JAR_NAME, DEV_BRANCH)
for jar in (tools_jar, tools_dependant_libs_jar):
cmd += "for file in %s; do CLASSPATH=$CLASSPATH:$file; done; " % jar
cmd += "export CLASSPATH; "
cmd += " export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\"; " % ProducerPerformanceService.LOG4J_CONFIG
cmd += "KAFKA_OPTS=%(kafka_opts)s KAFKA_HEAP_OPTS=\"-XX:+HeapDumpOnOutOfMemoryError\" %(kafka_run_class)s org.apache.kafka.tools.ProducerPerformance " \
"--topic %(topic)s --num-records %(num_records)d --record-size %(record_size)d --throughput %(throughput)d --producer-props bootstrap.servers=%(bootstrap_servers)s client.id=%(client_id)s %(metrics_props)s" % args
self.security_config.setup_node(node)
if self.security_config.security_protocol != SecurityConfig.PLAINTEXT:
self.settings.update(self.security_config.properties)
for key, value in self.settings.items():
cmd += " %s=%s" % (str(key), str(value))
cmd += " 2>>%s | tee %s" % (ProducerPerformanceService.STDERR_CAPTURE, ProducerPerformanceService.STDOUT_CAPTURE)
return cmd
def pids(self, node):
try:
cmd = "jps | grep -i ProducerPerformance | awk '{print $1}'"
pid_arr = [pid for pid in node.account.ssh_capture(cmd, allow_fail=True, callback=int)]
return pid_arr
except (RemoteCommandError, ValueError) as e:
return []
def alive(self, node):
return len(self.pids(node)) > 0
def _worker(self, idx, node):
node.account.ssh("mkdir -p %s" % ProducerPerformanceService.PERSISTENT_ROOT, allow_fail=False)
# Create and upload log properties
log_config = self.render('tools_log4j.properties', log_file=ProducerPerformanceService.LOG_FILE)
node.account.create_file(ProducerPerformanceService.LOG4J_CONFIG, log_config)
cmd = self.start_cmd(node)
self.logger.debug("Producer performance %d command: %s", idx, cmd)
# start ProducerPerformance process
start = time.time()
producer_output = node.account.ssh_capture(cmd)
wait_until(lambda: self.alive(node), timeout_sec=20, err_msg="ProducerPerformance failed to start")
# block until there is at least one line of output
first_line = next(producer_output, None)
if first_line is None:
raise Exception("No output from ProducerPerformance")
wait_until(lambda: not self.alive(node), timeout_sec=1200, backoff_sec=2, err_msg="ProducerPerformance failed to finish")
elapsed = time.time() - start
self.logger.debug("ProducerPerformance process ran for %s seconds" % elapsed)
# parse producer output from file
last = None
producer_output = node.account.ssh_capture("cat %s" % ProducerPerformanceService.STDOUT_CAPTURE)
for line in producer_output:
if self.intermediate_stats:
try:
self.stats[idx-1].append(self.parse_stats(line))
except:
# Sometimes there are extraneous log messages
pass
last = line
try:
self.results[idx-1] = self.parse_stats(last)
except:
raise Exception("Unable to parse aggregate performance statistics on node %d: %s" % (idx, last))
def parse_stats(self, 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]),
}

View File

@@ -0,0 +1,108 @@
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
from kafkatest.services.monitor.jmx import JmxMixin
from kafkatest.services.streams import StreamsTestBaseService
from kafkatest.services.kafka import KafkaConfig
from kafkatest.services import streams_property
#
# Class used to start the simple Kafka Streams benchmark
#
class StreamsSimpleBenchmarkService(StreamsTestBaseService):
"""Base class for simple Kafka Streams benchmark"""
def __init__(self, test_context, kafka, test_name, num_threads, num_recs_or_wait_ms, key_skew, value_size):
super(StreamsSimpleBenchmarkService, self).__init__(test_context,
kafka,
"org.apache.kafka.streams.perf.SimpleBenchmark",
test_name,
num_recs_or_wait_ms,
key_skew,
value_size)
self.jmx_option = ""
if test_name.startswith('stream') or test_name.startswith('table'):
self.jmx_option = "stream-jmx"
JmxMixin.__init__(self,
num_nodes=1,
jmx_object_names=['kafka.streams:type=stream-thread-metrics,thread-id=simple-benchmark-StreamThread-%d' %(i+1) for i in range(num_threads)],
jmx_attributes=['process-latency-avg',
'process-rate',
'commit-latency-avg',
'commit-rate',
'poll-latency-avg',
'poll-rate'],
root=StreamsTestBaseService.PERSISTENT_ROOT)
if test_name.startswith('consume'):
self.jmx_option = "consumer-jmx"
JmxMixin.__init__(self,
num_nodes=1,
jmx_object_names=['kafka.consumer:type=consumer-fetch-manager-metrics,client-id=simple-benchmark-consumer'],
jmx_attributes=['records-consumed-rate'],
root=StreamsTestBaseService.PERSISTENT_ROOT)
self.num_threads = num_threads
def prop_file(self):
cfg = KafkaConfig(**{streams_property.STATE_DIR: self.PERSISTENT_ROOT,
streams_property.KAFKA_SERVERS: self.kafka.bootstrap_servers(),
streams_property.NUM_THREADS: self.num_threads})
return cfg.render()
def start_cmd(self, node):
if self.jmx_option != "":
args = self.args.copy()
args['jmx_port'] = self.jmx_port
args['config_file'] = self.CONFIG_FILE
args['stdout'] = self.STDOUT_FILE
args['stderr'] = self.STDERR_FILE
args['pidfile'] = self.PID_FILE
args['log4j'] = self.LOG4J_CONFIG_FILE
args['kafka_run_class'] = self.path.script("kafka-run-class.sh", node)
cmd = "( export JMX_PORT=%(jmx_port)s; export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%(log4j)s\"; " \
"INCLUDE_TEST_JARS=true %(kafka_run_class)s %(streams_class_name)s " \
" %(config_file)s %(user_test_args1)s %(user_test_args2)s %(user_test_args3)s" \
" %(user_test_args4)s & echo $! >&3 ) 1>> %(stdout)s 2>> %(stderr)s 3> %(pidfile)s" % args
else:
cmd = super(StreamsSimpleBenchmarkService, self).start_cmd(node)
return cmd
def start_node(self, node):
super(StreamsSimpleBenchmarkService, self).start_node(node)
if self.jmx_option != "":
self.start_jmx_tool(1, node)
def clean_node(self, node):
if self.jmx_option != "":
JmxMixin.clean_node(self, node)
super(StreamsSimpleBenchmarkService, self).clean_node(node)
def collect_data(self, node, tag = None):
# Collect the data and return it to the framework
output = node.account.ssh_capture("grep Performance %s" % self.STDOUT_FILE)
data = {}
for line in output:
parts = line.split(':')
data[tag + parts[0]] = parts[1]
return data

View File

@@ -0,0 +1,25 @@
# 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.
# Define the root logger with appender file
log4j.rootLogger = {{ log_level|default("INFO") }}, FILE
log4j.appender.FILE=org.apache.log4j.FileAppender
log4j.appender.FILE.File={{ log_file }}
log4j.appender.FILE.ImmediateFlush=true
# Set the append to false, overwrite
log4j.appender.FILE.Append=false
log4j.appender.FILE.layout=org.apache.log4j.PatternLayout
log4j.appender.FILE.layout.conversionPattern=[%d] %p %m (%c)%n