Add km module kafka

This commit is contained in:
leewei
2023-02-14 14:57:39 +08:00
parent 229140f067
commit 469baad65b
4310 changed files with 736354 additions and 46204 deletions

View File

@@ -0,0 +1,14 @@
# 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.

View File

@@ -0,0 +1,228 @@
# 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 BaseHTTPServer import BaseHTTPRequestHandler, HTTPServer
from collections import defaultdict, namedtuple
import json
from threading import Thread
from select import select
import socket
MetricKey = namedtuple('MetricKey', ['host', 'client_id', 'name', 'group', 'tags'])
MetricValue = namedtuple('MetricValue', ['time', 'value'])
# Python's logging library doesn't define anything more detailed than DEBUG, but we'd like a finer-grained setting for
# for highly detailed messages, e.g. logging every single incoming request.
TRACE = 5
class HttpMetricsCollector(object):
"""
HttpMetricsCollector enables collection of metrics from various Kafka clients instrumented with the
PushHttpMetricsReporter. It starts a web server locally and provides the necessary configuration for clients
to automatically report metrics data to this server. It also provides basic functionality for querying the
recorded metrics. This class can be used either as a mixin or standalone object.
"""
# The port to listen on on the worker node, which will be forwarded to the port listening on this driver node
REMOTE_PORT = 6789
def __init__(self, **kwargs):
"""
Create a new HttpMetricsCollector
:param period the period, in seconds, between updates that the metrics reporter configuration should define.
defaults to reporting once per second
:param args:
:param kwargs:
"""
self._http_metrics_period = kwargs.pop('period', 1)
super(HttpMetricsCollector, self).__init__(**kwargs)
# TODO: currently we maintain just a simple map from all key info -> value. However, some key fields are far
# more common to filter on, so we'd want to index by them, e.g. host, client.id, metric name.
self._http_metrics = defaultdict(list)
self._httpd = HTTPServer(('', 0), _MetricsReceiver)
self._httpd.parent = self
self._httpd.metrics = self._http_metrics
self._http_metrics_thread = Thread(target=self._run_http_metrics_httpd,
name='http-metrics-thread[%s]' % str(self))
self._http_metrics_thread.start()
self._forwarders = {}
@property
def http_metrics_url(self):
"""
:return: the URL to use when reporting metrics
"""
return "http://%s:%d" % ("localhost", self.REMOTE_PORT)
@property
def http_metrics_client_configs(self):
"""
Get client configurations that can be used to report data to this collector. Put these in a properties file for
clients (e.g. console producer or consumer) to have them push metrics to this driver. Note that in some cases
(e.g. streams, connect) these settings may need to be prefixed.
:return: a dictionary of client configurations that will direct a client to report metrics to this collector
"""
return {
"metric.reporters": "org.apache.kafka.tools.PushHttpMetricsReporter",
"metrics.url": self.http_metrics_url,
"metrics.period": self._http_metrics_period,
}
def start_node(self, node):
local_port = self._httpd.socket.getsockname()[1]
self.logger.debug('HttpMetricsCollector listening on %s', local_port)
self._forwarders[self.idx(node)] = _ReverseForwarder(self.logger, node, self.REMOTE_PORT, local_port)
super(HttpMetricsCollector, self).start_node(node)
def stop(self):
super(HttpMetricsCollector, self).stop()
if self._http_metrics_thread:
self.logger.debug("Shutting down metrics httpd")
self._httpd.shutdown()
self._http_metrics_thread.join()
self.logger.debug("Finished shutting down metrics httpd")
def stop_node(self, node):
super(HttpMetricsCollector, self).stop_node(node)
idx = self.idx(node)
self._forwarders[idx].stop()
del self._forwarders[idx]
def metrics(self, host=None, client_id=None, name=None, group=None, tags=None):
"""
Get any collected metrics that match the specified parameters, yielding each as a tuple of
(key, [<timestamp, value>, ...]) values.
"""
for k, values in self._http_metrics.iteritems():
if ((host is None or host == k.host) and
(client_id is None or client_id == k.client_id) and
(name is None or name == k.name) and
(group is None or group == k.group) and
(tags is None or tags == k.tags)):
yield (k, values)
def _run_http_metrics_httpd(self):
self._httpd.serve_forever()
class _MetricsReceiver(BaseHTTPRequestHandler):
"""
HTTP request handler that accepts requests from the PushHttpMetricsReporter and stores them back into the parent
HttpMetricsCollector
"""
def log_message(self, format, *args, **kwargs):
# Don't do any logging here so we get rid of the mostly useless per-request Apache log-style info that spams
# the debug log
pass
def do_POST(self):
data = self.rfile.read(int(self.headers['Content-Length']))
data = json.loads(data)
self.server.parent.logger.log(TRACE, "POST %s\n\n%s\n%s", self.path, self.headers,
json.dumps(data, indent=4, separators=(',', ': ')))
self.send_response(204)
self.end_headers()
client = data['client']
host = client['host']
client_id = client['client_id']
ts = client['time']
metrics = data['metrics']
for raw_metric in metrics:
name = raw_metric['name']
group = raw_metric['group']
# Convert to tuple of pairs because dicts & lists are unhashable
tags = tuple([(k, v) for k, v in raw_metric['tags'].iteritems()]),
value = raw_metric['value']
key = MetricKey(host=host, client_id=client_id, name=name, group=group, tags=tags)
metric_value = MetricValue(time=ts, value=value)
self.server.metrics[key].append(metric_value)
class _ReverseForwarder(object):
"""
Runs reverse forwarding of a port on a node to a local port. This allows you to setup a server on the test driver
that only assumes we have basic SSH access that ducktape guarantees is available for worker nodes.
"""
def __init__(self, logger, node, remote_port, local_port):
self.logger = logger
self._node = node
self._local_port = local_port
self._remote_port = remote_port
self.logger.debug('Forwarding %s port %d to driver port %d', node, remote_port, local_port)
self._stopping = False
self._transport = node.account.ssh_client.get_transport()
self._transport.request_port_forward('', remote_port)
self._accept_thread = Thread(target=self._accept)
self._accept_thread.start()
def stop(self):
self._stopping = True
self._accept_thread.join(30)
if self._accept_thread.isAlive():
raise RuntimeError("Failed to stop reverse forwarder on %s", self._node)
self._transport.cancel_port_forward('', self._remote_port)
def _accept(self):
while not self._stopping:
chan = self._transport.accept(1)
if chan is None:
continue
thr = Thread(target=self._handler, args=(chan,))
thr.setDaemon(True)
thr.start()
def _handler(self, chan):
sock = socket.socket()
try:
sock.connect(("localhost", self._local_port))
except Exception as e:
self.logger.error('Forwarding request to port %d failed: %r', self._local_port, e)
return
self.logger.log(TRACE, 'Connected! Tunnel open %r -> %r -> %d', chan.origin_addr, chan.getpeername(),
self._local_port)
while True:
r, w, x = select([sock, chan], [], [])
if sock in r:
data = sock.recv(1024)
if len(data) == 0:
break
chan.send(data)
if chan in r:
data = chan.recv(1024)
if len(data) == 0:
break
sock.send(data)
chan.close()
sock.close()
self.logger.log(TRACE, 'Tunnel closed from %r', chan.origin_addr)

