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,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

View File

@@ -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
}

View File

@@ -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

View 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

View File

@@ -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]

View 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";

View 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

View 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

View 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

View 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)

View 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

View 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)