diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml
index 924cea7ce12a28..3c96c2d0308200 100644
--- a/.github/workflows/pulsar-ci.yaml
+++ b/.github/workflows/pulsar-ci.yaml
@@ -506,15 +506,10 @@ jobs:
cd $HOME
$GITHUB_WORKSPACE/build/pulsar_ci_tool.sh restore_tar_from_github_actions_artifacts pulsar-maven-repository-binaries
- - name: Pick ubuntu mirror for the docker image build
- run: |
- # pick the closest ubuntu mirror and set it to UBUNTU_MIRROR environment variable
- $GITHUB_WORKSPACE/build/pulsar_ci_tool.sh pick_ubuntu_mirror
-
- name: Build java-test-image docker image
run: |
# build docker image
- mvn -B -am -pl tests/docker-images/java-test-image install -Pcore-modules,-main,integrationTests,docker \
+ mvn -B -am -pl docker/pulsar,tests/docker-images/java-test-image install -Pcore-modules,-main,integrationTests,docker \
-Dmaven.test.skip=true -Ddocker.squash=true -DskipSourceReleaseAssembly=true \
-Dspotbugs.skip=true -Dlicense.skip=true -Dcheckstyle.skip=true -Drat.skip=true
@@ -863,17 +858,11 @@ jobs:
cd $HOME
$GITHUB_WORKSPACE/build/pulsar_ci_tool.sh restore_tar_from_github_actions_artifacts pulsar-maven-repository-binaries
- - name: Pick ubuntu mirror for the docker image build
- run: |
- # pick the closest ubuntu mirror and set it to UBUNTU_MIRROR environment variable
- $GITHUB_WORKSPACE/build/pulsar_ci_tool.sh pick_ubuntu_mirror
-
- name: Build latest-version-image docker image
run: |
# build docker image
# include building of Connectors, Offloaders and server distros
mvn -B -am -pl distribution/io,distribution/offloaders,distribution/server,distribution/shell,tests/docker-images/latest-version-image install \
- -DUBUNTU_MIRROR="${UBUNTU_MIRROR}" -DUBUNTU_SECURITY_MIRROR="${UBUNTU_SECURITY_MIRROR}" -DIMAGE_JDK_MAJOR_VERSION="${IMAGE_JDK_MAJOR_VERSION}" \
-Pmain,docker -Dmaven.test.skip=true -Ddocker.squash=true \
-Dspotbugs.skip=true -Dlicense.skip=true -Dcheckstyle.skip=true -Drat.skip=true
diff --git a/build/build_java_test_image.sh b/build/build_java_test_image.sh
index 459bf26f98eff4..0747e6dacb82ae 100755
--- a/build/build_java_test_image.sh
+++ b/build/build_java_test_image.sh
@@ -27,6 +27,5 @@ if [[ "$(docker version -f '{{.Server.Experimental}}' 2>/dev/null)" == "true" ]]
SQUASH_PARAM="-Ddocker.squash=true"
fi
mvn -am -pl tests/docker-images/java-test-image -Pcore-modules,-main,integrationTests,docker \
- -DUBUNTU_MIRROR="${UBUNTU_MIRROR}" -DUBUNTU_SECURITY_MIRROR="${UBUNTU_SECURITY_MIRROR}" \
-Dmaven.test.skip=true -DskipSourceReleaseAssembly=true -Dspotbugs.skip=true -Dlicense.skip=true $SQUASH_PARAM \
"$@" install
\ No newline at end of file
diff --git a/build/docker/Dockerfile b/build/docker/Dockerfile
deleted file mode 100644
index ed787f7b85ae89..00000000000000
--- a/build/docker/Dockerfile
+++ /dev/null
@@ -1,100 +0,0 @@
-#
-# 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 ubuntu:22.04
-
-ARG JDK_MAJOR_VERSION=17
-
-# prepare the directory for pulsar related files
-RUN mkdir /pulsar
-
-RUN apt-get update && \
- apt-get install -y software-properties-common && \
- apt-get update && \
- apt-get install -y tig g++ cmake libssl-dev libcurl4-openssl-dev \
- liblog4cxx-dev google-mock libgtest-dev \
- libboost-dev libboost-program-options-dev libboost-system-dev libboost-python-dev \
- libxml2-utils wget apt-transport-https \
- curl doxygen clang-format \
- gnupg2 golang-go zip unzip libzstd-dev libsnappy-dev wireshark-dev
-
-# Install Eclipse Temurin Package
-RUN mkdir -p /etc/apt/keyrings \
- && wget -O - https://packages.adoptium.net/artifactory/api/gpg/key/public | tee /etc/apt/keyrings/adoptium.asc \
- && echo "deb [signed-by=/etc/apt/keyrings/adoptium.asc] https://packages.adoptium.net/artifactory/deb $(awk -F= '/^VERSION_CODENAME/{print$2}' /etc/os-release) main" | tee /etc/apt/sources.list.d/adoptium.list \
- && apt-get update \
- && apt-get -y dist-upgrade \
- && apt-get -y install temurin-${JDK_MAJOR_VERSION:-17}-jdk
-
-# Compile and install gtest & gmock
-RUN cd /usr/src/googletest && \
- cmake . && \
- make && \
- make install
-
-# Include gtest parallel to speed up unit tests
-RUN git clone https://github.com/google/gtest-parallel.git
-
-# Build protobuf 3.x.y from source since the default protobuf from Ubuntu's apt source is 2.x.y
-RUN curl -O -L https://github.com/protocolbuffers/protobuf/releases/download/v3.17.3/protobuf-cpp-3.17.3.tar.gz && \
- tar xvfz protobuf-cpp-3.17.3.tar.gz && \
- cd protobuf-3.17.3/ && \
- CXXFLAGS=-fPIC ./configure && \
- make -j8 && make install && \
- cd .. && rm -rf protobuf-3.17.3/ protobuf-cpp-3.17.3.tar.gz
-ENV LD_LIBRARY_PATH /usr/local/lib
-
-## Website build dependencies
-
-# Install Ruby-2.4.1
-RUN (curl -sSL https://rvm.io/mpapis.asc | gpg --import -) && \
- (curl -sSL https://rvm.io/pkuczynski.asc | gpg --import -) && \
- (curl -sSL https://get.rvm.io | bash -s stable)
-ENV PATH "$PATH:/usr/local/rvm/bin"
-RUN rvm install 2.4.1
-
-# Install nodejs and yarn
-RUN curl -sL https://deb.nodesource.com/setup_12.x | bash -
-RUN apt-get install -y nodejs
-RUN curl -sS https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add -
-RUN echo "deb https://dl.yarnpkg.com/debian/ stable main" | tee /etc/apt/sources.list.d/yarn.list
-RUN apt-get update && apt-get install yarn
-
-# Install crowdin
-RUN wget https://artifacts.crowdin.com/repo/deb/crowdin.deb -O crowdin.deb
-RUN dpkg -i crowdin.deb
-
-# Install PIP
-RUN curl https://bootstrap.pypa.io/get-pip.py | python3 -
-RUN pip3 --no-cache-dir install pdoc
-#
-# Installation
-ARG MAVEN_VERSION=3.6.3
-ARG MAVEN_FILENAME="apache-maven-${MAVEN_VERSION}-bin.tar.gz"
-ARG MAVEN_HOME=/opt/maven
-ARG MAVEN_URL="http://archive.apache.org/dist/maven/maven-3/${MAVEN_VERSION}/binaries/${MAVEN_FILENAME}"
-ARG MAVEN_TMP="/tmp/${MAVEN_FILENAME}"
-RUN wget --no-verbose -O ${MAVEN_TMP} ${MAVEN_URL}
-
-# Cleanup
-RUN tar xzf ${MAVEN_TMP} -C /opt/ \
- && ln -s /opt/apache-maven-${MAVEN_VERSION} ${MAVEN_HOME} \
- && ln -s ${MAVEN_HOME}/bin/mvn /usr/local/bin
-
-RUN unset MAVEN_VERSION
diff --git a/build/docker/README.md b/build/docker/README.md
deleted file mode 100644
index bc2fe532f74e05..00000000000000
--- a/build/docker/README.md
+++ /dev/null
@@ -1,48 +0,0 @@
-
-
-This folder contains a Docker image that can used to compile the Pulsar C++ client library
-and website in a reproducible environment.
-
-```shell
-docker build -t pulsar-build .
-```
-
-The image is already available at https://hub.docker.com/r/apachepulsar/pulsar-build
-
-Example: `apachepulsar/pulsar-build:ubuntu-16.04`
-
-## Build and Publish pulsar-build image
-
-> Only committers have permissions on publishing pulsar images to `apachepulsar` docker hub.
-
-### Build pulsar-build image
-
-
-```shell
-docker build -t apachepulsar/pulsar-build:ubuntu-16.04 .
-```
-
-### Publish pulsar-build image
-
-```shell
-publish.sh
-```
diff --git a/build/docker/publish.sh b/build/docker/publish.sh
deleted file mode 100755
index 6bfa56bace6d86..00000000000000
--- a/build/docker/publish.sh
+++ /dev/null
@@ -1,57 +0,0 @@
-#!/bin/bash
-#
-# 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.
-#
-
-ROOT_DIR=$(git rev-parse --show-toplevel)
-cd $ROOT_DIR/build/docker
-
-if [ -z "$DOCKER_USER" ]; then
- echo "Docker user in variable \$DOCKER_USER was not set. Skipping image publishing"
- exit 1
-fi
-
-if [ -z "$DOCKER_PASSWORD" ]; then
- echo "Docker password in variable \$DOCKER_PASSWORD was not set. Skipping image publishing"
- exit 1
-fi
-
-DOCKER_ORG="${DOCKER_ORG:-apachepulsar}"
-
-docker login ${DOCKER_REGISTRY} -u="$DOCKER_USER" -p="$DOCKER_PASSWORD"
-if [ $? -ne 0 ]; then
- echo "Failed to loging to Docker Hub"
- exit 1
-fi
-
-if [[ -z ${DOCKER_REGISTRY} ]]; then
- docker_registry_org=${DOCKER_ORG}
-else
- docker_registry_org=${DOCKER_REGISTRY}/${DOCKER_ORG}
- echo "Starting to push images to ${docker_registry_org}..."
-fi
-
-set -x
-
-# Fail if any of the subsequent commands fail
-set -e
-
-# Push all images and tags
-docker push ${docker_registry_org}/pulsar-build:ubuntu-16.04
-
-echo "Finished pushing images to ${docker_registry_org}"
diff --git a/build/pulsar_ci_tool.sh b/build/pulsar_ci_tool.sh
index ae33c3e91667be..034b2ce60cf37d 100755
--- a/build/pulsar_ci_tool.sh
+++ b/build/pulsar_ci_tool.sh
@@ -46,8 +46,7 @@ function ci_print_thread_dumps() {
# runs maven
function _ci_mvn() {
- mvn -B -ntp -DUBUNTU_MIRROR="${UBUNTU_MIRROR}" -DUBUNTU_SECURITY_MIRROR="${UBUNTU_SECURITY_MIRROR}" \
- "$@"
+ mvn -B -ntp "$@"
}
# runs OWASP Dependency Check for all projects
@@ -55,50 +54,6 @@ function ci_dependency_check() {
_ci_mvn -Pmain,skip-all,skipDocker,owasp-dependency-check initialize verify -pl '!pulsar-client-tools-test' "$@"
}
-# Finds fastest up-to-date ubuntu mirror based on download speed
-function ci_find_fast_ubuntu_mirror() {
- local ubuntu_release=${1:-"$(lsb_release -c 2>/dev/null | cut -f2 || echo "jammy")"}
- local ubuntu_arch=${2:-"$(dpkg --print-architecture 2>/dev/null || echo "amd64")"}
- {
- # choose mirrors that are up-to-date by checking the Last-Modified header for
- {
- # randomly choose up to 10 mirrors using http:// protocol
- # (https isn't supported in docker containers that don't have ca-certificates installed)
- curl -s http://mirrors.ubuntu.com/mirrors.txt | grep '^http://' | shuf -n 10
- # also consider Azure's Ubuntu mirror
- echo http://azure.archive.ubuntu.com/ubuntu/
- } | xargs -I {} sh -c "ubuntu_release=$ubuntu_release ubuntu_arch=$ubuntu_arch;"'echo "$(curl -m 5 -sI {}dists/${ubuntu_release}/Contents-${ubuntu_arch}.gz|sed s/\\r\$//|grep Last-Modified|awk -F": " "{ print \$2 }" | LANG=C date -f- -u +%s)" "{}"' | sort -rg | awk '{ if (NR==1) TS=$1; if ($1 == TS) print $2 }'
- } | xargs -I {} sh -c 'echo `curl -r 0-102400 -m 5 -s -w %{speed_download} -o /dev/null {}ls-lR.gz` {}' \
- |sort -g -r |head -1| awk '{ print $2 }'
-}
-
-function ci_pick_ubuntu_mirror() {
- echo "Choosing fastest up-to-date ubuntu mirror based on download speed..."
- UBUNTU_MIRROR=$(ci_find_fast_ubuntu_mirror)
- if [ -z "$UBUNTU_MIRROR" ]; then
- # fallback to no mirror
- UBUNTU_MIRROR="http://archive.ubuntu.com/ubuntu/"
- UBUNTU_SECURITY_MIRROR="http://security.ubuntu.com/ubuntu/"
- else
- UBUNTU_SECURITY_MIRROR="${UBUNTU_MIRROR}"
- fi
- OLD_MIRROR=$(cat /etc/apt/sources.list | grep '^deb ' | head -1 | awk '{ print $2 }')
- echo "Picked '$UBUNTU_MIRROR'. Current mirror is '$OLD_MIRROR'."
- if [[ "$OLD_MIRROR" != "$UBUNTU_MIRROR" ]]; then
- sudo sed -i "s|$OLD_MIRROR|$UBUNTU_MIRROR|g" /etc/apt/sources.list
- sudo apt-get update
- fi
- # set the chosen mirror also in the UBUNTU_MIRROR and UBUNTU_SECURITY_MIRROR environment variables
- # that can be used by docker builds
- export UBUNTU_MIRROR
- export UBUNTU_SECURITY_MIRROR
- # make environment variables available for later GitHub Actions steps
- if [ -n "$GITHUB_ENV" ]; then
- echo "UBUNTU_MIRROR=$UBUNTU_MIRROR" >> $GITHUB_ENV
- echo "UBUNTU_SECURITY_MIRROR=$UBUNTU_SECURITY_MIRROR" >> $GITHUB_ENV
- fi
-}
-
# installs a tool executable if it's not found on the PATH
function ci_install_tool() {
local tool_executable=$1
@@ -108,7 +63,6 @@ function ci_install_tool() {
echo "::group::Installing ${tool_package}"
sudo apt-get -y install ${tool_package} >/dev/null || {
echo "Installing the package failed. Switching the ubuntu mirror and retrying..."
- ci_pick_ubuntu_mirror
# retry after picking the ubuntu mirror
sudo apt-get -y install ${tool_package}
}
diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile
index 4e5885ce55d17c..da0466db05ea3c 100644
--- a/docker/pulsar/Dockerfile
+++ b/docker/pulsar/Dockerfile
@@ -32,76 +32,64 @@ COPY scripts/gen-yml-from-env.py /pulsar/bin
COPY scripts/generate-zookeeper-config.sh /pulsar/bin
COPY scripts/pulsar-zookeeper-ruok.sh /pulsar/bin
COPY scripts/watch-znode.py /pulsar/bin
-COPY scripts/install-pulsar-client.sh /pulsar/bin
# The final image needs to give the root group sufficient permission for Pulsar components
# to write to specific directories within /pulsar
# The file permissions are preserved when copying files from this builder image to the target image.
-RUN for SUBDIRECTORY in conf data download logs; do \
+RUN for SUBDIRECTORY in conf data download logs trino; do \
[ -d /pulsar/$SUBDIRECTORY ] || mkdir /pulsar/$SUBDIRECTORY; \
chmod -R g+w /pulsar/$SUBDIRECTORY; \
done
-### Create 2nd stage from Ubuntu image
-### and add OpenJDK and Python dependencies (for Pulsar functions)
+RUN chmod -R o+rx /pulsar
-FROM ubuntu:22.04
+## Create 2nd stage to build the Python dependencies
+## Since it needs to have GCC available, we're doing it in a different layer
+FROM alpine:3.19 AS python-deps
-ARG DEBIAN_FRONTEND=noninteractive
-ARG UBUNTU_MIRROR=http://archive.ubuntu.com/ubuntu/
-ARG UBUNTU_SECURITY_MIRROR=http://security.ubuntu.com/ubuntu/
-ARG DEFAULT_USERNAME=pulsar
-ARG JDK_MAJOR_VERSION=17
+RUN apk add --no-cache \
+ bash \
+ python3-dev \
+ py3-pip \
+ g++ \
+ musl-dev \
+ libffi-dev \
+ py3-grpcio
+
+RUN pip3 install --break-system-packages \
+ kazoo
+
+ARG PULSAR_CLIENT_PYTHON_VERSION
+RUN pip3 install --break-system-packages \
+ pulsar-client==${PULSAR_CLIENT_PYTHON_VERSION}
+
+
+## Create final stage from Alpine image
+## and add OpenJDK and Python dependencies (for Pulsar functions)
+FROM amazoncorretto:21-alpine
# Install some utilities
-RUN sed -i -e "s|http://archive\.ubuntu\.com/ubuntu/|${UBUNTU_MIRROR:-http://archive.ubuntu.com/ubuntu/}|g" \
- -e "s|http://security\.ubuntu\.com/ubuntu/|${UBUNTU_SECURITY_MIRROR:-http://security.ubuntu.com/ubuntu/}|g" /etc/apt/sources.list \
- && echo 'Acquire::http::Timeout "30";\nAcquire::http::ConnectionAttemptDelayMsec "2000";\nAcquire::https::Timeout "30";\nAcquire::https::ConnectionAttemptDelayMsec "2000";\nAcquire::ftp::Timeout "30";\nAcquire::ftp::ConnectionAttemptDelayMsec "2000";\nAcquire::Retries "15";' > /etc/apt/apt.conf.d/99timeout_and_retries \
- && apt-get update \
- && apt-get -y dist-upgrade \
- && apt-get -y install netcat dnsutils less procps iputils-ping \
- curl ca-certificates wget apt-transport-https \
- && apt-get -y install --no-install-recommends python3 python3-kazoo python3-pip
-
-# Install Eclipse Temurin Package
-RUN mkdir -p /etc/apt/keyrings \
- && wget -O - https://packages.adoptium.net/artifactory/api/gpg/key/public | tee /etc/apt/keyrings/adoptium.asc \
- && echo "deb [signed-by=/etc/apt/keyrings/adoptium.asc] https://packages.adoptium.net/artifactory/deb $(awk -F= '/^VERSION_CODENAME/{print$2}' /etc/os-release) main" | tee /etc/apt/sources.list.d/adoptium.list \
- && apt-get update \
- && apt-get -y dist-upgrade \
- && apt-get -y install temurin-${JDK_MAJOR_VERSION:-17}-jdk \
- && export ARCH=$(uname -m | sed -r 's/aarch64/arm64/g' | awk '!/arm64/{$0="amd64"}1') \
- && echo networkaddress.cache.ttl=1 >> /usr/lib/jvm/temurin-${JDK_MAJOR_VERSION:-17}-jdk-$ARCH/conf/security/java.security \
- && echo networkaddress.cache.negative.ttl=1 >> /usr/lib/jvm/temurin-${JDK_MAJOR_VERSION:-17}-jdk-$ARCH/conf/security/java.security \
-
-# Cleanup apt
-RUN apt-get -y --purge autoremove \
- && apt-get autoclean \
- && apt-get clean \
- && rm -rf /var/lib/apt/lists/*
-
-RUN pip3 install pyyaml==6.0.1
-
-# Pulsar currently writes to the below directories, assuming the default configuration.
-# Note that number 4 is the reason that pulsar components need write access to the /pulsar directory.
-# 1. /pulsar/data - both bookkeepers and zookeepers use this directory
-# 2. /pulsar/logs - function workers write to this directory and pulsar-admin initializes this directory
-# 3. /pulsar/download - functions write to this directory
-# 4. /pulsar - hadoop writes to this directory
-RUN mkdir /pulsar && chmod g+w /pulsar
+RUN apk add --no-cache bash
+RUN apk add --no-cache python3
+RUN apk add --no-cache py3-pip
+RUN apk add --no-cache curl
+RUN apk add --no-cache bind-tools
+RUN apk add --no-cache ca-certificates
+
+ENV JAVA_HOME /usr/lib/jvm/default-jvm
+RUN echo networkaddress.cache.ttl=1 >> /usr/lib/jvm/default-jvm/conf/security/java.security
+
+# Copy Python depedencies from the other stage
+COPY --from=python-deps /usr/lib/python3.11/site-packages /usr/lib/python3.11/site-packages
ENV PULSAR_ROOT_LOGGER=INFO,CONSOLE
COPY --from=pulsar /pulsar /pulsar
-WORKDIR /pulsar
-
-ARG PULSAR_CLIENT_PYTHON_VERSION
-ENV PULSAR_CLIENT_PYTHON_VERSION ${PULSAR_CLIENT_PYTHON_VERSION}
-# This script is intentionally run as the root user to make the dependencies available for all UIDs.
-RUN chmod +x /pulsar/bin/install-pulsar-client.sh
-RUN /pulsar/bin/install-pulsar-client.sh
+WORKDIR /pulsar
+ENV PATH=$PATH:/pulsar/bin
# The UID must be non-zero. Otherwise, it is arbitrary. No logic should rely on its specific value.
-RUN useradd ${DEFAULT_USERNAME} -u 10000 -g 0
-USER 10000
+RUN adduser ${DEFAULT_USERNAME} -u 10000 -G root -D
+USER ${DEFAULT_USERNAME}
+
diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml
index 35cff16ac2c640..79ff4bd33b10c1 100644
--- a/docker/pulsar/pom.xml
+++ b/docker/pulsar/pom.xml
@@ -47,12 +47,6 @@
-
- http://archive.ubuntu.com/ubuntu/
- http://security.ubuntu.com/ubuntu/
- 17
-
-
git-commit-id-no-git
@@ -88,9 +82,6 @@
target/pulsar-server-distribution-${project.version}-bin.tar.gz
${pulsar.client.python.version}
- ${UBUNTU_MIRROR}
- ${UBUNTU_SECURITY_MIRROR}
- ${IMAGE_JDK_MAJOR_VERSION}
${project.basedir}
diff --git a/docker/pulsar/scripts/install-pulsar-client.sh b/docker/pulsar/scripts/install-pulsar-client.sh
deleted file mode 100755
index 0951b2aec1b606..00000000000000
--- a/docker/pulsar/scripts/install-pulsar-client.sh
+++ /dev/null
@@ -1,30 +0,0 @@
-#!/usr/bin/env bash
-#
-# 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.
-#
-
-set -x
-
-# TODO: remove these lines once grpcio doesn't need to compile from source on ARM64 platform
-ARCH=$(uname -m | sed -r 's/aarch64/arm64/g' | awk '!/arm64/{$0="amd64"}1')
-if [ "${ARCH}" == "arm64" ]; then
- apt update
- apt -y install build-essential python3-dev
-fi
-
-pip3 install pulsar-client[all]==${PULSAR_CLIENT_PYTHON_VERSION}
diff --git a/tests/docker-images/java-test-image/Dockerfile b/tests/docker-images/java-test-image/Dockerfile
index 5a1bbf15e93b54..805f20a0570db3 100644
--- a/tests/docker-images/java-test-image/Dockerfile
+++ b/tests/docker-images/java-test-image/Dockerfile
@@ -17,51 +17,16 @@
# under the License.
#
-FROM ubuntu:22.04
+ARG PULSAR_IMAGE
+FROM $PULSAR_IMAGE
-RUN groupadd -g 10001 pulsar
-RUN adduser -u 10000 --gid 10001 --disabled-login --disabled-password --gecos '' pulsar
-
-ARG PULSAR_TARBALL=target/pulsar-server-distribution-bin.tar.gz
-ADD ${PULSAR_TARBALL} /
-RUN mv /apache-pulsar-* /pulsar
-RUN chown -R root:root /pulsar
+# Base pulsar image is designed not be modified, though we need to add more scripts
+USER root
COPY target/scripts /pulsar/bin
RUN chmod a+rx /pulsar/bin/*
-WORKDIR /pulsar
-
-ARG DEBIAN_FRONTEND=noninteractive
-ARG UBUNTU_MIRROR=http://archive.ubuntu.com/ubuntu/
-ARG UBUNTU_SECURITY_MIRROR=http://security.ubuntu.com/ubuntu/
-ARG JDK_MAJOR_VERSION=17
-
-RUN sed -i -e "s|http://archive\.ubuntu\.com/ubuntu/|${UBUNTU_MIRROR:-http://archive.ubuntu.com/ubuntu/}|g" \
- -e "s|http://security\.ubuntu\.com/ubuntu/|${UBUNTU_SECURITY_MIRROR:-http://security.ubuntu.com/ubuntu/}|g" /etc/apt/sources.list \
- && echo 'Acquire::http::Timeout "30";\nAcquire::http::ConnectionAttemptDelayMsec "2000";\nAcquire::https::Timeout "30";\nAcquire::https::ConnectionAttemptDelayMsec "2000";\nAcquire::ftp::Timeout "30";\nAcquire::ftp::ConnectionAttemptDelayMsec "2000";\nAcquire::Retries "15";' > /etc/apt/apt.conf.d/99timeout_and_retries \
- && apt-get update \
- && apt-get -y dist-upgrade \
- && apt-get -y install ca-certificates wget apt-transport-https
-
-# Install Eclipse Temurin Package
-RUN mkdir -p /etc/apt/keyrings \
- && wget -O - https://packages.adoptium.net/artifactory/api/gpg/key/public | tee /etc/apt/keyrings/adoptium.asc \
- && echo "deb [signed-by=/etc/apt/keyrings/adoptium.asc] https://packages.adoptium.net/artifactory/deb $(awk -F= '/^VERSION_CODENAME/{print$2}' /etc/os-release) main" | tee /etc/apt/sources.list.d/adoptium.list \
- && apt-get update \
- && apt-get -y dist-upgrade \
- && apt-get -y install temurin-${JDK_MAJOR_VERSION:-17}-jdk \
- && export ARCH=$(uname -m | sed -r 's/aarch64/arm64/g' | awk '!/arm64/{$0="amd64"}1') \
- && echo networkaddress.cache.ttl=1 >> /usr/lib/jvm/temurin-${JDK_MAJOR_VERSION:-17}-jdk-$ARCH/conf/security/java.security
-
-# /pulsar/bin/watch-znode.py requires python3-kazoo
-# /pulsar/bin/pulsar-managed-ledger-admin requires python3-protobuf
-# gen-yml-from-env.py requires python3-yaml
-# make python3 the default
-RUN apt-get install -y python3-kazoo python3-protobuf python3-yaml \
- && update-alternatives --install /usr/bin/python python /usr/bin/python3 10
-
-RUN apt-get install -y supervisor procps curl less netcat dnsutils iputils-ping
+RUN apk add --no-cache supervisor
RUN mkdir -p /var/log/pulsar \
&& mkdir -p /var/run/supervisor/ \
@@ -74,13 +39,3 @@ RUN mv /etc/supervisord/conf.d/supervisord.conf /etc/supervisord.conf
COPY target/certificate-authority /pulsar/certificate-authority/
COPY target/java-test-functions.jar /pulsar/examples/
-
-ENV PULSAR_ROOT_LOGGER=INFO,CONSOLE
-
-RUN chown -R pulsar:0 /pulsar && chmod -R g=u /pulsar
-
-# cleanup
-RUN apt-get -y --purge autoremove \
- && apt-get autoclean \
- && apt-get clean \
- && rm -rf /var/lib/apt/lists/*
diff --git a/tests/docker-images/java-test-image/pom.xml b/tests/docker-images/java-test-image/pom.xml
index 8e27d4ed1101fa..61d8c9668e5f46 100644
--- a/tests/docker-images/java-test-image/pom.xml
+++ b/tests/docker-images/java-test-image/pom.xml
@@ -33,12 +33,6 @@
docker
-
- target/pulsar-server-distribution-bin.tar.gz
- ${env.UBUNTU_MIRROR}
- ${env.UBUNTU_SECURITY_MIRROR}
- ${env.IMAGE_JDK_MAJOR_VERSION}
-
integrationTests
@@ -149,12 +143,16 @@
package
build
+ tag
${docker.organization}/java-test-image
+
+ ${docker.organization}/pulsar:${project.version}-${git.commit.id.abbrev}
+
${project.basedir}
latest
diff --git a/tests/docker-images/latest-version-image/Dockerfile b/tests/docker-images/latest-version-image/Dockerfile
index 4973bec0441b59..41c700f162e8c3 100644
--- a/tests/docker-images/latest-version-image/Dockerfile
+++ b/tests/docker-images/latest-version-image/Dockerfile
@@ -40,9 +40,7 @@ FROM apachepulsar/pulsar:latest
# However, any processes exec'ing into the containers will run as root, by default.
USER root
-RUN rm -rf /var/lib/apt/lists/* && apt update
-
-RUN apt-get clean && apt-get update && apt-get install -y supervisor vim procps curl
+RUN apk add --no-cache supervisor
RUN mkdir -p /var/log/pulsar && mkdir -p /var/run/supervisor/
@@ -93,15 +91,15 @@ COPY --from=pulsar-all /pulsar/connectors/pulsar-io-kinesis-*.nar /pulsar/connec
# download Oracle JDBC driver for Oracle Debezium Connector tests
RUN mkdir -p META-INF/bundled-dependencies
-RUN cd META-INF/bundled-dependencies && curl -sSLO https://search.maven.org/remotecontent?filepath=com/oracle/ojdbc/ojdbc8/19.3.0.0/ojdbc8-19.3.0.0.jar
-RUN cd META-INF/bundled-dependencies && curl -sSLO https://search.maven.org/remotecontent?filepath=com/oracle/ojdbc/ucp/19.3.0.0/ucp-19.3.0.0.jar
-RUN cd META-INF/bundled-dependencies && curl -sSLO https://search.maven.org/remotecontent?filepath=com/oracle/ojdbc/oraclepki/19.3.0.0/oraclepki-19.3.0.0.jar
-RUN cd META-INF/bundled-dependencies && curl -sSLO https://search.maven.org/remotecontent?filepath=com/oracle/ojdbc/osdt_cert/19.3.0.0/osdt_cert-19.3.0.0.jar
-RUN cd META-INF/bundled-dependencies && curl -sSLO https://search.maven.org/remotecontent?filepath=com/oracle/ojdbc/osdt_core/19.3.0.0/osdt_core-19.3.0.0.jar
-RUN cd META-INF/bundled-dependencies && curl -sSLO https://search.maven.org/remotecontent?filepath=com/oracle/ojdbc/simplefan/19.3.0.0/simplefan-19.3.0.0.jar
-RUN cd META-INF/bundled-dependencies && curl -sSLO https://search.maven.org/remotecontent?filepath=com/oracle/ojdbc/orai18n/19.3.0.0/orai18n-19.3.0.0.jar
-RUN cd META-INF/bundled-dependencies && curl -sSLO https://search.maven.org/remotecontent?filepath=com/oracle/ojdbc/xdb/19.3.0.0/xdb-19.3.0.0.jar
-RUN cd META-INF/bundled-dependencies && curl -sSLO https://search.maven.org/remotecontent?filepath=com/oracle/ojdbc/xmlparserv2/19.3.0.0/xmlparserv2-19.3.0.0.jar
+RUN cd META-INF/bundled-dependencies && curl -sSL https://search.maven.org/remotecontent?filepath=com/oracle/ojdbc/ojdbc8/19.3.0.0/ojdbc8-19.3.0.0.jar -o ojdbc8-19.3.0.0.jar
+RUN cd META-INF/bundled-dependencies && curl -sSL https://search.maven.org/remotecontent?filepath=com/oracle/ojdbc/ucp/19.3.0.0/ucp-19.3.0.0.jar -o ucp-19.3.0.0.jar
+RUN cd META-INF/bundled-dependencies && curl -sSL https://search.maven.org/remotecontent?filepath=com/oracle/ojdbc/oraclepki/19.3.0.0/oraclepki-19.3.0.0.jar -o oraclepki-19.3.0.0.jar
+RUN cd META-INF/bundled-dependencies && curl -sSL https://search.maven.org/remotecontent?filepath=com/oracle/ojdbc/osdt_cert/19.3.0.0/osdt_cert-19.3.0.0.jar -o osdt_cert-19.3.0.0.jar
+RUN cd META-INF/bundled-dependencies && curl -sSL https://search.maven.org/remotecontent?filepath=com/oracle/ojdbc/osdt_core/19.3.0.0/osdt_core-19.3.0.0.jar -o osdt_core-19.3.0.0.jar
+RUN cd META-INF/bundled-dependencies && curl -sSL https://search.maven.org/remotecontent?filepath=com/oracle/ojdbc/simplefan/19.3.0.0/simplefan-19.3.0.0.jar -o simplefan-19.3.0.0.jar
+RUN cd META-INF/bundled-dependencies && curl -sSL https://search.maven.org/remotecontent?filepath=com/oracle/ojdbc/orai18n/19.3.0.0/orai18n-19.3.0.0.jar -o orai18n-19.3.0.0.jar
+RUN cd META-INF/bundled-dependencies && curl -sSL https://search.maven.org/remotecontent?filepath=com/oracle/ojdbc/xdb/19.3.0.0/xdb-19.3.0.0.jar -o xdb-19.3.0.0.jar
+RUN cd META-INF/bundled-dependencies && curl -sSL https://search.maven.org/remotecontent?filepath=com/oracle/ojdbc/xmlparserv2/19.3.0.0/xmlparserv2-19.3.0.0.jar -o xmlparserv2-19.3.0.0.jar
RUN jar uf connectors/pulsar-io-debezium-oracle-*.nar META-INF/bundled-dependencies/ojdbc8-19.3.0.0.jar META-INF/bundled-dependencies/ucp-19.3.0.0.jar META-INF/bundled-dependencies/oraclepki-19.3.0.0.jar META-INF/bundled-dependencies/osdt_cert-19.3.0.0.jar META-INF/bundled-dependencies/osdt_core-19.3.0.0.jar META-INF/bundled-dependencies/simplefan-19.3.0.0.jar META-INF/bundled-dependencies/orai18n-19.3.0.0.jar META-INF/bundled-dependencies/xdb-19.3.0.0.jar META-INF/bundled-dependencies/xmlparserv2-19.3.0.0.jar
diff --git a/tests/docker-images/pom.xml b/tests/docker-images/pom.xml
index 1a0eeb1424b8a8..770b490cdd390d 100644
--- a/tests/docker-images/pom.xml
+++ b/tests/docker-images/pom.xml
@@ -52,6 +52,20 @@
latest-version-image
java-test-image
+
+
+
+ pl.project13.maven
+ git-commit-id-plugin
+
+ false
+ true
+ true
+ false
+
+
+
+