View File

@@ -0,0 +1,141 @@
# 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 ducktape.cluster.remoteaccount import RemoteCommandError
from ducktape.utils.util import wait_until
from kafkatest.version import get_version, V_0_11_0_0, DEV_BRANCH
class JmxMixin(object):
"""This mixin helps existing service subclasses start JmxTool on their worker nodes and collect jmx stats.
A couple things worth noting:
- this is not a service in its own right.
- we assume the service using JmxMixin also uses KafkaPathResolverMixin
- this uses the --wait option for JmxTool, so the list of object names must be explicit; no patterns are permitted
"""
def __init__(self, num_nodes, jmx_object_names=None, jmx_attributes=None, jmx_poll_ms=1000, root="/mnt"):
self.jmx_object_names = jmx_object_names
self.jmx_attributes = jmx_attributes or []
self.jmx_poll_ms = jmx_poll_ms
self.jmx_port = 9192
self.started = [False] * num_nodes
self.jmx_stats = [{} for x in range(num_nodes)]
self.maximum_jmx_value = {} # map from object_attribute_name to maximum value observed over time
self.average_jmx_value = {} # map from object_attribute_name to average value observed over time
self.jmx_tool_log = os.path.join(root, "jmx_tool.log")
self.jmx_tool_err_log = os.path.join(root, "jmx_tool.err.log")
def clean_node(self, node):
node.account.kill_java_processes(self.jmx_class_name(), clean_shutdown=False,
allow_fail=True)
idx = self.idx(node)
self.started[idx-1] = False
node.account.ssh("rm -f -- %s %s" % (self.jmx_tool_log, self.jmx_tool_err_log), allow_fail=False)
def start_jmx_tool(self, idx, node):
if self.jmx_object_names is None:
self.logger.debug("%s: Not starting jmx tool because no jmx objects are defined" % node.account)
return
if self.started[idx-1]:
self.logger.debug("%s: jmx tool has been started already on this node" % node.account)
return
# JmxTool is not particularly robust to slow-starting processes. In order to ensure JmxTool doesn't fail if the
# process we're trying to monitor takes awhile before listening on the JMX port, wait until we can see that port
# listening before even launching JmxTool
def check_jmx_port_listening():
return 0 == node.account.ssh("nc -z 127.0.0.1 %d" % self.jmx_port, allow_fail=True)
wait_until(check_jmx_port_listening, timeout_sec=30, backoff_sec=.1,
err_msg="%s: Never saw JMX port for %s start listening" % (node.account, self))
# To correctly wait for requested JMX metrics to be added we need the --wait option for JmxTool. This option was
# not added until 0.11.0.1, so any earlier versions need to use JmxTool from a newer version.
use_jmxtool_version = get_version(node)
if use_jmxtool_version <= V_0_11_0_0:
use_jmxtool_version = DEV_BRANCH
cmd = "%s %s " % (self.path.script("kafka-run-class.sh", use_jmxtool_version), self.jmx_class_name())
cmd += "--reporting-interval %d --jmx-url service:jmx:rmi:///jndi/rmi://127.0.0.1:%d/jmxrmi" % (self.jmx_poll_ms, self.jmx_port)
cmd += " --wait"
for jmx_object_name in self.jmx_object_names:
cmd += " --object-name %s" % jmx_object_name
cmd += " --attributes "
for jmx_attribute in self.jmx_attributes:
cmd += "%s," % jmx_attribute
cmd += " 1>> %s" % self.jmx_tool_log
cmd += " 2>> %s &" % self.jmx_tool_err_log
self.logger.debug("%s: Start JmxTool %d command: %s" % (node.account, idx, cmd))
node.account.ssh(cmd, allow_fail=False)
wait_until(lambda: self._jmx_has_output(node), timeout_sec=30, backoff_sec=.5, err_msg="%s: Jmx tool took too long to start" % node.account)
self.started[idx-1] = True
def _jmx_has_output(self, node):
"""Helper used as a proxy to determine whether jmx is running by that jmx_tool_log contains output."""
try:
node.account.ssh("test -s %s" % self.jmx_tool_log, allow_fail=False)
return True
except RemoteCommandError:
return False
def read_jmx_output(self, idx, node):
if not self.started[idx-1]:
return
object_attribute_names = []
cmd = "cat %s" % self.jmx_tool_log
self.logger.debug("Read jmx output %d command: %s", idx, cmd)
lines = [line for line in node.account.ssh_capture(cmd, allow_fail=False)]
assert len(lines) > 1, "There don't appear to be any samples in the jmx tool log: %s" % lines
for line in lines:
if "time" in line:
object_attribute_names = line.strip()[1:-1].split("\",\"")[1:]
continue
stats = [float(field) for field in line.split(',')]
time_sec = int(stats[0]/1000)
self.jmx_stats[idx-1][time_sec] = {name: stats[i+1] for i, name in enumerate(object_attribute_names)}
# do not calculate average and maximum of jmx stats until we have read output from all nodes
# If the service is multithreaded, this means that the results will be aggregated only when the last
# service finishes
if any(len(time_to_stats) == 0 for time_to_stats in self.jmx_stats):
return
start_time_sec = min([min(time_to_stats.keys()) for time_to_stats in self.jmx_stats])
end_time_sec = max([max(time_to_stats.keys()) for time_to_stats in self.jmx_stats])
for name in object_attribute_names:
aggregates_per_time = []
for time_sec in xrange(start_time_sec, end_time_sec + 1):
# assume that value is 0 if it is not read by jmx tool at the given time. This is appropriate for metrics such as bandwidth
values_per_node = [time_to_stats.get(time_sec, {}).get(name, 0) for time_to_stats in self.jmx_stats]
# assume that value is aggregated across nodes by sum. This is appropriate for metrics such as bandwidth
aggregates_per_time.append(sum(values_per_node))
self.average_jmx_value[name] = sum(aggregates_per_time) / len(aggregates_per_time)
self.maximum_jmx_value[name] = max(aggregates_per_time)
def read_jmx_output_all_nodes(self):
for node in self.nodes:
self.read_jmx_output(self.idx(node), node)
def jmx_class_name(self):
return "kafka.tools.JmxTool"