This is an automated email from the ASF dual-hosted git repository.

chia7712 pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 767a62ade60 KAFKA-18737 KafkaDockerWrapper setup functions fails due 
to storage format command (#18844)
767a62ade60 is described below

commit 767a62ade605a1c133d35f676265e7745c99e501
Author: TengYao Chi <[email protected]>
AuthorDate: Fri Feb 21 20:43:41 2025 +0800

    KAFKA-18737 KafkaDockerWrapper setup functions fails due to storage format 
command (#18844)
    
    The current Docker Hub documentation for Kafka is based on the use of 
static voters. Since Kafka 4.0 utilizes dynamic voters, users following the doc 
of docker hub may encounter unexpected behavior. Due to the limited time 
available for the 4.0.0 release, a simple and quick solution is to revert to 
using static voters within the Docker image. This can be achieved by adding a 
configuration file with static voter definitions to the kafka/docker folder, 
keeping it separate from the main  [...]
    
    Reviewers: Vedarth Sharma 
<[email protected]>, Chia-Ping Tsai 
<[email protected]>
---
 .../scala/kafka/docker/KafkaDockerWrapper.scala    |  17 ++-
 core/src/main/scala/kafka/tools/TerseFailure.scala |   4 +-
 .../unit/kafka/docker/KafkaDockerWrapperTest.scala |   2 +-
 docker/common.py                                   |   3 +
 docker/jvm/Dockerfile                              |   3 +-
 docker/native/Dockerfile                           |   2 +-
 docker/server.properties                           | 138 +++++++++++++++++++++
 7 files changed, 162 insertions(+), 7 deletions(-)

diff --git a/core/src/main/scala/kafka/docker/KafkaDockerWrapper.scala 
b/core/src/main/scala/kafka/docker/KafkaDockerWrapper.scala
index 00c040488af..7cbcae5e9bb 100644
--- a/core/src/main/scala/kafka/docker/KafkaDockerWrapper.scala
+++ b/core/src/main/scala/kafka/docker/KafkaDockerWrapper.scala
@@ -22,12 +22,13 @@ import 
com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper}
 import com.fasterxml.jackson.dataformat.yaml.YAMLFactory
 import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator.Feature
 import kafka.Kafka
-import kafka.tools.StorageTool
+import kafka.tools.{StorageTool, TerseFailure}
 import kafka.utils.Logging
 import net.sourceforge.argparse4j.ArgumentParsers
 import net.sourceforge.argparse4j.impl.Arguments.store
 import net.sourceforge.argparse4j.inf.Namespace
 import org.apache.kafka.common.utils.Exit
+import org.apache.kafka.raft.QuorumConfig
 
 import java.nio.charset.StandardCharsets
 import java.nio.file.{Files, Path, Paths, StandardCopyOption, 
StandardOpenOption}
@@ -52,7 +53,15 @@ object KafkaDockerWrapper extends Logging {
         }
 
         val formatCmd = formatStorageCmd(finalConfigsPath, envVars)
-        StorageTool.main(formatCmd)
+        try {
+          StorageTool.main(formatCmd)
+        } catch {
+          case terseFailure: TerseFailure => if 
(terseFailure.getMessage.contains(QuorumConfig.QUORUM_VOTERS_CONFIG)) {
+            throw new TerseFailure("To maximize compatibility, the Docker 
image continues to use static voters, " +
+              "which are supported in 3.7 and later.", terseFailure)
+          } else throw terseFailure
+          case e: Throwable => throw e
+        }
       case "start" =>
         val configFile = namespace.getString("config")
         info("Starting Kafka server in the native mode.")
@@ -110,7 +119,9 @@ object KafkaDockerWrapper extends Logging {
       case Some(str) => str
       case None => throw new RuntimeException("CLUSTER_ID environment variable 
is not set.")
     }
-    Array("format", "--cluster-id=" + clusterId, "-c", 
s"${configsPath.toString}/server.properties", "--standalone")
+    // We maintain static voter configurations in Docker Hub images for better 
version compatibility and deployment stability,
+    // despite having dynamic voter support in the latest release.
+    Array("format", "--cluster-id=" + clusterId, "-c", 
s"${configsPath.toString}/server.properties")
   }
 
   private def prepareConfigs(defaultConfigsPath: Path, mountedConfigsPath: 
Path, finalConfigsPath: Path): Unit = {
diff --git a/core/src/main/scala/kafka/tools/TerseFailure.scala 
b/core/src/main/scala/kafka/tools/TerseFailure.scala
index c37b613d71f..2c4e41bebe3 100644
--- a/core/src/main/scala/kafka/tools/TerseFailure.scala
+++ b/core/src/main/scala/kafka/tools/TerseFailure.scala
@@ -25,6 +25,8 @@ import org.apache.kafka.common.KafkaException
  *
  * @param message     The message to print out before exiting.  A stack trace 
will not
  *                    be printed.
+ * @param cause       The exception's cause
  */
-class TerseFailure(message: String) extends KafkaException(message) {
+class TerseFailure(message: String, cause: Throwable) extends 
KafkaException(message, cause) {
+  def this(message: String) = this(message, null)
 }
diff --git a/core/src/test/scala/unit/kafka/docker/KafkaDockerWrapperTest.scala 
b/core/src/test/scala/unit/kafka/docker/KafkaDockerWrapperTest.scala
index 6940941a931..81dcad01a99 100644
--- a/core/src/test/scala/unit/kafka/docker/KafkaDockerWrapperTest.scala
+++ b/core/src/test/scala/unit/kafka/docker/KafkaDockerWrapperTest.scala
@@ -133,7 +133,7 @@ class KafkaDockerWrapperTest {
     val configsPath = Paths.get("/path/to/configs")
     val envVars = Map("CLUSTER_ID" -> "MYwKGPhXQZidgd0qMv8Mkw")
 
-    val expected = Array("format", "--cluster-id=MYwKGPhXQZidgd0qMv8Mkw", 
"-c", "/path/to/configs/server.properties", "--standalone")
+    val expected = Array("format", "--cluster-id=MYwKGPhXQZidgd0qMv8Mkw", 
"-c", "/path/to/configs/server.properties")
     val actual = KafkaDockerWrapper.formatStorageCmd(configsPath, envVars)
 
     assertArrayEquals(expected.toArray[Object], actual.toArray[Object])
diff --git a/docker/common.py b/docker/common.py
index aaab1eae1ff..9c0f901823f 100644
--- a/docker/common.py
+++ b/docker/common.py
@@ -18,6 +18,8 @@
 import subprocess
 import tempfile
 import os
+from distutils.file_util import copy_file
+
 from distutils.dir_util import copy_tree
 import shutil
 
@@ -36,6 +38,7 @@ def build_docker_image_runner(command, image_type):
     current_dir = os.path.dirname(os.path.realpath(__file__))
     copy_tree(f"{current_dir}/{image_type}", f"{temp_dir_path}/{image_type}")
     copy_tree(f"{current_dir}/resources", 
f"{temp_dir_path}/{image_type}/resources")
+    copy_file(f"{current_dir}/server.properties", 
f"{temp_dir_path}/{image_type}")
     command = command.replace("$DOCKER_FILE", 
f"{temp_dir_path}/{image_type}/Dockerfile")
     command = command.replace("$DOCKER_DIR", f"{temp_dir_path}/{image_type}")
     try:
diff --git a/docker/jvm/Dockerfile b/docker/jvm/Dockerfile
index e633237c873..f98f50a2e03 100644
--- a/docker/jvm/Dockerfile
+++ b/docker/jvm/Dockerfile
@@ -24,6 +24,7 @@ USER root
 ARG kafka_url
 
 COPY jsa_launch /etc/kafka/docker/jsa_launch
+COPY server.properties /etc/kafka/docker/server.properties
 
 RUN set -eux ; \
     apk update ; \
@@ -78,11 +79,11 @@ RUN set -eux ; \
     chmod -R ug+w /etc/kafka /var/lib/kafka /etc/kafka/secrets; \
     cp /opt/kafka/config/log4j2.yaml /etc/kafka/docker/log4j2.yaml; \
     cp /opt/kafka/config/tools-log4j2.yaml 
/etc/kafka/docker/tools-log4j2.yaml; \
-    cp /opt/kafka/config/server.properties 
/etc/kafka/docker/server.properties; \
     rm kafka.tgz kafka.tgz.asc KEYS; \
     apk del wget gpg gpg-agent; \
     apk cache clean;
 
+COPY server.properties /etc/kafka/docker/server.properties
 COPY --from=build-jsa kafka.jsa /opt/kafka/kafka.jsa
 COPY --from=build-jsa storage.jsa /opt/kafka/storage.jsa
 COPY --chown=appuser:appuser resources/common-scripts /etc/kafka/docker
diff --git a/docker/native/Dockerfile b/docker/native/Dockerfile
index 08db80781f2..ca85f35562d 100644
--- a/docker/native/Dockerfile
+++ b/docker/native/Dockerfile
@@ -62,8 +62,8 @@ RUN apk update ; \
     chown appuser:root -R /etc/kafka /opt/kafka /mnt/shared/config ; \
     chmod -R ug+w /etc/kafka /opt/kafka /mnt/shared/config ;
 
+COPY server.properties /etc/kafka/docker/server.properties
 COPY --chown=appuser:root --from=build-native-image /app/kafka/kafka.Kafka 
/opt/kafka/
-COPY --chown=appuser:root --from=build-native-image 
/app/kafka/config/server.properties /etc/kafka/docker/
 COPY --chown=appuser:root --from=build-native-image 
/app/kafka/config/log4j2.yaml /etc/kafka/docker/
 COPY --chown=appuser:root --from=build-native-image 
/app/kafka/config/tools-log4j2.yaml /etc/kafka/docker/
 COPY --chown=appuser:root resources/common-scripts /etc/kafka/docker/
diff --git a/docker/server.properties b/docker/server.properties
new file mode 100644
index 00000000000..eb0b445c344
--- /dev/null
+++ b/docker/server.properties
@@ -0,0 +1,138 @@
+# 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.
+
+################## Why We Need This Separate Config ####################
+
+# While our latest version supports dynamic voters configuration,
+# we will continue using static voter configurations in our Docker Hub images.
+# This decision ensures broader compatibility across different versions and
+# maintains consistent behavior for existing deployments.
+# By retaining static voter implementation in our Docker images, we can provide
+# a more stable and predictable environment for users across various versions 
of the application.
+
+############################# Server Basics #############################
+
+# The role of this server. Setting this puts us in KRaft mode
+process.roles=broker,controller
+
+# The node id associated with this instance's roles
+node.id=1
+
+# The connect string for the controller quorum
+controller.quorum.voters=1@localhost:9093
+
+############################# Socket Server Settings 
#############################
+
+# The address the socket server listens on.
+# Combined nodes (i.e. those with `process.roles=broker,controller`) must list 
the controller listener here at a minimum.
+# If the broker listener is not defined, the default listener will use a host 
name that is equal to the value of java.net.InetAddress.getCanonicalHostName(),
+# with PLAINTEXT listener name, and port 9092.
+#   FORMAT:
+#     listeners = listener_name://host_name:port
+#   EXAMPLE:
+#     listeners = PLAINTEXT://your.host.name:9092
+listeners=PLAINTEXT://:9092,CONTROLLER://:9093
+
+# Name of listener used for communication between brokers.
+inter.broker.listener.name=PLAINTEXT
+
+# Listener name, hostname and port the broker or the controller will advertise 
to clients.
+# If not set, it uses the value for "listeners".
+advertised.listeners=PLAINTEXT://localhost:9092
+
+# A comma-separated list of the names of the listeners used by the controller.
+# If no explicit mapping set in `listener.security.protocol.map`, default will 
be using PLAINTEXT protocol
+# This is required if running in KRaft mode.
+controller.listener.names=CONTROLLER
+
+# Maps listener names to security protocols, the default is for them to be the 
same. See the config documentation for more details
+listener.security.protocol.map=CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT,SSL:SSL,SASL_PLAINTEXT:SASL_PLAINTEXT,SASL_SSL:SASL_SSL
+
+# The number of threads that the server uses for receiving requests from the 
network and sending responses to the network
+num.network.threads=3
+
+# The number of threads that the server uses for processing requests, which 
may include disk I/O
+num.io.threads=8
+
+# The send buffer (SO_SNDBUF) used by the socket server
+socket.send.buffer.bytes=102400
+
+# The receive buffer (SO_RCVBUF) used by the socket server
+socket.receive.buffer.bytes=102400
+
+# The maximum size of a request that the socket server will accept (protection 
against OOM)
+socket.request.max.bytes=104857600
+
+
+############################# Log Basics #############################
+
+# A comma separated list of directories under which to store log files
+log.dirs=/tmp/kraft-combined-logs
+
+# The default number of log partitions per topic. More partitions allow greater
+# parallelism for consumption, but this will also result in more files across
+# the brokers.
+num.partitions=1
+
+# The number of threads per data directory to be used for log recovery at 
startup and flushing at shutdown.
+# This value is recommended to be increased for installations with data dirs 
located in RAID array.
+num.recovery.threads.per.data.dir=1
+
+############################# Internal Topic Settings  
#############################
+# The replication factor for the group metadata internal topics 
"__consumer_offsets", "__share_group_state" and "__transaction_state"
+# For anything other than development testing, a value greater than 1 is 
recommended to ensure availability such as 3.
+offsets.topic.replication.factor=1
+share.coordinator.state.topic.replication.factor=1
+share.coordinator.state.topic.min.isr=1
+transaction.state.log.replication.factor=1
+transaction.state.log.min.isr=1
+
+############################# Log Flush Policy #############################
+
+# Messages are immediately written to the filesystem but by default we only 
fsync() to sync
+# the OS cache lazily. The following configurations control the flush of data 
to disk.
+# There are a few important trade-offs here:
+#    1. Durability: Unflushed data may be lost if you are not using 
replication.
+#    2. Latency: Very large flush intervals may lead to latency spikes when 
the flush does occur as there will be a lot of data to flush.
+#    3. Throughput: The flush is generally the most expensive operation, and a 
small flush interval may lead to excessive seeks.
+# The settings below allow one to configure the flush policy to flush data 
after a period of time or
+# every N messages (or both). This can be done globally and overridden on a 
per-topic basis.
+
+# The number of messages to accept before forcing a flush of data to disk
+#log.flush.interval.messages=10000
+
+# The maximum amount of time a message can sit in a log before we force a flush
+#log.flush.interval.ms=1000
+
+############################# Log Retention Policy 
#############################
+
+# The following configurations control the disposal of log segments. The 
policy can
+# be set to delete segments after a period of time, or after a given size has 
accumulated.
+# A segment will be deleted whenever *either* of these criteria are met. 
Deletion always happens
+# from the end of the log.
+
+# The minimum age of a log file to be eligible for deletion due to age
+log.retention.hours=168
+
+# A size-based retention policy for logs. Segments are pruned from the log 
unless the remaining
+# segments drop below log.retention.bytes. Functions independently of 
log.retention.hours.
+#log.retention.bytes=1073741824
+
+# The maximum size of a log segment file. When this size is reached a new log 
segment will be created.
+log.segment.bytes=1073741824
+
+# The interval at which log segments are checked to see if they can be deleted 
according
+# to the retention policies
+log.retention.check.interval.ms=300000

Reply via email to