Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
133 changes: 86 additions & 47 deletions tests/docker/ducker-ak
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,9 @@ default_image_name="ducker-ak"
# The default kafka server mode.
default_kafka_mode="jvm"

# Container runtime command (docker or podman)
container_runtime=""

# Port to listen on when debugging
debugpy_port=5678

Expand All @@ -66,6 +69,9 @@ usage() {
cat <<EOF
ducker-ak: a tool for running Apache Kafka tests inside Docker images.

Supports Docker and Podman container runtimes. Auto-detects available runtime
or can be overridden with CONTAINER_RUNTIME environment variable.

Usage: ${script_path} [command] [options]

help|-h|--help
Expand Down Expand Up @@ -142,6 +148,29 @@ require_commands() {
done
}

# Detect and set the container runtime (docker or podman).
# Sets the global variable container_runtime.
# Can be overridden by setting CONTAINER_RUNTIME environment variable.
detect_container_runtime() {
if [[ -n "${container_runtime}" ]]; then
return # Already set
fi

# Allow override via environment variable
if [[ -n "${CONTAINER_RUNTIME}" ]]; then
container_runtime="${CONTAINER_RUNTIME}"
which "${container_runtime}" &> /dev/null || die "Specified container runtime '${container_runtime}' not found."
return
fi

container_runtime="docker"
if command -v podman &> /dev/null; then
container_runtime="podman"
else
die "No supported container runtime found. Please install docker or podman, or set CONTAINER_RUNTIME environment variable."
fi
}

# Set a global variable to a value.
#
# $1: The variable name to set. This function will die if the variable already has a value. The
Expand Down Expand Up @@ -256,7 +285,7 @@ ducker_build() {
must_pushd "${ducker_dir}"
# Tip: if you are scratching your head for some dependency problems that are referring to an old code version
# (for example java.lang.NoClassDefFoundError), add --no-cache flag to the build shall give you a clean start.
echo_and_do docker build --memory="${docker_build_memory_limit}" \
echo_and_do ${container_runtime} build --memory="${docker_build_memory_limit}" \
--build-arg "ducker_creator=${user_name}" \
--build-arg "jdk_version=${jdk_version}" \
--build-arg "UID=${UID}" \
Expand Down Expand Up @@ -298,9 +327,14 @@ docker_run() {
# Invoke docker-run. We need privileged mode to be able to run iptables
# and mount FUSE filesystems inside the container. We also need it to
# run iptables inside the container.
must_do -v docker run --privileged \
local memory_swappiness_option=""
if [[ "${container_runtime}" == "docker" ]]; then
# Only add memory-swappiness for Docker (not supported in podman/cgroups v2)
memory_swappiness_option="--memory-swappiness=1"
fi
must_do -v ${container_runtime} run --privileged \
-d -t -h "${node}" --network ducknet "${expose_ports}" \
--memory=${docker_run_memory_limit} --memory-swappiness=1 \
--memory=${docker_run_memory_limit} ${memory_swappiness_option} \
-v "${kafka_dir}:/opt/kafka-dev" --name "${node}" -- "${image_name}"
}

Expand All @@ -311,13 +345,13 @@ setup_custom_ducktape() {
[[ -f "${custom_ducktape}/ducktape/__init__.py" ]] || \
die "You must supply a valid ducktape directory to --custom-ducktape"
docker_run ducker01 "${image_name}"
local running_container="$(docker ps -f=network=ducknet -q)"
must_do -v -o docker cp "${custom_ducktape}" "${running_container}:/opt/ducktape"
docker exec --user=root ducker01 bash -c 'set -x && cd /opt/kafka-dev/tests && sudo python3 ./setup.py develop install && cd /opt/ducktape && sudo python3 ./setup.py develop install'
local running_container="$(${container_runtime} ps -f=network=ducknet -q)"
must_do -v -o ${container_runtime} cp "${custom_ducktape}" "${running_container}:/opt/ducktape"
${container_runtime} exec --user=root ducker01 bash -c 'set -x && cd /opt/kafka-dev/tests && sudo python3 ./setup.py develop install && cd /opt/ducktape && sudo python3 ./setup.py develop install'
[[ $? -ne 0 ]] && die "failed to install the new ducktape."
must_do -v -o docker commit ducker01 "${image_name}"
must_do -v docker kill "${running_container}"
must_do -v docker rm ducker01
must_do -v -o ${container_runtime} commit ducker01 "${image_name}"
must_do -v ${container_runtime} kill "${running_container}"
must_do -v ${container_runtime} rm ducker01
}

cleanup_native_dir() {
Expand Down Expand Up @@ -347,7 +381,8 @@ prepare_native_dir() {
}

ducker_up() {
require_commands docker
detect_container_runtime
require_commands ${container_runtime}
while [[ $# -ge 1 ]]; do
case "${1}" in
-C|--custom-ducktape) set_once custom_ducktape "${2}" "the custom ducktape directory"; shift 2;;
Expand All @@ -357,7 +392,7 @@ ducker_up() {
-e|--expose-ports) set_once expose_ports "${2}" "the ports to expose"; shift 2;;
-m|--kafka_mode) set_once kafka_mode "${2}" "the mode in which kafka will run"; shift 2;;
--ipv6) set_once ipv6 "true" "enable IPv6"; shift;;
*) set_once image_name "${1}" "docker image name"; shift;;
*) set_once image_name "${1}" "container image name"; shift;;
esac
done
[[ -n "${num_nodes}" ]] || num_nodes="${default_num_nodes}"
Expand All @@ -377,13 +412,13 @@ use only ${num_nodes}."
fi
fi

docker ps >/dev/null || die "ducker_up: failed to run docker. Please check that the daemon is started."
${container_runtime} ps >/dev/null || die "ducker_up: failed to run ${container_runtime}. Please check that the daemon is started."

prepare_native_dir
ducker_build "${image_name}"
cleanup_native_dir

docker inspect --format='{{.Config.Labels}}' --type=image "${image_name}" | grep -q 'ducker.type'
${container_runtime} inspect --format='{{.Config.Labels}}' --type=image "${image_name}" | grep -q 'ducker.type'
local docker_status=${PIPESTATUS[0]}
local grep_status=${PIPESTATUS[1]}
[[ "${docker_status}" -eq 0 ]] || die "ducker_up: failed to inspect image ${image_name}. \
Expand All @@ -396,7 +431,7 @@ it up anyway."
exit 1
fi
fi
local running_containers="$(docker ps -f=network=ducknet -q)"
local running_containers="$(${container_runtime} ps -f=network=ducknet -q)"
local num_running_containers=$(count ${running_containers})
if [[ ${num_running_containers} -gt 0 ]]; then
die "ducker_up: there are ${num_running_containers} ducker containers \
Expand All @@ -405,15 +440,15 @@ attempting to start new ones."
fi

echo "ducker_up: Bringing up ${image_name} with ${num_nodes} nodes..."
if docker network inspect ducknet &>/dev/null; then
must_do -v docker network rm ducknet
if ${container_runtime} network inspect ducknet &>/dev/null; then
must_do -v ${container_runtime} network rm ducknet
fi
network_create_args=""
if [[ "${ipv6}" == "true" ]]; then
subnet_cidr_prefix="${DUCKER_SUBNET_CIDR:-"fc00:cf17"}"
network_create_args="--ipv6 --subnet ${subnet_cidr_prefix}::/64"
fi
must_do -v docker network create ${network_create_args} ducknet
must_do -v ${container_runtime} network create ${network_create_args} ducknet
if [[ -n "${custom_ducktape}" ]]; then
setup_custom_ducktape "${custom_ducktape}" "${image_name}"
fi
Expand All @@ -427,28 +462,28 @@ attempting to start new ones."
for n in $(seq -f %02g 1 ${num_nodes}); do
local node="ducker${n}"
if [[ "${ipv6}" == "true" ]]; then
docker exec --user=root "${node}" grep "${node}" /etc/hosts | grep "${subnet_cidr_prefix}" >&3
${container_runtime} exec --user=root "${node}" grep "${node}" /etc/hosts | grep "${subnet_cidr_prefix}" >&3
else
docker exec --user=root "${node}" grep "${node}" /etc/hosts >&3
${container_runtime} exec --user=root "${node}" grep "${node}" /etc/hosts >&3
fi
[[ $? -ne 0 ]] && die "failed to find the /etc/hosts entry for ${node}"
done
exec 3>&-
for n in $(seq -f %02g 1 ${num_nodes}); do
local node="ducker${n}"
docker exec --user=root "${node}" \
${container_runtime} exec --user=root "${node}" \
bash -c "grep -v ${node} /opt/kafka-dev/tests/docker/build/node_hosts >> /etc/hosts"
[[ $? -ne 0 ]] && die "failed to append to the /etc/hosts file on ${node}"
# Filter out ipv4 addresses if ipv6
if [[ "${ipv6}" == "true" ]]; then
docker exec --user=root "${node}" \
${container_runtime} exec --user=root "${node}" \
bash -c "grep -v -E '([0-9]{1,3}\.){3}[0-9]{1,3}' /opt/kafka-dev/tests/docker/build/node_hosts >> /etc/hosts"
[[ $? -ne 0 ]] && die "failed to append to the /etc/hosts file on ${node}"
fi
done

if [ "$kafka_mode" == "native" ]; then
docker exec --user=root ducker01 bash -c 'cp /opt/kafka-binary/kafka.Kafka /opt/kafka-dev/kafka.Kafka'
${container_runtime} exec --user=root ducker01 bash -c 'cp /opt/kafka-binary/kafka.Kafka /opt/kafka-dev/kafka.Kafka'
fi

echo "ducker_up: added the latest entries to /etc/hosts on each node."
Expand Down Expand Up @@ -526,8 +561,9 @@ correct_latest_link() {
}

ducker_test() {
require_commands docker
docker inspect ducker01 &>/dev/null || \
detect_container_runtime
require_commands ${container_runtime}
${container_runtime} inspect ducker01 &>/dev/null || \
die "ducker_test: the ducker01 instance appears to be down. Did you run 'ducker up'?"
declare -a test_name_args=()
local debug=0
Expand Down Expand Up @@ -564,15 +600,16 @@ ducker_test() {
fi

cmd="cd /opt/kafka-dev && ${ducktape_cmd} --cluster-file /opt/kafka-dev/tests/docker/build/cluster.json $test_names $ducktape_args"
echo "docker exec ducker01 bash -c \"${cmd}\""
docker exec --user=ducker ducker01 bash -c "${cmd}"
echo "${container_runtime} exec ducker01 bash -c \"${cmd}\""
${container_runtime} exec --user=ducker ducker01 bash -c "${cmd}"
docker_status=$?
correct_latest_link
exit "${docker_status}"
}

ducker_ssh() {
require_commands docker
detect_container_runtime
require_commands ${container_runtime}
[[ $# -eq 0 ]] && die "ducker_ssh: Please specify a container name to log into. \
Currently active containers: $(echo_running_container_names)"
local node_info="${1}"
Expand All @@ -597,21 +634,21 @@ Currently active containers: $(echo_running_container_names)"
local nodes=$(echo_running_container_names)
[[ "${nodes}" == "(none)" ]] && die "ducker_ssh: can't locate any running ducker nodes."
for node in ${nodes}; do
docker exec --user=${user_name} -i ${docker_flags} "${node}" \
${guest_command_prefix} "${guest_command}" || die "docker exec ${node} failed"
${container_runtime} exec --user=${user_name} -i ${docker_flags} "${node}" \
${guest_command_prefix} "${guest_command}" || die "${container_runtime} exec ${node} failed"
done
else
docker inspect --type=container -- "${node_name}" &>/dev/null || \
${container_runtime} inspect --type=container -- "${node_name}" &>/dev/null || \
die "ducker_ssh: can't locate node ${node_name}. Currently running nodes: \
$(echo_running_container_names)"
exec docker exec --user=${user_name} -i ${docker_flags} "${node_name}" \
exec ${container_runtime} exec --user=${user_name} -i ${docker_flags} "${node_name}" \
${guest_command_prefix} "${guest_command}"
fi
}

# Echo all the running Ducker container names, or (none) if there are no running Ducker containers.
echo_running_container_names() {
node_names="$(docker ps -f=network=ducknet -q --format '{{.Names}}' | sort)"
node_names="$(${container_runtime} ps -f=network=ducknet -q --format '{{.Names}}' | sort)"
if [[ -z "${node_names}" ]]; then
echo "(none)"
else
Expand All @@ -620,7 +657,8 @@ echo_running_container_names() {
}

ducker_down() {
require_commands docker
detect_container_runtime
require_commands ${container_runtime}
local verbose=1
local force_str=""
while [[ $# -ge 1 ]]; do
Expand All @@ -631,10 +669,10 @@ ducker_down() {
esac
done
local running_containers
running_containers="$(docker ps -f=network=ducknet -q)"
[[ $? -eq 0 ]] || die "ducker_down: docker command failed. Is the docker daemon running?"
running_containers="$(${container_runtime} ps -f=network=ducknet -q)"
[[ $? -eq 0 ]] || die "ducker_down: ${container_runtime} command failed. Is the ${container_runtime} daemon running?"
running_containers=${running_containers//$'\n'/ }
local all_containers="$(docker ps -a -f=network=ducknet -q)"
local all_containers="$(${container_runtime} ps -a -f=network=ducknet -q)"
all_containers=${all_containers//$'\n'/ }
if [[ -z "${all_containers}" ]]; then
maybe_echo "${verbose}" "No ducker containers found."
Expand All @@ -645,18 +683,19 @@ ducker_down() {
verbose_flag="-v"
fi
if [[ -n "${running_containers}" ]]; then
must_do ${verbose_flag} docker kill "${running_containers}"
must_do ${verbose_flag} ${container_runtime} kill "${running_containers}"
fi
must_do ${verbose_flag} docker rm ${force_str} "${all_containers}"
must_do ${verbose_flag} ${container_runtime} rm ${force_str} "${all_containers}"
must_do ${verbose_flag} -o rm -f -- "${ducker_dir}/build/node_hosts" "${ducker_dir}/build/cluster.json"
if docker network inspect ducknet &>/dev/null; then
must_do -v docker network rm ducknet
if ${container_runtime} network inspect ducknet &>/dev/null; then
must_do -v ${container_runtime} network rm ducknet
fi
maybe_echo "${verbose}" "ducker_down: removed $(count ${all_containers}) containers."
}

ducker_purge() {
require_commands docker
detect_container_runtime
require_commands ${container_runtime}
local force_str=""
while [[ $# -ge 1 ]]; do
case "${1}" in
Expand All @@ -666,8 +705,8 @@ ducker_purge() {
done
echo "** ducker_purge: attempting to locate ducker images to purge"
local images
images=$(docker images -q -a -f label=ducker.creator)
[[ $? -ne 0 ]] && die "docker images command failed"
images=$(${container_runtime} images -q -a -f label=ducker.creator)
[[ $? -ne 0 ]] && die "${container_runtime} images command failed"
images=${images//$'\n'/ }
declare -a purge_images=()
if [[ -z "${images}" ]]; then
Expand All @@ -677,12 +716,12 @@ ducker_purge() {
echo "** ducker_purge: images to delete:"
for image in ${images}; do
echo -n "${image} "
docker inspect --format='{{.Config.Labels}} {{.Created}}' --type=image "${image}"
[[ $? -ne 0 ]] && die "docker inspect ${image} failed"
${container_runtime} inspect --format='{{.Config.Labels}} {{.Created}}' --type=image "${image}"
[[ $? -ne 0 ]] && die "${container_runtime} inspect ${image} failed"
done
ask_yes_no "Delete these docker images? [y/n]"
ask_yes_no "Delete these container images? [y/n]"
[[ "${_return}" -eq 0 ]] && exit 0
must_do -v -o docker rmi ${force_str} ${images}
must_do -v -o ${container_runtime} rmi ${force_str} ${images}
}

# Parse command-line arguments
Expand Down
8 changes: 8 additions & 0 deletions tests/docker/run_tests.sh
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,14 @@ KAFKA_NUM_CONTAINERS=${KAFKA_NUM_CONTAINERS:-14}
TC_PATHS=${TC_PATHS:-./kafkatest/}
REBUILD=${REBUILD:f}

# Auto-detect container runtime if not set
if [[ -z "${CONTAINER_RUNTIME}" ]]; then
export CONTAINER_RUNTIME="docker"
if command -v podman &> /dev/null; then
export CONTAINER_RUNTIME="podman"
fi
fi

die() {
echo $@
exit 1
Expand Down