mirror of
https://github.com/didi/KnowStreaming.git
synced 2026-01-09 00:09:35 +08:00
Add km module kafka
This commit is contained in:
14
tests/kafkatest/services/trogdor/__init__.py
Normal file
14
tests/kafkatest/services/trogdor/__init__.py
Normal 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.
|
||||
56
tests/kafkatest/services/trogdor/consume_bench_workload.py
Normal file
56
tests/kafkatest/services/trogdor/consume_bench_workload.py
Normal file
@@ -0,0 +1,56 @@
|
||||
# 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.service import Service
|
||||
from kafkatest.services.trogdor.task_spec import TaskSpec
|
||||
|
||||
|
||||
class ConsumeBenchWorkloadSpec(TaskSpec):
|
||||
def __init__(self, start_ms, duration_ms, consumer_node, bootstrap_servers,
|
||||
target_messages_per_sec, max_messages, active_topics,
|
||||
consumer_conf, common_client_conf, admin_client_conf, consumer_group=None, threads_per_worker=1):
|
||||
super(ConsumeBenchWorkloadSpec, self).__init__(start_ms, duration_ms)
|
||||
self.message["class"] = "org.apache.kafka.trogdor.workload.ConsumeBenchSpec"
|
||||
self.message["consumerNode"] = consumer_node
|
||||
self.message["bootstrapServers"] = bootstrap_servers
|
||||
self.message["targetMessagesPerSec"] = target_messages_per_sec
|
||||
self.message["maxMessages"] = max_messages
|
||||
self.message["consumerConf"] = consumer_conf
|
||||
self.message["adminClientConf"] = admin_client_conf
|
||||
self.message["commonClientConf"] = common_client_conf
|
||||
self.message["activeTopics"] = active_topics
|
||||
self.message["threadsPerWorker"] = threads_per_worker
|
||||
if consumer_group is not None:
|
||||
self.message["consumerGroup"] = consumer_group
|
||||
|
||||
|
||||
class ConsumeBenchWorkloadService(Service):
|
||||
def __init__(self, context, kafka):
|
||||
Service.__init__(self, context, num_nodes=1)
|
||||
self.bootstrap_servers = kafka.bootstrap_servers(validate=False)
|
||||
self.consumer_node = self.nodes[0].account.hostname
|
||||
|
||||
def free(self):
|
||||
Service.free(self)
|
||||
|
||||
def wait_node(self, node, timeout_sec=None):
|
||||
pass
|
||||
|
||||
def stop_node(self, node):
|
||||
pass
|
||||
|
||||
def clean_node(self, node):
|
||||
pass
|
||||
@@ -0,0 +1,48 @@
|
||||
# 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.trogdor.task_spec import TaskSpec
|
||||
|
||||
|
||||
class DegradedNetworkFaultSpec(TaskSpec):
|
||||
"""
|
||||
The specification for a network degradation fault.
|
||||
|
||||
Degrades the network so that traffic on a subset of nodes has higher latency
|
||||
"""
|
||||
|
||||
def __init__(self, start_ms, duration_ms):
|
||||
"""
|
||||
Create a new NetworkDegradeFaultSpec.
|
||||
|
||||
:param start_ms: The start time, as described in task_spec.py
|
||||
:param duration_ms: The duration in milliseconds.
|
||||
"""
|
||||
super(DegradedNetworkFaultSpec, self).__init__(start_ms, duration_ms)
|
||||
self.message["class"] = "org.apache.kafka.trogdor.fault.DegradedNetworkFaultSpec"
|
||||
self.message["nodeSpecs"] = {}
|
||||
|
||||
def add_node_spec(self, node, networkDevice, latencyMs=0, rateLimitKbit=0):
|
||||
"""
|
||||
Add a node spec to this fault spec
|
||||
:param node: The node name which is to be degraded
|
||||
:param networkDevice: The network device name (e.g., eth0) to apply the degradation to
|
||||
:param latencyMs: Optional. How much latency to add to each packet
|
||||
:param rateLimitKbit: Optional. Maximum throughput in kilobits per second to allow
|
||||
:return:
|
||||
"""
|
||||
self.message["nodeSpecs"][node] = {
|
||||
"rateLimitKbit": rateLimitKbit, "latencyMs": latencyMs, "networkDevice": networkDevice
|
||||
}
|
||||
@@ -0,0 +1,46 @@
|
||||
# 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.trogdor.task_spec import TaskSpec
|
||||
|
||||
|
||||
class FilesUnreadableFaultSpec(TaskSpec):
|
||||
"""
|
||||
The specification for a fault which makes files unreadable.
|
||||
"""
|
||||
|
||||
def __init__(self, start_ms, duration_ms, node_names, mount_path,
|
||||
prefix, error_code):
|
||||
"""
|
||||
Create a new FilesUnreadableFaultSpec.
|
||||
|
||||
:param start_ms: The start time, as described in task_spec.py
|
||||
:param duration_ms: The duration in milliseconds.
|
||||
:param node_names: The names of the node(s) to create the fault on.
|
||||
:param mount_path: The mount path.
|
||||
:param prefix: The prefix within the mount point to make unreadable.
|
||||
:param error_code: The error code to use.
|
||||
"""
|
||||
super(FilesUnreadableFaultSpec, self).__init__(start_ms, duration_ms)
|
||||
self.message["class"] = "org.apache.kafka.trogdor.fault.FilesUnreadableFaultSpec"
|
||||
self.message["nodeNames"] = node_names
|
||||
self.message["mountPath"] = mount_path
|
||||
self.message["prefix"] = prefix
|
||||
self.message["errorCode"] = error_code
|
||||
|
||||
self.kibosh_message = {}
|
||||
self.kibosh_message["type"] = "unreadable"
|
||||
self.kibosh_message["prefix"] = prefix
|
||||
self.kibosh_message["code"] = error_code
|
||||
156
tests/kafkatest/services/trogdor/kibosh.py
Normal file
156
tests/kafkatest/services/trogdor/kibosh.py
Normal file
@@ -0,0 +1,156 @@
|
||||
# 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 json
|
||||
import os.path
|
||||
|
||||
from ducktape.services.service import Service
|
||||
from ducktape.utils import util
|
||||
|
||||
|
||||
class KiboshService(Service):
|
||||
"""
|
||||
Kibosh is a fault-injecting FUSE filesystem.
|
||||
|
||||
Attributes:
|
||||
INSTALL_ROOT The path of where Kibosh is installed.
|
||||
BINARY_NAME The Kibosh binary name.
|
||||
BINARY_PATH The path to the kibosh binary.
|
||||
"""
|
||||
INSTALL_ROOT = "/opt/kibosh/build"
|
||||
BINARY_NAME = "kibosh"
|
||||
BINARY_PATH = os.path.join(INSTALL_ROOT, BINARY_NAME)
|
||||
|
||||
def __init__(self, context, nodes, target, mirror, persist="/mnt/kibosh"):
|
||||
"""
|
||||
Create a Kibosh service.
|
||||
|
||||
:param context: The TestContext object.
|
||||
:param nodes: The nodes to put the Kibosh FS on. Kibosh allocates no
|
||||
nodes of its own.
|
||||
:param target: The target directory, which Kibosh exports a view of.
|
||||
:param mirror: The mirror directory, where Kibosh injects faults.
|
||||
:param persist: Where the log files and pid files will be created.
|
||||
"""
|
||||
Service.__init__(self, context, num_nodes=0)
|
||||
if (len(nodes) == 0):
|
||||
raise RuntimeError("You must supply at least one node to run the service on.")
|
||||
for node in nodes:
|
||||
self.nodes.append(node)
|
||||
|
||||
self.target = target
|
||||
self.mirror = mirror
|
||||
self.persist = persist
|
||||
|
||||
self.control_path = os.path.join(self.mirror, "kibosh_control")
|
||||
self.pidfile_path = os.path.join(self.persist, "pidfile")
|
||||
self.stdout_stderr_path = os.path.join(self.persist, "kibosh-stdout-stderr.log")
|
||||
self.log_path = os.path.join(self.persist, "kibosh.log")
|
||||
self.logs = {
|
||||
"kibosh-stdout-stderr.log": {
|
||||
"path": self.stdout_stderr_path,
|
||||
"collect_default": True},
|
||||
"kibosh.log": {
|
||||
"path": self.log_path,
|
||||
"collect_default": True}
|
||||
}
|
||||
|
||||
def free(self):
|
||||
"""Clear the nodes list."""
|
||||
# Because the filesystem runs on nodes which have been allocated by other services, those nodes
|
||||
# are not deallocated here.
|
||||
self.nodes = []
|
||||
Service.free(self)
|
||||
|
||||
def kibosh_running(self, node):
|
||||
return 0 == node.account.ssh("test -e '%s'" % self.control_path, allow_fail=True)
|
||||
|
||||
def start_node(self, node):
|
||||
node.account.mkdirs(self.persist)
|
||||
cmd = "sudo -E "
|
||||
cmd += " %s" % KiboshService.BINARY_PATH
|
||||
cmd += " --target %s" % self.target
|
||||
cmd += " --pidfile %s" % self.pidfile_path
|
||||
cmd += " --log %s" % self.log_path
|
||||
cmd += " --control-mode 666"
|
||||
cmd += " --verbose"
|
||||
cmd += " %s" % self.mirror
|
||||
cmd += " &> %s" % self.stdout_stderr_path
|
||||
node.account.ssh(cmd)
|
||||
util.wait_until(lambda: self.kibosh_running(node), 20, backoff_sec=.1,
|
||||
err_msg="Timed out waiting for kibosh to start on %s" % node.account.hostname)
|
||||
|
||||
def pids(self, node):
|
||||
return [pid for pid in node.account.ssh_capture("test -e '%s' && test -e /proc/$(cat '%s')" %
|
||||
(self.pidfile_path, self.pidfile_path), allow_fail=True)]
|
||||
|
||||
def wait_node(self, node, timeout_sec=None):
|
||||
return len(self.pids(node)) == 0
|
||||
|
||||
def kibosh_process_running(self, node):
|
||||
pids = self.pids(node)
|
||||
if len(pids) == 0:
|
||||
return True
|
||||
return False
|
||||
|
||||
def stop_node(self, node):
|
||||
"""Halt kibosh process(es) on this node."""
|
||||
node.account.logger.debug("stop_node(%s): unmounting %s" % (node.name, self.mirror))
|
||||
node.account.ssh("sudo fusermount -u %s" % self.mirror, allow_fail=True)
|
||||
# Wait for the kibosh process to terminate.
|
||||
try:
|
||||
util.wait_until(lambda: self.kibosh_process_running(node), 20, backoff_sec=.1,
|
||||
err_msg="Timed out waiting for kibosh to stop on %s" % node.account.hostname)
|
||||
except TimeoutError:
|
||||
# If the process won't terminate, use kill -9 to shut it down.
|
||||
node.account.logger.debug("stop_node(%s): killing the kibosh process managing %s" % (node.name, self.mirror))
|
||||
node.account.ssh("sudo kill -9 %s" % (" ".join(self.pids(node))), allow_fail=True)
|
||||
node.account.ssh("sudo fusermount -u %s" % self.mirror)
|
||||
util.wait_until(lambda: self.kibosh_process_running(node), 20, backoff_sec=.1,
|
||||
err_msg="Timed out waiting for kibosh to stop on %s" % node.account.hostname)
|
||||
|
||||
def clean_node(self, node):
|
||||
"""Clean up persistent state on this node - e.g. service logs, configuration files etc."""
|
||||
self.stop_node(node)
|
||||
node.account.ssh("rm -rf -- %s" % self.persist)
|
||||
|
||||
def set_faults(self, node, specs):
|
||||
"""
|
||||
Set the currently active faults.
|
||||
|
||||
:param node: The node.
|
||||
:param spec: An array of FaultSpec objects describing the faults.
|
||||
"""
|
||||
if len(specs) == 0:
|
||||
obj_json = "{}"
|
||||
else:
|
||||
fault_array = [spec.kibosh_message for spec in specs]
|
||||
obj = { 'faults': fault_array }
|
||||
obj_json = json.dumps(obj)
|
||||
node.account.create_file(self.control_path, obj_json)
|
||||
|
||||
def get_fault_json(self, node):
|
||||
"""
|
||||
Return a JSON string which contains the currently active faults.
|
||||
|
||||
:param node: The node.
|
||||
|
||||
:returns: The fault JSON describing the faults.
|
||||
"""
|
||||
iter = node.account.ssh_capture("cat '%s'" % self.control_path)
|
||||
text = ""
|
||||
for line in iter:
|
||||
text = "%s%s" % (text, line.rstrip("\r\n"))
|
||||
return text
|
||||
@@ -0,0 +1,39 @@
|
||||
# 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.trogdor.task_spec import TaskSpec
|
||||
|
||||
|
||||
class NetworkPartitionFaultSpec(TaskSpec):
|
||||
"""
|
||||
The specification for a network partition fault.
|
||||
|
||||
Network partition faults fracture the network into different partitions
|
||||
that cannot communicate with each other.
|
||||
"""
|
||||
|
||||
def __init__(self, start_ms, duration_ms, partitions):
|
||||
"""
|
||||
Create a new NetworkPartitionFaultSpec.
|
||||
|
||||
:param start_ms: The start time, as described in task_spec.py
|
||||
:param duration_ms: The duration in milliseconds.
|
||||
:param partitions: An array of arrays describing the partitions.
|
||||
The inner arrays may contain either node names,
|
||||
or ClusterNode objects.
|
||||
"""
|
||||
super(NetworkPartitionFaultSpec, self).__init__(start_ms, duration_ms)
|
||||
self.message["class"] = "org.apache.kafka.trogdor.fault.NetworkPartitionFaultSpec"
|
||||
self.message["partitions"] = [TaskSpec.to_node_names(p) for p in partitions]
|
||||
35
tests/kafkatest/services/trogdor/no_op_task_spec.py
Normal file
35
tests/kafkatest/services/trogdor/no_op_task_spec.py
Normal file
@@ -0,0 +1,35 @@
|
||||
# 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.trogdor.task_spec import TaskSpec
|
||||
|
||||
|
||||
class NoOpTaskSpec(TaskSpec):
|
||||
"""
|
||||
The specification for a nop-op task.
|
||||
|
||||
No-op faults are used to test Trogdor. They don't do anything,
|
||||
but must be propagated to all Trogdor agents.
|
||||
"""
|
||||
|
||||
def __init__(self, start_ms, duration_ms):
|
||||
"""
|
||||
Create a new NoOpFault.
|
||||
|
||||
:param start_ms: The start time, as described in task_spec.py
|
||||
:param duration_ms: The duration in milliseconds.
|
||||
"""
|
||||
super(NoOpTaskSpec, self).__init__(start_ms, duration_ms)
|
||||
self.message["class"] = "org.apache.kafka.trogdor.task.NoOpTaskSpec";
|
||||
38
tests/kafkatest/services/trogdor/process_stop_fault_spec.py
Normal file
38
tests/kafkatest/services/trogdor/process_stop_fault_spec.py
Normal file
@@ -0,0 +1,38 @@
|
||||
# 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.trogdor.task_spec import TaskSpec
|
||||
|
||||
|
||||
class ProcessStopFaultSpec(TaskSpec):
|
||||
"""
|
||||
The specification for a process stop fault.
|
||||
"""
|
||||
|
||||
def __init__(self, start_ms, duration_ms, nodes, java_process_name):
|
||||
"""
|
||||
Create a new ProcessStopFaultSpec.
|
||||
|
||||
:param start_ms: The start time, as described in task_spec.py
|
||||
:param duration_ms: The duration in milliseconds.
|
||||
:param node_names: An array describing the nodes to stop processes on. The array
|
||||
may contain either node names, or ClusterNode objects.
|
||||
:param java_process_name: The name of the java process to stop. This is the name which
|
||||
is reported by jps, etc., not the OS-level process name.
|
||||
"""
|
||||
super(ProcessStopFaultSpec, self).__init__(start_ms, duration_ms)
|
||||
self.message["class"] = "org.apache.kafka.trogdor.fault.ProcessStopFaultSpec"
|
||||
self.message["nodeNames"] = TaskSpec.to_node_names(nodes)
|
||||
self.message["javaProcessName"] = java_process_name
|
||||
56
tests/kafkatest/services/trogdor/produce_bench_workload.py
Normal file
56
tests/kafkatest/services/trogdor/produce_bench_workload.py
Normal file
@@ -0,0 +1,56 @@
|
||||
# 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.service import Service
|
||||
from kafkatest.services.trogdor.task_spec import TaskSpec
|
||||
|
||||
|
||||
class ProduceBenchWorkloadSpec(TaskSpec):
|
||||
def __init__(self, start_ms, duration_ms, producer_node, bootstrap_servers,
|
||||
target_messages_per_sec, max_messages, producer_conf, admin_client_conf,
|
||||
common_client_conf, inactive_topics, active_topics,
|
||||
transaction_generator=None):
|
||||
super(ProduceBenchWorkloadSpec, self).__init__(start_ms, duration_ms)
|
||||
self.message["class"] = "org.apache.kafka.trogdor.workload.ProduceBenchSpec"
|
||||
self.message["producerNode"] = producer_node
|
||||
self.message["bootstrapServers"] = bootstrap_servers
|
||||
self.message["targetMessagesPerSec"] = target_messages_per_sec
|
||||
self.message["maxMessages"] = max_messages
|
||||
self.message["producerConf"] = producer_conf
|
||||
self.message["transactionGenerator"] = transaction_generator
|
||||
self.message["adminClientConf"] = admin_client_conf
|
||||
self.message["commonClientConf"] = common_client_conf
|
||||
self.message["inactiveTopics"] = inactive_topics
|
||||
self.message["activeTopics"] = active_topics
|
||||
|
||||
|
||||
class ProduceBenchWorkloadService(Service):
|
||||
def __init__(self, context, kafka):
|
||||
Service.__init__(self, context, num_nodes=1)
|
||||
self.bootstrap_servers = kafka.bootstrap_servers(validate=False)
|
||||
self.producer_node = self.nodes[0].account.hostname
|
||||
|
||||
def free(self):
|
||||
Service.free(self)
|
||||
|
||||
def wait_node(self, node, timeout_sec=None):
|
||||
pass
|
||||
|
||||
def stop_node(self, node):
|
||||
pass
|
||||
|
||||
def clean_node(self, node):
|
||||
pass
|
||||
49
tests/kafkatest/services/trogdor/round_trip_workload.py
Normal file
49
tests/kafkatest/services/trogdor/round_trip_workload.py
Normal file
@@ -0,0 +1,49 @@
|
||||
# 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.service import Service
|
||||
from kafkatest.services.trogdor.task_spec import TaskSpec
|
||||
|
||||
|
||||
class RoundTripWorkloadSpec(TaskSpec):
|
||||
def __init__(self, start_ms, duration_ms, client_node, bootstrap_servers,
|
||||
target_messages_per_sec, max_messages, active_topics):
|
||||
super(RoundTripWorkloadSpec, self).__init__(start_ms, duration_ms)
|
||||
self.message["class"] = "org.apache.kafka.trogdor.workload.RoundTripWorkloadSpec"
|
||||
self.message["clientNode"] = client_node
|
||||
self.message["bootstrapServers"] = bootstrap_servers
|
||||
self.message["targetMessagesPerSec"] = target_messages_per_sec
|
||||
self.message["maxMessages"] = max_messages
|
||||
self.message["activeTopics"] = active_topics
|
||||
|
||||
|
||||
class RoundTripWorkloadService(Service):
|
||||
def __init__(self, context, kafka):
|
||||
Service.__init__(self, context, num_nodes=1)
|
||||
self.bootstrap_servers = kafka.bootstrap_servers(validate=False)
|
||||
self.client_node = self.nodes[0].account.hostname
|
||||
|
||||
def free(self):
|
||||
Service.free(self)
|
||||
|
||||
def wait_node(self, node, timeout_sec=None):
|
||||
pass
|
||||
|
||||
def stop_node(self, node):
|
||||
pass
|
||||
|
||||
def clean_node(self, node):
|
||||
pass
|
||||
54
tests/kafkatest/services/trogdor/task_spec.py
Normal file
54
tests/kafkatest/services/trogdor/task_spec.py
Normal file
@@ -0,0 +1,54 @@
|
||||
# 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 json
|
||||
|
||||
|
||||
class TaskSpec(object):
|
||||
"""
|
||||
The base class for a task specification.
|
||||
|
||||
MAX_DURATION_MS The longest duration we should use for a task specification.
|
||||
"""
|
||||
|
||||
MAX_DURATION_MS=10000000
|
||||
|
||||
def __init__(self, start_ms, duration_ms):
|
||||
"""
|
||||
Create a new task specification.
|
||||
|
||||
:param start_ms: The target start time in milliseconds since the epoch.
|
||||
:param duration_ms: The duration in milliseconds.
|
||||
"""
|
||||
self.message = {
|
||||
'startMs': start_ms,
|
||||
'durationMs': duration_ms
|
||||
}
|
||||
|
||||
@staticmethod
|
||||
def to_node_names(nodes):
|
||||
"""
|
||||
Convert an array of nodes or node names to an array of node names.
|
||||
"""
|
||||
node_names = []
|
||||
for obj in nodes:
|
||||
if isinstance(obj, basestring):
|
||||
node_names.append(obj)
|
||||
else:
|
||||
node_names.append(obj.name)
|
||||
return node_names
|
||||
|
||||
def __str__(self):
|
||||
return json.dumps(self.message)
|
||||
23
tests/kafkatest/services/trogdor/templates/log4j.properties
Normal file
23
tests/kafkatest/services/trogdor/templates/log4j.properties
Normal file
@@ -0,0 +1,23 @@
|
||||
# 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.
|
||||
|
||||
log4j.rootLogger=DEBUG, mylogger
|
||||
log4j.logger.kafka=DEBUG
|
||||
log4j.logger.org.apache.kafka=DEBUG
|
||||
log4j.logger.org.eclipse=INFO
|
||||
log4j.appender.mylogger=org.apache.log4j.FileAppender
|
||||
log4j.appender.mylogger.File={{ log_path }}
|
||||
log4j.appender.mylogger.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.mylogger.layout.ConversionPattern=[%d] %p %m (%c)%n
|
||||
354
tests/kafkatest/services/trogdor/trogdor.py
Normal file
354
tests/kafkatest/services/trogdor/trogdor.py
Normal file
@@ -0,0 +1,354 @@
|
||||
# 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 json
|
||||
import os.path
|
||||
import requests
|
||||
from requests.adapters import HTTPAdapter
|
||||
from requests.packages.urllib3 import Retry
|
||||
|
||||
from ducktape.services.service import Service
|
||||
from ducktape.utils.util import wait_until
|
||||
from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin
|
||||
|
||||
|
||||
class TrogdorService(KafkaPathResolverMixin, Service):
|
||||
"""
|
||||
A ducktape service for running the trogdor fault injection daemons.
|
||||
|
||||
Attributes:
|
||||
PERSISTENT_ROOT The root filesystem path to store service files under.
|
||||
COORDINATOR_STDOUT_STDERR The path where we store the coordinator's stdout/stderr output.
|
||||
AGENT_STDOUT_STDERR The path where we store the agents's stdout/stderr output.
|
||||
COORDINATOR_LOG The path where we store the coordinator's log4j output.
|
||||
AGENT_LOG The path where we store the agent's log4j output.
|
||||
AGENT_LOG4J_PROPERTIES The path to the agent log4j.properties file for log config.
|
||||
COORDINATOR_LOG4J_PROPERTIES The path to the coordinator log4j.properties file for log config.
|
||||
CONFIG_PATH The path to the trogdor configuration file.
|
||||
DEFAULT_AGENT_PORT The default port to use for trogdor_agent daemons.
|
||||
DEFAULT_COORDINATOR_PORT The default port to use for trogdor_coordinator daemons.
|
||||
REQUEST_TIMEOUT The request timeout in seconds to use for REST requests.
|
||||
REQUEST_HEADERS The request headers to use when communicating with trogdor.
|
||||
"""
|
||||
|
||||
PERSISTENT_ROOT="/mnt/trogdor"
|
||||
COORDINATOR_STDOUT_STDERR = os.path.join(PERSISTENT_ROOT, "trogdor-coordinator-stdout-stderr.log")
|
||||
AGENT_STDOUT_STDERR = os.path.join(PERSISTENT_ROOT, "trogdor-agent-stdout-stderr.log")
|
||||
COORDINATOR_LOG = os.path.join(PERSISTENT_ROOT, "trogdor-coordinator.log")
|
||||
AGENT_LOG = os.path.join(PERSISTENT_ROOT, "trogdor-agent.log")
|
||||
COORDINATOR_LOG4J_PROPERTIES = os.path.join(PERSISTENT_ROOT, "trogdor-coordinator-log4j.properties")
|
||||
AGENT_LOG4J_PROPERTIES = os.path.join(PERSISTENT_ROOT, "trogdor-agent-log4j.properties")
|
||||
CONFIG_PATH = os.path.join(PERSISTENT_ROOT, "trogdor.conf")
|
||||
DEFAULT_AGENT_PORT=8888
|
||||
DEFAULT_COORDINATOR_PORT=8889
|
||||
REQUEST_TIMEOUT=5
|
||||
REQUEST_HEADERS = {"Content-type": "application/json"}
|
||||
|
||||
logs = {
|
||||
"trogdor_coordinator_stdout_stderr": {
|
||||
"path": COORDINATOR_STDOUT_STDERR,
|
||||
"collect_default": True},
|
||||
"trogdor_agent_stdout_stderr": {
|
||||
"path": AGENT_STDOUT_STDERR,
|
||||
"collect_default": True},
|
||||
"trogdor_coordinator_log": {
|
||||
"path": COORDINATOR_LOG,
|
||||
"collect_default": True},
|
||||
"trogdor_agent_log": {
|
||||
"path": AGENT_LOG,
|
||||
"collect_default": True},
|
||||
}
|
||||
|
||||
|
||||
def __init__(self, context, agent_nodes=None, client_services=None,
|
||||
agent_port=DEFAULT_AGENT_PORT, coordinator_port=DEFAULT_COORDINATOR_PORT):
|
||||
"""
|
||||
Create a Trogdor service.
|
||||
|
||||
:param context: The test context.
|
||||
:param agent_nodes: The nodes to run the agents on.
|
||||
:param client_services: Services whose nodes we should run agents on.
|
||||
:param agent_port: The port to use for the trogdor_agent daemons.
|
||||
:param coordinator_port: The port to use for the trogdor_coordinator daemons.
|
||||
"""
|
||||
Service.__init__(self, context, num_nodes=1)
|
||||
self.coordinator_node = self.nodes[0]
|
||||
if client_services is not None:
|
||||
for client_service in client_services:
|
||||
for node in client_service.nodes:
|
||||
self.nodes.append(node)
|
||||
if agent_nodes is not None:
|
||||
for agent_node in agent_nodes:
|
||||
self.nodes.append(agent_node)
|
||||
if (len(self.nodes) == 1):
|
||||
raise RuntimeError("You must supply at least one agent node to run the service on.")
|
||||
self.agent_port = agent_port
|
||||
self.coordinator_port = coordinator_port
|
||||
|
||||
def free(self):
|
||||
# We only want to deallocate the coordinator node, not the agent nodes. So we
|
||||
# change self.nodes to include only the coordinator node, and then invoke
|
||||
# the base class' free method.
|
||||
if self.coordinator_node is not None:
|
||||
self.nodes = [self.coordinator_node]
|
||||
self.coordinator_node = None
|
||||
Service.free(self)
|
||||
|
||||
def _create_config_dict(self):
|
||||
"""
|
||||
Create a dictionary with the Trogdor configuration.
|
||||
|
||||
:return: The configuration dictionary.
|
||||
"""
|
||||
dict_nodes = {}
|
||||
for node in self.nodes:
|
||||
dict_nodes[node.name] = {
|
||||
"hostname": node.account.ssh_hostname,
|
||||
}
|
||||
if node.name == self.coordinator_node.name:
|
||||
dict_nodes[node.name]["trogdor.coordinator.port"] = self.coordinator_port
|
||||
else:
|
||||
dict_nodes[node.name]["trogdor.agent.port"] = self.agent_port
|
||||
|
||||
return {
|
||||
"platform": "org.apache.kafka.trogdor.basic.BasicPlatform",
|
||||
"nodes": dict_nodes,
|
||||
}
|
||||
|
||||
def start_node(self, node):
|
||||
node.account.mkdirs(TrogdorService.PERSISTENT_ROOT)
|
||||
|
||||
# Create the configuration file on the node.
|
||||
str = json.dumps(self._create_config_dict(), indent=2)
|
||||
self.logger.info("Creating configuration file %s with %s" % (TrogdorService.CONFIG_PATH, str))
|
||||
node.account.create_file(TrogdorService.CONFIG_PATH, str)
|
||||
|
||||
if self.is_coordinator(node):
|
||||
self._start_coordinator_node(node)
|
||||
else:
|
||||
self._start_agent_node(node)
|
||||
|
||||
def _start_coordinator_node(self, node):
|
||||
node.account.create_file(TrogdorService.COORDINATOR_LOG4J_PROPERTIES,
|
||||
self.render('log4j.properties',
|
||||
log_path=TrogdorService.COORDINATOR_LOG))
|
||||
self._start_trogdor_daemon("coordinator", TrogdorService.COORDINATOR_STDOUT_STDERR,
|
||||
TrogdorService.COORDINATOR_LOG4J_PROPERTIES,
|
||||
TrogdorService.COORDINATOR_LOG, node)
|
||||
self.logger.info("Started trogdor coordinator on %s." % node.name)
|
||||
|
||||
def _start_agent_node(self, node):
|
||||
node.account.create_file(TrogdorService.AGENT_LOG4J_PROPERTIES,
|
||||
self.render('log4j.properties',
|
||||
log_path=TrogdorService.AGENT_LOG))
|
||||
self._start_trogdor_daemon("agent", TrogdorService.AGENT_STDOUT_STDERR,
|
||||
TrogdorService.AGENT_LOG4J_PROPERTIES,
|
||||
TrogdorService.AGENT_LOG, node)
|
||||
self.logger.info("Started trogdor agent on %s." % node.name)
|
||||
|
||||
def _start_trogdor_daemon(self, daemon_name, stdout_stderr_capture_path,
|
||||
log4j_properties_path, log_path, node):
|
||||
cmd = "export KAFKA_LOG4J_OPTS='-Dlog4j.configuration=file:%s'; " % log4j_properties_path
|
||||
cmd += "%s %s --%s.config %s --node-name %s 1>> %s 2>> %s &" % \
|
||||
(self.path.script("trogdor.sh", node),
|
||||
daemon_name,
|
||||
daemon_name,
|
||||
TrogdorService.CONFIG_PATH,
|
||||
node.name,
|
||||
stdout_stderr_capture_path,
|
||||
stdout_stderr_capture_path)
|
||||
node.account.ssh(cmd)
|
||||
with node.account.monitor_log(log_path) as monitor:
|
||||
monitor.wait_until("Starting %s process." % daemon_name, timeout_sec=60, backoff_sec=.10,
|
||||
err_msg=("%s on %s didn't finish startup" % (daemon_name, node.name)))
|
||||
|
||||
def wait_node(self, node, timeout_sec=None):
|
||||
if self.is_coordinator(node):
|
||||
return len(node.account.java_pids(self.coordinator_class_name())) == 0
|
||||
else:
|
||||
return len(node.account.java_pids(self.agent_class_name())) == 0
|
||||
|
||||
def stop_node(self, node):
|
||||
"""Halt trogdor processes on this node."""
|
||||
if self.is_coordinator(node):
|
||||
node.account.kill_java_processes(self.coordinator_class_name())
|
||||
else:
|
||||
node.account.kill_java_processes(self.agent_class_name())
|
||||
|
||||
def clean_node(self, node):
|
||||
"""Clean up persistent state on this node - e.g. service logs, configuration files etc."""
|
||||
self.stop_node(node)
|
||||
node.account.ssh("rm -rf -- %s" % TrogdorService.PERSISTENT_ROOT)
|
||||
|
||||
def _coordinator_url(self, path):
|
||||
return "http://%s:%d/coordinator/%s" % \
|
||||
(self.coordinator_node.account.ssh_hostname, self.coordinator_port, path)
|
||||
|
||||
def request_session(self):
|
||||
"""
|
||||
Creates a new request session which will retry for a while.
|
||||
"""
|
||||
session = requests.Session()
|
||||
session.mount('http://',
|
||||
HTTPAdapter(max_retries=Retry(total=5, backoff_factor=0.3)))
|
||||
return session
|
||||
|
||||
def _coordinator_post(self, path, message):
|
||||
"""
|
||||
Make a POST request to the Trogdor coordinator.
|
||||
|
||||
:param path: The URL path to use.
|
||||
:param message: The message object to send.
|
||||
:return: The response as an object.
|
||||
"""
|
||||
url = self._coordinator_url(path)
|
||||
self.logger.info("POST %s %s" % (url, message))
|
||||
response = self.request_session().post(url, json=message,
|
||||
timeout=TrogdorService.REQUEST_TIMEOUT,
|
||||
headers=TrogdorService.REQUEST_HEADERS)
|
||||
response.raise_for_status()
|
||||
return response.json()
|
||||
|
||||
def _coordinator_put(self, path, message):
|
||||
"""
|
||||
Make a PUT request to the Trogdor coordinator.
|
||||
|
||||
:param path: The URL path to use.
|
||||
:param message: The message object to send.
|
||||
:return: The response as an object.
|
||||
"""
|
||||
url = self._coordinator_url(path)
|
||||
self.logger.info("PUT %s %s" % (url, message))
|
||||
response = self.request_session().put(url, json=message,
|
||||
timeout=TrogdorService.REQUEST_TIMEOUT,
|
||||
headers=TrogdorService.REQUEST_HEADERS)
|
||||
response.raise_for_status()
|
||||
return response.json()
|
||||
|
||||
def _coordinator_get(self, path, message):
|
||||
"""
|
||||
Make a GET request to the Trogdor coordinator.
|
||||
|
||||
:param path: The URL path to use.
|
||||
:param message: The message object to send.
|
||||
:return: The response as an object.
|
||||
"""
|
||||
url = self._coordinator_url(path)
|
||||
self.logger.info("GET %s %s" % (url, message))
|
||||
response = self.request_session().get(url, json=message,
|
||||
timeout=TrogdorService.REQUEST_TIMEOUT,
|
||||
headers=TrogdorService.REQUEST_HEADERS)
|
||||
response.raise_for_status()
|
||||
return response.json()
|
||||
|
||||
def create_task(self, id, spec):
|
||||
"""
|
||||
Create a new task.
|
||||
|
||||
:param id: The task id.
|
||||
:param spec: The task spec.
|
||||
"""
|
||||
self._coordinator_post("task/create", { "id": id, "spec": spec.message})
|
||||
return TrogdorTask(id, self)
|
||||
|
||||
def stop_task(self, id):
|
||||
"""
|
||||
Stop a task.
|
||||
|
||||
:param id: The task id.
|
||||
"""
|
||||
self._coordinator_put("task/stop", { "id": id })
|
||||
|
||||
def tasks(self):
|
||||
"""
|
||||
Get the tasks which are on the coordinator.
|
||||
|
||||
:returns: A map of task id strings to task state objects.
|
||||
Task state objects contain a 'spec' field with the spec
|
||||
and a 'state' field with the state.
|
||||
"""
|
||||
return self._coordinator_get("tasks", {})
|
||||
|
||||
def is_coordinator(self, node):
|
||||
return node == self.coordinator_node
|
||||
|
||||
def agent_class_name(self):
|
||||
return "org.apache.kafka.trogdor.agent.Agent"
|
||||
|
||||
def coordinator_class_name(self):
|
||||
return "org.apache.kafka.trogdor.coordinator.Coordinator"
|
||||
|
||||
class TrogdorTask(object):
|
||||
PENDING_STATE = "PENDING"
|
||||
RUNNING_STATE = "RUNNING"
|
||||
STOPPING_STATE = "STOPPING"
|
||||
DONE_STATE = "DONE"
|
||||
|
||||
def __init__(self, id, trogdor):
|
||||
self.id = id
|
||||
self.trogdor = trogdor
|
||||
|
||||
def task_state_or_error(self):
|
||||
task_state = self.trogdor.tasks()["tasks"][self.id]
|
||||
if task_state is None:
|
||||
raise RuntimeError("Coordinator did not know about %s." % self.id)
|
||||
error = task_state.get("error")
|
||||
if error is None or error == "":
|
||||
return task_state["state"], None
|
||||
else:
|
||||
return None, error
|
||||
|
||||
def done(self):
|
||||
"""
|
||||
Check if this task is done.
|
||||
|
||||
:raises RuntimeError: If the task encountered an error.
|
||||
:returns: True if the task is in DONE_STATE;
|
||||
False if it is in a different state.
|
||||
"""
|
||||
(task_state, error) = self.task_state_or_error()
|
||||
if task_state is not None:
|
||||
return task_state == TrogdorTask.DONE_STATE
|
||||
else:
|
||||
raise RuntimeError("Failed to gracefully stop %s: got task error: %s" % (self.id, error))
|
||||
|
||||
def running(self):
|
||||
"""
|
||||
Check if this task is running.
|
||||
|
||||
:raises RuntimeError: If the task encountered an error.
|
||||
:returns: True if the task is in RUNNING_STATE;
|
||||
False if it is in a different state.
|
||||
"""
|
||||
(task_state, error) = self.task_state_or_error()
|
||||
if task_state is not None:
|
||||
return task_state == TrogdorTask.RUNNING_STATE
|
||||
else:
|
||||
raise RuntimeError("Failed to start %s: got task error: %s" % (self.id, error))
|
||||
|
||||
def stop(self):
|
||||
"""
|
||||
Stop this task.
|
||||
|
||||
:raises RuntimeError: If the task encountered an error.
|
||||
"""
|
||||
if self.done():
|
||||
return
|
||||
self.trogdor.stop_task(self.id)
|
||||
|
||||
def wait_for_done(self, timeout_sec=360):
|
||||
wait_until(lambda: self.done(),
|
||||
timeout_sec=timeout_sec,
|
||||
err_msg="%s failed to finish in the expected amount of time." % self.id)
|
||||
Reference in New Issue
Block a user