diff --git a/automq-shell/src/main/java/com/automq/shell/util/CLIUtils.java b/automq-shell/src/main/java/com/automq/shell/util/CLIUtils.java deleted file mode 100644 index f6609d31c3..0000000000 --- a/automq-shell/src/main/java/com/automq/shell/util/CLIUtils.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Copyright 2024, AutoMQ HK Limited. - * - * The use of this file is governed by the Business Source License, - * as detailed in the file "/LICENSE.S3Stream" included in this repository. - * - * As of the Change Date specified in that file, in accordance with - * the Business Source License, use of this software will be governed - * by the Apache License, Version 2.0 - */ - -package com.automq.shell.util; - -import java.util.Collections; -import org.apache.kafka.clients.ApiVersions; -import org.apache.kafka.clients.ClientUtils; -import org.apache.kafka.clients.CommonClientConfigs; -import org.apache.kafka.clients.ManualMetadataUpdater; -import org.apache.kafka.clients.NetworkClient; -import org.apache.kafka.clients.admin.AdminClientConfig; -import org.apache.kafka.common.metrics.Metrics; -import org.apache.kafka.common.network.ChannelBuilder; -import org.apache.kafka.common.network.NetworkReceive; -import org.apache.kafka.common.network.Selector; -import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.common.utils.Time; - -public class CLIUtils { - public static NetworkClient buildNetworkClient( - String prefix, - AdminClientConfig config, - Metrics metrics, - Time time, - LogContext logContext) { - ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(config, time, logContext); - - String metricGroupPrefix = prefix + "-channel"; - - Selector selector = new Selector( - NetworkReceive.UNLIMITED, - config.getLong(CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG), - metrics, - time, - metricGroupPrefix, - Collections.emptyMap(), - false, - channelBuilder, - logContext - ); - - String clientId = prefix + "-network-client"; - return new NetworkClient( - selector, - new ManualMetadataUpdater(), - clientId, - 100, - config.getLong(CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG), - config.getLong(CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_CONFIG), - config.getInt(CommonClientConfigs.SEND_BUFFER_CONFIG), - config.getInt(CommonClientConfigs.RECEIVE_BUFFER_CONFIG), - config.getInt(CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG), - config.getLong(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG), - config.getLong(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG), - time, - false, - new ApiVersions(), - logContext - ); - } -} diff --git a/automq-shell/src/main/java/com/automq/shell/util/S3PropUtil.java b/automq-shell/src/main/java/com/automq/shell/util/S3PropUtil.java deleted file mode 100644 index 6b277273c2..0000000000 --- a/automq-shell/src/main/java/com/automq/shell/util/S3PropUtil.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Copyright 2024, AutoMQ HK Limited. - * - * The use of this file is governed by the Business Source License, - * as detailed in the file "/LICENSE.S3Stream" included in this repository. - * - * As of the Change Date specified in that file, in accordance with - * the Business Source License, use of this software will be governed - * by the Apache License, Version 2.0 - */ -package com.automq.shell.util; - -import java.io.File; -import java.io.IOException; -import java.io.PrintWriter; -import java.nio.charset.Charset; -import java.util.Enumeration; -import java.util.Properties; - -public class S3PropUtil { - public static final String BROKER_PROPS_PATH = "template/broker.properties"; - public static final String CONTROLLER_PROPS_PATH = "template/controller.properties"; - public static final String SERVER_PROPS_PATH = "template/server.properties"; - - public static void persist(Properties props, String fileName) throws IOException { - File directory = new File("generated"); - if (!directory.exists() && !directory.mkdirs()) { - throw new IOException("Can't create directory " + directory.getAbsolutePath()); - } - - String targetPath = "generated/" + fileName; - File file = new File(targetPath); - try (PrintWriter pw = new PrintWriter(file, Charset.forName("utf-8"))) { - for (Enumeration e = props.propertyNames(); e.hasMoreElements(); ) { - String key = (String) e.nextElement(); - pw.println(key + "=" + props.getProperty(key)); - } - } - } - - public static Properties loadTemplateProps(String propsPath) throws IOException { - try (var in = S3PropUtil.class.getClassLoader().getResourceAsStream(propsPath)) { - if (in != null) { - Properties props = new Properties(); - props.load(in); - return props; - } else { - throw new IOException(String.format("Can not find resource file under path: %s", propsPath)); - } - } - } -} diff --git a/build.gradle b/build.gradle index 6248e4025e..1cb318afd2 100644 --- a/build.gradle +++ b/build.gradle @@ -937,6 +937,7 @@ project(':core') { api libs.scalaLibrary implementation project(':server-common') + implementation project(':group-coordinator:group-coordinator-api') implementation project(':group-coordinator') implementation project(':transaction-coordinator') implementation project(':metadata') @@ -1367,6 +1368,66 @@ project(':metadata') { } } +project(':group-coordinator:group-coordinator-api') { + base { + archivesName = "kafka-group-coordinator-api" + } + + dependencies { + implementation project(':clients') + } + + task createVersionFile() { + def receiptFile = file("$buildDir/kafka/$buildVersionFileName") + inputs.property "commitId", commitId + inputs.property "version", version + outputs.file receiptFile + + doLast { + def data = [ + commitId: commitId, + version: version, + ] + + receiptFile.parentFile.mkdirs() + def content = data.entrySet().collect { "$it.key=$it.value" }.sort().join("\n") + receiptFile.setText(content, "ISO-8859-1") + } + } + + sourceSets { + main { + java { + srcDirs = ["src/main/java"] + } + } + test { + java { + srcDirs = ["src/test/java"] + } + } + } + + jar { + dependsOn createVersionFile + from("$buildDir") { + include "kafka/$buildVersionFileName" + } + } + + clean.doFirst { + delete "$buildDir/kafka/" + } + + javadoc { + include "**/org/apache/kafka/coordinator/group/api/**" + } + + checkstyle { + configProperties = checkstyleConfigProperties("import-control-group-coordinator.xml") + } +} + project(':group-coordinator') { base { archivesName = "kafka-group-coordinator" @@ -1380,6 +1441,7 @@ project(':group-coordinator') { implementation project(':server-common') implementation project(':clients') implementation project(':metadata') + implementation project(':group-coordinator:group-coordinator-api') implementation project(':storage') implementation libs.jacksonDatabind implementation libs.jacksonJDK8Datatypes @@ -2597,10 +2659,6 @@ project(':streams:test-utils') { testRuntimeOnly libs.slf4jlog4j } - javadoc { - include "**/org/apache/kafka/streams/test/**" - } - tasks.create(name: "copyDependantLibs", type: Copy) { from (configurations.runtimeClasspath) { exclude('kafka-streams*') @@ -3023,6 +3081,7 @@ project(':jmh-benchmarks') { implementation project(':raft') implementation project(':clients') implementation project(':group-coordinator') + implementation project(':group-coordinator:group-coordinator-api') implementation project(':metadata') implementation project(':storage') implementation project(':streams') diff --git a/checkstyle/import-control-core.xml b/checkstyle/import-control-core.xml index ed6c53a322..a30de55e41 100644 --- a/checkstyle/import-control-core.xml +++ b/checkstyle/import-control-core.xml @@ -38,6 +38,7 @@ + diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 71c8e0dd4b..a663195985 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -443,6 +443,7 @@ + diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index ce25c63f28..fe1c805176 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -202,7 +202,7 @@ files="StreamThread.java"/> + files="(InternalTopologyBuilder|KafkaStreams|KStreamImpl|KTableImpl|StreamsPartitionAssignor).java"/> @@ -211,7 +211,7 @@ files="StreamsMetricsImpl.java"/> + files="(KafkaStreams|StreamsPartitionAssignor|StreamThread|TaskManager|TaskAssignmentUtils|GlobalStateManagerImpl|KStreamImplJoin|TopologyConfig|KTableKTableOuterJoin).java"/> diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index 6b6b56059c..b08bc12950 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -245,7 +245,9 @@ public static NetworkClient createNetworkClient(AbstractConfig config, throttleTimeSensor, logContext, hostResolver, - clientTelemetrySender); + clientTelemetrySender, + MetadataRecoveryStrategy.forName(config.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)) + ); } catch (Throwable t) { closeQuietly(selector, "Selector"); closeQuietly(channelBuilder, "ChannelBuilder"); diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java index 6f5be8a006..40b806c4e9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -220,6 +220,19 @@ public class CommonClientConfigs { public static final String DEFAULT_API_TIMEOUT_MS_DOC = "Specifies the timeout (in milliseconds) for client APIs. " + "This configuration is used as the default timeout for all client operations that do not specify a timeout parameter."; + public static final String METADATA_RECOVERY_STRATEGY_CONFIG = "metadata.recovery.strategy"; + public static final String METADATA_RECOVERY_STRATEGY_DOC = "Controls how the client recovers when none of the brokers known to it is available. " + + "If set to none, the client fails. If set to rebootstrap, " + + "the client repeats the bootstrap process using bootstrap.servers. " + + "Rebootstrapping is useful when a client communicates with brokers so infrequently " + + "that the set of brokers may change entirely before the client refreshes metadata. " + + "Metadata recovery is triggered when all last-known brokers appear unavailable simultaneously. " + + "Brokers appear unavailable when disconnected and no current retry attempt is in-progress. " + + "Consider increasing reconnect.backoff.ms and reconnect.backoff.max.ms and " + + "decreasing socket.connection.setup.timeout.ms and socket.connection.setup.timeout.max.ms " + + "for the client."; + public static final String DEFAULT_METADATA_RECOVERY_STRATEGY = MetadataRecoveryStrategy.NONE.name; + /** * Postprocess the configuration so that exponential backoff is disabled when reconnect backoff * is explicitly configured but the maximum reconnect backoff is not explicitly configured. diff --git a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java index a03d57b40f..46b6498606 100644 --- a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java @@ -130,7 +130,7 @@ public interface KafkaClient extends Closeable { * @param now The current time in ms * @return The node with the fewest in-flight requests. */ - Node leastLoadedNode(long now); + LeastLoadedNode leastLoadedNode(long now); /** * The number of currently in-flight requests for which we have not yet returned a response diff --git a/clients/src/main/java/org/apache/kafka/clients/LeastLoadedNode.java b/clients/src/main/java/org/apache/kafka/clients/LeastLoadedNode.java new file mode 100644 index 0000000000..b2b93e6c94 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/LeastLoadedNode.java @@ -0,0 +1,43 @@ +/* + * 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. + */ +package org.apache.kafka.clients; + +import org.apache.kafka.common.Node; + +public class LeastLoadedNode { + private final Node node; + private final boolean atLeastOneConnectionReady; + + public LeastLoadedNode(Node node, boolean atLeastOneConnectionReady) { + this.node = node; + this.atLeastOneConnectionReady = atLeastOneConnectionReady; + } + + public Node node() { + return node; + } + + /** + * Indicates if the least loaded node is available or at least a ready connection exists. + * + *

There may be no node available while ready connections to live nodes exist. This may happen when + * the connections are overloaded with in-flight requests. This function takes this into account. + */ + public boolean hasNodeAvailableOrConnectionReady() { + return node != null || atLeastOneConnectionReady; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java index 30cad44a4b..9246da0100 100644 --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -82,6 +82,8 @@ public class Metadata implements Closeable { private final ClusterResourceListeners clusterResourceListeners; private boolean isClosed; private final Map lastSeenLeaderEpochs; + /** Addresses with which the metadata was originally bootstrapped. */ + private List bootstrapAddresses; /** * Create a new Metadata instance @@ -304,6 +306,12 @@ public synchronized void bootstrap(List addresses) { this.needFullUpdate = true; this.updateVersion += 1; this.metadataSnapshot = MetadataSnapshot.bootstrap(addresses); + this.bootstrapAddresses = addresses; + } + + public synchronized void rebootstrap() { + log.info("Rebootstrapping with {}", this.bootstrapAddresses); + this.bootstrap(this.bootstrapAddresses); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/MetadataRecoveryStrategy.java b/clients/src/main/java/org/apache/kafka/clients/MetadataRecoveryStrategy.java new file mode 100644 index 0000000000..a4e0340c24 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/MetadataRecoveryStrategy.java @@ -0,0 +1,44 @@ +/* + * 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. + */ +package org.apache.kafka.clients; + +import java.util.Locale; + +/** + * Defines the strategies which clients can follow to deal with the situation when none of the known nodes is available. + */ +public enum MetadataRecoveryStrategy { + NONE("none"), + REBOOTSTRAP("rebootstrap"); + + public final String name; + + MetadataRecoveryStrategy(String name) { + this.name = name; + } + + public static MetadataRecoveryStrategy forName(String name) { + if (name == null) { + throw new IllegalArgumentException("Illegal MetadataRecoveryStrategy: null"); + } + try { + return MetadataRecoveryStrategy.valueOf(name.toUpperCase(Locale.ROOT)); + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException("Illegal MetadataRecoveryStrategy: " + name); + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 3a7af6617e..8ac92acd88 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -114,6 +114,8 @@ private enum State { /* time in ms to wait before retrying to create connection to a server */ private final long reconnectBackoffMs; + private final MetadataRecoveryStrategy metadataRecoveryStrategy; + private final Time time; /** @@ -147,7 +149,8 @@ public NetworkClient(Selectable selector, Time time, boolean discoverBrokerVersions, ApiVersions apiVersions, - LogContext logContext) { + LogContext logContext, + MetadataRecoveryStrategy metadataRecoveryStrategy) { this(selector, metadata, clientId, @@ -163,7 +166,8 @@ public NetworkClient(Selectable selector, discoverBrokerVersions, apiVersions, null, - logContext); + logContext, + metadataRecoveryStrategy); } public NetworkClient(Selectable selector, @@ -181,7 +185,8 @@ public NetworkClient(Selectable selector, boolean discoverBrokerVersions, ApiVersions apiVersions, Sensor throttleTimeSensor, - LogContext logContext) { + LogContext logContext, + MetadataRecoveryStrategy metadataRecoveryStrategy) { this(null, metadata, selector, @@ -200,7 +205,8 @@ public NetworkClient(Selectable selector, throttleTimeSensor, logContext, new DefaultHostResolver(), - null); + null, + metadataRecoveryStrategy); } public NetworkClient(Selectable selector, @@ -217,7 +223,8 @@ public NetworkClient(Selectable selector, Time time, boolean discoverBrokerVersions, ApiVersions apiVersions, - LogContext logContext) { + LogContext logContext, + MetadataRecoveryStrategy metadataRecoveryStrategy) { this(metadataUpdater, null, selector, @@ -236,7 +243,8 @@ public NetworkClient(Selectable selector, null, logContext, new DefaultHostResolver(), - null); + null, + metadataRecoveryStrategy); } public NetworkClient(MetadataUpdater metadataUpdater, @@ -257,7 +265,8 @@ public NetworkClient(MetadataUpdater metadataUpdater, Sensor throttleTimeSensor, LogContext logContext, HostResolver hostResolver, - ClientTelemetrySender clientTelemetrySender) { + ClientTelemetrySender clientTelemetrySender, + MetadataRecoveryStrategy metadataRecoveryStrategy) { /* It would be better if we could pass `DefaultMetadataUpdater` from the public constructor, but it's not * possible because `DefaultMetadataUpdater` is an inner class and it can only be instantiated after the * super constructor is invoked. @@ -288,6 +297,7 @@ public NetworkClient(MetadataUpdater metadataUpdater, this.log = logContext.logger(NetworkClient.class); this.state = new AtomicReference<>(State.ACTIVE); this.telemetrySender = (clientTelemetrySender != null) ? new TelemetrySender(clientTelemetrySender) : null; + this.metadataRecoveryStrategy = metadataRecoveryStrategy; } /** @@ -695,7 +705,7 @@ public void close() { * @return The node with the fewest in-flight requests. */ @Override - public Node leastLoadedNode(long now) { + public LeastLoadedNode leastLoadedNode(long now) { List nodes = this.metadataUpdater.fetchNodes(); if (nodes.isEmpty()) throw new IllegalStateException("There are no nodes in the Kafka cluster"); @@ -705,16 +715,25 @@ public Node leastLoadedNode(long now) { Node foundCanConnect = null; Node foundReady = null; + boolean atLeastOneConnectionReady = false; + int offset = this.randOffset.nextInt(nodes.size()); for (int i = 0; i < nodes.size(); i++) { int idx = (offset + i) % nodes.size(); Node node = nodes.get(idx); + + if (!atLeastOneConnectionReady + && connectionStates.isReady(node.idString(), now) + && selector.isChannelReady(node.idString())) { + atLeastOneConnectionReady = true; + } + if (canSendRequest(node.idString(), now)) { int currInflight = this.inFlightRequests.count(node.idString()); if (currInflight == 0) { // if we find an established connection with no in-flight requests we can stop right away log.trace("Found least loaded node {} connected with no in-flight requests", node); - return node; + return new LeastLoadedNode(node, true); } else if (currInflight < inflight) { // otherwise if this is the best we have found so far, record that inflight = currInflight; @@ -738,16 +757,16 @@ public Node leastLoadedNode(long now) { // which are being established before connecting to new nodes. if (foundReady != null) { log.trace("Found least loaded node {} with {} inflight requests", foundReady, inflight); - return foundReady; + return new LeastLoadedNode(foundReady, atLeastOneConnectionReady); } else if (foundConnecting != null) { log.trace("Found least loaded connecting node {}", foundConnecting); - return foundConnecting; + return new LeastLoadedNode(foundConnecting, atLeastOneConnectionReady); } else if (foundCanConnect != null) { log.trace("Found least loaded node {} with no active connection", foundCanConnect); - return foundCanConnect; + return new LeastLoadedNode(foundCanConnect, atLeastOneConnectionReady); } else { log.trace("Least loaded node selection failed to find an available node"); - return null; + return new LeastLoadedNode(null, atLeastOneConnectionReady); } } @@ -1122,13 +1141,22 @@ public long maybeUpdate(long now) { // Beware that the behavior of this method and the computation of timeouts for poll() are // highly dependent on the behavior of leastLoadedNode. - Node node = leastLoadedNode(now); - if (node == null) { + LeastLoadedNode leastLoadedNode = leastLoadedNode(now); + + // Rebootstrap if needed and configured. + if (metadataRecoveryStrategy == MetadataRecoveryStrategy.REBOOTSTRAP + && !leastLoadedNode.hasNodeAvailableOrConnectionReady()) { + metadata.rebootstrap(); + + leastLoadedNode = leastLoadedNode(now); + } + + if (leastLoadedNode.node() == null) { log.debug("Give up sending metadata request since no node is available"); return reconnectBackoffMs; } - return maybeUpdate(now, node); + return maybeUpdate(now, leastLoadedNode.node()); } @Override @@ -1266,7 +1294,7 @@ public long maybeUpdate(long now) { // Per KIP-714, let's continue to re-use the same broker for as long as possible. if (stickyNode == null) { - stickyNode = leastLoadedNode(now); + stickyNode = leastLoadedNode(now).node(); if (stickyNode == null) { log.debug("Give up sending telemetry request since no node is available"); return reconnectBackoffMs; diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java index 38b3355fd9..2f73b8c1ac 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java @@ -911,7 +911,7 @@ default DescribeConsumerGroupsResult describeConsumerGroups(Collection g * List the consumer groups available in the cluster. * * @param options The options to use when listing the consumer groups. - * @return The ListGroupsResult. + * @return The ListConsumerGroupsResult. */ ListConsumerGroupsResult listConsumerGroups(ListConsumerGroupsOptions options); @@ -921,7 +921,7 @@ default DescribeConsumerGroupsResult describeConsumerGroups(Collection g * This is a convenience method for {@link #listConsumerGroups(ListConsumerGroupsOptions)} with default options. * See the overload for more details. * - * @return The ListGroupsResult. + * @return The ListConsumerGroupsResult. */ default ListConsumerGroupsResult listConsumerGroups() { return listConsumerGroups(new ListConsumerGroupsOptions()); @@ -931,7 +931,7 @@ default ListConsumerGroupsResult listConsumerGroups() { * List the consumer group offsets available in the cluster. * * @param options The options to use when listing the consumer group offsets. - * @return The ListGroupOffsetsResult + * @return The ListConsumerGroupOffsetsResult */ default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId, ListConsumerGroupOffsetsOptions options) { @SuppressWarnings("deprecation") @@ -949,7 +949,7 @@ default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId, * This is a convenience method for {@link #listConsumerGroupOffsets(Map, ListConsumerGroupOffsetsOptions)} * to list offsets of all partitions of one group with default options. * - * @return The ListGroupOffsetsResult. + * @return The ListConsumerGroupOffsetsResult. */ default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId) { return listConsumerGroupOffsets(groupId, new ListConsumerGroupOffsetsOptions()); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java b/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java index d740283f50..b64338b1d1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java @@ -19,6 +19,7 @@ import org.apache.kafka.clients.ClientDnsLookup; import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.MetadataRecoveryStrategy; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; @@ -139,6 +140,10 @@ public class AdminClientConfig extends AbstractConfig { public static final String RETRIES_CONFIG = CommonClientConfigs.RETRIES_CONFIG; public static final String DEFAULT_API_TIMEOUT_MS_CONFIG = CommonClientConfigs.DEFAULT_API_TIMEOUT_MS_CONFIG; + public static final String METADATA_RECOVERY_STRATEGY_CONFIG = CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG; + public static final String METADATA_RECOVERY_STRATEGY_DOC = CommonClientConfigs.METADATA_RECOVERY_STRATEGY_DOC; + public static final String DEFAULT_METADATA_RECOVERY_STRATEGY = CommonClientConfigs.DEFAULT_METADATA_RECOVERY_STRATEGY; + /** * security.providers */ @@ -262,7 +267,14 @@ public class AdminClientConfig extends AbstractConfig { Importance.MEDIUM, SECURITY_PROTOCOL_DOC) .withClientSslSupport() - .withClientSaslSupport(); + .withClientSaslSupport() + .define(METADATA_RECOVERY_STRATEGY_CONFIG, + Type.STRING, + DEFAULT_METADATA_RECOVERY_STRATEGY, + ConfigDef.CaseInsensitiveValidString + .in(Utils.enumOptions(MetadataRecoveryStrategy.class)), + Importance.LOW, + METADATA_RECOVERY_STRATEGY_DOC); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 249417da68..5982c08192 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -25,6 +25,8 @@ import org.apache.kafka.clients.DefaultHostResolver; import org.apache.kafka.clients.HostResolver; import org.apache.kafka.clients.KafkaClient; +import org.apache.kafka.clients.LeastLoadedNode; +import org.apache.kafka.clients.MetadataRecoveryStrategy; import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.clients.StaleMetadataException; import org.apache.kafka.clients.admin.CreateTopicsResult.TopicMetadataAndConfig; @@ -277,7 +279,6 @@ import java.util.OptionalLong; import java.util.Set; import java.util.TreeMap; -import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -400,6 +401,7 @@ public class KafkaAdminClient extends AdminClient { private final long retryBackoffMaxMs; private final ExponentialBackoff retryBackoff; private final boolean clientTelemetryEnabled; + private final MetadataRecoveryStrategy metadataRecoveryStrategy; /** * The telemetry requests client instance id. @@ -613,6 +615,7 @@ private KafkaAdminClient(AdminClientConfig config, retryBackoffMaxMs, CommonClientConfigs.RETRY_BACKOFF_JITTER); this.clientTelemetryEnabled = config.getBoolean(AdminClientConfig.ENABLE_METRICS_PUSH_CONFIG); + this.metadataRecoveryStrategy = MetadataRecoveryStrategy.forName(config.getString(AdminClientConfig.METADATA_RECOVERY_STRATEGY_CONFIG)); config.logUnused(); AppInfoParser.registerAppInfo(JMX_PREFIX, clientId, metrics, time.milliseconds()); log.debug("Kafka admin client initialized"); @@ -699,7 +702,13 @@ private interface NodeProvider { private class MetadataUpdateNodeIdProvider implements NodeProvider { @Override public Node provide() { - return client.leastLoadedNode(time.milliseconds()); + LeastLoadedNode leastLoadedNode = client.leastLoadedNode(time.milliseconds()); + if (metadataRecoveryStrategy == MetadataRecoveryStrategy.REBOOTSTRAP + && !leastLoadedNode.hasNodeAvailableOrConnectionReady()) { + metadataManager.rebootstrap(time.milliseconds()); + } + + return leastLoadedNode.node(); } @Override @@ -781,7 +790,7 @@ public Node provide() { if (metadataManager.isReady()) { // This may return null if all nodes are busy. // In that case, we will postpone node assignment. - return client.leastLoadedNode(time.milliseconds()); + return client.leastLoadedNode(time.milliseconds()).node(); } metadataManager.requestUpdate(); return null; @@ -836,7 +845,7 @@ public Node provide() { } else { // This may return null if all nodes are busy. // In that case, we will postpone node assignment. - return client.leastLoadedNode(time.milliseconds()); + return client.leastLoadedNode(time.milliseconds()).node(); } } metadataManager.requestUpdate(); @@ -2130,7 +2139,7 @@ else if (topics instanceof TopicNameCollection) throw new IllegalArgumentException("The TopicCollection: " + topics + " provided did not match any supported classes for describeTopics."); } - Call generateDescribeTopicsCallWithMetadataApi( + private Call generateDescribeTopicsCallWithMetadataApi( List topicNamesList, Map> topicFutures, DescribeTopicsOptions options, @@ -2193,7 +2202,7 @@ void handleFailure(Throwable throwable) { }; } - Call generateDescribeTopicsCallWithDescribeTopicPartitionsApi( + private Call generateDescribeTopicsCallWithDescribeTopicPartitionsApi( List topicNamesList, Map> topicFutures, Map nodes, @@ -2247,7 +2256,7 @@ void handleResponse(AbstractResponse abstractResponse) { continue; } - TopicDescription currentTopicDescription = getTopicDescriptionFromDescribeTopicsResponseTopic(topic, nodes); + TopicDescription currentTopicDescription = getTopicDescriptionFromDescribeTopicsResponseTopic(topic, nodes, options.includeAuthorizedOperations()); if (partiallyFinishedTopicDescription != null && partiallyFinishedTopicDescription.name().equals(topicName)) { // Add the partitions for the cursor topic of the previous batch. @@ -2320,27 +2329,27 @@ private Map> handleDescribeTopicsByNamesWi } if (topicNamesList.isEmpty()) { - return new HashMap<>(topicFutures); + return Collections.unmodifiableMap(topicFutures); } // First, we need to retrieve the node info. DescribeClusterResult clusterResult = describeCluster(); - Map nodes; - try { - nodes = clusterResult.nodes().get().stream().collect(Collectors.toMap(Node::id, node -> node)); - } catch (InterruptedException | ExecutionException e) { - completeAllExceptionally(topicFutures.values(), e.getCause()); - return new HashMap<>(topicFutures); - } - - final long now = time.milliseconds(); + clusterResult.nodes().whenComplete( + (nodes, exception) -> { + if (exception != null) { + completeAllExceptionally(topicFutures.values(), exception); + return; + } - runnable.call( - generateDescribeTopicsCallWithDescribeTopicPartitionsApi(topicNamesList, topicFutures, nodes, options, now), - now - ); + final long now = time.milliseconds(); + Map nodeIdMap = nodes.stream().collect(Collectors.toMap(Node::id, node -> node)); + runnable.call( + generateDescribeTopicsCallWithDescribeTopicPartitionsApi(topicNamesList, topicFutures, nodeIdMap, options, now), + now + ); + }); - return new HashMap<>(topicFutures); + return Collections.unmodifiableMap(topicFutures); } private Map> handleDescribeTopicsByIds(Collection topicIds, DescribeTopicsOptions options) { @@ -2410,14 +2419,16 @@ void handleFailure(Throwable throwable) { private TopicDescription getTopicDescriptionFromDescribeTopicsResponseTopic( DescribeTopicPartitionsResponseTopic topic, - Map nodes + Map nodes, + boolean includeAuthorizedOperations ) { List partitionInfos = topic.partitions(); List partitions = new ArrayList<>(partitionInfos.size()); for (DescribeTopicPartitionsResponsePartition partitionInfo : partitionInfos) { partitions.add(DescribeTopicPartitionsResponse.partitionToTopicPartitionInfo(partitionInfo, nodes)); } - return new TopicDescription(topic.name(), topic.isInternal(), partitions, validAclOperations(topic.topicAuthorizedOperations()), topic.topicId()); + Set authorisedOperations = includeAuthorizedOperations ? validAclOperations(topic.topicAuthorizedOperations()) : null; + return new TopicDescription(topic.name(), topic.isInternal(), partitions, authorisedOperations, topic.topicId()); } // AutoMQ for Kafka inject start @@ -4604,7 +4615,7 @@ public ListTransactionsResult listTransactions(ListTransactionsOptions options) public FenceProducersResult fenceProducers(Collection transactionalIds, FenceProducersOptions options) { AdminApiFuture.SimpleAdminApiFuture future = FenceProducersHandler.newFuture(transactionalIds); - FenceProducersHandler handler = new FenceProducersHandler(logContext); + FenceProducersHandler handler = new FenceProducersHandler(options, logContext, requestTimeoutMs); invokeDriver(handler, future, options.timeoutMs); return new FenceProducersResult(future.all()); } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java index f8123c40ef..239f6eecef 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java @@ -92,6 +92,11 @@ public class AdminMetadataManager { */ private ApiException fatalException = null; + /** + * The cluster with which the metadata was bootstrapped. + */ + private Cluster bootstrapCluster; + public class AdminMetadataUpdater implements MetadataUpdater { @Override public List fetchNodes() { @@ -275,6 +280,7 @@ public void updateFailed(Throwable exception) { public void update(Cluster cluster, long now) { if (cluster.isBootstrapConfigured()) { log.debug("Setting bootstrap cluster metadata {}.", cluster); + bootstrapCluster = cluster; } else { log.debug("Updating cluster metadata to {}", cluster); this.lastMetadataUpdateMs = now; @@ -287,4 +293,12 @@ public void update(Cluster cluster, long now) { this.cluster = cluster; } } + + /** + * Rebootstrap metadata with the cluster previously used for bootstrapping. + */ + public void rebootstrap(long now) { + log.info("Rebootstrapping with {}", this.bootstrapCluster); + update(bootstrapCluster, now); + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/FenceProducersHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/FenceProducersHandler.java index 23572dd441..826db904a3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/FenceProducersHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/FenceProducersHandler.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients.admin.internals; +import org.apache.kafka.clients.admin.FenceProducersOptions; import org.apache.kafka.common.Node; import org.apache.kafka.common.errors.ClusterAuthorizationException; import org.apache.kafka.common.errors.TransactionalIdAuthorizationException; @@ -38,12 +39,16 @@ public class FenceProducersHandler extends AdminApiHandler.Unbatched { private final Logger log; private final AdminApiLookupStrategy lookupStrategy; + private final int txnTimeoutMs; public FenceProducersHandler( - LogContext logContext + FenceProducersOptions options, + LogContext logContext, + int requestTimeoutMs ) { this.log = logContext.logger(FenceProducersHandler.class); this.lookupStrategy = new CoordinatorStrategy(FindCoordinatorRequest.CoordinatorType.TRANSACTION, logContext); + this.txnTimeoutMs = options.timeoutMs() != null ? options.timeoutMs() : requestTimeoutMs; } public static AdminApiFuture.SimpleAdminApiFuture newFuture( @@ -82,9 +87,8 @@ InitProducerIdRequest.Builder buildSingleRequest(int brokerId, CoordinatorKey ke .setProducerEpoch(ProducerIdAndEpoch.NONE.epoch) .setProducerId(ProducerIdAndEpoch.NONE.producerId) .setTransactionalId(key.idValue) - // Set transaction timeout to 1 since it's only being initialized to fence out older producers with the same transactional ID, - // and shouldn't be used for any actual record writes - .setTransactionTimeoutMs(1); + // This timeout is used by the coordinator to append the record with the new producer epoch to the transaction log. + .setTransactionTimeoutMs(txnTimeoutMs); return new InitProducerIdRequest.Builder(data); } @@ -130,6 +134,10 @@ private ApiResult handleError( "coordinator is still in the process of loading state. Will retry", transactionalIdKey.idValue); return ApiResult.empty(); + case CONCURRENT_TRANSACTIONS: + log.debug("InitProducerId request for transactionalId `{}` failed because of " + + "a concurrent transaction. Will retry", transactionalIdKey.idValue); + return ApiResult.empty(); case NOT_COORDINATOR: case COORDINATOR_NOT_AVAILABLE: diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 7ec147e9e3..c4c10c404b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.ClientDnsLookup; import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.MetadataRecoveryStrategy; import org.apache.kafka.common.IsolationLevel; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; @@ -198,7 +199,10 @@ public class ConsumerConfig extends AbstractConfig { * fetch.max.wait.ms */ public static final String FETCH_MAX_WAIT_MS_CONFIG = "fetch.max.wait.ms"; - private static final String FETCH_MAX_WAIT_MS_DOC = "The maximum amount of time the server will block before answering the fetch request if there isn't sufficient data to immediately satisfy the requirement given by fetch.min.bytes."; + private static final String FETCH_MAX_WAIT_MS_DOC = "The maximum amount of time the server will block before " + + "answering the fetch request there isn't sufficient data to immediately satisfy the requirement given by " + + "fetch.min.bytes. This config is used only for local log fetch. To tune the remote fetch maximum wait " + + "time, please refer to 'remote.fetch.max.wait.ms' broker config"; public static final int DEFAULT_FETCH_MAX_WAIT_MS = 500; /** metadata.max.age.ms */ @@ -653,7 +657,14 @@ public class ConsumerConfig extends AbstractConfig { Importance.MEDIUM, CommonClientConfigs.SECURITY_PROTOCOL_DOC) .withClientSslSupport() - .withClientSaslSupport(); + .withClientSaslSupport() + .define(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG, + Type.STRING, + CommonClientConfigs.DEFAULT_METADATA_RECOVERY_STRATEGY, + ConfigDef.CaseInsensitiveValidString + .in(Utils.enumOptions(MetadataRecoveryStrategy.class)), + Importance.LOW, + CommonClientConfigs.METADATA_RECOVERY_STRATEGY_DOC); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignor.java index ed1871a3d8..06bd17f4eb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignor.java @@ -297,7 +297,7 @@ static List getAssignorInstances(List assigno // first try to get the class if passed in as a string if (klass instanceof String) { try { - klass = Class.forName((String) klass, true, Utils.getContextOrKafkaClassLoader()); + klass = Utils.loadClass((String) klass, Object.class); } catch (ClassNotFoundException classNotFound) { throw new KafkaException(klass + " ClassNotFoundException exception occurred", classNotFound); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 6930cd0295..6622cb286d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -26,7 +26,6 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; import org.apache.kafka.clients.consumer.ConsumerInterceptor; -import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.GroupProtocol; @@ -54,7 +53,6 @@ import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.clients.consumer.internals.events.EventProcessor; import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsEvent; -import org.apache.kafka.clients.consumer.internals.events.LeaveOnCloseEvent; import org.apache.kafka.clients.consumer.internals.events.ListOffsetsEvent; import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; import org.apache.kafka.clients.consumer.internals.events.PollEvent; @@ -91,7 +89,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; -import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import org.slf4j.event.Level; @@ -110,7 +107,6 @@ import java.util.OptionalLong; import java.util.Set; import java.util.SortedSet; -import java.util.TreeSet; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; @@ -235,12 +231,12 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { private final SubscriptionState subscriptions; private final ConsumerMetadata metadata; + private int metadataVersionSnapshot; private final Metrics metrics; private final long retryBackoffMs; private final int defaultApiTimeoutMs; private final boolean autoCommitEnabled; private volatile boolean closed = false; - private final List assignors; private final Optional clientTelemetryReporter; // to keep from repeatedly scanning subscriptions in poll(), cache the result during metadata updates @@ -257,6 +253,8 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { private final AtomicLong currentThread = new AtomicLong(NO_CURRENT_THREAD); private final AtomicInteger refCount = new AtomicInteger(0); + private FetchCommittedOffsetsEvent pendingOffsetFetchEvent; + AsyncKafkaConsumer(final ConsumerConfig config, final Deserializer keyDeserializer, final Deserializer valueDeserializer) { @@ -313,6 +311,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.metadata = metadataFactory.build(config, subscriptions, logContext, clusterResourceListeners); final List addresses = ClientUtils.parseAndValidateAddresses(config); metadata.bootstrap(addresses); + this.metadataVersionSnapshot = metadata.updateVersion(); FetchMetricsManager fetchMetricsManager = createFetchMetricsManager(metrics); FetchConfig fetchConfig = new FetchConfig(config); @@ -373,10 +372,6 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { rebalanceListenerInvoker ); this.backgroundEventReaper = backgroundEventReaperFactory.build(logContext); - this.assignors = ConsumerPartitionAssignor.getAssignorInstances( - config.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), - config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)) - ); // The FetchCollector is only used on the application thread. this.fetchCollector = fetchCollectorFactory.build(logContext, @@ -424,7 +419,6 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { ConsumerMetadata metadata, long retryBackoffMs, int defaultApiTimeoutMs, - List assignors, String groupId, boolean autoCommitEnabled) { this.log = logContext.logger(getClass()); @@ -441,11 +435,11 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.metrics = metrics; this.groupMetadata.set(initializeGroupMetadata(groupId, Optional.empty())); this.metadata = metadata; + this.metadataVersionSnapshot = metadata.updateVersion(); this.retryBackoffMs = retryBackoffMs; this.defaultApiTimeoutMs = defaultApiTimeoutMs; this.deserializers = deserializers; this.applicationEventHandler = applicationEventHandler; - this.assignors = assignors; this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer"); this.clientTelemetryReporter = Optional.empty(); this.autoCommitEnabled = autoCommitEnabled; @@ -460,8 +454,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { Deserializer valueDeserializer, KafkaClient client, SubscriptionState subscriptions, - ConsumerMetadata metadata, - List assignors) { + ConsumerMetadata metadata) { this.log = logContext.logger(getClass()); this.subscriptions = subscriptions; this.clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG); @@ -472,10 +465,10 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.time = time; this.metrics = new Metrics(time); this.metadata = metadata; + this.metadataVersionSnapshot = metadata.updateVersion(); this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); this.defaultApiTimeoutMs = config.getInt(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG); this.deserializers = new Deserializers<>(keyDeserializer, valueDeserializer); - this.assignors = assignors; this.clientTelemetryReporter = Optional.empty(); ConsumerMetrics metricsRegistry = new ConsumerMetrics(CONSUMER_METRIC_GROUP_PREFIX); @@ -1237,8 +1230,8 @@ private void close(Duration timeout, boolean swallowException) { clientTelemetryReporter.ifPresent(reporter -> reporter.initiateClose(timeout.toMillis())); closeTimer.update(); // Prepare shutting down the network thread - prepareShutdown(closeTimer, firstException); - closeTimer.update(); + swallow(log, Level.ERROR, "Failed to release assignment before closing consumer", + () -> releaseAssignmentAndLeaveGroup(closeTimer), firstException); swallow(log, Level.ERROR, "Failed invoking asynchronous commit callback.", () -> awaitPendingAsyncCommitsAndExecuteCommitCallbacks(closeTimer, false), firstException); if (applicationEventHandler != null) @@ -1270,27 +1263,34 @@ private void close(Duration timeout, boolean swallowException) { /** * Prior to closing the network thread, we need to make sure the following operations happen in the right sequence: * 1. autocommit offsets - * 2. revoke all partitions - * 3. if partition revocation completes successfully, send leave group + * 2. release assignment. This is done via a background unsubscribe event that will + * trigger the callbacks, clear the assignment on the subscription state and send the leave group request to the broker */ - void prepareShutdown(final Timer timer, final AtomicReference firstException) { + private void releaseAssignmentAndLeaveGroup(final Timer timer) { if (!groupMetadata.get().isPresent()) return; if (autoCommitEnabled) - autoCommitSync(timer); + commitSyncAllConsumed(timer); applicationEventHandler.add(new CommitOnCloseEvent()); - completeQuietly( - () -> { - maybeRevokePartitions(); - applicationEventHandler.addAndGet(new LeaveOnCloseEvent(calculateDeadlineMs(timer))); - }, - "Failed to send leaveGroup heartbeat with a timeout(ms)=" + timer.timeoutMs(), firstException); + + log.info("Releasing assignment and leaving group before closing consumer"); + UnsubscribeEvent unsubscribeEvent = new UnsubscribeEvent(calculateDeadlineMs(timer)); + applicationEventHandler.add(unsubscribeEvent); + try { + processBackgroundEvents(unsubscribeEvent.future(), timer); + log.info("Completed releasing assignment and sending leave group to close consumer"); + } catch (TimeoutException e) { + log.warn("Consumer triggered an unsubscribe event to leave the group but couldn't " + + "complete it within {} ms. It will proceed to close.", timer.timeoutMs()); + } finally { + timer.update(); + } } // Visible for testing - void autoCommitSync(final Timer timer) { + void commitSyncAllConsumed(final Timer timer) { Map allConsumed = subscriptions.allConsumed(); log.debug("Sending synchronous auto-commit of offsets {} on closing", allConsumed); try { @@ -1302,35 +1302,6 @@ void autoCommitSync(final Timer timer) { timer.update(); } - // Visible for testing - void maybeRevokePartitions() { - if (!subscriptions.hasAutoAssignedPartitions() || subscriptions.assignedPartitions().isEmpty()) - return; - try { - SortedSet droppedPartitions = new TreeSet<>(MembershipManagerImpl.TOPIC_PARTITION_COMPARATOR); - droppedPartitions.addAll(subscriptions.assignedPartitions()); - if (subscriptions.rebalanceListener().isPresent()) - subscriptions.rebalanceListener().get().onPartitionsRevoked(droppedPartitions); - } catch (Exception e) { - throw new KafkaException(e); - } finally { - subscriptions.assignFromSubscribed(Collections.emptySet()); - } - } - - // Visible for testing - void completeQuietly(final Utils.ThrowingRunnable function, - final String msg, - final AtomicReference firstException) { - try { - function.run(); - } catch (TimeoutException e) { - log.debug("Timeout expired before the {} operation could complete.", msg); - } catch (Exception e) { - firstException.compareAndSet(null, e); - } - } - @Override public void wakeup() { wakeupTrigger.wakeup(); @@ -1478,12 +1449,11 @@ public void assign(Collection partitions) { } /** - * TODO: remove this when we implement the KIP-848 protocol. - * *

- * The contents of this method are shamelessly stolen from - * {@link ConsumerCoordinator#updatePatternSubscription(Cluster)} and are used here because we won't have access - * to a {@link ConsumerCoordinator} in this code. Perhaps it could be moved to a ConsumerUtils class? + * + * This function evaluates the regex that the consumer subscribed to + * against the list of topic names from metadata, and updates + * the list of topics in subscription state accordingly * * @param cluster Cluster from which we get the topics */ @@ -1493,7 +1463,7 @@ private void updatePatternSubscription(Cluster cluster) { .collect(Collectors.toSet()); if (subscriptions.subscribeFromPattern(topicsToSubscribe)) { applicationEventHandler.add(new SubscriptionChangeEvent()); - metadata.requestUpdateForNewTopics(); + this.metadataVersionSnapshot = metadata.requestUpdateForNewTopics(); } } @@ -1506,7 +1476,8 @@ public void unsubscribe() { Timer timer = time.timer(Long.MAX_VALUE); UnsubscribeEvent unsubscribeEvent = new UnsubscribeEvent(calculateDeadlineMs(timer)); applicationEventHandler.add(unsubscribeEvent); - log.info("Unsubscribing all topics or patterns and assigned partitions"); + log.info("Unsubscribing all topics or patterns and assigned partitions {}", + subscriptions.assignedPartitions()); try { processBackgroundEvents(unsubscribeEvent.future(), timer); @@ -1516,7 +1487,9 @@ public void unsubscribe() { } resetGroupMetadata(); } - subscriptions.unsubscribe(); + } catch (Exception e) { + log.error("Unsubscribe failed", e); + throw e; } finally { release(); } @@ -1670,27 +1643,64 @@ private boolean initWithCommittedOffsetsIfNeeded(Timer timer) { return true; log.debug("Refreshing committed offsets for partitions {}", initializingPartitions); + + // The shorter the timeout provided to poll(), the more likely the offsets fetch will time out. To handle + // this case, on the first attempt to fetch the committed offsets, a FetchCommittedOffsetsEvent is created + // (with potentially a longer timeout) and stored. The event is used for the first attempt, but in the + // case it times out, subsequent attempts will also use the event in order to wait for the results. + if (!canReusePendingOffsetFetchEvent(initializingPartitions)) { + // Give the event a reasonable amount of time to complete. + final long timeoutMs = Math.max(defaultApiTimeoutMs, timer.remainingMs()); + final long deadlineMs = calculateDeadlineMs(time, timeoutMs); + pendingOffsetFetchEvent = new FetchCommittedOffsetsEvent(initializingPartitions, deadlineMs); + applicationEventHandler.add(pendingOffsetFetchEvent); + } + + final CompletableFuture> future = pendingOffsetFetchEvent.future(); + try { - final FetchCommittedOffsetsEvent event = - new FetchCommittedOffsetsEvent( - initializingPartitions, - calculateDeadlineMs(timer)); - wakeupTrigger.setActiveTask(event.future()); - final Map offsets = applicationEventHandler.addAndGet(event); + wakeupTrigger.setActiveTask(future); + final Map offsets = ConsumerUtils.getResult(future, timer); + + // Clear the pending event once its result is successfully retrieved. + pendingOffsetFetchEvent = null; + refreshCommittedOffsets(offsets, metadata, subscriptions); return true; } catch (TimeoutException e) { - log.error("Couldn't refresh committed offsets before timeout expired"); + log.debug( + "The committed offsets for the following partition(s) could not be refreshed within the timeout: {} ", + initializingPartitions + ); return false; + } catch (InterruptException e) { + throw e; + } catch (Throwable t) { + pendingOffsetFetchEvent = null; + throw ConsumerUtils.maybeWrapAsKafkaException(t); } finally { wakeupTrigger.clearTask(); } } - private void throwIfNoAssignorsConfigured() { - if (assignors.isEmpty()) - throw new IllegalStateException("Must configure at least one partition assigner class name to " + - ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG + " configuration property"); + /** + * This determines if the {@link #pendingOffsetFetchEvent pending offset fetch event} can be reused. Reuse + * is only possible if all the following conditions are true: + * + *

    + *
  • A pending offset fetch event exists
  • + *
  • The partition set of the pending offset fetch event is the same as the given partition set
  • + *
  • The pending offset fetch event has not expired
  • + *
+ */ + private boolean canReusePendingOffsetFetchEvent(Set partitions) { + if (pendingOffsetFetchEvent == null) + return false; + + if (!pendingOffsetFetchEvent.partitions().equals(partitions)) + return false; + + return pendingOffsetFetchEvent.deadlineMs() > time.milliseconds(); } private void updateLastSeenEpochIfNewer(TopicPartition topicPartition, OffsetAndMetadata offsetAndMetadata) { @@ -1780,7 +1790,6 @@ private void subscribeInternal(Pattern pattern, Optional topics, Optional currentTopicPartitions = new HashSet<>(); @@ -1818,7 +1825,7 @@ private void subscribeInternal(Collection topics, Optional(topics), listener)) - metadata.requestUpdateForNewTopics(); + this.metadataVersionSnapshot = metadata.requestUpdateForNewTopics(); // Trigger subscribe event to effectively join the group if not already part of it, // or just send the new subscription to the broker. @@ -1996,10 +2003,17 @@ SubscriptionState subscriptions() { return subscriptions; } + boolean hasPendingOffsetFetchEvent() { + return pendingOffsetFetchEvent != null; + } + private void maybeUpdateSubscriptionMetadata() { - if (subscriptions.hasPatternSubscription()) { - updatePatternSubscription(metadata.fetch()); + if (this.metadataVersionSnapshot < metadata.updateVersion()) { + this.metadataVersionSnapshot = metadata.updateVersion(); + if (subscriptions.hasPatternSubscription()) { + updatePatternSubscription(metadata.fetch()); + } } } - } + diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java index 577cf7dee6..000797dba0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java @@ -69,6 +69,7 @@ import static org.apache.kafka.common.protocol.Errors.COORDINATOR_LOAD_IN_PROGRESS; public class CommitRequestManager implements RequestManager, MemberStateListener { + private final Time time; private final SubscriptionState subscriptions; private final LogContext logContext; private final Logger log; @@ -133,6 +134,7 @@ public CommitRequestManager( final OptionalDouble jitter, final Metrics metrics) { Objects.requireNonNull(coordinatorRequestManager, "Coordinator is needed upon committing offsets"); + this.time = time; this.logContext = logContext; this.log = logContext.logger(getClass()); this.pendingRequests = new PendingRequests(); @@ -205,6 +207,13 @@ private static long findMinTime(final Collection request .orElse(Long.MAX_VALUE); } + private KafkaException maybeWrapAsTimeoutException(Throwable t) { + if (t instanceof TimeoutException) + return (TimeoutException) t; + else + return new TimeoutException(t); + } + /** * Generate a request to commit consumed offsets. Add the request to the queue of pending * requests to be sent out on the next call to {@link #poll(long)}. If there are empty @@ -245,7 +254,7 @@ public void maybeAutoCommitAsync() { if (autoCommitEnabled() && autoCommitState.get().shouldAutoCommit()) { OffsetCommitRequestState requestState = createOffsetCommitRequest( subscriptions.allConsumed(), - Optional.empty()); + Long.MAX_VALUE); CompletableFuture> result = requestAutoCommit(requestState); // Reset timer to the interval (even if no request was generated), but ensure that if // the request completes with a retriable error, the timer is reset to send the next @@ -294,14 +303,14 @@ private void maybeResetTimerWithBackoff(final CompletableFuture maybeAutoCommitSyncBeforeRevocation(final long retryExpirationTimeMs) { + public CompletableFuture maybeAutoCommitSyncBeforeRevocation(final long deadlineMs) { if (!autoCommitEnabled()) { return CompletableFuture.completedFuture(null); } CompletableFuture result = new CompletableFuture<>(); OffsetCommitRequestState requestState = - createOffsetCommitRequest(subscriptions.allConsumed(), Optional.of(retryExpirationTimeMs)); + createOffsetCommitRequest(subscriptions.allConsumed(), deadlineMs); autoCommitSyncBeforeRevocationWithRetries(requestState, result); return result; } @@ -314,9 +323,9 @@ private void autoCommitSyncBeforeRevocationWithRetries(OffsetCommitRequestState result.complete(null); } else { if (error instanceof RetriableException || isStaleEpochErrorAndValidEpochAvailable(error)) { - if (error instanceof TimeoutException && requestAttempt.isExpired) { + if (requestAttempt.isExpired()) { log.debug("Auto-commit sync before revocation timed out and won't be retried anymore"); - result.completeExceptionally(error); + result.completeExceptionally(maybeWrapAsTimeoutException(error)); } else if (error instanceof UnknownTopicOrPartitionException) { log.debug("Auto-commit sync before revocation failed because topic or partition were deleted"); result.completeExceptionally(error); @@ -367,7 +376,7 @@ public CompletableFuture commitAsync(final Map asyncCommitResult = new CompletableFuture<>(); @@ -385,28 +394,26 @@ public CompletableFuture commitAsync(final Map commitSync(final Map offsets, - final long retryExpirationTimeMs) { + final long deadlineMs) { CompletableFuture result = new CompletableFuture<>(); - OffsetCommitRequestState requestState = createOffsetCommitRequest( - offsets, - Optional.of(retryExpirationTimeMs)); + OffsetCommitRequestState requestState = createOffsetCommitRequest(offsets, deadlineMs); commitSyncWithRetries(requestState, result); return result; } private OffsetCommitRequestState createOffsetCommitRequest(final Map offsets, - final Optional expirationTimeMs) { + final long deadlineMs) { return jitter.isPresent() ? new OffsetCommitRequestState( offsets, groupId, groupInstanceId, - expirationTimeMs, + deadlineMs, retryBackoffMs, retryBackoffMaxMs, jitter.getAsDouble(), @@ -415,7 +422,7 @@ private OffsetCommitRequestState createOffsetCommitRequest(final Map> fetchOffsets( final Set partitions, - final long expirationTimeMs) { + final long deadlineMs) { if (partitions.isEmpty()) { return CompletableFuture.completedFuture(Collections.emptyMap()); } CompletableFuture> result = new CompletableFuture<>(); - OffsetFetchRequestState request = createOffsetFetchRequest(partitions, expirationTimeMs); + OffsetFetchRequestState request = createOffsetFetchRequest(partitions, deadlineMs); fetchOffsetsWithRetries(request, result); return result; } private OffsetFetchRequestState createOffsetFetchRequest(final Set partitions, - final long expirationTimeMs) { + final long deadlineMs) { return jitter.isPresent() ? new OffsetFetchRequestState( partitions, retryBackoffMs, retryBackoffMaxMs, - expirationTimeMs, + deadlineMs, jitter.getAsDouble(), memberInfo) : new OffsetFetchRequestState( partitions, retryBackoffMs, retryBackoffMaxMs, - expirationTimeMs, + deadlineMs, memberInfo); } @@ -516,8 +523,9 @@ private void fetchOffsetsWithRetries(final OffsetFetchRequestState fetchRequest, result.complete(res); } else { if (error instanceof RetriableException || isStaleEpochErrorAndValidEpochAvailable(error)) { - if (error instanceof TimeoutException && fetchRequest.isExpired) { - result.completeExceptionally(error); + if (fetchRequest.isExpired()) { + log.debug("OffsetFetch request for {} timed out and won't be retried anymore", fetchRequest.requestedPartitions); + result.completeExceptionally(maybeWrapAsTimeoutException(error)); } else { fetchRequest.resetFuture(); fetchOffsetsWithRetries(fetchRequest, result); @@ -612,12 +620,12 @@ private class OffsetCommitRequestState extends RetriableRequestState { OffsetCommitRequestState(final Map offsets, final String groupId, final Optional groupInstanceId, - final Optional expirationTimeMs, + final long deadlineMs, final long retryBackoffMs, final long retryBackoffMaxMs, final MemberInfo memberInfo) { super(logContext, CommitRequestManager.class.getSimpleName(), retryBackoffMs, - retryBackoffMaxMs, memberInfo, expirationTimeMs); + retryBackoffMaxMs, memberInfo, deadlineTimer(time, deadlineMs)); this.offsets = offsets; this.groupId = groupId; this.groupInstanceId = groupInstanceId; @@ -628,13 +636,13 @@ private class OffsetCommitRequestState extends RetriableRequestState { OffsetCommitRequestState(final Map offsets, final String groupId, final Optional groupInstanceId, - final Optional expirationTimeMs, + final long deadlineMs, final long retryBackoffMs, final long retryBackoffMaxMs, final double jitter, final MemberInfo memberInfo) { super(logContext, CommitRequestManager.class.getSimpleName(), retryBackoffMs, 2, - retryBackoffMaxMs, jitter, memberInfo, expirationTimeMs); + retryBackoffMaxMs, jitter, memberInfo, deadlineTimer(time, deadlineMs)); this.offsets = offsets; this.groupId = groupId; this.groupInstanceId = groupInstanceId; @@ -780,40 +788,24 @@ void removeRequest() { * Represents a request that can be retried or aborted, based on member ID and epoch * information. */ - abstract class RetriableRequestState extends RequestState { + abstract class RetriableRequestState extends TimedRequestState { /** * Member info (ID and epoch) to be included in the request if present. */ final MemberInfo memberInfo; - /** - * Time until which the request should be retried if it fails with retriable - * errors. If not present, the request is triggered without waiting for a response or - * retrying. - */ - private final Optional expirationTimeMs; - - /** - * True if the request expiration time has been reached. This is set when validating the - * request expiration on {@link #poll(long)} before sending it. It is used to know if a - * request should be retried on TimeoutException. - */ - boolean isExpired; - RetriableRequestState(LogContext logContext, String owner, long retryBackoffMs, - long retryBackoffMaxMs, MemberInfo memberInfo, Optional expirationTimeMs) { - super(logContext, owner, retryBackoffMs, retryBackoffMaxMs); + long retryBackoffMaxMs, MemberInfo memberInfo, Timer timer) { + super(logContext, owner, retryBackoffMs, retryBackoffMaxMs, timer); this.memberInfo = memberInfo; - this.expirationTimeMs = expirationTimeMs; } // Visible for testing RetriableRequestState(LogContext logContext, String owner, long retryBackoffMs, int retryBackoffExpBase, - long retryBackoffMaxMs, double jitter, MemberInfo memberInfo, Optional expirationTimeMs) { - super(logContext, owner, retryBackoffMs, retryBackoffExpBase, retryBackoffMaxMs, jitter); + long retryBackoffMaxMs, double jitter, MemberInfo memberInfo, Timer timer) { + super(logContext, owner, retryBackoffMs, retryBackoffExpBase, retryBackoffMaxMs, jitter, timer); this.memberInfo = memberInfo; - this.expirationTimeMs = expirationTimeMs; } /** @@ -828,13 +820,12 @@ abstract class RetriableRequestState extends RequestState { abstract CompletableFuture future(); /** - * Complete the request future with a TimeoutException if the request timeout has been - * reached, based on the provided current time. + * Complete the request future with a TimeoutException if the request has been sent out + * at least once and the timeout has been reached. */ - void maybeExpire(long currentTimeMs) { - if (retryTimeoutExpired(currentTimeMs)) { + void maybeExpire() { + if (numAttempts > 0 && isExpired()) { removeRequest(); - isExpired = true; future().completeExceptionally(new TimeoutException(requestDescription() + " could not complete before timeout expired.")); } @@ -846,11 +837,12 @@ void maybeExpire(long currentTimeMs) { NetworkClientDelegate.UnsentRequest buildRequestWithResponseHandling(final AbstractRequest.Builder builder) { NetworkClientDelegate.UnsentRequest request = new NetworkClientDelegate.UnsentRequest( builder, - coordinatorRequestManager.coordinator()); + coordinatorRequestManager.coordinator() + ); request.whenComplete( (response, throwable) -> { - long currentTimeMs = request.handler().completionTimeMs(); - handleClientResponse(response, throwable, currentTimeMs); + long completionTimeMs = request.handler().completionTimeMs(); + handleClientResponse(response, throwable, completionTimeMs); }); return request; } @@ -875,10 +867,6 @@ private void handleClientResponse(final ClientResponse response, abstract void onResponse(final ClientResponse response); - boolean retryTimeoutExpired(long currentTimeMs) { - return expirationTimeMs.isPresent() && expirationTimeMs.get() <= currentTimeMs; - } - abstract void removeRequest(); } @@ -898,10 +886,10 @@ class OffsetFetchRequestState extends RetriableRequestState { public OffsetFetchRequestState(final Set partitions, final long retryBackoffMs, final long retryBackoffMaxMs, - final long expirationTimeMs, + final long deadlineMs, final MemberInfo memberInfo) { super(logContext, CommitRequestManager.class.getSimpleName(), retryBackoffMs, - retryBackoffMaxMs, memberInfo, Optional.of(expirationTimeMs)); + retryBackoffMaxMs, memberInfo, deadlineTimer(time, deadlineMs)); this.requestedPartitions = partitions; this.future = new CompletableFuture<>(); } @@ -909,11 +897,11 @@ public OffsetFetchRequestState(final Set partitions, public OffsetFetchRequestState(final Set partitions, final long retryBackoffMs, final long retryBackoffMaxMs, - final long expirationTimeMs, + final long deadlineMs, final double jitter, final MemberInfo memberInfo) { super(logContext, CommitRequestManager.class.getSimpleName(), retryBackoffMs, 2, - retryBackoffMaxMs, jitter, memberInfo, Optional.of(expirationTimeMs)); + retryBackoffMaxMs, jitter, memberInfo, deadlineTimer(time, deadlineMs)); this.requestedPartitions = partitions; this.future = new CompletableFuture<>(); } @@ -1145,9 +1133,10 @@ private CompletableFuture> addOffsetFetch inflightOffsetFetches.stream().filter(r -> r.sameRequest(request)).findAny(); if (dupe.isPresent() || inflight.isPresent()) { - log.info("Duplicated OffsetFetchRequest: " + request.requestedPartitions); + log.debug("Duplicated unsent offset fetch request found for partitions: {}", request.requestedPartitions); dupe.orElseGet(inflight::get).chainFuture(request.future); } else { + log.debug("Enqueuing offset fetch request for partitions: {}", request.requestedPartitions); this.unsentOffsetFetches.add(request); } return request.future; @@ -1165,7 +1154,7 @@ List drain(final long currentTimeMs) { .filter(request -> !request.canSendRequest(currentTimeMs)) .collect(Collectors.toList()); - failAndRemoveExpiredCommitRequests(currentTimeMs); + failAndRemoveExpiredCommitRequests(); // Add all unsent offset commit requests to the unsentRequests list List unsentRequests = unsentOffsetCommits.stream() @@ -1179,7 +1168,7 @@ List drain(final long currentTimeMs) { unsentOffsetFetches.stream() .collect(Collectors.partitioningBy(request -> request.canSendRequest(currentTimeMs))); - failAndRemoveExpiredFetchRequests(currentTimeMs); + failAndRemoveExpiredFetchRequests(); // Add all sendable offset fetch requests to the unsentRequests list and to the inflightOffsetFetches list for (OffsetFetchRequestState request : partitionedBySendability.get(true)) { @@ -1200,18 +1189,18 @@ List drain(final long currentTimeMs) { * Find the unsent commit requests that have expired, remove them and complete their * futures with a TimeoutException. */ - private void failAndRemoveExpiredCommitRequests(final long currentTimeMs) { + private void failAndRemoveExpiredCommitRequests() { Queue requestsToPurge = new LinkedList<>(unsentOffsetCommits); - requestsToPurge.forEach(req -> req.maybeExpire(currentTimeMs)); + requestsToPurge.forEach(RetriableRequestState::maybeExpire); } /** * Find the unsent fetch requests that have expired, remove them and complete their * futures with a TimeoutException. */ - private void failAndRemoveExpiredFetchRequests(final long currentTimeMs) { + private void failAndRemoveExpiredFetchRequests() { Queue requestsToPurge = new LinkedList<>(unsentOffsetFetches); - requestsToPurge.forEach(req -> req.maybeExpire(currentTimeMs)); + requestsToPurge.forEach(RetriableRequestState::maybeExpire); } private void clearAll() { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerDelegateCreator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerDelegateCreator.java index bd95e06c86..81c45aba69 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerDelegateCreator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerDelegateCreator.java @@ -91,8 +91,7 @@ public ConsumerDelegate create(LogContext logContext, valueDeserializer, client, subscriptions, - metadata, - assignors + metadata ); else return new LegacyKafkaConsumer<>( diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java index 4699f00c15..50c7eb5b02 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java @@ -139,7 +139,7 @@ public RequestFuture send(Node node, public Node leastLoadedNode() { lock.lock(); try { - return client.leastLoadedNode(time.milliseconds()); + return client.leastLoadedNode(time.milliseconds()).node(); } finally { lock.unlock(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java index adee659460..64bba14837 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -207,8 +207,7 @@ private void reapExpiredApplicationEvents(long currentTimeMs) { */ // Visible for testing static void runAtClose(final Collection> requestManagers, - final NetworkClientDelegate networkClientDelegate, - final Timer timer) { + final NetworkClientDelegate networkClientDelegate) { // These are the optional outgoing requests at the requestManagers.stream() .filter(Optional::isPresent) @@ -293,21 +292,28 @@ private void closeInternal(final Duration timeout) { * Check the unsent queue one last time and poll until all requests are sent or the timer runs out. */ private void sendUnsentRequests(final Timer timer) { - if (networkClientDelegate.unsentRequests().isEmpty()) + if (!networkClientDelegate.hasAnyPendingRequests()) return; + do { networkClientDelegate.poll(timer.remainingMs(), timer.currentTimeMs()); timer.update(); - } while (timer.notExpired() && !networkClientDelegate.unsentRequests().isEmpty()); + } while (timer.notExpired() && networkClientDelegate.hasAnyPendingRequests()); + + if (networkClientDelegate.hasAnyPendingRequests()) { + log.warn("Close timeout of {} ms expired before the consumer network thread was able " + + "to complete pending requests. Inflight request count: {}, Unsent request count: {}", + timer.timeoutMs(), networkClientDelegate.inflightRequestCount(), networkClientDelegate.unsentRequests().size()); + } } void cleanup() { log.trace("Closing the consumer network thread"); Timer timer = time.timer(closeTimeout); try { - runAtClose(requestManagers.entries(), networkClientDelegate, timer); + runAtClose(requestManagers.entries(), networkClientDelegate); } catch (Exception e) { - log.error("Unexpected error during shutdown. Proceed with closing.", e); + log.error("Unexpected error during shutdown. Proceed with closing.", e); } finally { sendUnsentRequests(timer); applicationEventReaper.reap(applicationEventQueue); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java index a956ef3a93..43e15a4def 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java @@ -45,6 +45,7 @@ import java.util.TreeSet; import java.util.stream.Collectors; + /** *

Manages the request creation and response handling for the heartbeat. The module creates a * {@link ConsumerGroupHeartbeatRequest} using the state stored in the {@link MembershipManager} and enqueue it to @@ -208,7 +209,11 @@ public NetworkClientDelegate.PollResult poll(long currentTimeMs) { return new NetworkClientDelegate.PollResult(heartbeatRequestState.heartbeatIntervalMs, Collections.singletonList(leaveHeartbeat)); } - boolean heartbeatNow = membershipManager.shouldHeartbeatNow() && !heartbeatRequestState.requestInFlight(); + // Case 1: The member is leaving + boolean heartbeatNow = membershipManager.state() == MemberState.LEAVING || + // Case 2: The member state indicates it should send a heartbeat without waiting for the interval, and there is no heartbeat request currently in-flight + (membershipManager.shouldHeartbeatNow() && !heartbeatRequestState.requestInFlight()); + if (!heartbeatRequestState.canSendRequest(currentTimeMs) && !heartbeatNow) { return new NetworkClientDelegate.PollResult(heartbeatRequestState.timeToNextHeartbeatMs(currentTimeMs)); } @@ -258,7 +263,8 @@ public void resetPollTimer(final long pollMs) { pollTimer.update(pollMs); if (pollTimer.isExpired()) { logger.warn("Time between subsequent calls to poll() was longer than the configured " + - "max.poll.interval.ms, exceeded approximately by {} ms.", pollTimer.isExpiredBy()); + "max.poll.interval.ms, exceeded approximately by {} ms. Member {} will rejoin the group now.", + pollTimer.isExpiredBy(), membershipManager.memberId()); membershipManager.maybeRejoinStaleMember(); } pollTimer.reset(maxPollIntervalMs); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java index 76a550ad71..763fc02769 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java @@ -46,6 +46,7 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -520,12 +521,15 @@ private void replaceTargetAssignmentWithNewAssignment( @Override public void transitionToFenced() { if (state == MemberState.PREPARE_LEAVING) { - log.debug("Member {} with epoch {} got fenced but it is already preparing to leave " + + log.info("Member {} with epoch {} got fenced but it is already preparing to leave " + "the group, so it will stop sending heartbeat and won't attempt to rejoin.", memberId, memberEpoch); - // Transition to UNSUBSCRIBED, ensuring that the member (that is not part of the - // group anymore from the broker point of view) will stop sending heartbeats while it - // completes the ongoing leaving operation. + // Briefly transition to LEAVING to ensure all required actions are applied even + // though there is no need to send a leave group heartbeat (ex. clear epoch and + // notify epoch listeners). Then transition to UNSUBSCRIBED, ensuring that the member + // (that is not part of the group anymore from the broker point of view) will stop + // sending heartbeats while it completes the ongoing leaving operation. + transitionToSendingLeaveGroup(false); transitionTo(MemberState.UNSUBSCRIBED); return; } @@ -552,7 +556,7 @@ public void transitionToFenced() { log.error("onPartitionsLost callback invocation failed while releasing assignment" + " after member got fenced. Member will rejoin the group anyways.", error); } - clearSubscription(); + clearAssignment(); if (state == MemberState.FENCED) { transitionToJoining(); } else { @@ -589,7 +593,7 @@ public void transitionToFatal() { log.error("onPartitionsLost callback invocation failed while releasing assignment" + "after member failed with fatal error.", error); } - clearSubscription(); + clearAssignment(); }); } @@ -605,7 +609,7 @@ public void onSubscriptionUpdated() { /** * Clear the assigned partitions in the member subscription, pending assignments and metadata cache. */ - private void clearSubscription() { + private void clearAssignment() { if (subscriptions.hasAutoAssignedPartitions()) { subscriptions.assignFromSubscribed(Collections.emptySet()); } @@ -655,7 +659,7 @@ public void transitionToJoining() { public CompletableFuture leaveGroup() { if (isNotInGroup()) { if (state == MemberState.FENCED) { - clearSubscription(); + clearAssignment(); transitionTo(MemberState.UNSUBSCRIBED); } return CompletableFuture.completedFuture(null); @@ -664,6 +668,7 @@ public CompletableFuture leaveGroup() { if (state == MemberState.PREPARE_LEAVING || state == MemberState.LEAVING) { // Member already leaving. No-op and return existing leave group future that will // complete when the ongoing leave operation completes. + log.debug("Leave group operation already in progress for member {}", memberId); return leaveGroupInProgress.get(); } @@ -673,8 +678,16 @@ public CompletableFuture leaveGroup() { CompletableFuture callbackResult = invokeOnPartitionsRevokedOrLostToReleaseAssignment(); callbackResult.whenComplete((result, error) -> { + if (error != null) { + log.error("Member {} callback to release assignment failed. Member will proceed " + + "to send leave group heartbeat", memberId, error); + } else { + log.debug("Member {} completed callback to release assignment and will send leave " + + "group heartbeat", memberId); + } // Clear the subscription, no matter if the callback execution failed or succeeded. - clearSubscription(); + subscriptions.unsubscribe(); + clearAssignment(); // Transition to ensure that a heartbeat request is sent out to effectively leave the // group (even in the case where the member had no assignment to release or when the @@ -705,6 +718,9 @@ private CompletableFuture invokeOnPartitionsRevokedOrLostToReleaseAssignme SortedSet droppedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); droppedPartitions.addAll(subscriptions.assignedPartitions()); + log.debug("Member {} is triggering callbacks to release assignment {} and leave group", + memberId, droppedPartitions); + CompletableFuture callbackResult; if (droppedPartitions.isEmpty()) { // No assignment to release. @@ -764,7 +780,7 @@ public void transitionToSendingLeaveGroup(boolean dueToExpiredPollTimer) { * This also includes the latest member ID in the notification. If the member fails or leaves * the group, this will be invoked with empty epoch and member ID. */ - private void notifyEpochChange(Optional epoch, Optional memberId) { + void notifyEpochChange(Optional epoch, Optional memberId) { stateUpdatesListeners.forEach(stateListener -> stateListener.onMemberEpochUpdated(epoch, memberId)); } @@ -794,8 +810,12 @@ public void onHeartbeatRequestSent() { } } else if (state == MemberState.LEAVING) { if (isPollTimerExpired) { + log.debug("Member {} sent heartbeat to leave due to expired poll timer. It will " + + "remain stale (no heartbeat) until it rejoins the group on the next consumer " + + "poll.", memberId); transitionToStale(); } else { + log.debug("Member {} sent heartbeat to leave group.", memberId); transitionToUnsubscribed(); } } @@ -875,7 +895,7 @@ private void transitionToStale() { log.error("onPartitionsLost callback invocation failed while releasing assignment" + " after member left group due to expired poll timer.", error); } - clearSubscription(); + clearAssignment(); log.debug("Member {} sent leave group heartbeat and released its assignment. It will remain " + "in {} state until the poll timer is reset, and it will then rejoin the group", memberId, MemberState.STALE); @@ -939,11 +959,13 @@ void maybeReconcile() { revokedPartitions.addAll(ownedPartitions); revokedPartitions.removeAll(assignedTopicPartitions); - log.info("Updating assignment with local epoch {}\n" + + log.info("Reconciling assignment with local epoch {}\n" + + "\tMember: {}\n" + "\tAssigned partitions: {}\n" + "\tCurrent owned partitions: {}\n" + "\tAdded partitions (assigned - owned): {}\n" + "\tRevoked partitions (owned - assigned): {}\n", + memberId, resolvedAssignment.localEpoch, assignedTopicPartitions, ownedPartitions, @@ -960,7 +982,7 @@ void maybeReconcile() { // best effort to commit the offsets in the case where the epoch might have changed while // the current reconciliation is in process. Note this is using the rebalance timeout as // it is the limit enforced by the broker to complete the reconciliation process. - commitResult = commitRequestManager.maybeAutoCommitSyncBeforeRevocation(getExpirationTimeForTimeout(rebalanceTimeoutMs)); + commitResult = commitRequestManager.maybeAutoCommitSyncBeforeRevocation(getDeadlineMsForTimeout(rebalanceTimeoutMs)); // Execute commit -> onPartitionsRevoked -> onPartitionsAssigned. commitResult.whenComplete((__, commitReqError) -> { @@ -986,7 +1008,7 @@ void maybeReconcile() { }); } - long getExpirationTimeForTimeout(final long timeoutMs) { + long getDeadlineMsForTimeout(final long timeoutMs) { long expiration = time.milliseconds() + timeoutMs; if (expiration < 0) { return Long.MAX_VALUE; @@ -1175,11 +1197,14 @@ private Optional findTopicNameInGlobalOrLocalCache(Uuid topicId) { * request fails, this will proceed to invoke the user callbacks anyway, * returning a future that will complete or fail depending on the callback execution only. * - * @param revokedPartitions Partitions to revoke. + * @param partitionsToRevoke Partitions to revoke. * @return Future that will complete when the commit request and user callback completes. * Visible for testing */ - CompletableFuture revokePartitions(Set revokedPartitions) { + CompletableFuture revokePartitions(Set partitionsToRevoke) { + // Ensure the set of partitions to revoke are still assigned + Set revokedPartitions = new HashSet<>(partitionsToRevoke); + revokedPartitions.retainAll(subscriptions.assignedPartitions()); log.info("Revoking previously assigned partitions {}", revokedPartitions.stream().map(TopicPartition::toString).collect(Collectors.joining(", "))); logPausedPartitionsBeingRevoked(revokedPartitions); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java index 141f5f955c..83015acd89 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java @@ -81,6 +81,10 @@ Queue unsentRequests() { return unsentRequests; } + public int inflightRequestCount() { + return client.inFlightRequestCount(); + } + /** * Check if the node is disconnected and unavailable for immediate reconnection (i.e. if it is in * reconnect backoff window following the disconnect). @@ -130,6 +134,13 @@ public void poll(final long timeoutMs, final long currentTimeMs) { checkDisconnects(currentTimeMs); } + /** + * Return true if there is at least one in-flight request or unsent request. + */ + public boolean hasAnyPendingRequests() { + return client.hasInFlightRequests() || !unsentRequests.isEmpty(); + } + /** * Tries to send the requests in the unsentRequest queue. If the request doesn't have an assigned node, it will * find the leastLoadedOne, and will be retried in the next {@code poll()}. If the request is expired, a @@ -156,7 +167,7 @@ private void trySend(final long currentTimeMs) { } boolean doSend(final UnsentRequest r, final long currentTimeMs) { - Node node = r.node.orElse(client.leastLoadedNode(currentTimeMs)); + Node node = r.node.orElse(client.leastLoadedNode(currentTimeMs).node()); if (node == null || nodeUnavailable(node)) { log.debug("No broker available to send the request: {}. Retrying.", r); return false; @@ -201,7 +212,7 @@ private ClientRequest makeClientRequest( } public Node leastLoadedNode() { - return this.client.leastLoadedNode(time.milliseconds()); + return this.client.leastLoadedNode(time.milliseconds()).node(); } public void wakeup() { @@ -309,11 +320,20 @@ Optional node() { @Override public String toString() { + String remainingMs; + + if (timer != null) { + timer.update(); + remainingMs = String.valueOf(timer.remainingMs()); + } else { + remainingMs = ""; + } + return "UnsentRequest{" + "requestBuilder=" + requestBuilder + ", handler=" + handler + ", node=" + node + - ", timer=" + timer + + ", remainingMs=" + remainingMs + '}'; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java index 75d87432db..edd8ca9721 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java @@ -155,6 +155,7 @@ protected RequestManagers create() { apiVersions); final TopicMetadataRequestManager topic = new TopicMetadataRequestManager( logContext, + time, config); HeartbeatRequestManager heartbeatRequestManager = null; MembershipManager membershipManager = null; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TimedRequestState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TimedRequestState.java new file mode 100644 index 0000000000..c61032cea7 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TimedRequestState.java @@ -0,0 +1,71 @@ +/* + * 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. + */ +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; + +/** + * {@code TimedRequestState} adds to a {@link RequestState} a {@link Timer} with which to keep track + * of the request's expiration. + */ +public class TimedRequestState extends RequestState { + + private final Timer timer; + + public TimedRequestState(final LogContext logContext, + final String owner, + final long retryBackoffMs, + final long retryBackoffMaxMs, + final Timer timer) { + super(logContext, owner, retryBackoffMs, retryBackoffMaxMs); + this.timer = timer; + } + + public TimedRequestState(final LogContext logContext, + final String owner, + final long retryBackoffMs, + final int retryBackoffExpBase, + final long retryBackoffMaxMs, + final double jitter, + final Timer timer) { + super(logContext, owner, retryBackoffMs, retryBackoffExpBase, retryBackoffMaxMs, jitter); + this.timer = timer; + } + + public boolean isExpired() { + timer.update(); + return timer.isExpired(); + } + + public long remainingMs() { + timer.update(); + return timer.remainingMs(); + } + + public static Timer deadlineTimer(final Time time, final long deadlineMs) { + long diff = Math.max(0, deadlineMs - time.milliseconds()); + return time.timer(diff); + } + + + @Override + protected String toStringBase() { + return super.toStringBase() + ", remainingMs=" + remainingMs(); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java index 75a5ed08d1..a555d6ce7f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java @@ -29,6 +29,7 @@ import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; import org.slf4j.Logger; import java.util.Collections; @@ -61,6 +62,7 @@ */ public class TopicMetadataRequestManager implements RequestManager { + private final Time time; private final boolean allowAutoTopicCreation; private final List inflightRequests; private final long retryBackoffMs; @@ -68,9 +70,10 @@ public class TopicMetadataRequestManager implements RequestManager { private final Logger log; private final LogContext logContext; - public TopicMetadataRequestManager(final LogContext context, final ConsumerConfig config) { + public TopicMetadataRequestManager(final LogContext context, final Time time, final ConsumerConfig config) { logContext = context; log = logContext.logger(getClass()); + this.time = time; inflightRequests = new LinkedList<>(); retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); retryBackoffMaxMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG); @@ -81,7 +84,7 @@ public TopicMetadataRequestManager(final LogContext context, final ConsumerConfi public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { // Prune any requests which have timed out List expiredRequests = inflightRequests.stream() - .filter(req -> req.isExpired(currentTimeMs)) + .filter(TimedRequestState::isExpired) .collect(Collectors.toList()); expiredRequests.forEach(TopicMetadataRequestState::expire); @@ -99,10 +102,10 @@ public NetworkClientDelegate.PollResult poll(final long currentTimeMs) { * * @return the future of the metadata request. */ - public CompletableFuture>> requestAllTopicsMetadata(final long expirationTimeMs) { + public CompletableFuture>> requestAllTopicsMetadata(final long deadlineMs) { TopicMetadataRequestState newRequest = new TopicMetadataRequestState( logContext, - expirationTimeMs, + deadlineMs, retryBackoffMs, retryBackoffMaxMs); inflightRequests.add(newRequest); @@ -115,11 +118,11 @@ public CompletableFuture>> requestAllTopicsMetad * @param topic to be requested. * @return the future of the metadata request. */ - public CompletableFuture>> requestTopicMetadata(final String topic, final long expirationTimeMs) { + public CompletableFuture>> requestTopicMetadata(final String topic, final long deadlineMs) { TopicMetadataRequestState newRequest = new TopicMetadataRequestState( logContext, topic, - expirationTimeMs, + deadlineMs, retryBackoffMs, retryBackoffMaxMs); inflightRequests.add(newRequest); @@ -131,35 +134,32 @@ List inflightRequests() { return inflightRequests; } - class TopicMetadataRequestState extends RequestState { + class TopicMetadataRequestState extends TimedRequestState { private final String topic; private final boolean allTopics; - private final long expirationTimeMs; CompletableFuture>> future; public TopicMetadataRequestState(final LogContext logContext, - final long expirationTimeMs, + final long deadlineMs, final long retryBackoffMs, final long retryBackoffMaxMs) { super(logContext, TopicMetadataRequestState.class.getSimpleName(), retryBackoffMs, - retryBackoffMaxMs); + retryBackoffMaxMs, deadlineTimer(time, deadlineMs)); future = new CompletableFuture<>(); this.topic = null; this.allTopics = true; - this.expirationTimeMs = expirationTimeMs; } public TopicMetadataRequestState(final LogContext logContext, final String topic, - final long expirationTimeMs, + final long deadlineMs, final long retryBackoffMs, final long retryBackoffMaxMs) { super(logContext, TopicMetadataRequestState.class.getSimpleName(), retryBackoffMs, - retryBackoffMaxMs); + retryBackoffMaxMs, deadlineTimer(time, deadlineMs)); future = new CompletableFuture<>(); this.topic = topic; this.allTopics = false; - this.expirationTimeMs = expirationTimeMs; } /** @@ -167,10 +167,6 @@ public TopicMetadataRequestState(final LogContext logContext, * {@link org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.UnsentRequest} if needed. */ private Optional send(final long currentTimeMs) { - if (currentTimeMs >= expirationTimeMs) { - return Optional.empty(); - } - if (!canSendRequest(currentTimeMs)) { return Optional.empty(); } @@ -183,10 +179,6 @@ private Optional send(final long currentTim return Optional.of(createUnsentRequest(request)); } - private boolean isExpired(final long currentTimeMs) { - return currentTimeMs >= expirationTimeMs; - } - private void expire() { completeFutureAndRemoveRequest( new TimeoutException("Timeout expired while fetching topic metadata")); @@ -210,9 +202,8 @@ private NetworkClientDelegate.UnsentRequest createUnsentRequest( private void handleError(final Throwable exception, final long completionTimeMs) { if (exception instanceof RetriableException) { - if (completionTimeMs >= expirationTimeMs) { - completeFutureAndRemoveRequest( - new TimeoutException("Timeout expired while fetching topic metadata")); + if (isExpired()) { + completeFutureAndRemoveRequest(new TimeoutException("Timeout expired while fetching topic metadata")); } else { onFailedAttempt(completionTimeMs); } @@ -222,20 +213,12 @@ private void handleError(final Throwable exception, } private void handleResponse(final ClientResponse response) { - long responseTimeMs = response.receivedTimeMs(); try { Map> res = handleTopicMetadataResponse((MetadataResponse) response.responseBody()); future.complete(res); inflightRequests.remove(this); - } catch (RetriableException e) { - if (responseTimeMs >= expirationTimeMs) { - completeFutureAndRemoveRequest( - new TimeoutException("Timeout expired while fetching topic metadata")); - } else { - onFailedAttempt(responseTimeMs); - } - } catch (Exception t) { - completeFutureAndRemoveRequest(t); + } catch (Exception e) { + handleError(e, response.receivedTimeMs()); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java index 2897117da8..ae5f7faeae 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java @@ -31,7 +31,7 @@ public enum Type { COMMIT_ASYNC, COMMIT_SYNC, POLL, FETCH_COMMITTED_OFFSETS, NEW_TOPICS_METADATA_UPDATE, ASSIGNMENT_CHANGE, LIST_OFFSETS, RESET_POSITIONS, VALIDATE_POSITIONS, TOPIC_METADATA, ALL_TOPICS_METADATA, SUBSCRIPTION_CHANGE, UNSUBSCRIBE, CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED, - COMMIT_ON_CLOSE, LEAVE_ON_CLOSE + COMMIT_ON_CLOSE } private final Type type; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 7ee0c09d40..9d0062f546 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -32,7 +32,6 @@ import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.function.BiConsumer; import java.util.function.Supplier; @@ -119,10 +118,6 @@ public void process(ApplicationEvent event) { process((CommitOnCloseEvent) event); return; - case LEAVE_ON_CLOSE: - process((LeaveOnCloseEvent) event); - return; - default: log.warn("Application event type " + event.type() + " was not expected"); } @@ -268,20 +263,6 @@ private void process(@SuppressWarnings("unused") final CommitOnCloseEvent event) requestManagers.commitRequestManager.get().signalClose(); } - private void process(final LeaveOnCloseEvent event) { - if (!requestManagers.heartbeatRequestManager.isPresent()) { - event.future().complete(null); - return; - } - MembershipManager membershipManager = - Objects.requireNonNull(requestManagers.heartbeatRequestManager.get().membershipManager(), "Expecting " + - "membership manager to be non-null"); - log.debug("Leaving group before closing"); - CompletableFuture future = membershipManager.leaveGroup(); - // The future will be completed on heartbeat sent - future.whenComplete(complete(event.future())); - } - private BiConsumer complete(final CompletableFuture b) { return (value, exception) -> { if (exception != null) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index dffac12902..8cd17d19fe 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -43,6 +43,7 @@ public CompletableFuture future() { return future; } + @Override public long deadlineMs() { return deadlineMs; } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index c67d60a180..a59ee81b4a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.ClientDnsLookup; import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.MetadataRecoveryStrategy; import org.apache.kafka.common.compress.GzipCompression; import org.apache.kafka.common.compress.Lz4Compression; import org.apache.kafka.common.compress.ZstdCompression; @@ -528,7 +529,14 @@ public class ProducerConfig extends AbstractConfig { null, new ConfigDef.NonEmptyString(), Importance.LOW, - TRANSACTIONAL_ID_DOC); + TRANSACTIONAL_ID_DOC) + .define(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG, + Type.STRING, + CommonClientConfigs.DEFAULT_METADATA_RECOVERY_STRATEGY, + ConfigDef.CaseInsensitiveValidString + .in(Utils.enumOptions(MetadataRecoveryStrategy.class)), + Importance.LOW, + CommonClientConfigs.METADATA_RECOVERY_STRATEGY_DOC); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index c4e2b73e8b..b1a3ab9e29 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -484,7 +484,7 @@ private boolean maybeSendAndPollTransactionalRequest() { FindCoordinatorRequest.CoordinatorType coordinatorType = nextRequestHandler.coordinatorType(); targetNode = coordinatorType != null ? transactionManager.coordinator(coordinatorType) : - client.leastLoadedNode(time.milliseconds()); + client.leastLoadedNode(time.milliseconds()).node(); if (targetNode != null) { if (!awaitNodeReady(targetNode, coordinatorType)) { log.trace("Target node {} not ready within request timeout, will retry when node is ready.", targetNode); diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java index d82d06fa16..fd40c031ee 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java @@ -766,14 +766,7 @@ else if (value instanceof String) if (value instanceof Class) return value; else if (value instanceof String) { - ClassLoader contextOrKafkaClassLoader = Utils.getContextOrKafkaClassLoader(); - // Use loadClass here instead of Class.forName because the name we use here may be an alias - // and not match the name of the class that gets loaded. If that happens, Class.forName can - // throw an exception. - Class klass = contextOrKafkaClassLoader.loadClass(trimmed); - // Invoke forName here with the true name of the requested class to cause class - // initialization to take place. - return Class.forName(klass.getName(), true, contextOrKafkaClassLoader); + return Utils.loadClass(trimmed, Object.class); } else throw new ConfigException(name, value, "Expected a Class instance or class name."); default: diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java index c3083457ad..84289a0cd3 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java @@ -871,6 +871,18 @@ public boolean hasRoomFor(long timestamp, ByteBuffer key, ByteBuffer value, Head return this.writeLimit >= estimatedBytesWritten() + recordSize; } + /** + * Check if we have room for a given number of bytes. + */ + public boolean hasRoomFor(int estimatedRecordsSize) { + if (isFull()) return false; + return this.writeLimit >= estimatedBytesWritten() + estimatedRecordsSize; + } + + public int maxAllowedBytes() { + return this.writeLimit - this.batchHeaderSizeInBytes; + } + public boolean isClosed() { return builtRecords != null; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java index fc996453d6..efdc7da2af 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java @@ -47,8 +47,6 @@ public class ListOffsetsRequest extends AbstractRequest { */ public static final long EARLIEST_LOCAL_TIMESTAMP = -4L; - public static final long LATEST_TIERED_TIMESTAMP = -5L; - public static final int CONSUMER_REPLICA_ID = -1; public static final int DEBUGGING_REPLICA_ID = -2; diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index b25383aaad..75a024e197 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -94,7 +94,7 @@ private Utils() {} // This matches URIs of formats: host:port and protocol://host:port // IPv6 is supported with [ip] pattern - private static final Pattern HOST_PORT_PATTERN = Pattern.compile("^(?:[a-zA-Z][a-zA-Z\\d+-.]*://)?\\[?([0-9a-zA-Z\\-._%:]+)\\]?:([0-9]+)$"); + private static final Pattern HOST_PORT_PATTERN = Pattern.compile("^(?:[0-9a-zA-Z\\-%._]*://)?\\[?([0-9a-zA-Z\\-%._:]*)]?:([0-9]+)"); private static final Pattern VALID_HOST_CHARACTERS = Pattern.compile("([0-9a-zA-Z\\-%._:]*)"); @@ -451,7 +451,14 @@ public static T newInstance(String klass, Class base) throws ClassNotFoun * @return the new class */ public static Class loadClass(String klass, Class base) throws ClassNotFoundException { - return Class.forName(klass, true, Utils.getContextOrKafkaClassLoader()).asSubclass(base); + ClassLoader contextOrKafkaClassLoader = Utils.getContextOrKafkaClassLoader(); + // Use loadClass here instead of Class.forName because the name we use here may be an alias + // and not match the name of the class that gets loaded. If that happens, Class.forName can + // throw an exception. + Class loadedClass = contextOrKafkaClassLoader.loadClass(klass); + // Invoke forName here with the true name of the requested class to cause class + // initialization to take place. + return Class.forName(loadedClass.getName(), true, contextOrKafkaClassLoader).asSubclass(base); } /** @@ -480,7 +487,7 @@ public static T newParameterizedInstance(String className, Object... params) Class[] argTypes = new Class[params.length / 2]; Object[] args = new Object[params.length / 2]; try { - Class c = Class.forName(className, true, Utils.getContextOrKafkaClassLoader()); + Class c = Utils.loadClass(className, Object.class); for (int i = 0; i < params.length / 2; i++) { argTypes[i] = (Class) params[2 * i]; args[i] = params[(2 * i) + 1]; diff --git a/clients/src/test/java/org/apache/kafka/clients/MockClient.java b/clients/src/test/java/org/apache/kafka/clients/MockClient.java index d6cdd14f36..86d1ddf5f4 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java @@ -319,7 +319,7 @@ public List poll(long timeoutMs, long now) { checkTimeoutOfPendingRequests(now); // We skip metadata updates if all nodes are currently blacked out - if (metadataUpdater.isUpdateNeeded() && leastLoadedNode(now) != null) { + if (metadataUpdater.isUpdateNeeded() && leastLoadedNode(now).node() != null) { MetadataUpdate metadataUpdate = metadataUpdates.poll(); if (metadataUpdate != null) { metadataUpdater.update(time, metadataUpdate); @@ -588,13 +588,13 @@ public void close(String node) { } @Override - public Node leastLoadedNode(long now) { + public LeastLoadedNode leastLoadedNode(long now) { // Consistent with NetworkClient, we do not return nodes awaiting reconnect backoff for (Node node : metadataUpdater.fetchNodes()) { if (!connectionState(node.idString()).isBackingOff(now)) - return node; + return new LeastLoadedNode(node, true); } - return null; + return new LeastLoadedNode(null, false); } public void setWakeupHook(Runnable wakeupHook) { diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index cd3ec36f38..4369c8404e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -128,7 +128,16 @@ public class NetworkClientTest { private NetworkClient createNetworkClient(long reconnectBackoffMaxMs) { return new NetworkClient(selector, metadataUpdater, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMs, 64 * 1024, 64 * 1024, - defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext()); + defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext(), + MetadataRecoveryStrategy.NONE); + } + + private NetworkClient createNetworkClientWithMaxInFlightRequestsPerConnection( + int maxInFlightRequestsPerConnection, long reconnectBackoffMaxMs) { + return new NetworkClient(selector, metadataUpdater, "mock", maxInFlightRequestsPerConnection, + reconnectBackoffMsTest, reconnectBackoffMaxMs, 64 * 1024, 64 * 1024, + defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext(), + MetadataRecoveryStrategy.NONE); } private NetworkClient createNetworkClientWithMultipleNodes(long reconnectBackoffMaxMs, long connectionSetupTimeoutMsTest, int nodeNumber) { @@ -136,26 +145,30 @@ private NetworkClient createNetworkClientWithMultipleNodes(long reconnectBackoff TestMetadataUpdater metadataUpdater = new TestMetadataUpdater(nodes); return new NetworkClient(selector, metadataUpdater, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMs, 64 * 1024, 64 * 1024, - defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext()); + defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext(), + MetadataRecoveryStrategy.NONE); } private NetworkClient createNetworkClientWithStaticNodes() { return new NetworkClient(selector, metadataUpdater, "mock-static", Integer.MAX_VALUE, 0, 0, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, - connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext()); + connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext(), + MetadataRecoveryStrategy.NONE); } private NetworkClient createNetworkClientWithNoVersionDiscovery(Metadata metadata) { return new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, - defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, false, new ApiVersions(), new LogContext()); + defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, false, new ApiVersions(), new LogContext(), + MetadataRecoveryStrategy.NONE); } private NetworkClient createNetworkClientWithNoVersionDiscovery() { return new NetworkClient(selector, metadataUpdater, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, - connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, false, new ApiVersions(), new LogContext()); + connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, false, new ApiVersions(), new LogContext(), + MetadataRecoveryStrategy.NONE); } @BeforeEach @@ -698,14 +711,18 @@ private void sendThrottledProduceResponse(int correlationId, int throttleMs, sho public void testLeastLoadedNode() { client.ready(node, time.milliseconds()); assertFalse(client.isReady(node, time.milliseconds())); - assertEquals(node, client.leastLoadedNode(time.milliseconds())); + LeastLoadedNode leastLoadedNode = client.leastLoadedNode(time.milliseconds()); + assertEquals(node, leastLoadedNode.node()); + assertTrue(leastLoadedNode.hasNodeAvailableOrConnectionReady()); awaitReady(client, node); client.poll(1, time.milliseconds()); assertTrue(client.isReady(node, time.milliseconds()), "The client should be ready"); // leastloadednode should be our single node - Node leastNode = client.leastLoadedNode(time.milliseconds()); + leastLoadedNode = client.leastLoadedNode(time.milliseconds()); + assertTrue(leastLoadedNode.hasNodeAvailableOrConnectionReady()); + Node leastNode = leastLoadedNode.node(); assertEquals(leastNode.id(), node.id(), "There should be one leastloadednode"); // sleep for longer than reconnect backoff @@ -716,8 +733,29 @@ public void testLeastLoadedNode() { client.poll(1, time.milliseconds()); assertFalse(client.ready(node, time.milliseconds()), "After we forced the disconnection the client is no longer ready."); - leastNode = client.leastLoadedNode(time.milliseconds()); - assertNull(leastNode, "There should be NO leastloadednode"); + leastLoadedNode = client.leastLoadedNode(time.milliseconds()); + assertFalse(leastLoadedNode.hasNodeAvailableOrConnectionReady()); + assertNull(leastLoadedNode.node(), "There should be NO leastloadednode"); + } + + @Test + public void testHasNodeAvailableOrConnectionReady() { + NetworkClient client = createNetworkClientWithMaxInFlightRequestsPerConnection(1, reconnectBackoffMaxMsTest); + awaitReady(client, node); + + long now = time.milliseconds(); + LeastLoadedNode leastLoadedNode = client.leastLoadedNode(now); + assertEquals(node, leastLoadedNode.node()); + assertTrue(leastLoadedNode.hasNodeAvailableOrConnectionReady()); + + MetadataRequest.Builder builder = new MetadataRequest.Builder(Collections.emptyList(), true); + ClientRequest request = client.newClientRequest(node.idString(), builder, now, true); + client.send(request, now); + client.poll(defaultRequestTimeoutMs, now); + + leastLoadedNode = client.leastLoadedNode(now); + assertNull(leastLoadedNode.node()); + assertTrue(leastLoadedNode.hasNodeAvailableOrConnectionReady()); } @Test @@ -727,7 +765,7 @@ public void testLeastLoadedNodeProvideDisconnectedNodesPrioritizedByLastConnecti Set providedNodeIds = new HashSet<>(); for (int i = 0; i < nodeNumber * 10; i++) { - Node node = client.leastLoadedNode(time.milliseconds()); + Node node = client.leastLoadedNode(time.milliseconds()).node(); assertNotNull(node, "Should provide a node"); providedNodeIds.add(node); client.ready(node, time.milliseconds()); @@ -800,7 +838,7 @@ public void testLeastLoadedNodeConsidersThrottledConnections() { client.poll(1, time.milliseconds()); // leastloadednode should return null since the node is throttled - assertNull(client.leastLoadedNode(time.milliseconds())); + assertNull(client.leastLoadedNode(time.milliseconds()).node()); } @Test @@ -1046,7 +1084,8 @@ public void testReconnectAfterAddressChange() { NetworkClient client = new NetworkClient(metadataUpdater, null, selector, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, - time, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender); + time, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender, + MetadataRecoveryStrategy.NONE); // Connect to one the initial addresses, then change the addresses and disconnect client.ready(node, time.milliseconds()); @@ -1106,7 +1145,8 @@ public void testFailedConnectionToFirstAddress() { NetworkClient client = new NetworkClient(metadataUpdater, null, selector, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, - time, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender); + time, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender, + MetadataRecoveryStrategy.NONE); // First connection attempt should fail client.ready(node, time.milliseconds()); @@ -1158,7 +1198,8 @@ public void testFailedConnectionToFirstAddressAfterReconnect() { NetworkClient client = new NetworkClient(metadataUpdater, null, selector, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, - time, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender); + time, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender, + MetadataRecoveryStrategy.NONE); // Connect to one the initial addresses, then change the addresses and disconnect client.ready(node, time.milliseconds()); @@ -1266,7 +1307,8 @@ public void testTelemetryRequest() { NetworkClient client = new NetworkClient(metadataUpdater, null, selector, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, - time, true, new ApiVersions(), null, new LogContext(), new DefaultHostResolver(), mockClientTelemetrySender); + time, true, new ApiVersions(), null, new LogContext(), new DefaultHostResolver(), mockClientTelemetrySender, + MetadataRecoveryStrategy.NONE); // Send the ApiVersionsRequest client.ready(node, time.milliseconds()); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientConfigTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientConfigTest.java new file mode 100644 index 0000000000..92dc56fde4 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientConfigTest.java @@ -0,0 +1,47 @@ +/* + * 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. + */ + +package org.apache.kafka.clients.admin; + +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.MetadataRecoveryStrategy; +import org.apache.kafka.common.config.ConfigException; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class AdminClientConfigTest { + @Test + public void testDefaultMetadataRecoveryStrategy() { + Map configs = new HashMap<>(); + final AdminClientConfig adminClientConfig = new AdminClientConfig(configs); + assertEquals(MetadataRecoveryStrategy.NONE.name, adminClientConfig.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)); + } + + @Test + public void testInvalidMetadataRecoveryStrategy() { + Map configs = new HashMap<>(); + configs.put(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG, "abc"); + ConfigException ce = assertThrows(ConfigException.class, () -> new AdminClientConfig(configs)); + assertTrue(ce.getMessage().contains(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)); + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index ea1305e533..e04de635f7 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -1464,6 +1464,7 @@ public void testDescribeTopicsWithDescribeTopicPartitionsApiBasic() { assertEquals(0, topicDescription.partitions().get(0).partition()); assertEquals(1, topicDescription.partitions().get(1).partition()); topicDescription = topicDescriptions.get(topicName1); + assertNull(topicDescription.authorizedOperations()); assertEquals(1, topicDescription.partitions().size()); } catch (Exception e) { fail("describe using DescribeTopics API should not fail", e); @@ -1471,6 +1472,77 @@ public void testDescribeTopicsWithDescribeTopicPartitionsApiBasic() { } } + @Test + public void testDescribeTopicPartitionsApiWithAuthorizedOps() throws ExecutionException, InterruptedException { + try (AdminClientUnitTestEnv env = mockClientEnv()) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + String topicName0 = "test-0"; + Uuid topicId = Uuid.randomUuid(); + + int authorisedOperations = Utils.to32BitField(Utils.mkSet(AclOperation.DESCRIBE.code(), AclOperation.ALTER.code())); + env.kafkaClient().prepareResponse( + prepareDescribeClusterResponse(0, + env.cluster().nodes(), + env.cluster().clusterResource().clusterId(), + 2, + authorisedOperations) + ); + + DescribeTopicPartitionsResponseData responseData = new DescribeTopicPartitionsResponseData(); + responseData.topics().add(new DescribeTopicPartitionsResponseTopic() + .setErrorCode((short) 0) + .setTopicId(topicId) + .setName(topicName0) + .setIsInternal(false) + .setTopicAuthorizedOperations(authorisedOperations)); + env.kafkaClient().prepareResponse(new DescribeTopicPartitionsResponse(responseData)); + + DescribeTopicsResult result = env.adminClient().describeTopics( + singletonList(topicName0), new DescribeTopicsOptions().includeAuthorizedOperations(true) + ); + + Map topicDescriptions = result.allTopicNames().get(); + TopicDescription topicDescription = topicDescriptions.get(topicName0); + assertEquals(new HashSet<>(asList(AclOperation.DESCRIBE, AclOperation.ALTER)), + topicDescription.authorizedOperations()); + } + } + + @Test + public void testDescribeTopicPartitionsApiWithoutAuthorizedOps() throws ExecutionException, InterruptedException { + try (AdminClientUnitTestEnv env = mockClientEnv()) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + String topicName0 = "test-0"; + Uuid topicId = Uuid.randomUuid(); + + int authorisedOperations = Utils.to32BitField(Utils.mkSet(AclOperation.DESCRIBE.code(), AclOperation.ALTER.code())); + env.kafkaClient().prepareResponse( + prepareDescribeClusterResponse(0, + env.cluster().nodes(), + env.cluster().clusterResource().clusterId(), + 2, + authorisedOperations) + ); + + DescribeTopicPartitionsResponseData responseData = new DescribeTopicPartitionsResponseData(); + responseData.topics().add(new DescribeTopicPartitionsResponseTopic() + .setErrorCode((short) 0) + .setTopicId(topicId) + .setName(topicName0) + .setIsInternal(false) + .setTopicAuthorizedOperations(authorisedOperations)); + env.kafkaClient().prepareResponse(new DescribeTopicPartitionsResponse(responseData)); + + DescribeTopicsResult result = env.adminClient().describeTopics( + singletonList(topicName0), new DescribeTopicsOptions().includeAuthorizedOperations(false) + ); + + Map topicDescriptions = result.allTopicNames().get(); + TopicDescription topicDescription = topicDescriptions.get(topicName0); + assertNull(topicDescription.authorizedOperations()); + } + } + @SuppressWarnings({"NPathComplexity", "CyclomaticComplexity"}) @Test public void testDescribeTopicsWithDescribeTopicPartitionsApiEdgeCase() { @@ -1547,6 +1619,7 @@ public void testDescribeTopicsWithDescribeTopicPartitionsApiEdgeCase() { assertEquals(2, topicDescription.partitions().size()); topicDescription = topicDescriptions.get(topicName2); assertEquals(2, topicDescription.partitions().size()); + assertNull(topicDescription.authorizedOperations()); } catch (Exception e) { fail("describe using DescribeTopics API should not fail", e); } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/FenceProducersHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/FenceProducersHandlerTest.java index 34ed2e6772..62258289a9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/FenceProducersHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/FenceProducersHandlerTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients.admin.internals; +import org.apache.kafka.clients.admin.FenceProducersOptions; import org.apache.kafka.clients.admin.internals.AdminApiHandler.ApiResult; import org.apache.kafka.common.Node; import org.apache.kafka.common.message.InitProducerIdResponseData; @@ -39,11 +40,21 @@ public class FenceProducersHandlerTest { private final LogContext logContext = new LogContext(); private final Node node = new Node(1, "host", 1234); + private final int requestTimeoutMs = 30000; + private final FenceProducersOptions options = new FenceProducersOptions(); @Test public void testBuildRequest() { - FenceProducersHandler handler = new FenceProducersHandler(logContext); - mkSet("foo", "bar", "baz").forEach(transactionalId -> assertLookup(handler, transactionalId)); + FenceProducersHandler handler = new FenceProducersHandler(options, logContext, requestTimeoutMs); + mkSet("foo", "bar", "baz").forEach(transactionalId -> assertLookup(handler, transactionalId, requestTimeoutMs)); + } + + @Test + public void testBuildRequestOptionsTimeout() { + final int optionsTimeoutMs = 50000; + options.timeoutMs(optionsTimeoutMs); + FenceProducersHandler handler = new FenceProducersHandler(options, logContext, requestTimeoutMs); + mkSet("foo", "bar", "baz").forEach(transactionalId -> assertLookup(handler, transactionalId, optionsTimeoutMs)); } @Test @@ -51,7 +62,7 @@ public void testHandleSuccessfulResponse() { String transactionalId = "foo"; CoordinatorKey key = CoordinatorKey.byTransactionalId(transactionalId); - FenceProducersHandler handler = new FenceProducersHandler(logContext); + FenceProducersHandler handler = new FenceProducersHandler(options, logContext, requestTimeoutMs); short epoch = 57; long producerId = 7; @@ -73,7 +84,7 @@ public void testHandleSuccessfulResponse() { @Test public void testHandleErrorResponse() { String transactionalId = "foo"; - FenceProducersHandler handler = new FenceProducersHandler(logContext); + FenceProducersHandler handler = new FenceProducersHandler(options, logContext, requestTimeoutMs); assertFatalError(handler, transactionalId, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED); assertFatalError(handler, transactionalId, Errors.CLUSTER_AUTHORIZATION_FAILED); assertFatalError(handler, transactionalId, Errors.UNKNOWN_SERVER_ERROR); @@ -83,6 +94,7 @@ public void testHandleErrorResponse() { assertRetriableError(handler, transactionalId, Errors.COORDINATOR_LOAD_IN_PROGRESS); assertUnmappedKey(handler, transactionalId, Errors.NOT_COORDINATOR); assertUnmappedKey(handler, transactionalId, Errors.COORDINATOR_NOT_AVAILABLE); + assertRetriableError(handler, transactionalId, Errors.CONCURRENT_TRANSACTIONS); } private void assertFatalError( @@ -136,10 +148,10 @@ private ApiResult handleResponseError( return result; } - private void assertLookup(FenceProducersHandler handler, String transactionalId) { + private void assertLookup(FenceProducersHandler handler, String transactionalId, int txnTimeoutMs) { CoordinatorKey key = CoordinatorKey.byTransactionalId(transactionalId); InitProducerIdRequest.Builder request = handler.buildSingleRequest(1, key); assertEquals(transactionalId, request.data.transactionalId()); - assertEquals(1, request.data.transactionTimeoutMs()); + assertEquals(txnTimeoutMs, request.data.transactionTimeoutMs()); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java index 8e9fa5722f..0fc8e6ca48 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.consumer; import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.MetadataRecoveryStrategy; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.security.auth.SecurityProtocol; @@ -191,6 +192,25 @@ public void testRemoteAssignorConfig() { assertEquals(remoteAssignorName, consumerConfig.getString(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); } + @Test + public void testDefaultMetadataRecoveryStrategy() { + Map configs = new HashMap<>(); + configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass); + configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); + final ConsumerConfig consumerConfig = new ConsumerConfig(configs); + assertEquals(MetadataRecoveryStrategy.NONE.name, consumerConfig.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)); + } + + @Test + public void testInvalidMetadataRecoveryStrategy() { + Map configs = new HashMap<>(); + configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass); + configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); + configs.put(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG, "abc"); + ConfigException ce = assertThrows(ConfigException.class, () -> new ConsumerConfig(configs)); + assertTrue(ce.getMessage().contains(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)); + } + @ParameterizedTest @CsvSource({"consumer, true", "classic, true", "Consumer, true", "Classic, true", "invalid, false"}) public void testProtocolConfigValidation(String protocol, boolean isValid) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 850ac2bd8f..cfe4d03e6a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -435,7 +435,7 @@ public void shouldIgnoreGroupInstanceIdForEmptyGroupId(GroupProtocol groupProtoc } @ParameterizedTest - @EnumSource(GroupProtocol.class) + @EnumSource(value = GroupProtocol.class, names = "CLASSIC") public void testSubscription(GroupProtocol groupProtocol) { consumer = newConsumer(groupProtocol, groupId); @@ -495,7 +495,7 @@ public void testSubscriptionOnEmptyPattern(GroupProtocol groupProtocol) { } @ParameterizedTest - @EnumSource(GroupProtocol.class) + @EnumSource(value = GroupProtocol.class, names = "CLASSIC") public void testSubscriptionWithEmptyPartitionAssignment(GroupProtocol groupProtocol) { Properties props = new Properties(); props.setProperty(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol.name()); @@ -3227,7 +3227,7 @@ public void testUnusedConfigs(GroupProtocol groupProtocol) { } @ParameterizedTest - @EnumSource(GroupProtocol.class) + @EnumSource(value = GroupProtocol.class, names = "CLASSIC") public void testAssignorNameConflict(GroupProtocol groupProtocol) { Map configs = new HashMap<>(); configs.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol.name()); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 66ee724a0e..09efcbbad9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -19,7 +19,6 @@ import org.apache.kafka.clients.Metadata.LeaderAndEpoch; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; -import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -30,7 +29,6 @@ import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.clients.consumer.RetriableCommitFailedException; -import org.apache.kafka.clients.consumer.RoundRobinAssignor; import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeEvent; @@ -44,7 +42,6 @@ import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.clients.consumer.internals.events.EventProcessor; import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsEvent; -import org.apache.kafka.clients.consumer.internals.events.LeaveOnCloseEvent; import org.apache.kafka.clients.consumer.internals.events.ListOffsetsEvent; import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; import org.apache.kafka.clients.consumer.internals.events.PollEvent; @@ -56,6 +53,7 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Metric; import org.apache.kafka.common.Node; +import org.apache.kafka.common.Cluster; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.FencedInstanceIdException; import org.apache.kafka.common.errors.GroupAuthorizationException; @@ -105,10 +103,10 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import java.util.regex.Pattern; import java.util.stream.Stream; import static java.util.Arrays.asList; -import static java.util.Collections.emptySet; import static java.util.Collections.singleton; import static java.util.Collections.singletonList; import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED; @@ -131,7 +129,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; @@ -139,8 +136,10 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mockStatic; import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.mockito.Mockito.clearInvocations; @SuppressWarnings("unchecked") public class AsyncKafkaConsumerTest { @@ -157,9 +156,15 @@ public class AsyncKafkaConsumerTest { public void resetAll() { backgroundEventQueue.clear(); if (consumer != null) { - consumer.close(Duration.ZERO); + try { + consumer.close(Duration.ZERO); + } catch (Exception e) { + // best effort to clean up after each test, but may throw (ex. if callbacks where + // throwing errors) + } } consumer = null; + Mockito.framework().clearInlineMocks(); MockConsumerInterceptor.resetCounters(); } @@ -205,7 +210,6 @@ private AsyncKafkaConsumer newConsumer( ConsumerInterceptors interceptors, ConsumerRebalanceListenerInvoker rebalanceListenerInvoker, SubscriptionState subscriptions, - List assignors, String groupId, String clientId) { long retryBackoffMs = 100L; @@ -228,7 +232,6 @@ private AsyncKafkaConsumer newConsumer( metadata, retryBackoffMs, defaultApiTimeoutMs, - assignors, groupId, autoCommitEnabled); } @@ -236,6 +239,7 @@ private AsyncKafkaConsumer newConsumer( @Test public void testSuccessfulStartupShutdown() { consumer = newConsumer(); + completeUnsubscribeApplicationEventSuccessfully(); assertDoesNotThrow(() -> consumer.close()); } @@ -248,6 +252,7 @@ public void testInvalidGroupId() { @Test public void testFailOnClosedConsumer() { consumer = newConsumer(); + completeUnsubscribeApplicationEventSuccessfully(); consumer.close(); final IllegalStateException res = assertThrows(IllegalStateException.class, consumer::assignment); assertEquals("This consumer has already been closed.", res.getMessage()); @@ -480,6 +485,44 @@ public void onPartitionsAssigned(final Collection partitions) { assertTrue(callbackExecuted.get()); } + @Test + public void testSubscriptionRegexEvalOnPollOnlyIfMetadataChanges() { + SubscriptionState subscriptions = mock(SubscriptionState.class); + Cluster cluster = mock(Cluster.class); + + consumer = newConsumer( + mock(FetchBuffer.class), + mock(ConsumerInterceptors.class), + mock(ConsumerRebalanceListenerInvoker.class), + subscriptions, + "group-id", + "client-id"); + + final String topicName = "foo"; + final int partition = 3; + final TopicPartition tp = new TopicPartition(topicName, partition); + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + Map offsets = Collections.singletonMap(tp, new OffsetAndMetadata(1)); + completeFetchedCommittedOffsetApplicationEventSuccessfully(offsets); + doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); + doReturn(cluster).when(metadata).fetch(); + doReturn(Collections.singleton(topicName)).when(cluster).topics(); + + consumer.subscribe(Pattern.compile("f*")); + verify(metadata).requestUpdateForNewTopics(); + verify(subscriptions).matchesSubscribedPattern(topicName); + clearInvocations(subscriptions); + + when(subscriptions.hasPatternSubscription()).thenReturn(true); + consumer.poll(Duration.ZERO); + verify(subscriptions, never()).matchesSubscribedPattern(topicName); + + when(metadata.updateVersion()).thenReturn(2); + when(subscriptions.hasPatternSubscription()).thenReturn(true); + consumer.poll(Duration.ZERO); + verify(subscriptions).matchesSubscribedPattern(topicName); + } + @Test public void testClearWakeupTriggerAfterPoll() { consumer = newConsumer(); @@ -537,6 +580,151 @@ public void testPollLongThrowsException() { "This method is deprecated and will be removed in the next major release.", e.getMessage()); } + @Test + public void testOffsetFetchStoresPendingEvent() { + consumer = newConsumer(); + long timeoutMs = 0; + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + consumer.assign(Collections.singleton(new TopicPartition("topic1", 0))); + + // The first attempt at poll() creates an event, enqueues it, but its Future does not complete within the + // timeout, leaving a pending fetch. + consumer.poll(Duration.ofMillis(timeoutMs)); + verify(applicationEventHandler).add(any(FetchCommittedOffsetsEvent.class)); + CompletableApplicationEvent> event = getLastEnqueuedEvent(); + assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event.future(), time.timer(timeoutMs))); + assertTrue(consumer.hasPendingOffsetFetchEvent()); + + clearInvocations(applicationEventHandler); + + // For the second attempt, the event is reused, so first verify that another FetchCommittedOffsetsEvent + // was not enqueued. On this attempt the Future returns successfully, clearing the pending fetch. + event.future().complete(Collections.emptyMap()); + consumer.poll(Duration.ofMillis(timeoutMs)); + verify(applicationEventHandler, never()).add(any(FetchCommittedOffsetsEvent.class)); + assertDoesNotThrow(() -> ConsumerUtils.getResult(event.future(), time.timer(timeoutMs))); + assertFalse(consumer.hasPendingOffsetFetchEvent()); + } + + @Test + public void testOffsetFetchDoesNotReuseMismatchedPendingEvent() { + consumer = newConsumer(); + long timeoutMs = 0; + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + + // The first attempt at poll() retrieves data for partition 0 of the topic. poll() creates an event, + // enqueues it, but its Future does not complete within the timeout, leaving a pending fetch. + consumer.assign(Collections.singleton(new TopicPartition("topic1", 0))); + consumer.poll(Duration.ofMillis(timeoutMs)); + verify(applicationEventHandler).add(any(FetchCommittedOffsetsEvent.class)); + CompletableApplicationEvent> event1 = getLastEnqueuedEvent(); + assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event1.future(), time.timer(timeoutMs))); + assertTrue(consumer.hasPendingOffsetFetchEvent()); + + clearInvocations(applicationEventHandler); + + // For the second attempt, the set of partitions is reassigned, causing the pending offset to be replaced. + // Verify that another FetchCommittedOffsetsEvent is enqueued. + consumer.assign(Collections.singleton(new TopicPartition("topic1", 1))); + consumer.poll(Duration.ofMillis(timeoutMs)); + verify(applicationEventHandler).add(any(FetchCommittedOffsetsEvent.class)); + CompletableApplicationEvent> event2 = getLastEnqueuedEvent(); + assertNotEquals(event1, event2); + assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event2.future(), time.timer(timeoutMs))); + assertTrue(consumer.hasPendingOffsetFetchEvent()); + + clearInvocations(applicationEventHandler); + + // For the third attempt, the event from attempt 2 is reused, so there should not have been another + // FetchCommittedOffsetsEvent enqueued. The Future is completed to make it return successfully in poll(). + // This will finally clear out the pending fetch. + event2.future().complete(Collections.emptyMap()); + consumer.poll(Duration.ofMillis(timeoutMs)); + verify(applicationEventHandler, never()).add(any(FetchCommittedOffsetsEvent.class)); + assertDoesNotThrow(() -> ConsumerUtils.getResult(event2.future(), time.timer(timeoutMs))); + assertFalse(consumer.hasPendingOffsetFetchEvent()); + } + + @Test + public void testOffsetFetchDoesNotReuseExpiredPendingEvent() { + consumer = newConsumer(); + long timeoutMs = 0; + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + consumer.assign(Collections.singleton(new TopicPartition("topic1", 0))); + + // The first attempt at poll() creates an event, enqueues it, but its Future does not complete within + // the timeout, leaving a pending fetch. + consumer.poll(Duration.ofMillis(timeoutMs)); + verify(applicationEventHandler).add(any(FetchCommittedOffsetsEvent.class)); + CompletableApplicationEvent> event1 = getLastEnqueuedEvent(); + assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event1.future(), time.timer(timeoutMs))); + assertTrue(consumer.hasPendingOffsetFetchEvent()); + + clearInvocations(applicationEventHandler); + + // Sleep past the event's expiration, causing the poll() to *not* reuse the pending fetch. A new event + // is created and added to the application event queue. + time.sleep(event1.deadlineMs() - time.milliseconds()); + consumer.poll(Duration.ofMillis(timeoutMs)); + verify(applicationEventHandler).add(any(FetchCommittedOffsetsEvent.class)); + CompletableApplicationEvent> event2 = getLastEnqueuedEvent(); + assertNotEquals(event1, event2); + assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event2.future(), time.timer(timeoutMs))); + assertTrue(consumer.hasPendingOffsetFetchEvent()); + } + + @Test + public void testOffsetFetchTimeoutExceptionKeepsPendingEvent() { + consumer = newConsumer(); + long timeoutMs = 0; + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + consumer.assign(Collections.singleton(new TopicPartition("topic1", 0))); + + consumer.poll(Duration.ofMillis(timeoutMs)); + verify(applicationEventHandler).add(any(FetchCommittedOffsetsEvent.class)); + CompletableApplicationEvent> event = getLastEnqueuedEvent(); + assertTrue(consumer.hasPendingOffsetFetchEvent()); + + event.future().completeExceptionally(new TimeoutException("Test error")); + assertDoesNotThrow(() -> consumer.poll(Duration.ofMillis(timeoutMs))); + assertTrue(consumer.hasPendingOffsetFetchEvent()); + } + + @Test + public void testOffsetFetchInterruptExceptionKeepsPendingEvent() { + consumer = newConsumer(); + long timeoutMs = 0; + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + consumer.assign(Collections.singleton(new TopicPartition("topic1", 0))); + + consumer.poll(Duration.ofMillis(timeoutMs)); + verify(applicationEventHandler).add(any(FetchCommittedOffsetsEvent.class)); + CompletableApplicationEvent> event = getLastEnqueuedEvent(); + assertTrue(consumer.hasPendingOffsetFetchEvent()); + + event.future().completeExceptionally(new InterruptException("Test error")); + assertThrows(InterruptException.class, () -> consumer.poll(Duration.ofMillis(timeoutMs))); + assertTrue(Thread.interrupted()); + assertTrue(consumer.hasPendingOffsetFetchEvent()); + } + + @Test + public void testOffsetFetchUnexpectedExceptionClearsPendingEvent() { + consumer = newConsumer(); + long timeoutMs = 0; + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + consumer.assign(Collections.singleton(new TopicPartition("topic1", 0))); + + consumer.poll(Duration.ofMillis(timeoutMs)); + verify(applicationEventHandler).add(any(FetchCommittedOffsetsEvent.class)); + CompletableApplicationEvent> event = getLastEnqueuedEvent(); + assertTrue(consumer.hasPendingOffsetFetchEvent()); + + event.future().completeExceptionally(new NullPointerException("Test error")); + assertThrows(KafkaException.class, () -> consumer.poll(Duration.ofMillis(timeoutMs))); + assertFalse(consumer.hasPendingOffsetFetchEvent()); + } + @Test public void testCommitSyncLeaderEpochUpdate() { consumer = newConsumer(); @@ -564,7 +752,6 @@ public void testCommitAsyncLeaderEpochUpdate() { new ConsumerInterceptors<>(Collections.emptyList()), mock(ConsumerRebalanceListenerInvoker.class), subscriptions, - singletonList(new RoundRobinAssignor()), "group-id", "client-id"); completeCommitSyncApplicationEventSuccessfully(); @@ -705,17 +892,17 @@ private CompletableFuture setUpConsumerWithIncompleteAsyncCommit(TopicPart consumer.seek(tp, 20); consumer.commitAsync(); - return getLastEnqueuedEventFuture(); + CompletableApplicationEvent event = getLastEnqueuedEvent(); + return event.future(); } // ArgumentCaptor's type-matching does not work reliably with Java 8, so we cannot directly capture the AsyncCommitEvent // Instead, we capture the super-class CompletableApplicationEvent and fetch the last captured event. - private CompletableFuture getLastEnqueuedEventFuture() { - final ArgumentCaptor> eventArgumentCaptor = ArgumentCaptor.forClass(CompletableApplicationEvent.class); + private CompletableApplicationEvent getLastEnqueuedEvent() { + final ArgumentCaptor> eventArgumentCaptor = ArgumentCaptor.forClass(CompletableApplicationEvent.class); verify(applicationEventHandler, atLeast(1)).add(eventArgumentCaptor.capture()); - final List> allValues = eventArgumentCaptor.getAllValues(); - final CompletableApplicationEvent lastEvent = allValues.get(allValues.size() - 1); - return lastEvent.future(); + final List> allValues = eventArgumentCaptor.getAllValues(); + return allValues.get(allValues.size() - 1); } @Test @@ -758,6 +945,7 @@ public void testEnsurePollExecutedCommitAsyncCallbacks() { @Test public void testEnsureShutdownExecutedCommitAsyncCallbacks() { consumer = newConsumer(); + completeUnsubscribeApplicationEventSuccessfully(); MockCommitCallback callback = new MockCommitCallback(); completeCommitAsyncApplicationEventSuccessfully(); assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); @@ -769,70 +957,45 @@ public void testEnsureShutdownExecutedCommitAsyncCallbacks() { @Test public void testVerifyApplicationEventOnShutdown() { consumer = newConsumer(); + completeUnsubscribeApplicationEventSuccessfully(); doReturn(null).when(applicationEventHandler).addAndGet(any()); consumer.close(); - verify(applicationEventHandler).addAndGet(any(LeaveOnCloseEvent.class)); + verify(applicationEventHandler).add(any(UnsubscribeEvent.class)); verify(applicationEventHandler).add(any(CommitOnCloseEvent.class)); } @Test - public void testPartitionRevocationOnClose() { - MockRebalanceListener listener = new MockRebalanceListener(); - SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE); - consumer = newConsumer( + public void testUnsubscribeOnClose() { + SubscriptionState subscriptions = mock(SubscriptionState.class); + consumer = spy(newConsumer( mock(FetchBuffer.class), mock(ConsumerInterceptors.class), mock(ConsumerRebalanceListenerInvoker.class), subscriptions, - singletonList(new RoundRobinAssignor()), "group-id", - "client-id"); - - consumer.subscribe(singleton("topic"), listener); - subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); + "client-id")); + completeUnsubscribeApplicationEventSuccessfully(); consumer.close(Duration.ZERO); - assertTrue(subscriptions.assignedPartitions().isEmpty()); - assertEquals(1, listener.revokedCount); + verifyUnsubscribeEvent(subscriptions); } @Test public void testFailedPartitionRevocationOnClose() { - // If rebalance listener failed to execute during close, we will skip sending leave group and proceed with - // closing the consumer. - ConsumerRebalanceListener listener = mock(ConsumerRebalanceListener.class); - SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE); - consumer = newConsumer( + // If rebalance listener failed to execute during close, we still send the leave group, + // and proceed with closing the consumer. + SubscriptionState subscriptions = mock(SubscriptionState.class); + consumer = spy(newConsumer( mock(FetchBuffer.class), new ConsumerInterceptors<>(Collections.emptyList()), mock(ConsumerRebalanceListenerInvoker.class), subscriptions, - singletonList(new RoundRobinAssignor()), "group-id", - "client-id"); - subscriptions.subscribe(singleton("topic"), Optional.of(listener)); - TopicPartition tp = new TopicPartition("topic", 0); - subscriptions.assignFromSubscribed(singleton(tp)); - doThrow(new KafkaException()).when(listener).onPartitionsRevoked(eq(singleton(tp))); + "client-id")); + doThrow(new KafkaException()).when(consumer).processBackgroundEvents(any(), any()); assertThrows(KafkaException.class, () -> consumer.close(Duration.ZERO)); - verify(applicationEventHandler, never()).addAndGet(any(LeaveOnCloseEvent.class)); - verify(listener).onPartitionsRevoked(eq(singleton(tp))); - assertEquals(emptySet(), subscriptions.assignedPartitions()); - } - - @Test - public void testCompleteQuietly() { - AtomicReference exception = new AtomicReference<>(); - CompletableFuture future = CompletableFuture.completedFuture(null); - consumer = newConsumer(); - assertDoesNotThrow(() -> consumer.completeQuietly(() -> { - future.get(0, TimeUnit.MILLISECONDS); - }, "test", exception)); - assertNull(exception.get()); - - assertDoesNotThrow(() -> consumer.completeQuietly(() -> { - throw new KafkaException("Test exception"); - }, "test", exception)); - assertInstanceOf(KafkaException.class, exception.get()); + verifyUnsubscribeEvent(subscriptions); + // Close operation should carry on even if the unsubscribe fails + verify(applicationEventHandler).close(any(Duration.class)); } @Test @@ -844,13 +1007,12 @@ public void testAutoCommitSyncEnabled() { mock(ConsumerInterceptors.class), mock(ConsumerRebalanceListenerInvoker.class), subscriptions, - singletonList(new RoundRobinAssignor()), "group-id", "client-id"); consumer.subscribe(singleton("topic"), mock(ConsumerRebalanceListener.class)); subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); subscriptions.seek(new TopicPartition("topic", 0), 100); - consumer.autoCommitSync(time.timer(100)); + consumer.commitSyncAllConsumed(time.timer(100)); verify(applicationEventHandler).add(any(SyncCommitEvent.class)); } @@ -862,7 +1024,6 @@ public void testAutoCommitSyncDisabled() { mock(ConsumerInterceptors.class), mock(ConsumerRebalanceListenerInvoker.class), subscriptions, - singletonList(new RoundRobinAssignor()), "group-id", "client-id"); consumer.subscribe(singleton("topic"), mock(ConsumerRebalanceListener.class)); @@ -1135,7 +1296,7 @@ public void testNoWakeupInCloseCommit() { } return null; }).when(applicationEventHandler).add(any()); - + completeUnsubscribeApplicationEventSuccessfully(); consumer.close(Duration.ZERO); // A commit was triggered and not completed exceptionally by the wakeup @@ -1173,6 +1334,7 @@ public void testCloseAwaitPendingAsyncCommitComplete() { completeCommitAsyncApplicationEventSuccessfully(); consumer.commitAsync(cb); + completeUnsubscribeApplicationEventSuccessfully(); assertDoesNotThrow(() -> consumer.close(Duration.ofMillis(10))); assertEquals(1, cb.invoked); } @@ -1187,6 +1349,7 @@ public void testInterceptorAutoCommitOnClose() { consumer = newConsumer(props); assertEquals(1, MockConsumerInterceptor.INIT_COUNT.get()); completeCommitSyncApplicationEventSuccessfully(); + completeUnsubscribeApplicationEventSuccessfully(); consumer.close(Duration.ZERO); @@ -1624,6 +1787,18 @@ public void testGroupRemoteAssignorUsedInConsumerProtocol() { assertFalse(config.unused().contains(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); } + @Test + public void testPartitionAssignmentStrategyUnusedInAsyncConsumer() { + final Properties props = requiredConsumerConfig(); + props.put(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroup1"); + props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CONSUMER.name().toLowerCase(Locale.ROOT)); + props.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "CooperativeStickyAssignor"); + final ConsumerConfig config = new ConsumerConfig(props); + consumer = newConsumer(config); + + assertTrue(config.unused().contains(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG)); + } + @Test public void testGroupIdNull() { final Properties props = requiredConsumerConfig(); @@ -1666,7 +1841,6 @@ public void testEnsurePollEventSentOnConsumerPoll() { new ConsumerInterceptors<>(Collections.emptyList()), mock(ConsumerRebalanceListenerInvoker.class), subscriptions, - singletonList(new RoundRobinAssignor()), "group-id", "client-id"); final TopicPartition tp = new TopicPartition("topic", 0); @@ -1713,13 +1887,13 @@ private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean co if (committedOffsetsEnabled) { // Verify there was an FetchCommittedOffsets event and no ResetPositions event verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class)); + .add(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class)); verify(applicationEventHandler, never()) .addAndGet(ArgumentMatchers.isA(ResetPositionsEvent.class)); } else { // Verify there was not any FetchCommittedOffsets event but there should be a ResetPositions verify(applicationEventHandler, never()) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class)); + .add(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class)); verify(applicationEventHandler, atLeast(1)) .addAndGet(ArgumentMatchers.isA(ResetPositionsEvent.class)); } @@ -1738,7 +1912,7 @@ private void testRefreshCommittedOffsetsSuccess(Set partitions, verify(applicationEventHandler, atLeast(1)) .addAndGet(ArgumentMatchers.isA(ValidatePositionsEvent.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class)); + .add(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class)); verify(applicationEventHandler, atLeast(1)) .addAndGet(ArgumentMatchers.isA(ResetPositionsEvent.class)); } @@ -1880,6 +2054,7 @@ public void testPollThrowsInterruptExceptionIfInterrupted() { @Test void testReaperInvokedInClose() { consumer = newConsumer(); + completeUnsubscribeApplicationEventSuccessfully(); consumer.close(); verify(backgroundEventReaper).reap(backgroundEventQueue); } @@ -1901,6 +2076,18 @@ void testReaperInvokedInPoll() { verify(backgroundEventReaper).reap(time.milliseconds()); } + private void verifyUnsubscribeEvent(SubscriptionState subscriptions) { + // Check that an unsubscribe event was generated, and that the consumer waited for it to + // complete processing background events. + verify(applicationEventHandler).add(any(UnsubscribeEvent.class)); + verify(consumer).processBackgroundEvents(any(), any()); + + // The consumer should not clear the assignment in the app thread. The unsubscribe + // event is the one responsible for updating the assignment in the background when it + // completes. + verify(subscriptions, never()).assignFromSubscribed(any()); + } + private Map mockTopicPartitionOffset() { final TopicPartition t0 = new TopicPartition("t0", 2); final TopicPartition t1 = new TopicPartition("t0", 3); @@ -1964,6 +2151,12 @@ private void completeFetchedCommittedOffsetApplicationEventSuccessfully(final Ma doReturn(committedOffsets) .when(applicationEventHandler) .addAndGet(any(FetchCommittedOffsetsEvent.class)); + + doAnswer(invocation -> { + FetchCommittedOffsetsEvent event = invocation.getArgument(0); + event.future().complete(committedOffsets); + return null; + }).when(applicationEventHandler).add(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class)); } private void completeFetchedCommittedOffsetApplicationEventExceptionally(Exception ex) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java index b1db0297a1..8e61d61cbc 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java @@ -193,11 +193,11 @@ public void testPollEnsureCorrectInflightRequestBufferSize() { offsets2.put(new TopicPartition("test", 4), new OffsetAndMetadata(20L)); // Add the requests to the CommitRequestManager and store their futures - long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; - commitManager.commitSync(offsets1, expirationTimeMs); - commitManager.fetchOffsets(Collections.singleton(new TopicPartition("test", 0)), expirationTimeMs); - commitManager.commitSync(offsets2, expirationTimeMs); - commitManager.fetchOffsets(Collections.singleton(new TopicPartition("test", 1)), expirationTimeMs); + long deadlineMs = time.milliseconds() + defaultApiTimeoutMs; + commitManager.commitSync(offsets1, deadlineMs); + commitManager.fetchOffsets(Collections.singleton(new TopicPartition("test", 0)), deadlineMs); + commitManager.commitSync(offsets2, deadlineMs); + commitManager.fetchOffsets(Collections.singleton(new TopicPartition("test", 1)), deadlineMs); // Poll the CommitRequestManager and verify that the inflightOffsetFetches size is correct NetworkClientDelegate.PollResult result = commitManager.poll(time.milliseconds()); @@ -287,8 +287,8 @@ public void testCommitSyncRetriedAfterExpectedRetriableException(Errors error) { Map offsets = Collections.singletonMap( new TopicPartition("topic", 1), new OffsetAndMetadata(0)); - long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; - CompletableFuture commitResult = commitRequestManager.commitSync(offsets, expirationTimeMs); + long deadlineMs = time.milliseconds() + defaultApiTimeoutMs; + CompletableFuture commitResult = commitRequestManager.commitSync(offsets, deadlineMs); sendAndVerifyOffsetCommitRequestFailedAndMaybeRetried(commitRequestManager, error, commitResult); // We expect that request should have been retried on this sync commit. @@ -307,8 +307,8 @@ public void testCommitSyncFailsWithExpectedException(Errors commitError, new OffsetAndMetadata(0)); // Send sync offset commit that fails and verify it propagates the expected exception. - long expirationTimeMs = time.milliseconds() + retryBackoffMs; - CompletableFuture commitResult = commitRequestManager.commitSync(offsets, expirationTimeMs); + long deadlineMs = time.milliseconds() + retryBackoffMs; + CompletableFuture commitResult = commitRequestManager.commitSync(offsets, deadlineMs); completeOffsetCommitRequestWithError(commitRequestManager, commitError); assertFutureThrows(commitResult, expectedException); } @@ -332,8 +332,8 @@ public void testCommitSyncFailsWithCommitFailedExceptionIfUnknownMemberId() { Map offsets = Collections.singletonMap( new TopicPartition("topic", 1), new OffsetAndMetadata(0)); - long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; - CompletableFuture commitResult = commitRequestManager.commitSync(offsets, expirationTimeMs); + long deadlineMs = time.milliseconds() + defaultApiTimeoutMs; + CompletableFuture commitResult = commitRequestManager.commitSync(offsets, deadlineMs); completeOffsetCommitRequestWithError(commitRequestManager, Errors.UNKNOWN_MEMBER_ID); NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds()); @@ -594,7 +594,7 @@ public void testOffsetFetchRequestEnsureDuplicatedRequestSucceed() { @ParameterizedTest @MethodSource("offsetFetchExceptionSupplier") - public void testOffsetFetchRequestErroredRequests(final Errors error, final boolean isRetriable) { + public void testOffsetFetchRequestErroredRequests(final Errors error) { CommitRequestManager commitRequestManager = create(true, 100); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode)); @@ -606,7 +606,7 @@ public void testOffsetFetchRequestErroredRequests(final Errors error, final bool 1, error); // we only want to make sure to purge the outbound buffer for non-retriables, so retriable will be re-queued. - if (isRetriable) + if (isRetriableOnOffsetFetch(error)) testRetriable(commitRequestManager, futures); else { testNonRetriable(futures); @@ -614,15 +614,49 @@ public void testOffsetFetchRequestErroredRequests(final Errors error, final bool } } + @ParameterizedTest + @MethodSource("offsetFetchExceptionSupplier") + public void testOffsetFetchRequestTimeoutRequests(final Errors error) { + CommitRequestManager commitRequestManager = create(true, 100); + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode)); + + Set partitions = new HashSet<>(); + partitions.add(new TopicPartition("t1", 0)); + List>> futures = sendAndVerifyDuplicatedOffsetFetchRequests( + commitRequestManager, + partitions, + 1, + error); + + if (isRetriableOnOffsetFetch(error)) { + futures.forEach(f -> assertFalse(f.isDone())); + + // Insert a long enough sleep to force a timeout of the operation. Invoke poll() again so that each + // OffsetFetchRequestState is evaluated via isExpired(). + time.sleep(defaultApiTimeoutMs); + assertFalse(commitRequestManager.pendingRequests.unsentOffsetFetches.isEmpty()); + commitRequestManager.poll(time.milliseconds()); + futures.forEach(f -> assertFutureThrows(f, TimeoutException.class)); + assertTrue(commitRequestManager.pendingRequests.unsentOffsetFetches.isEmpty()); + } else { + futures.forEach(f -> assertFutureThrows(f, KafkaException.class)); + assertEmptyPendingRequests(commitRequestManager); + } + } + + private boolean isRetriableOnOffsetFetch(Errors error) { + return error == Errors.NOT_COORDINATOR || error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error == Errors.COORDINATOR_NOT_AVAILABLE; + } + @Test public void testSuccessfulOffsetFetch() { CommitRequestManager commitManager = create(false, 100); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode)); - long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; + long deadlineMs = time.milliseconds() + defaultApiTimeoutMs; CompletableFuture> fetchResult = commitManager.fetchOffsets(Collections.singleton(new TopicPartition("test", 0)), - expirationTimeMs); + deadlineMs); // Send fetch request NetworkClientDelegate.PollResult result = commitManager.poll(time.milliseconds()); @@ -667,8 +701,8 @@ public void testOffsetFetchMarksCoordinatorUnknownOnRetriableCoordinatorErrors(E Set partitions = new HashSet<>(); partitions.add(new TopicPartition("t1", 0)); - long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; - CompletableFuture> result = commitRequestManager.fetchOffsets(partitions, expirationTimeMs); + long deadlineMs = time.milliseconds() + defaultApiTimeoutMs; + CompletableFuture> result = commitRequestManager.fetchOffsets(partitions, deadlineMs); completeOffsetFetchRequestWithError(commitRequestManager, partitions, error); @@ -694,8 +728,8 @@ public void testOffsetFetchMarksCoordinatorUnknownOnCoordinatorDisconnectedAndRe Set partitions = new HashSet<>(); partitions.add(new TopicPartition("t1", 0)); - long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; - CompletableFuture> result = commitRequestManager.fetchOffsets(partitions, expirationTimeMs); + long deadlineMs = time.milliseconds() + defaultApiTimeoutMs; + CompletableFuture> result = commitRequestManager.fetchOffsets(partitions, deadlineMs); NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds()); assertEquals(1, res.unsentRequests.size()); @@ -748,8 +782,8 @@ public void testOffsetCommitSyncTimeoutNotReturnedOnPollAndFails() { new OffsetAndMetadata(0)); // Send sync offset commit request that fails with retriable error. - long expirationTimeMs = time.milliseconds() + retryBackoffMs * 2; - CompletableFuture commitResult = commitRequestManager.commitSync(offsets, expirationTimeMs); + long deadlineMs = time.milliseconds() + retryBackoffMs * 2; + CompletableFuture commitResult = commitRequestManager.commitSync(offsets, deadlineMs); completeOffsetCommitRequestWithError(commitRequestManager, Errors.REQUEST_TIMED_OUT); // Request retried after backoff, and fails with retriable again. Should not complete yet @@ -770,8 +804,9 @@ public void testOffsetCommitSyncTimeoutNotReturnedOnPollAndFails() { * Sync commit requests that fail with an expected retriable error should be retried * while there is time. When time expires, they should fail with a TimeoutException. */ - @Test - public void testOffsetCommitSyncFailedWithRetriableThrowsTimeoutWhenRetryTimeExpires() { + @ParameterizedTest + @MethodSource("offsetCommitExceptionSupplier") + public void testOffsetCommitSyncFailedWithRetriableThrowsTimeoutWhenRetryTimeExpires(final Errors error) { CommitRequestManager commitRequestManager = create(false, 100); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode)); @@ -780,17 +815,21 @@ public void testOffsetCommitSyncFailedWithRetriableThrowsTimeoutWhenRetryTimeExp new OffsetAndMetadata(0)); // Send offset commit request that fails with retriable error. - long expirationTimeMs = time.milliseconds() + retryBackoffMs * 2; - CompletableFuture commitResult = commitRequestManager.commitSync(offsets, expirationTimeMs); - completeOffsetCommitRequestWithError(commitRequestManager, Errors.COORDINATOR_NOT_AVAILABLE); + long deadlineMs = time.milliseconds() + retryBackoffMs * 2; + CompletableFuture commitResult = commitRequestManager.commitSync(offsets, deadlineMs); + completeOffsetCommitRequestWithError(commitRequestManager, error); // Sleep to expire the request timeout. Request should fail on the next poll with a // TimeoutException. - time.sleep(expirationTimeMs); + time.sleep(deadlineMs); NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds()); assertEquals(0, res.unsentRequests.size()); assertTrue(commitResult.isDone()); - assertFutureThrows(commitResult, TimeoutException.class); + + if (error.exception() instanceof RetriableException) + assertFutureThrows(commitResult, TimeoutException.class); + else + assertFutureThrows(commitResult, KafkaException.class); } /** @@ -829,8 +868,8 @@ public void testEnsureBackoffRetryOnOffsetCommitRequestTimeout() { Map offsets = Collections.singletonMap(new TopicPartition("topic", 1), new OffsetAndMetadata(0)); - long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; - commitRequestManager.commitSync(offsets, expirationTimeMs); + long deadlineMs = time.milliseconds() + defaultApiTimeoutMs; + commitRequestManager.commitSync(offsets, deadlineMs); NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds()); assertEquals(1, res.unsentRequests.size()); res.unsentRequests.get(0).handler().onFailure(time.milliseconds(), new TimeoutException()); @@ -911,8 +950,8 @@ public void testSyncOffsetFetchFailsWithStaleEpochAndRetriesWithNewEpoch() { when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode)); // Send request that is expected to fail with invalid epoch. - long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; - commitRequestManager.fetchOffsets(partitions, expirationTimeMs); + long deadlineMs = time.milliseconds() + defaultApiTimeoutMs; + commitRequestManager.fetchOffsets(partitions, deadlineMs); // Mock member has new a valid epoch. int newEpoch = 8; @@ -950,9 +989,9 @@ public void testSyncOffsetFetchFailsWithStaleEpochAndNotRetriedIfMemberNotInGrou when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode)); // Send request that is expected to fail with invalid epoch. - long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; + long deadlineMs = time.milliseconds() + defaultApiTimeoutMs; CompletableFuture> requestResult = - commitRequestManager.fetchOffsets(partitions, expirationTimeMs); + commitRequestManager.fetchOffsets(partitions, deadlineMs); // Mock member not having a valid epoch anymore (left/failed/fenced). commitRequestManager.onMemberEpochUpdated(Optional.empty(), Optional.empty()); @@ -983,10 +1022,10 @@ public void testAutoCommitSyncBeforeRevocationRetriesOnRetriableAndStaleEpoch(Er TopicPartition tp = new TopicPartition("topic", 1); subscriptionState.assignFromUser(singleton(tp)); subscriptionState.seek(tp, 5); - long expirationTimeMs = time.milliseconds() + retryBackoffMs * 2; + long deadlineMs = time.milliseconds() + retryBackoffMs * 2; // Send commit request expected to be retried on STALE_MEMBER_EPOCH error while it does not expire - commitRequestManager.maybeAutoCommitSyncBeforeRevocation(expirationTimeMs); + commitRequestManager.maybeAutoCommitSyncBeforeRevocation(deadlineMs); int newEpoch = 8; String memberId = "member1"; @@ -1094,7 +1133,7 @@ private void testNonRetriable(final List offsetCommitExceptionSupplier() { return Stream.of( @@ -1113,25 +1152,27 @@ private static Stream offsetCommitExceptionSupplier() { Arguments.of(Errors.UNKNOWN_MEMBER_ID)); } - // Supplies (error, isRetriable) + /** + * @return {@link Errors} that could be received in {@link ApiKeys#OFFSET_FETCH} responses. + */ private static Stream offsetFetchExceptionSupplier() { - // fetchCommit is only retrying on a subset of RetriableErrors return Stream.of( - Arguments.of(Errors.NOT_COORDINATOR, true), - Arguments.of(Errors.COORDINATOR_LOAD_IN_PROGRESS, true), - Arguments.of(Errors.UNKNOWN_SERVER_ERROR, false), - Arguments.of(Errors.GROUP_AUTHORIZATION_FAILED, false), - Arguments.of(Errors.OFFSET_METADATA_TOO_LARGE, false), - Arguments.of(Errors.INVALID_COMMIT_OFFSET_SIZE, false), - Arguments.of(Errors.UNKNOWN_TOPIC_OR_PARTITION, false), - Arguments.of(Errors.COORDINATOR_NOT_AVAILABLE, true), - Arguments.of(Errors.REQUEST_TIMED_OUT, false), - Arguments.of(Errors.FENCED_INSTANCE_ID, false), - Arguments.of(Errors.TOPIC_AUTHORIZATION_FAILED, false), - Arguments.of(Errors.UNKNOWN_MEMBER_ID, false), + Arguments.of(Errors.NOT_COORDINATOR), + Arguments.of(Errors.COORDINATOR_LOAD_IN_PROGRESS), + Arguments.of(Errors.UNKNOWN_SERVER_ERROR), + Arguments.of(Errors.GROUP_AUTHORIZATION_FAILED), + Arguments.of(Errors.OFFSET_METADATA_TOO_LARGE), + Arguments.of(Errors.INVALID_COMMIT_OFFSET_SIZE), + Arguments.of(Errors.UNKNOWN_TOPIC_OR_PARTITION), + Arguments.of(Errors.COORDINATOR_NOT_AVAILABLE), + Arguments.of(Errors.REQUEST_TIMED_OUT), + Arguments.of(Errors.FENCED_INSTANCE_ID), + Arguments.of(Errors.TOPIC_AUTHORIZATION_FAILED), + Arguments.of(Errors.UNKNOWN_MEMBER_ID), // Adding STALE_MEMBER_EPOCH as non-retriable here because it is only retried if a new // member epoch is received. Tested separately. - Arguments.of(Errors.STALE_MEMBER_EPOCH, false)); + Arguments.of(Errors.STALE_MEMBER_EPOCH), + Arguments.of(Errors.UNSTABLE_OFFSET_COMMIT)); } /** @@ -1155,9 +1196,9 @@ public void testOffsetFetchRequestPartitionDataError(final Errors error, final b TopicPartition tp2 = new TopicPartition("t2", 3); partitions.add(tp1); partitions.add(tp2); - long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; + long deadlineMs = time.milliseconds() + defaultApiTimeoutMs; CompletableFuture> future = - commitRequestManager.fetchOffsets(partitions, expirationTimeMs); + commitRequestManager.fetchOffsets(partitions, deadlineMs); NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds()); assertEquals(1, res.unsentRequests.size()); @@ -1215,9 +1256,9 @@ private List>> sendAndV int numRequest, final Errors error) { List>> futures = new ArrayList<>(); - long expirationTimeMs = time.milliseconds() + defaultApiTimeoutMs; + long deadlineMs = time.milliseconds() + defaultApiTimeoutMs; for (int i = 0; i < numRequest; i++) { - futures.add(commitRequestManager.fetchOffsets(partitions, expirationTimeMs)); + futures.add(commitRequestManager.fetchOffsets(partitions, deadlineMs)); } NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds()); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java index 8c3f97dd64..b6780fd427 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java @@ -339,6 +339,27 @@ void testRunOnceInvokesReaper() { verify(applicationEventReaper).reap(any(Long.class)); } + @Test + void testSendUnsentRequest() { + String groupId = "group-id"; + NetworkClientDelegate.UnsentRequest request = new NetworkClientDelegate.UnsentRequest( + new FindCoordinatorRequest.Builder( + new FindCoordinatorRequestData() + .setKeyType(FindCoordinatorRequest.CoordinatorType.TRANSACTION.id()) + .setKey(groupId)), + Optional.empty()); + + networkClient.add(request); + assertTrue(networkClient.hasAnyPendingRequests()); + assertFalse(networkClient.unsentRequests().isEmpty()); + assertFalse(client.hasInFlightRequests()); + consumerNetworkThread.cleanup(); + + assertTrue(networkClient.unsentRequests().isEmpty()); + assertFalse(client.hasInFlightRequests()); + assertFalse(networkClient.hasAnyPendingRequests()); + } + private void prepareOffsetCommitRequest(final Map expectedOffsets, final Errors error, final boolean disconnected) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java index 9f6fd4a764..dabd697b89 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java @@ -174,7 +174,7 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA backgroundEventHandler, logContext)); - this.topicMetadataRequestManager = spy(new TopicMetadataRequestManager(logContext, config)); + this.topicMetadataRequestManager = spy(new TopicMetadataRequestManager(logContext, time, config)); if (groupInfo.isPresent()) { GroupInformation gi = groupInfo.get(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java index eaabcb8f81..204f87e1c1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.KafkaClient; import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.MetadataRecoveryStrategy; import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.clients.NodeApiVersions; @@ -369,7 +370,7 @@ public void testFetcherCloseClosesFetchSessionsInBroker() { // NOTE: by design the FetchRequestManager doesn't perform network I/O internally. That means that calling // the close() method with a Timer will NOT send out the close session requests on close. The network // I/O logic is handled inside ConsumerNetworkThread.runAtClose, so we need to run that logic here. - ConsumerNetworkThread.runAtClose(singletonList(Optional.of(fetcher)), networkClientDelegate, timer); + ConsumerNetworkThread.runAtClose(singletonList(Optional.of(fetcher)), networkClientDelegate); // the network is polled during the last state of clean up. networkClientDelegate.poll(time.timer(1)); // validate that closing the fetcher has sent a request with final epoch. 2 requests are sent, one for the @@ -1909,7 +1910,8 @@ public void testQuotaMetrics() { Node node = cluster.nodes().get(0); NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, 1000, 1000, 64 * 1024, 64 * 1024, 1000, 10 * 1000, 127 * 1000, - time, true, new ApiVersions(), metricsManager.throttleTimeSensor(), new LogContext()); + time, true, new ApiVersions(), metricsManager.throttleTimeSensor(), new LogContext(), + MetadataRecoveryStrategy.NONE); ApiVersionsResponse apiVersionsResponse = TestUtils.defaultApiVersionsResponse( 400, ApiMessageType.ListenerType.ZK_BROKER); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 1eac870993..d0167e9b98 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.FetchSessionHandler; import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.MetadataRecoveryStrategy; import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.clients.NodeApiVersions; @@ -1905,7 +1906,8 @@ public void testQuotaMetrics() { Node node = cluster.nodes().get(0); NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, 1000, 1000, 64 * 1024, 64 * 1024, 1000, 10 * 1000, 127 * 1000, - time, true, new ApiVersions(), metricsManager.throttleTimeSensor(), new LogContext()); + time, true, new ApiVersions(), metricsManager.throttleTimeSensor(), new LogContext(), + MetadataRecoveryStrategy.NONE); ApiVersionsResponse apiVersionsResponse = TestUtils.defaultApiVersionsResponse( 400, ApiMessageType.ListenerType.ZK_BROKER); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 8334fb2360..f63dd55754 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -277,7 +277,7 @@ public void testHeartbeatNotSentIfAnotherOneInFlight() { result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent while a " + "previous one is in-flight"); - + time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent when the " + @@ -752,6 +752,25 @@ public void testFencedMemberStopHeartbeatUntilItReleasesAssignmentToRejoin() { assertEquals(1, result.unsentRequests.size(), "Fenced member should resume heartbeat after transitioning to JOINING"); } + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) + public void testSendingLeaveGroupHeartbeatWhenPreviousOneInFlight(final short version) { + mockStableMember(); + time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(1, result.unsentRequests.size()); + result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent while a previous one is in-flight"); + + membershipManager.leaveGroup(); + + ConsumerGroupHeartbeatRequest heartbeatToLeave = getHeartbeatRequest(heartbeatRequestManager, version); + assertEquals(ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH, heartbeatToLeave.data().memberEpoch()); + + NetworkClientDelegate.PollResult pollAgain = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(0, pollAgain.unsentRequests.size()); + } + private void assertHeartbeat(HeartbeatRequestManager hrm, int nextPollMs) { NetworkClientDelegate.PollResult pollResult = hrm.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java index cef608a902..44de312087 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java @@ -365,6 +365,8 @@ public void testFencingWhenStateIsPrepareLeaving() { // because member is already out of the group in the broker). completeCallback(callbackEvent, membershipManager); assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); + assertEquals(ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH, membershipManager.memberEpoch()); + verify(membershipManager).notifyEpochChange(Optional.empty(), Optional.empty()); assertTrue(membershipManager.shouldSkipHeartbeat()); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java index 70f33bfdf4..7d53ca385b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java @@ -43,6 +43,7 @@ import static org.apache.kafka.clients.consumer.ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -140,6 +141,34 @@ public void testEnsureTimerSetOnAdd() { assertEquals(REQUEST_TIMEOUT_MS, ncd.unsentRequests().poll().timer().timeoutMs()); } + @Test + public void testHasAnyPendingRequests() throws Exception { + try (NetworkClientDelegate networkClientDelegate = newNetworkClientDelegate()) { + NetworkClientDelegate.UnsentRequest unsentRequest = newUnsentFindCoordinatorRequest(); + networkClientDelegate.add(unsentRequest); + + // unsent + assertTrue(networkClientDelegate.hasAnyPendingRequests()); + assertFalse(networkClientDelegate.unsentRequests().isEmpty()); + assertFalse(client.hasInFlightRequests()); + + networkClientDelegate.poll(0, time.milliseconds()); + + // in-flight + assertTrue(networkClientDelegate.hasAnyPendingRequests()); + assertTrue(networkClientDelegate.unsentRequests().isEmpty()); + assertTrue(client.hasInFlightRequests()); + + client.respond(FindCoordinatorResponse.prepareResponse(Errors.NONE, GROUP_ID, mockNode())); + networkClientDelegate.poll(0, time.milliseconds()); + + // get response + assertFalse(networkClientDelegate.hasAnyPendingRequests()); + assertTrue(networkClientDelegate.unsentRequests().isEmpty()); + assertFalse(client.hasInFlightRequests()); + } + } + public NetworkClientDelegate newNetworkClientDelegate() { LogContext logContext = new LogContext(); Properties properties = new Properties(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TimedRequestStateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TimedRequestStateTest.java new file mode 100644 index 0000000000..ddde3ae84d --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TimedRequestStateTest.java @@ -0,0 +1,96 @@ +/* + * 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. + */ +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TimedRequestStateTest { + + private final static long DEFAULT_TIMEOUT_MS = 30000; + private final Time time = new MockTime(); + + @Test + public void testIsExpired() { + TimedRequestState state = new TimedRequestState( + new LogContext(), + this.getClass().getSimpleName(), + 100, + 1000, + time.timer(DEFAULT_TIMEOUT_MS) + ); + assertFalse(state.isExpired()); + time.sleep(DEFAULT_TIMEOUT_MS); + assertTrue(state.isExpired()); + } + + @Test + public void testRemainingMs() { + TimedRequestState state = new TimedRequestState( + new LogContext(), + this.getClass().getSimpleName(), + 100, + 1000, + time.timer(DEFAULT_TIMEOUT_MS) + ); + assertEquals(DEFAULT_TIMEOUT_MS, state.remainingMs()); + time.sleep(DEFAULT_TIMEOUT_MS); + assertEquals(0, state.remainingMs()); + } + + @Test + public void testDeadlineTimer() { + long deadlineMs = time.milliseconds() + DEFAULT_TIMEOUT_MS; + Timer timer = TimedRequestState.deadlineTimer(time, deadlineMs); + assertEquals(DEFAULT_TIMEOUT_MS, timer.remainingMs()); + timer.sleep(DEFAULT_TIMEOUT_MS); + assertEquals(0, timer.remainingMs()); + } + + @Test + public void testAllowOverdueDeadlineTimer() { + long deadlineMs = time.milliseconds() - DEFAULT_TIMEOUT_MS; + Timer timer = TimedRequestState.deadlineTimer(time, deadlineMs); + assertEquals(0, timer.remainingMs()); + } + + @Test + public void testToStringUpdatesTimer() { + TimedRequestState state = new TimedRequestState( + new LogContext(), + this.getClass().getSimpleName(), + 100, + 1000, + time.timer(DEFAULT_TIMEOUT_MS) + ); + + assertToString(state, DEFAULT_TIMEOUT_MS); + time.sleep(DEFAULT_TIMEOUT_MS); + assertToString(state, 0); + } + + private void assertToString(TimedRequestState state, long timerMs) { + assertTrue(state.toString().contains("remainingMs=" + timerMs + "}")); + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java index 3f2b2c3d98..56eff5b4f4 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java @@ -74,6 +74,7 @@ public void setup() { props.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); this.topicMetadataRequestManager = spy(new TopicMetadataRequestManager( new LogContext(), + time, new ConsumerConfig(props))); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index 451743ae2a..bb021fa8b9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -36,14 +36,10 @@ import java.util.List; import java.util.Optional; import java.util.concurrent.BlockingQueue; -import java.util.concurrent.CompletableFuture; -import static org.apache.kafka.clients.consumer.internals.events.CompletableEvent.calculateDeadlineMs; -import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; public class ApplicationEventProcessorTest { private final Time time = new MockTime(1); @@ -90,16 +86,6 @@ public void testPrepClosingCommitEvents() { verify(commitRequestManager).signalClose(); } - @Test - public void testPrepClosingLeaveGroupEvent() { - LeaveOnCloseEvent event = new LeaveOnCloseEvent(calculateDeadlineMs(time, 100)); - when(heartbeatRequestManager.membershipManager()).thenReturn(membershipManager); - when(membershipManager.leaveGroup()).thenReturn(CompletableFuture.completedFuture(null)); - processor.process(event); - verify(membershipManager).leaveGroup(); - assertTrue(event.future().isDone()); - } - private List mockCommitResults() { return Collections.singletonList(mock(NetworkClientDelegate.UnsentRequest.class)); } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index 7d4aa5e3a8..17119a2529 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.KafkaClient; +import org.apache.kafka.clients.LeastLoadedNode; import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.NodeApiVersions; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -735,8 +736,8 @@ private static KafkaProducer producerWithOverrideNewSender(Map configs = new HashMap<>(); + configs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass); + configs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass); + final ProducerConfig producerConfig = new ProducerConfig(configs); + assertEquals(MetadataRecoveryStrategy.NONE.name, producerConfig.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)); + } + + @Test + public void testInvalidMetadataRecoveryStrategy() { + Map configs = new HashMap<>(); + configs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass); + configs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass); + configs.put(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG, "abc"); + ConfigException ce = assertThrows(ConfigException.class, () -> new ProducerConfig(configs)); + assertTrue(ce.getMessage().contains(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)); + } + @Test public void testCaseInsensitiveSecurityProtocol() { final String saslSslLowerCase = SecurityProtocol.SASL_SSL.name.toLowerCase(Locale.ROOT); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index 5c1088987e..cfeefc0ae5 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -19,7 +19,9 @@ import org.apache.kafka.clients.ApiVersions; import org.apache.kafka.clients.ClientRequest; import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.LeastLoadedNode; import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.MetadataRecoveryStrategy; import org.apache.kafka.clients.MetadataSnapshot; import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.NetworkClient; @@ -299,7 +301,8 @@ public void testQuotaMetrics() { Node node = cluster.nodes().get(0); NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, 1000, 1000, 64 * 1024, 64 * 1024, 1000, 10 * 1000, 127 * 1000, - time, true, new ApiVersions(), throttleTimeSensor, logContext); + time, true, new ApiVersions(), throttleTimeSensor, logContext, + MetadataRecoveryStrategy.NONE); ApiVersionsResponse apiVersionsResponse = TestUtils.defaultApiVersionsResponse( 400, ApiMessageType.ListenerType.ZK_BROKER); @@ -3797,12 +3800,12 @@ private void createMockClientWithMaxFlightOneMetadataPending() { client = new MockClient(time, metadata) { volatile boolean canSendMore = true; @Override - public Node leastLoadedNode(long now) { + public LeastLoadedNode leastLoadedNode(long now) { for (Node node : metadata.fetch().nodes()) { if (isReady(node, now) && canSendMore) - return node; + return new LeastLoadedNode(node, true); } - return null; + return new LeastLoadedNode(null, false); } @Override @@ -3821,7 +3824,7 @@ public List poll(long timeoutMs, long now) { while (!client.ready(node, time.milliseconds())) client.poll(0, time.milliseconds()); client.send(request, time.milliseconds()); - while (client.leastLoadedNode(time.milliseconds()) != null) + while (client.leastLoadedNode(time.milliseconds()).node() != null) client.poll(0, time.milliseconds()); } diff --git a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java index 08e2cebc31..bd38ccabd9 100755 --- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java @@ -21,6 +21,8 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.function.Executable; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; import org.mockito.stubbing.OngoingStubbing; import java.io.Closeable; @@ -106,31 +108,35 @@ public void testMurmur2() { } } - @Test - public void testGetHost() { - // valid + @ParameterizedTest + @CsvSource(value = {"PLAINTEXT", "SASL_PLAINTEXT", "SSL", "SASL_SSL"}) + public void testGetHostValid(String protocol) { + assertEquals("mydomain.com", getHost(protocol + "://mydomain.com:8080")); + assertEquals("MyDomain.com", getHost(protocol + "://MyDomain.com:8080")); + assertEquals("My_Domain.com", getHost(protocol + "://My_Domain.com:8080")); + assertEquals("::1", getHost(protocol + "://[::1]:1234")); + assertEquals("2001:db8:85a3:8d3:1319:8a2e:370:7348", getHost(protocol + "://[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678")); + assertEquals("2001:DB8:85A3:8D3:1319:8A2E:370:7348", getHost(protocol + "://[2001:DB8:85A3:8D3:1319:8A2E:370:7348]:5678")); + assertEquals("fe80::b1da:69ca:57f7:63d8%3", getHost(protocol + "://[fe80::b1da:69ca:57f7:63d8%3]:5678")); assertEquals("127.0.0.1", getHost("127.0.0.1:8000")); - assertEquals("mydomain.com", getHost("PLAINTEXT://mydomain.com:8080")); - assertEquals("MyDomain.com", getHost("PLAINTEXT://MyDomain.com:8080")); - assertEquals("My_Domain.com", getHost("PLAINTEXT://My_Domain.com:8080")); assertEquals("::1", getHost("[::1]:1234")); - assertEquals("2001:db8:85a3:8d3:1319:8a2e:370:7348", getHost("PLAINTEXT://[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678")); - assertEquals("2001:DB8:85A3:8D3:1319:8A2E:370:7348", getHost("PLAINTEXT://[2001:DB8:85A3:8D3:1319:8A2E:370:7348]:5678")); - assertEquals("fe80::b1da:69ca:57f7:63d8%3", getHost("PLAINTEXT://[fe80::b1da:69ca:57f7:63d8%3]:5678")); + } - // invalid - assertNull(getHost("PLAINTEXT://mydo)main.com:8080")); - assertNull(getHost("PLAINTEXT://mydo(main.com:8080")); - assertNull(getHost("PLAINTEXT://mydo()main.com:8080")); - assertNull(getHost("PLAINTEXT://mydo(main).com:8080")); + @ParameterizedTest + @CsvSource(value = {"PLAINTEXT", "SASL_PLAINTEXT", "SSL", "SASL_SSL"}) + public void testGetHostInvalid(String protocol) { + assertNull(getHost(protocol + "://mydo)main.com:8080")); + assertNull(getHost(protocol + "://mydo(main.com:8080")); + assertNull(getHost(protocol + "://mydo()main.com:8080")); + assertNull(getHost(protocol + "://mydo(main).com:8080")); + assertNull(getHost(protocol + "://[2001:db)8:85a3:8d3:1319:8a2e:370:7348]:5678")); + assertNull(getHost(protocol + "://[2001:db(8:85a3:8d3:1319:8a2e:370:7348]:5678")); + assertNull(getHost(protocol + "://[2001:db()8:85a3:8d3:1319:8a2e:370:7348]:5678")); + assertNull(getHost(protocol + "://[2001:db(8:85a3:)8d3:1319:8a2e:370:7348]:5678")); assertNull(getHost("ho)st:9092")); assertNull(getHost("ho(st:9092")); assertNull(getHost("ho()st:9092")); assertNull(getHost("ho(st):9092")); - assertNull(getHost("PLAINTEXT://[2001:db)8:85a3:8d3:1319:8a2e:370:7348]:5678")); - assertNull(getHost("PLAINTEXT://[2001:db(8:85a3:8d3:1319:8a2e:370:7348]:5678")); - assertNull(getHost("PLAINTEXT://[2001:db()8:85a3:8d3:1319:8a2e:370:7348]:5678")); - assertNull(getHost("PLAINTEXT://[2001:db(8:85a3:)8d3:1319:8a2e:370:7348]:5678")); } @Test diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java b/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java index 2a6e71c381..cf5f01502c 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java @@ -213,11 +213,15 @@ public static void validateValue(Schema schema, Object value) { validateValue(null, schema, value); } - public static void validateValue(String name, Schema schema, Object value) { + public static void validateValue(String field, Schema schema, Object value) { + validateValue(schema, value, field == null ? "value" : "field: \"" + field + "\""); + } + + private static void validateValue(Schema schema, Object value, String location) { if (value == null) { if (!schema.isOptional()) - throw new DataException("Invalid value: null used for required field: \"" + name - + "\", schema type: " + schema.type()); + throw new DataException("Invalid value: null used for required " + location + + ", schema type: " + schema.type()); return; } @@ -236,8 +240,8 @@ public static void validateValue(String name, Schema schema, Object value) { exceptionMessage.append(" \"").append(schema.name()).append("\""); } exceptionMessage.append(" with type ").append(schema.type()).append(": ").append(value.getClass()); - if (name != null) { - exceptionMessage.append(" for field: \"").append(name).append("\""); + if (location != null) { + exceptionMessage.append(" for ").append(location); } throw new DataException(exceptionMessage.toString()); } @@ -251,19 +255,33 @@ public static void validateValue(String name, Schema schema, Object value) { break; case ARRAY: List array = (List) value; - for (Object entry : array) - validateValue(schema.valueSchema(), entry); + String entryLocation = "element of array " + location; + Schema arrayValueSchema = assertSchemaNotNull(schema.valueSchema(), entryLocation); + for (Object entry : array) { + validateValue(arrayValueSchema, entry, entryLocation); + } break; case MAP: Map map = (Map) value; + String keyLocation = "key of map " + location; + String valueLocation = "value of map " + location; + Schema mapKeySchema = assertSchemaNotNull(schema.keySchema(), keyLocation); + Schema mapValueSchema = assertSchemaNotNull(schema.valueSchema(), valueLocation); for (Map.Entry entry : map.entrySet()) { - validateValue(schema.keySchema(), entry.getKey()); - validateValue(schema.valueSchema(), entry.getValue()); + validateValue(mapKeySchema, entry.getKey(), keyLocation); + validateValue(mapValueSchema, entry.getValue(), valueLocation); } break; } } + private static Schema assertSchemaNotNull(Schema schema, String location) { + if (schema == null) { + throw new DataException("No schema defined for " + location); + } + return schema; + } + private static List> expectedClassesFor(Schema schema) { List> expectedClasses = LOGICAL_TYPE_CLASSES.get(schema.name()); if (expectedClasses == null) diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java index 25e6db3469..43c2342fe3 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java @@ -330,4 +330,144 @@ public void testEmptyStruct() { new Struct(emptyStruct); } + private void assertInvalidValueForSchema(String fieldName, Schema schema, Object value, String message) { + Exception e = assertThrows(DataException.class, () -> ConnectSchema.validateValue(fieldName, schema, value)); + assertEquals(message, e.getMessage()); + } + + @Test + public void testValidateFieldWithInvalidValueType() { + String fieldName = "field"; + assertInvalidValueForSchema(fieldName, new FakeSchema(), new Object(), + "Invalid Java object for schema \"fake\" with type null: class java.lang.Object for field: \"field\""); + assertInvalidValueForSchema(null, Schema.INT8_SCHEMA, new Object(), + "Invalid Java object for schema with type INT8: class java.lang.Object for value"); + assertInvalidValueForSchema(fieldName, Schema.INT8_SCHEMA, new Object(), + "Invalid Java object for schema with type INT8: class java.lang.Object for field: \"field\""); + } + + @Test + public void testValidateFieldWithInvalidValueMismatchTimestamp() { + long longValue = 1000L; + String fieldName = "field"; + + ConnectSchema.validateValue(fieldName, Schema.INT64_SCHEMA, longValue); + + assertInvalidValueForSchema(fieldName, Timestamp.SCHEMA, longValue, + "Invalid Java object for schema \"org.apache.kafka.connect.data.Timestamp\" " + + "with type INT64: class java.lang.Long for field: \"field\""); + } + + @Test + public void testValidateList() { + String fieldName = "field"; + + // Optional element schema + Schema optionalStrings = SchemaBuilder.array(Schema.OPTIONAL_STRING_SCHEMA); + ConnectSchema.validateValue(fieldName, optionalStrings, Collections.emptyList()); + ConnectSchema.validateValue(fieldName, optionalStrings, Collections.singletonList("hello")); + ConnectSchema.validateValue(fieldName, optionalStrings, Collections.singletonList(null)); + ConnectSchema.validateValue(fieldName, optionalStrings, Arrays.asList("hello", "world")); + ConnectSchema.validateValue(fieldName, optionalStrings, Arrays.asList("hello", null)); + ConnectSchema.validateValue(fieldName, optionalStrings, Arrays.asList(null, "world")); + assertInvalidValueForSchema(fieldName, optionalStrings, Collections.singletonList(true), + "Invalid Java object for schema with type STRING: class java.lang.Boolean for element of array field: \"field\""); + + // Required element schema + Schema requiredStrings = SchemaBuilder.array(Schema.STRING_SCHEMA); + ConnectSchema.validateValue(fieldName, requiredStrings, Collections.emptyList()); + ConnectSchema.validateValue(fieldName, requiredStrings, Collections.singletonList("hello")); + assertInvalidValueForSchema(fieldName, requiredStrings, Collections.singletonList(null), + "Invalid value: null used for required element of array field: \"field\", schema type: STRING"); + ConnectSchema.validateValue(fieldName, requiredStrings, Arrays.asList("hello", "world")); + assertInvalidValueForSchema(fieldName, requiredStrings, Arrays.asList("hello", null), + "Invalid value: null used for required element of array field: \"field\", schema type: STRING"); + assertInvalidValueForSchema(fieldName, requiredStrings, Arrays.asList(null, "world"), + "Invalid value: null used for required element of array field: \"field\", schema type: STRING"); + assertInvalidValueForSchema(fieldName, optionalStrings, Collections.singletonList(true), + "Invalid Java object for schema with type STRING: class java.lang.Boolean for element of array field: \"field\""); + + // Null element schema + Schema nullElements = SchemaBuilder.type(Schema.Type.ARRAY); + assertInvalidValueForSchema(fieldName, nullElements, Collections.emptyList(), + "No schema defined for element of array field: \"field\""); + assertInvalidValueForSchema(fieldName, nullElements, Collections.singletonList("hello"), + "No schema defined for element of array field: \"field\""); + assertInvalidValueForSchema(fieldName, nullElements, Collections.singletonList(null), + "No schema defined for element of array field: \"field\""); + assertInvalidValueForSchema(fieldName, nullElements, Arrays.asList("hello", "world"), + "No schema defined for element of array field: \"field\""); + assertInvalidValueForSchema(fieldName, nullElements, Arrays.asList("hello", null), + "No schema defined for element of array field: \"field\""); + assertInvalidValueForSchema(fieldName, nullElements, Arrays.asList(null, "world"), + "No schema defined for element of array field: \"field\""); + assertInvalidValueForSchema(fieldName, nullElements, Collections.singletonList(true), + "No schema defined for element of array field: \"field\""); + } + + @Test + public void testValidateMap() { + String fieldName = "field"; + + // Optional element schema + Schema optionalStrings = SchemaBuilder.map(Schema.OPTIONAL_STRING_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA); + ConnectSchema.validateValue(fieldName, optionalStrings, Collections.emptyMap()); + ConnectSchema.validateValue(fieldName, optionalStrings, Collections.singletonMap("key", "value")); + ConnectSchema.validateValue(fieldName, optionalStrings, Collections.singletonMap("key", null)); + ConnectSchema.validateValue(fieldName, optionalStrings, Collections.singletonMap(null, "value")); + ConnectSchema.validateValue(fieldName, optionalStrings, Collections.singletonMap(null, null)); + assertInvalidValueForSchema(fieldName, optionalStrings, Collections.singletonMap("key", true), + "Invalid Java object for schema with type STRING: class java.lang.Boolean for value of map field: \"field\""); + assertInvalidValueForSchema(fieldName, optionalStrings, Collections.singletonMap(true, "value"), + "Invalid Java object for schema with type STRING: class java.lang.Boolean for key of map field: \"field\""); + + // Required element schema + Schema requiredStrings = SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.STRING_SCHEMA); + ConnectSchema.validateValue(fieldName, requiredStrings, Collections.emptyMap()); + ConnectSchema.validateValue(fieldName, requiredStrings, Collections.singletonMap("key", "value")); + assertInvalidValueForSchema(fieldName, requiredStrings, Collections.singletonMap("key", null), + "Invalid value: null used for required value of map field: \"field\", schema type: STRING"); + assertInvalidValueForSchema(fieldName, requiredStrings, Collections.singletonMap(null, "value"), + "Invalid value: null used for required key of map field: \"field\", schema type: STRING"); + assertInvalidValueForSchema(fieldName, requiredStrings, Collections.singletonMap(null, null), + "Invalid value: null used for required key of map field: \"field\", schema type: STRING"); + assertInvalidValueForSchema(fieldName, requiredStrings, Collections.singletonMap("key", true), + "Invalid Java object for schema with type STRING: class java.lang.Boolean for value of map field: \"field\""); + assertInvalidValueForSchema(fieldName, requiredStrings, Collections.singletonMap(true, "value"), + "Invalid Java object for schema with type STRING: class java.lang.Boolean for key of map field: \"field\""); + + // Null key schema + Schema nullKeys = SchemaBuilder.type(Schema.Type.MAP); + assertInvalidValueForSchema(fieldName, nullKeys, Collections.emptyMap(), + "No schema defined for key of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullKeys, Collections.singletonMap("key", "value"), + "No schema defined for key of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullKeys, Collections.singletonMap("key", null), + "No schema defined for key of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullKeys, Collections.singletonMap(null, "value"), + "No schema defined for key of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullKeys, Collections.singletonMap(null, null), + "No schema defined for key of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullKeys, Collections.singletonMap("key", true), + "No schema defined for key of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullKeys, Collections.singletonMap(true, "value"), + "No schema defined for key of map field: \"field\""); + + // Null value schema + Schema nullValues = SchemaBuilder.mapWithNullValues(Schema.OPTIONAL_STRING_SCHEMA); + assertInvalidValueForSchema(fieldName, nullValues, Collections.emptyMap(), + "No schema defined for value of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullValues, Collections.singletonMap("key", "value"), + "No schema defined for value of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullValues, Collections.singletonMap("key", null), + "No schema defined for value of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullValues, Collections.singletonMap(null, "value"), + "No schema defined for value of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullValues, Collections.singletonMap(null, null), + "No schema defined for value of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullValues, Collections.singletonMap("key", true), + "No schema defined for value of map field: \"field\""); + assertInvalidValueForSchema(fieldName, nullValues, Collections.singletonMap(true, "value"), + "No schema defined for value of map field: \"field\""); + } } diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/StructTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/StructTest.java index 55ccc81bed..6dee26ca83 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/StructTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/StructTest.java @@ -304,39 +304,6 @@ public void testValidateStructWithNullValue() { e.getMessage()); } - @Test - public void testValidateFieldWithInvalidValueType() { - String fieldName = "field"; - FakeSchema fakeSchema = new FakeSchema(); - - Exception e = assertThrows(DataException.class, () -> ConnectSchema.validateValue(fieldName, - fakeSchema, new Object())); - assertEquals("Invalid Java object for schema \"fake\" with type null: class java.lang.Object for field: \"field\"", - e.getMessage()); - - e = assertThrows(DataException.class, () -> ConnectSchema.validateValue(fieldName, - Schema.INT8_SCHEMA, new Object())); - assertEquals("Invalid Java object for schema with type INT8: class java.lang.Object for field: \"field\"", - e.getMessage()); - - e = assertThrows(DataException.class, () -> ConnectSchema.validateValue(Schema.INT8_SCHEMA, new Object())); - assertEquals("Invalid Java object for schema with type INT8: class java.lang.Object", e.getMessage()); - } - - @Test - public void testValidateFieldWithInvalidValueMismatchTimestamp() { - String fieldName = "field"; - long longValue = 1000L; - - // Does not throw - ConnectSchema.validateValue(fieldName, Schema.INT64_SCHEMA, longValue); - - Exception e = assertThrows(DataException.class, () -> ConnectSchema.validateValue(fieldName, - Timestamp.SCHEMA, longValue)); - assertEquals("Invalid Java object for schema \"org.apache.kafka.connect.data.Timestamp\" " + - "with type INT64: class java.lang.Long for field: \"field\"", e.getMessage()); - } - @Test public void testPutNullField() { final String fieldName = "fieldName"; diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java index b623aaf14e..b77eeaf066 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java @@ -241,15 +241,15 @@ public JsonConverter() { /** * Creates a JsonConvert initializing serializer and deserializer. * - * @param enableModules permits to enable/disable the registration of additional Jackson modules. + * @param enableAfterburner permits to enable/disable the registration of Jackson Afterburner module. *

* NOTE: This is visible only for testing */ - public JsonConverter(boolean enableModules) { + public JsonConverter(boolean enableAfterburner) { serializer = new JsonSerializer( mkSet(), JSON_NODE_FACTORY, - enableModules + enableAfterburner ); deserializer = new JsonDeserializer( @@ -259,7 +259,7 @@ public JsonConverter(boolean enableModules) { DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS ), JSON_NODE_FACTORY, - enableModules + enableAfterburner ); } diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonDeserializer.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonDeserializer.java index d63ae7808c..40d4631dfb 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonDeserializer.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonDeserializer.java @@ -16,13 +16,15 @@ */ package org.apache.kafka.connect.json; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.serialization.Deserializer; + import com.fasterxml.jackson.core.json.JsonReadFeature; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.JsonNodeFactory; -import org.apache.kafka.common.errors.SerializationException; -import org.apache.kafka.common.serialization.Deserializer; +import com.fasterxml.jackson.module.afterburner.AfterburnerModule; import java.util.Collections; import java.util.Set; @@ -51,13 +53,13 @@ public JsonDeserializer() { JsonDeserializer( final Set deserializationFeatures, final JsonNodeFactory jsonNodeFactory, - final boolean enableModules + final boolean enableAfterburner ) { objectMapper.enable(JsonReadFeature.ALLOW_LEADING_ZEROS_FOR_NUMBERS.mappedFeature()); deserializationFeatures.forEach(objectMapper::enable); objectMapper.setNodeFactory(jsonNodeFactory); - if (enableModules) { - objectMapper.findAndRegisterModules(); + if (enableAfterburner) { + objectMapper.registerModule(new AfterburnerModule()); } } diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonSerializer.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonSerializer.java index f8b13fbb6a..64c4414148 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonSerializer.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonSerializer.java @@ -16,12 +16,14 @@ */ package org.apache.kafka.connect.json; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.serialization.Serializer; + import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializationFeature; import com.fasterxml.jackson.databind.node.JsonNodeFactory; -import org.apache.kafka.common.errors.SerializationException; -import org.apache.kafka.common.serialization.Serializer; +import com.fasterxml.jackson.module.afterburner.AfterburnerModule; import java.util.Collections; import java.util.Set; @@ -50,12 +52,12 @@ public JsonSerializer() { JsonSerializer( final Set serializationFeatures, final JsonNodeFactory jsonNodeFactory, - final boolean enableModules + final boolean enableAfterburner ) { serializationFeatures.forEach(objectMapper::enable); objectMapper.setNodeFactory(jsonNodeFactory); - if (enableModules) { - objectMapper.findAndRegisterModules(); + if (enableAfterburner) { + objectMapper.registerModule(new AfterburnerModule()); } } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java index 4fbd282d11..5398f21c62 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java @@ -219,16 +219,12 @@ public void startClusters(Map additionalMM2Config) throws Except .build(); primary.start(); - primary.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, - "Workers of " + PRIMARY_CLUSTER_ALIAS + "-connect-cluster did not start in time."); waitForTopicCreated(primary, "mm2-status.backup.internal"); waitForTopicCreated(primary, "mm2-offsets.backup.internal"); waitForTopicCreated(primary, "mm2-configs.backup.internal"); backup.start(); - backup.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, - "Workers of " + BACKUP_CLUSTER_ALIAS + "-connect-cluster did not start in time."); primaryProducer = initializeProducer(primary); backupProducer = initializeProducer(backup); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java index 2a27103079..e50019676a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java @@ -124,7 +124,7 @@ */ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, ConnectorStatus.Listener { - private final Logger log = LoggerFactory.getLogger(AbstractHerder.class); + private static final Logger log = LoggerFactory.getLogger(AbstractHerder.class); private final String workerId; protected final Worker worker; @@ -1039,21 +1039,30 @@ public static List> reverseTransform(String connName, return result; } - public boolean taskConfigsChanged(ClusterConfigState configState, String connName, List> taskProps) { + public static boolean taskConfigsChanged(ClusterConfigState configState, String connName, List> rawTaskProps) { int currentNumTasks = configState.taskCount(connName); boolean result = false; - if (taskProps.size() != currentNumTasks) { - log.debug("Connector {} task count changed from {} to {}", connName, currentNumTasks, taskProps.size()); + if (rawTaskProps.size() != currentNumTasks) { + log.debug("Connector {} task count changed from {} to {}", connName, currentNumTasks, rawTaskProps.size()); result = true; - } else { + } + if (!result) { for (int index = 0; index < currentNumTasks; index++) { ConnectorTaskId taskId = new ConnectorTaskId(connName, index); - if (!taskProps.get(index).equals(configState.taskConfig(taskId))) { + if (!rawTaskProps.get(index).equals(configState.rawTaskConfig(taskId))) { log.debug("Connector {} has change in configuration for task {}-{}", connName, connName, index); result = true; } } } + if (!result) { + Map appliedConnectorConfig = configState.appliedConnectorConfig(connName); + Map currentConnectorConfig = configState.connectorConfig(connName); + if (!Objects.equals(appliedConnectorConfig, currentConnectorConfig)) { + log.debug("Forcing task restart for connector {} as its configuration appears to be updated", connName); + result = true; + } + } if (result) { log.debug("Reconfiguring connector {}: writing new updated configurations for tasks", connName); } else { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java index 14826e982d..1880fa512d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.runtime.distributed; import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.MetadataRecoveryStrategy; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigException; @@ -96,6 +97,10 @@ public class DistributedConfig extends WorkerConfig { public static final String REBALANCE_TIMEOUT_MS_CONFIG = CommonClientConfigs.REBALANCE_TIMEOUT_MS_CONFIG; private static final String REBALANCE_TIMEOUT_MS_DOC = CommonClientConfigs.REBALANCE_TIMEOUT_MS_DOC; + public static final String METADATA_RECOVERY_STRATEGY_CONFIG = CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG; + private static final String METADATA_RECOVERY_STRATEGY_DOC = CommonClientConfigs.METADATA_RECOVERY_STRATEGY_DOC; + public static final String DEFAULT_METADATA_RECOVERY_STRATEGY = CommonClientConfigs.DEFAULT_METADATA_RECOVERY_STRATEGY; + /** * worker.sync.timeout.ms */ @@ -512,7 +517,14 @@ private static ConfigDef config(Crypto crypto) { (name, value) -> validateVerificationAlgorithms(crypto, name, (List) value), () -> "A list of one or more MAC algorithms, each supported by the worker JVM"), ConfigDef.Importance.LOW, - INTER_WORKER_VERIFICATION_ALGORITHMS_DOC); + INTER_WORKER_VERIFICATION_ALGORITHMS_DOC) + .define(METADATA_RECOVERY_STRATEGY_CONFIG, + ConfigDef.Type.STRING, + DEFAULT_METADATA_RECOVERY_STRATEGY, + ConfigDef.CaseInsensitiveValidString + .in(Utils.enumOptions(MetadataRecoveryStrategy.class)), + ConfigDef.Importance.LOW, + METADATA_RECOVERY_STRATEGY_DOC); } private final ExactlyOnceSourceSupport exactlyOnceSourceSupport; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java index cdffbb8787..ab46ee536a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java @@ -2229,11 +2229,11 @@ private void reconfigureConnector(final String connName, final Callback cb } private void publishConnectorTaskConfigs(String connName, List> taskProps, Callback cb) { - if (!taskConfigsChanged(configState, connName, taskProps)) { + List> rawTaskProps = reverseTransform(connName, configState, taskProps); + if (!taskConfigsChanged(configState, connName, rawTaskProps)) { return; } - List> rawTaskProps = reverseTransform(connName, configState, taskProps); if (isLeader()) { writeTaskConfigs(connName, rawTaskProps); cb.onCompletion(null, null); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java index 2c3537ea67..2ea83daf04 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java @@ -20,6 +20,7 @@ import org.apache.kafka.clients.ClientUtils; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.MetadataRecoveryStrategy; import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient; import org.apache.kafka.clients.GroupRebalanceConfig; @@ -119,7 +120,9 @@ public WorkerGroupMember(DistributedConfig config, time, true, new ApiVersions(), - logContext); + logContext, + MetadataRecoveryStrategy.forName(config.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)) + ); this.client = new ConsumerNetworkClient( logContext, netClient, diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/CreateConnectorRequest.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/CreateConnectorRequest.java index da8e235e42..9d338936db 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/CreateConnectorRequest.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/CreateConnectorRequest.java @@ -47,7 +47,7 @@ public Map config() { return config; } - @JsonProperty + @JsonProperty("initial_state") public InitialState initialState() { return initialState; } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java index e773eeefd5..2768d910d4 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java @@ -519,10 +519,10 @@ private synchronized void updateConnectorTasks(String connName) { } List> newTaskConfigs = recomputeTaskConfigs(connName); + List> rawTaskConfigs = reverseTransform(connName, configState, newTaskConfigs); - if (taskConfigsChanged(configState, connName, newTaskConfigs)) { + if (taskConfigsChanged(configState, connName, rawTaskConfigs)) { removeConnectorTasks(connName); - List> rawTaskConfigs = reverseTransform(connName, configState, newTaskConfigs); configBackingStore.putTaskConfigs(connName, rawTaskConfigs); createConnectorTasks(connName); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/AppliedConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/AppliedConnectorConfig.java new file mode 100644 index 0000000000..22f20e4b4a --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/AppliedConnectorConfig.java @@ -0,0 +1,66 @@ +/* + * 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. + */ +package org.apache.kafka.connect.storage; + +import org.apache.kafka.connect.runtime.WorkerConfigTransformer; + +import java.util.Map; + +/** + * Wrapper class for a connector configuration that has been used to generate task configurations + * Supports lazy {@link WorkerConfigTransformer#transform(Map) transformation}. + */ +public class AppliedConnectorConfig { + + private final Map rawConfig; + private Map transformedConfig; + + /** + * Create a new applied config that has not yet undergone + * {@link WorkerConfigTransformer#transform(Map) transformation}. + * @param rawConfig the non-transformed connector configuration; may be null + */ + public AppliedConnectorConfig(Map rawConfig) { + this.rawConfig = rawConfig; + } + + /** + * If necessary, {@link WorkerConfigTransformer#transform(Map) transform} the raw + * connector config, then return the result. Transformed configurations are cached and + * returned in all subsequent calls. + *

+ * This method is thread-safe: different threads may invoke it at any time and the same + * transformed config should always be returned, with transformation still only ever + * taking place once before its results are cached. + * @param configTransformer the transformer to use, if no transformed connector + * config has been cached yet; may be null + * @return the possibly-cached, transformed, connector config; may be null + */ + public synchronized Map transformedConfig(WorkerConfigTransformer configTransformer) { + if (transformedConfig != null || rawConfig == null) + return transformedConfig; + + if (configTransformer != null) { + transformedConfig = configTransformer.transform(rawConfig); + } else { + transformedConfig = rawConfig; + } + + return transformedConfig; + } + +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ClusterConfigState.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ClusterConfigState.java index 1025373042..df5b00926f 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ClusterConfigState.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ClusterConfigState.java @@ -43,6 +43,7 @@ public class ClusterConfigState { Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap(), + Collections.emptyMap(), Collections.emptySet(), Collections.emptySet()); @@ -55,6 +56,7 @@ public class ClusterConfigState { final Map> taskConfigs; final Map connectorTaskCountRecords; final Map connectorTaskConfigGenerations; + final Map appliedConnectorConfigs; final Set connectorsPendingFencing; final Set inconsistentConnectors; @@ -66,6 +68,7 @@ public ClusterConfigState(long offset, Map> taskConfigs, Map connectorTaskCountRecords, Map connectorTaskConfigGenerations, + Map appliedConnectorConfigs, Set connectorsPendingFencing, Set inconsistentConnectors) { this(offset, @@ -76,6 +79,7 @@ public ClusterConfigState(long offset, taskConfigs, connectorTaskCountRecords, connectorTaskConfigGenerations, + appliedConnectorConfigs, connectorsPendingFencing, inconsistentConnectors, null); @@ -89,6 +93,7 @@ public ClusterConfigState(long offset, Map> taskConfigs, Map connectorTaskCountRecords, Map connectorTaskConfigGenerations, + Map appliedConnectorConfigs, Set connectorsPendingFencing, Set inconsistentConnectors, WorkerConfigTransformer configTransformer) { @@ -100,6 +105,7 @@ public ClusterConfigState(long offset, this.taskConfigs = taskConfigs; this.connectorTaskCountRecords = connectorTaskCountRecords; this.connectorTaskConfigGenerations = connectorTaskConfigGenerations; + this.appliedConnectorConfigs = appliedConnectorConfigs; this.connectorsPendingFencing = connectorsPendingFencing; this.inconsistentConnectors = inconsistentConnectors; this.configTransformer = configTransformer; @@ -158,6 +164,19 @@ public Map rawConnectorConfig(String connector) { return connectorConfigs.get(connector); } + /** + * Get the most recent configuration for the connector from which task configs have + * been generated. The configuration will have been transformed by + * {@link org.apache.kafka.common.config.ConfigTransformer} + * @param connector name of the connector + * @return the connector config, or null if no config exists from which task configs have + * been generated + */ + public Map appliedConnectorConfig(String connector) { + AppliedConnectorConfig appliedConfig = appliedConnectorConfigs.get(connector); + return appliedConfig != null ? appliedConfig.transformedConfig(configTransformer) : null; + } + /** * Get the target state of the connector * @param connector name of the connector @@ -303,4 +322,5 @@ public int hashCode() { inconsistentConnectors, configTransformer); } + } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java index c24dd6c790..1360765964 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java @@ -318,6 +318,7 @@ public static String LOGGER_CLUSTER_KEY(String namespace) { final Map connectorTaskCountRecords = new HashMap<>(); final Map connectorTaskConfigGenerations = new HashMap<>(); + final Map appliedConnectorConfigs = new HashMap<>(); final Set connectorsPendingFencing = new HashSet<>(); private final WorkerConfigTransformer configTransformer; @@ -478,6 +479,7 @@ public ClusterConfigState snapshot() { new HashMap<>(taskConfigs), new HashMap<>(connectorTaskCountRecords), new HashMap<>(connectorTaskConfigGenerations), + new HashMap<>(appliedConnectorConfigs), new HashSet<>(connectorsPendingFencing), new HashSet<>(inconsistent), configTransformer @@ -997,11 +999,8 @@ private void processConnectorConfigRecord(String connectorName, SchemaAndValue v synchronized (lock) { if (value.value() == null) { // Connector deletion will be written as a null value + processConnectorRemoval(connectorName); log.info("Successfully processed removal of connector '{}'", connectorName); - connectorConfigs.remove(connectorName); - connectorTaskCounts.remove(connectorName); - taskConfigs.keySet().removeIf(taskId -> taskId.connector().equals(connectorName)); - deferredTaskUpdates.remove(connectorName); removed = true; } else { // Connector configs can be applied and callbacks invoked immediately @@ -1064,6 +1063,22 @@ private void processTaskConfigRecord(ConnectorTaskId taskId, SchemaAndValue valu private void processTasksCommitRecord(String connectorName, SchemaAndValue value) { List updatedTasks = new ArrayList<>(); synchronized (lock) { + // Edge case: connector was deleted before these task configs were published, + // but compaction took place and both the original connector config and the + // tombstone message for it have been removed from the config topic + // We should ignore these task configs + Map appliedConnectorConfig = connectorConfigs.get(connectorName); + if (appliedConnectorConfig == null) { + processConnectorRemoval(connectorName); + log.debug( + "Ignoring task configs for connector {}; it appears that the connector was deleted previously " + + "and that log compaction has since removed any trace of its previous configurations " + + "from the config topic", + connectorName + ); + return; + } + // Apply any outstanding deferred task updates for the given connector. Note that just because we // encounter a commit message does not mean it will result in consistent output. In particular due to // compaction, there may be cases where . For example if we have the following sequence of writes: @@ -1111,6 +1126,11 @@ private void processTasksCommitRecord(String connectorName, SchemaAndValue value connectorTaskConfigGenerations.compute(connectorName, (ignored, generation) -> generation != null ? generation + 1 : 0); } inconsistent.remove(connectorName); + + appliedConnectorConfigs.put( + connectorName, + new AppliedConnectorConfig(appliedConnectorConfig) + ); } // Always clear the deferred entries, even if we didn't apply them. If they represented an inconsistent // update, then we need to see a completely fresh set of configs after this commit message, so we don't @@ -1168,7 +1188,7 @@ private void processTaskCountRecord(String connectorName, SchemaAndValue value) log.debug("Setting task count record for connector '{}' to {}", connectorName, taskCount); connectorTaskCountRecords.put(connectorName, taskCount); - // If a task count record appears after the latest task configs, the connectors doesn't need a round of zombie + // If a task count record appears after the latest task configs, the connector doesn't need a round of zombie // fencing before it can start tasks with the latest configs connectorsPendingFencing.remove(connectorName); } @@ -1244,6 +1264,14 @@ private void processLoggerLevelRecord(String namespace, SchemaAndValue value) { } } + private void processConnectorRemoval(String connectorName) { + connectorConfigs.remove(connectorName); + connectorTaskCounts.remove(connectorName); + taskConfigs.keySet().removeIf(taskId -> taskId.connector().equals(connectorName)); + deferredTaskUpdates.remove(connectorName); + appliedConnectorConfigs.remove(connectorName); + } + private ConnectorTaskId parseTaskId(String key) { String[] parts = key.split("-"); if (parts.length < 3) return null; @@ -1314,5 +1342,6 @@ else if (value instanceof Long) else throw new ConnectException("Expected integer value to be either Integer or Long"); } + } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java index 3b9ba966ca..0fe3a5d811 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java @@ -21,6 +21,8 @@ import org.apache.kafka.connect.runtime.TargetState; import org.apache.kafka.connect.runtime.WorkerConfigTransformer; import org.apache.kafka.connect.util.ConnectorTaskId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.Collections; import java.util.HashMap; @@ -36,6 +38,8 @@ */ public class MemoryConfigBackingStore implements ConfigBackingStore { + private static final Logger log = LoggerFactory.getLogger(MemoryConfigBackingStore.class); + private final Map connectors = new HashMap<>(); private UpdateListener updateListener; private WorkerConfigTransformer configTransformer; @@ -61,6 +65,7 @@ public synchronized ClusterConfigState snapshot() { Map> connectorConfigs = new HashMap<>(); Map connectorTargetStates = new HashMap<>(); Map> taskConfigs = new HashMap<>(); + Map appliedConnectorConfigs = new HashMap<>(); for (Map.Entry connectorStateEntry : connectors.entrySet()) { String connector = connectorStateEntry.getKey(); @@ -69,6 +74,9 @@ public synchronized ClusterConfigState snapshot() { connectorConfigs.put(connector, connectorState.connConfig); connectorTargetStates.put(connector, connectorState.targetState); taskConfigs.putAll(connectorState.taskConfigs); + if (connectorState.appliedConnConfig != null) { + appliedConnectorConfigs.put(connector, connectorState.appliedConnConfig); + } } return new ClusterConfigState( @@ -80,6 +88,7 @@ public synchronized ClusterConfigState snapshot() { taskConfigs, Collections.emptyMap(), Collections.emptyMap(), + appliedConnectorConfigs, Collections.emptySet(), Collections.emptySet(), configTransformer @@ -123,6 +132,7 @@ public synchronized void removeTaskConfigs(String connector) { HashSet taskIds = new HashSet<>(state.taskConfigs.keySet()); state.taskConfigs.clear(); + state.appliedConnConfig = null; if (updateListener != null) updateListener.onTaskConfigUpdate(taskIds); @@ -137,6 +147,8 @@ public synchronized void putTaskConfigs(String connector, List> taskConfigsMap = taskConfigListAsMap(connector, configs); state.taskConfigs = taskConfigsMap; + state.applyConfig(); + if (updateListener != null) updateListener.onTaskConfigUpdate(taskConfigsMap.keySet()); } @@ -187,6 +199,7 @@ private static class ConnectorState { private TargetState targetState; private Map connConfig; private Map> taskConfigs; + private AppliedConnectorConfig appliedConnConfig; /** * @param connConfig the connector's configuration @@ -197,6 +210,11 @@ public ConnectorState(Map connConfig, TargetState targetState) { this.targetState = targetState == null ? TargetState.STARTED : targetState; this.connConfig = connConfig; this.taskConfigs = new HashMap<>(); + this.appliedConnConfig = null; + } + + public void applyConfig() { + this.appliedConnConfig = new AppliedConnectorConfig(connConfig); } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/BlockingConnectorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/BlockingConnectorTest.java index 3eefee64c0..532ab1baaf 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/BlockingConnectorTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/BlockingConnectorTest.java @@ -134,11 +134,6 @@ public void setup() throws Exception { // start the clusters connect.start(); - - connect.assertions().assertAtLeastNumWorkersAreUp( - NUM_WORKERS, - "Initial group of workers did not start in time" - ); } @After diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java index 8a730c198d..84ff88013f 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java @@ -17,19 +17,28 @@ package org.apache.kafka.connect.integration; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.config.provider.FileConfigProvider; +import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.utils.LogCaptureAppender; +import org.apache.kafka.connect.connector.Task; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.json.JsonConverter; import org.apache.kafka.connect.json.JsonConverterConfig; import org.apache.kafka.connect.runtime.distributed.DistributedConfig; import org.apache.kafka.connect.runtime.distributed.DistributedHerder; +import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffset; +import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffsets; import org.apache.kafka.connect.runtime.rest.entities.CreateConnectorRequest; import org.apache.kafka.connect.runtime.rest.resources.ConnectorsResource; import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException; +import org.apache.kafka.connect.sink.SinkConnector; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTask; import org.apache.kafka.connect.storage.KafkaConfigBackingStore; import org.apache.kafka.connect.storage.StringConverter; import org.apache.kafka.connect.util.ConnectorTaskId; +import org.apache.kafka.connect.util.SinkUtils; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; import org.apache.kafka.connect.util.clusters.WorkerHandle; import org.apache.kafka.test.IntegrationTest; @@ -39,11 +48,15 @@ import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.rules.TemporaryFolder; import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; +import java.io.File; +import java.io.FileOutputStream; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -53,9 +66,14 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import static javax.ws.rs.core.Response.Status.INTERNAL_SERVER_ERROR; import static org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.kafka.common.config.AbstractConfig.CONFIG_PROVIDERS_CONFIG; +import static org.apache.kafka.common.config.TopicConfig.DELETE_RETENTION_MS_CONFIG; +import static org.apache.kafka.common.config.TopicConfig.SEGMENT_MS_CONFIG; import static org.apache.kafka.connect.integration.BlockingConnectorTest.TASK_STOP; import static org.apache.kafka.connect.integration.MonitorableSourceConnector.TOPIC_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; @@ -71,6 +89,7 @@ import static org.apache.kafka.connect.runtime.WorkerConfig.CONNECTOR_CLIENT_POLICY_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.CONFIG_STORAGE_PREFIX; import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.CONFIG_TOPIC_CONFIG; import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.SCHEDULED_REBALANCE_MAX_DELAY_MS_CONFIG; import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.REBALANCE_TIMEOUT_MS_CONFIG; @@ -108,6 +127,9 @@ public class ConnectWorkerIntegrationTest { @Rule public TestRule watcher = ConnectIntegrationTestUtils.newTestWatcher(log); + @Rule + public TemporaryFolder tmp = new TemporaryFolder(); + @Before public void setup() { // setup Connect worker properties @@ -150,9 +172,6 @@ public void testAddAndRemoveWorker() throws Exception { // set up props for the source connector Map props = defaultSourceConnectorProps(TOPIC_NAME); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, - "Initial group of workers did not start in time."); - // start a source connector connect.configureConnector(CONNECTOR_NAME, props); @@ -196,9 +215,6 @@ public void testRestartFailedTask() throws Exception { props.put(TASKS_MAX_CONFIG, Objects.toString(numTasks)); props.put(CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX + BOOTSTRAP_SERVERS_CONFIG, "nobrokerrunningatthisaddress"); - connect.assertions().assertExactlyNumWorkersAreUp(NUM_WORKERS, - "Initial group of workers did not start in time."); - // Try to start the connector and its single task. connect.configureConnector(CONNECTOR_NAME, props); @@ -236,9 +252,6 @@ public void testBrokerCoordinator() throws Exception { // set up props for the source connector Map props = defaultSourceConnectorProps(TOPIC_NAME); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, - "Initial group of workers did not start in time."); - // start a source connector connect.configureConnector(CONNECTOR_NAME, props); @@ -290,9 +303,6 @@ public void testTaskStatuses() throws Exception { // start the clusters connect.start(); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, - "Initial group of workers did not start in time."); - // base connector props Map props = defaultSourceConnectorProps(TOPIC_NAME); props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName()); @@ -330,8 +340,6 @@ public void testSourceTaskNotBlockedOnShutdownWithNonExistentTopic() throws Exce .build(); connect.start(); - connect.assertions().assertAtLeastNumWorkersAreUp(1, "Initial group of workers did not start in time."); - // and when the connector is not configured to create topics Map props = defaultSourceConnectorProps("nonexistenttopic"); props.remove(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG); @@ -383,9 +391,6 @@ public void testPauseStopResume() throws Exception { // start the clusters connect.start(); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, - "Initial group of workers did not start in time."); - // Want to make sure to use multiple tasks final int numTasks = 4; Map props = defaultSourceConnectorProps(TOPIC_NAME); @@ -475,9 +480,6 @@ public void testStoppedState() throws Exception { // start the clusters connect.start(); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, - "Initial group of workers did not start in time."); - Map props = defaultSourceConnectorProps(TOPIC_NAME); // Fail the connector on startup props.put("connector.start.inject.error", "true"); @@ -550,11 +552,6 @@ public void testTasksConfigDeprecation() throws Exception { // start the clusters connect.start(); - connect.assertions().assertAtLeastNumWorkersAreUp( - NUM_WORKERS, - "Initial group of workers did not start in time." - ); - connect.configureConnector(CONNECTOR_NAME, defaultSourceConnectorProps(TOPIC_NAME)); connect.assertions().assertConnectorAndExactlyNumTasksAreRunning( CONNECTOR_NAME, @@ -578,9 +575,6 @@ public void testCreateConnectorWithPausedInitialState() throws Exception { // start the clusters connect.start(); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, - "Initial group of workers did not start in time."); - CreateConnectorRequest createConnectorRequest = new CreateConnectorRequest( CONNECTOR_NAME, defaultSourceConnectorProps(TOPIC_NAME), @@ -612,9 +606,6 @@ public void testCreateSourceConnectorWithStoppedInitialStateAndModifyOffsets() t // start the clusters connect.start(); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, - "Initial group of workers did not start in time."); - Map props = defaultSourceConnectorProps(TOPIC_NAME); // Configure the connector to produce a maximum of 10 messages @@ -665,9 +656,6 @@ public void testCreateSinkConnectorWithStoppedInitialStateAndModifyOffsets() thr // start the clusters connect.start(); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, - "Initial group of workers did not start in time."); - // Create topic and produce 10 messages connect.kafka().createTopic(TOPIC_NAME); for (int i = 0; i < 10; i++) { @@ -732,9 +720,6 @@ public void testDeleteConnectorCreatedWithPausedOrStoppedInitialState() throws E // start the clusters connect.start(); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, - "Initial group of workers did not start in time."); - // Create a connector with PAUSED initial state CreateConnectorRequest createConnectorRequest = new CreateConnectorRequest( CONNECTOR_NAME, @@ -784,9 +769,6 @@ public void testPatchConnectorConfig() throws Exception { // start the clusters connect.start(); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, - "Initial group of workers did not start in time."); - connect.kafka().createTopic(TOPIC_NAME); Map props = defaultSinkConnectorProps(TOPIC_NAME); @@ -837,8 +819,6 @@ public void testRequestTimeouts() throws Exception { .numWorkers(1) .build(); connect.start(); - connect.assertions().assertAtLeastNumWorkersAreUp(1, - "Worker did not start in time"); Map connectorConfig1 = defaultSourceConnectorProps(TOPIC_NAME); Map connectorConfig2 = new HashMap<>(connectorConfig1); @@ -905,8 +885,6 @@ public void testPollTimeoutExpiry() throws Exception { connect.start(); - connect.assertions().assertExactlyNumWorkersAreUp(1, "Worker not brought up in time"); - Map connectorWithBlockingTaskStopConfig = new HashMap<>(); connectorWithBlockingTaskStopConfig.put(CONNECTOR_CLASS_CONFIG, BlockingConnectorTest.BlockingSourceConnector.class.getName()); connectorWithBlockingTaskStopConfig.put(TASKS_MAX_CONFIG, "1"); @@ -986,11 +964,6 @@ public void testTasksMaxEnforcement() throws Exception { // start the clusters connect.start(); - connect.assertions().assertAtLeastNumWorkersAreUp( - NUM_WORKERS, - "Initial group of workers did not start in time." - ); - Map connectorProps = defaultSourceConnectorProps(TOPIC_NAME); int maxTasks = 1; connectorProps.put(TASKS_MAX_CONFIG, Integer.toString(maxTasks)); @@ -1123,6 +1096,249 @@ public void testTasksMaxEnforcement() throws Exception { ); } + /** + * Task configs are not removed from the config topic after a connector is deleted. + * When topic compaction takes place, this can cause the tombstone message for the + * connector config to be deleted, leaving the task configs in the config topic with no + * explicit record of the connector's deletion. + *

+ * This test guarantees that those older task configs are never used, even when the + * connector is recreated later. + */ + @Test + public void testCompactedDeletedOlderConnectorConfig() throws Exception { + brokerProps.put("log.cleaner.backoff.ms", "100"); + brokerProps.put("log.cleaner.delete.retention.ms", "1"); + brokerProps.put("log.cleaner.max.compaction.lag.ms", "1"); + brokerProps.put("log.cleaner.min.cleanable.ratio", "0"); + brokerProps.put("log.cleaner.min.compaction.lag.ms", "1"); + brokerProps.put("log.cleaner.threads", "1"); + + final String configTopic = "kafka-16838-configs"; + final int offsetCommitIntervalMs = 100; + workerProps.put(CONFIG_TOPIC_CONFIG, configTopic); + workerProps.put(CONFIG_STORAGE_PREFIX + SEGMENT_MS_CONFIG, "100"); + workerProps.put(CONFIG_STORAGE_PREFIX + DELETE_RETENTION_MS_CONFIG, "1"); + workerProps.put(OFFSET_COMMIT_INTERVAL_MS_CONFIG, Integer.toString(offsetCommitIntervalMs)); + + final int numWorkers = 1; + connect = connectBuilder + .numWorkers(numWorkers) + .build(); + // start the clusters + connect.start(); + + final String connectorTopic = "connector-topic"; + connect.kafka().createTopic(connectorTopic, 1); + + ConnectorHandle connectorHandle = RuntimeHandles.get().connectorHandle(CONNECTOR_NAME); + connectorHandle.expectedCommits(NUM_TASKS * 2); + + Map connectorConfig = defaultSourceConnectorProps(connectorTopic); + connect.configureConnector(CONNECTOR_NAME, connectorConfig); + connect.assertions().assertConnectorAndExactlyNumTasksAreRunning( + CONNECTOR_NAME, + NUM_TASKS, + "Connector or its tasks did not start in time" + ); + connectorHandle.awaitCommits(offsetCommitIntervalMs * 3); + + connect.deleteConnector(CONNECTOR_NAME); + + // Roll the entire cluster + connect.activeWorkers().forEach(connect::removeWorker); + + // Miserable hack: produce directly to the config topic and then wait a little bit + // in order to trigger segment rollover and allow compaction to take place + connect.kafka().produce(configTopic, "garbage-key-1", null); + Thread.sleep(1_000); + connect.kafka().produce(configTopic, "garbage-key-2", null); + Thread.sleep(1_000); + + for (int i = 0; i < numWorkers; i++) + connect.addWorker(); + + connect.assertions().assertAtLeastNumWorkersAreUp( + numWorkers, + "Workers did not start in time after cluster was rolled." + ); + + final TopicPartition connectorTopicPartition = new TopicPartition(connectorTopic, 0); + final long initialEndOffset = connect.kafka().endOffset(connectorTopicPartition); + assertTrue( + "Source connector should have published at least one record to Kafka", + initialEndOffset > 0 + ); + + connectorHandle.expectedCommits(NUM_TASKS * 2); + + // Re-create the connector with a different config (targets a different topic) + final String otherConnectorTopic = "other-topic"; + connect.kafka().createTopic(otherConnectorTopic, 1); + connectorConfig.put(TOPIC_CONFIG, otherConnectorTopic); + connect.configureConnector(CONNECTOR_NAME, connectorConfig); + connect.assertions().assertConnectorAndExactlyNumTasksAreRunning( + CONNECTOR_NAME, + NUM_TASKS, + "Connector or its tasks did not start in time" + ); + connectorHandle.awaitCommits(offsetCommitIntervalMs * 3); + + // See if any new records got written to the old topic + final long nextEndOffset = connect.kafka().endOffset(connectorTopicPartition); + assertEquals( + "No new records should have been written to the older topic", + initialEndOffset, + nextEndOffset + ); + } + + /** + * If a connector has existing tasks, and then generates new task configs, workers compare the + * new and existing configs before publishing them to the config topic. If there is no difference, + * workers do not publish task configs (this is a workaround to prevent infinite loops with eager + * rebalancing). + *

+ * This test tries to guarantee that, if the old task configs become invalid because of + * an invalid config provider reference, it will still be possible to reconfigure the connector. + */ + @Test + public void testReconfigureConnectorWithFailingTaskConfigs() throws Exception { + final int offsetCommitIntervalMs = 100; + workerProps.put(CONFIG_PROVIDERS_CONFIG, "file"); + workerProps.put(CONFIG_PROVIDERS_CONFIG + ".file.class", FileConfigProvider.class.getName()); + workerProps.put(OFFSET_COMMIT_INTERVAL_MS_CONFIG, Integer.toString(offsetCommitIntervalMs)); + + final int numWorkers = 1; + connect = connectBuilder + .numWorkers(numWorkers) + .build(); + // start the clusters + connect.start(); + + final String firstConnectorTopic = "connector-topic-1"; + connect.kafka().createTopic(firstConnectorTopic); + + final File secretsFile = tmp.newFile("test-secrets"); + final Properties secrets = new Properties(); + final String throughputSecretKey = "secret-throughput"; + secrets.put(throughputSecretKey, "10"); + try (FileOutputStream secretsOutputStream = new FileOutputStream(secretsFile)) { + secrets.store(secretsOutputStream, null); + } + + ConnectorHandle connectorHandle = RuntimeHandles.get().connectorHandle(CONNECTOR_NAME); + connectorHandle.expectedCommits(NUM_TASKS * 2); + + Map connectorConfig = defaultSourceConnectorProps(firstConnectorTopic); + connectorConfig.put( + "throughput", + "${file:" + secretsFile.getAbsolutePath() + ":" + throughputSecretKey + "}" + ); + connect.configureConnector(CONNECTOR_NAME, connectorConfig); + connect.assertions().assertConnectorAndExactlyNumTasksAreRunning( + CONNECTOR_NAME, + NUM_TASKS, + "Connector or its tasks did not start in time" + ); + connectorHandle.awaitCommits(offsetCommitIntervalMs * 3); + + // Delete the secrets file, which should render the old task configs invalid + assertTrue("Failed to delete secrets file", secretsFile.delete()); + + // Use a start latch here instead of assertConnectorAndExactlyNumTasksAreRunning + // since failure to reconfigure the tasks (which may occur if the bug this test was written + // to help catch resurfaces) will not cause existing tasks to fail or stop running + StartAndStopLatch restarts = connectorHandle.expectedStarts(1); + + final String secondConnectorTopic = "connector-topic-2"; + connect.kafka().createTopic(secondConnectorTopic, 1); + + // Stop using the config provider for this connector, and instruct it to start writing to the + // old topic again + connectorConfig.put("throughput", "10"); + connectorConfig.put(TOPIC_CONFIG, secondConnectorTopic); + connect.configureConnector(CONNECTOR_NAME, connectorConfig); + assertTrue( + "Connector tasks were not restarted in time", + restarts.await(10, TimeUnit.SECONDS) + ); + + // Wait for at least one task to commit offsets after being restarted + connectorHandle.expectedCommits(1); + connectorHandle.awaitCommits(offsetCommitIntervalMs * 3); + + final long endOffset = connect.kafka().endOffset(new TopicPartition(secondConnectorTopic, 0)); + assertTrue( + "Source connector should have published at least one record to new Kafka topic " + + "after being reconfigured", + endOffset > 0 + ); + } + + @Test + public void testRuntimePropertyReconfiguration() throws Exception { + final int offsetCommitIntervalMs = 1_000; + // force fast offset commits + workerProps.put(OFFSET_COMMIT_INTERVAL_MS_CONFIG, Integer.toString(offsetCommitIntervalMs)); + connect = connectBuilder.build(); + // start the clusters + connect.start(); + + final String topic = "kafka9228"; + connect.kafka().createTopic(topic, 1); + connect.kafka().produce(topic, "non-json-value"); + + Map connectorConfig = new HashMap<>(); + connectorConfig.put(CONNECTOR_CLASS_CONFIG, EmptyTaskConfigsConnector.class.getName()); + connectorConfig.put(TASKS_MAX_CONFIG, "1"); + connectorConfig.put(TOPICS_CONFIG, topic); + // Initially configure the connector to use the JSON converter, which should cause task failure(s) + connectorConfig.put(VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); + connectorConfig.put( + VALUE_CONVERTER_CLASS_CONFIG + "." + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, + "false" + ); + + connect.configureConnector(CONNECTOR_NAME, connectorConfig); + connect.assertions().assertConnectorIsRunningAndTasksHaveFailed( + CONNECTOR_NAME, + 1, + "Connector did not start or task did not fail in time" + ); + assertEquals( + "Connector should not have any committed offsets when only task fails on first record", + new ConnectorOffsets(Collections.emptyList()), + connect.connectorOffsets(CONNECTOR_NAME) + ); + + // Reconfigure the connector to use the string converter, which should not cause any more task failures + connectorConfig.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + connectorConfig.remove( + KEY_CONVERTER_CLASS_CONFIG + "." + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG + ); + connect.configureConnector(CONNECTOR_NAME, connectorConfig); + connect.assertions().assertConnectorAndExactlyNumTasksAreRunning( + CONNECTOR_NAME, + 1, + "Connector or tasks did not start in time" + ); + + Map expectedOffsetKey = new HashMap<>(); + expectedOffsetKey.put(SinkUtils.KAFKA_TOPIC_KEY, topic); + expectedOffsetKey.put(SinkUtils.KAFKA_PARTITION_KEY, 0); + Map expectedOffsetValue = Collections.singletonMap(SinkUtils.KAFKA_OFFSET_KEY, 1); + ConnectorOffset expectedOffset = new ConnectorOffset(expectedOffsetKey, expectedOffsetValue); + ConnectorOffsets expectedOffsets = new ConnectorOffsets(Collections.singletonList(expectedOffset)); + + // Wait for it to commit offsets, signaling that it has successfully processed the record we produced earlier + waitForCondition( + () -> expectedOffsets.equals(connect.connectorOffsets(CONNECTOR_NAME)), + offsetCommitIntervalMs * 2, + "Task did not successfully process record and/or commit offsets in time" + ); + } + private Map defaultSourceConnectorProps(String topic) { // setup props for the source connector Map props = new HashMap<>(); @@ -1137,4 +1353,60 @@ private Map defaultSourceConnectorProps(String topic) { props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(1)); return props; } + + public static class EmptyTaskConfigsConnector extends SinkConnector { + @Override + public String version() { + return "0.0"; + } + + @Override + public void start(Map props) { + // no-op + } + + @Override + public Class taskClass() { + return SimpleTask.class; + } + + @Override + public List> taskConfigs(int maxTasks) { + return IntStream.range(0, maxTasks) + .mapToObj(i -> Collections.emptyMap()) + .collect(Collectors.toList()); + } + + @Override + public void stop() { + // no-op + } + + @Override + public ConfigDef config() { + return new ConfigDef(); + } + } + + public static class SimpleTask extends SinkTask { + @Override + public String version() { + return "0.0"; + } + + @Override + public void start(Map props) { + // no-op + } + + @Override + public void put(Collection records) { + // no-op + } + + @Override + public void stop() { + // no-op + } + } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorClientPolicyIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorClientPolicyIntegrationTest.java index a0abece17d..f09a949010 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorClientPolicyIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorClientPolicyIntegrationTest.java @@ -121,8 +121,6 @@ private EmbeddedConnectCluster connectClusterWithPolicy(String policy) throws In // start the clusters connect.start(); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, - "Initial group of workers did not start in time."); return connect; } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorRestartApiIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorRestartApiIntegrationTest.java index a512eeaae0..e957b97f61 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorRestartApiIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorRestartApiIntegrationTest.java @@ -113,8 +113,6 @@ private void startOrReuseConnectWithNumWorkers(int numWorkers) throws Exception connect.start(); return connect; }); - connect.assertions().assertExactlyNumWorkersAreUp(numWorkers, - "Initial group of workers did not start in time."); } @After diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorTopicsIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorTopicsIntegrationTest.java index eb055ab13f..e9fdd91c38 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorTopicsIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorTopicsIntegrationTest.java @@ -115,8 +115,6 @@ public void testGetActiveTopics() throws InterruptedException { connect.kafka().createTopic(FOO_TOPIC, NUM_TOPIC_PARTITIONS); connect.kafka().createTopic(BAR_TOPIC, NUM_TOPIC_PARTITIONS); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Initial group of workers did not start in time."); - connect.assertions().assertConnectorActiveTopics(FOO_CONNECTOR, Collections.emptyList(), "Active topic set is not empty for connector: " + FOO_CONNECTOR); @@ -179,8 +177,6 @@ public void testTopicTrackingResetIsDisabled() throws InterruptedException { connect.kafka().createTopic(FOO_TOPIC, NUM_TOPIC_PARTITIONS); connect.kafka().createTopic(BAR_TOPIC, NUM_TOPIC_PARTITIONS); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Initial group of workers did not start in time."); - connect.assertions().assertConnectorActiveTopics(FOO_CONNECTOR, Collections.emptyList(), "Active topic set is not empty for connector: " + FOO_CONNECTOR); @@ -235,8 +231,6 @@ public void testTopicTrackingIsDisabled() throws InterruptedException { connect.kafka().createTopic(FOO_TOPIC, NUM_TOPIC_PARTITIONS); connect.kafka().createTopic(BAR_TOPIC, NUM_TOPIC_PARTITIONS); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Initial group of workers did not start in time."); - // start a source connector connect.configureConnector(FOO_CONNECTOR, defaultSourceConnectorProps(FOO_TOPIC)); connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(FOO_CONNECTOR, NUM_TASKS, diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java index 1a76956be6..30fe48116f 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java @@ -94,8 +94,6 @@ public void setup() throws InterruptedException { // start Connect cluster connect.start(); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, - "Initial group of workers did not start in time."); // get connector handles before starting test. connectorHandle = RuntimeHandles.get().connectorHandle(CONNECTOR_NAME); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java index 04dd4c7de6..84ee814ae4 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java @@ -99,6 +99,7 @@ import static org.apache.kafka.connect.source.SourceTask.TransactionBoundary.CONNECTOR; import static org.apache.kafka.connect.source.SourceTask.TransactionBoundary.INTERVAL; import static org.apache.kafka.connect.source.SourceTask.TransactionBoundary.POLL; +import static org.apache.kafka.test.TestUtils.waitForCondition; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -500,9 +501,6 @@ public void testFencedLeaderRecovery() throws Exception { connectorHandle.expectedRecords(MINIMUM_MESSAGES); connectorHandle.expectedCommits(MINIMUM_MESSAGES); - // make sure the worker is actually up (otherwise, it may fence out our simulated zombie leader, instead of the other way around) - connect.assertions().assertExactlyNumWorkersAreUp(1, "Connect worker did not complete startup in time"); - // fence out the leader of the cluster Producer zombieLeader = transactionalProducer( "simulated-zombie-leader", @@ -750,9 +748,18 @@ public void testSeparateOffsetsTopic() throws Exception { workerProps.put(DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG, globalOffsetsTopic); startConnect(); - EmbeddedKafkaCluster connectorTargetedCluster = new EmbeddedKafkaCluster(1, brokerProps); + + int numConnectorTargetedBrokers = 1; + EmbeddedKafkaCluster connectorTargetedCluster = new EmbeddedKafkaCluster(numConnectorTargetedBrokers, brokerProps); try (Closeable clusterShutdown = connectorTargetedCluster::stop) { connectorTargetedCluster.start(); + // Wait for the connector-targeted Kafka cluster to get on its feet + waitForCondition( + () -> connectorTargetedCluster.runningBrokers().size() == numConnectorTargetedBrokers, + ConnectAssertions.WORKER_SETUP_DURATION_MS, + "Separate Kafka cluster did not start in time" + ); + String topic = "test-topic"; connectorTargetedCluster.createTopic(topic, 3); @@ -780,6 +787,11 @@ public void testSeparateOffsetsTopic() throws Exception { // start a source connector connect.configureConnector(CONNECTOR_NAME, props); + connect.assertions().assertConnectorAndExactlyNumTasksAreRunning( + CONNECTOR_NAME, + numTasks, + "connector and tasks did not start in time" + ); log.info("Waiting for records to be provided to worker by task"); // wait for the connector tasks to produce enough records diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/InternalTopicsIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/InternalTopicsIntegrationTest.java index d73d1c4ed0..c044bb8229 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/InternalTopicsIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/InternalTopicsIntegrationTest.java @@ -72,8 +72,6 @@ public void testCreateInternalTopicsWithDefaultSettings() throws InterruptedExce // Start the Connect cluster connect.start(); - connect.assertions().assertExactlyNumBrokersAreUp(numBrokers, "Brokers did not start in time."); - connect.assertions().assertExactlyNumWorkersAreUp(numWorkers, "Worker did not start in time."); log.info("Completed startup of {} Kafka brokers and {} Connect workers", numBrokers, numWorkers); // Check the topics @@ -111,9 +109,6 @@ public void testCreateInternalTopicsWithFewerReplicasThanBrokers() throws Interr // Start the Connect cluster connect.start(); - connect.assertions().assertExactlyNumBrokersAreUp(numBrokers, "Broker did not start in time."); - connect.assertions().assertAtLeastNumWorkersAreUp(numWorkers, "Worker did not start in time."); - log.info("Completed startup of {} Kafka brokers and {} Connect workers", numBrokers, numWorkers); // Check the topics log.info("Verifying the internal topics for Connect"); @@ -126,7 +121,7 @@ public void testFailToCreateInternalTopicsWithMoreReplicasThanBrokers() throws I workerProps.put(DistributedConfig.CONFIG_STORAGE_REPLICATION_FACTOR_CONFIG, "3"); workerProps.put(DistributedConfig.OFFSET_STORAGE_REPLICATION_FACTOR_CONFIG, "2"); workerProps.put(DistributedConfig.STATUS_STORAGE_REPLICATION_FACTOR_CONFIG, "1"); - int numWorkers = 1; + int numWorkers = 0; int numBrokers = 1; connect = new EmbeddedConnectCluster.Builder().name("connect-cluster-1") .workerProps(workerProps) @@ -137,11 +132,14 @@ public void testFailToCreateInternalTopicsWithMoreReplicasThanBrokers() throws I // Start the brokers and Connect, but Connect should fail to create config and offset topic connect.start(); - connect.assertions().assertExactlyNumBrokersAreUp(numBrokers, "Broker did not start in time."); log.info("Completed startup of {} Kafka broker. Expected Connect worker to fail", numBrokers); + // Try to start a worker + connect.addWorker(); + // Verify that the offset and config topic don't exist; // the status topic may have been created if timing was right but we don't care + // TODO: Synchronously await and verify that the worker fails during startup log.info("Verifying the internal topics for Connect"); connect.assertions().assertTopicsDoNotExist(configTopic(), offsetTopic()); } @@ -169,7 +167,6 @@ public void testFailToStartWhenInternalTopicsAreNotCompacted() throws Interrupte // Start the brokers but not Connect log.info("Starting {} Kafka brokers, but no Connect workers yet", numBrokers); connect.start(); - connect.assertions().assertExactlyNumBrokersAreUp(numBrokers, "Broker did not start in time."); log.info("Completed startup of {} Kafka broker. Expected Connect worker to fail", numBrokers); // Create the good topics @@ -243,7 +240,6 @@ public void testStartWhenInternalTopicsCreatedManuallyWithCompactForBrokersDefau // Start the brokers but not Connect log.info("Starting {} Kafka brokers, but no Connect workers yet", numBrokers); connect.start(); - connect.assertions().assertExactlyNumBrokersAreUp(numBrokers, "Broker did not start in time."); log.info("Completed startup of {} Kafka broker. Expected Connect worker to fail", numBrokers); // Create the valid internal topics w/o topic settings, so these will use the broker's diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RebalanceSourceConnectorsIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RebalanceSourceConnectorsIntegrationTest.java index 82004c8dc3..4c803a3492 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RebalanceSourceConnectorsIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RebalanceSourceConnectorsIntegrationTest.java @@ -116,9 +116,6 @@ public void testStartTwoConnectors() throws Exception { // setup up props for the source connector Map props = defaultSourceConnectorProps(TOPIC_NAME); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, - "Connect workers did not start in time."); - // start a source connector connect.configureConnector(CONNECTOR_NAME, props); @@ -147,9 +144,6 @@ public void testReconfigConnector() throws Exception { // setup up props for the source connector Map props = defaultSourceConnectorProps(TOPIC_NAME); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, - "Connect workers did not start in time."); - // start a source connector connect.configureConnector(CONNECTOR_NAME, props); @@ -194,9 +188,6 @@ public void testDeleteConnector() throws Exception { // setup up props for the source connector Map props = defaultSourceConnectorProps(TOPIC_NAME); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, - "Connect workers did not start in time."); - // start several source connectors IntStream.range(0, 4).forEachOrdered(i -> connect.configureConnector(CONNECTOR_NAME + i, props)); @@ -221,9 +212,6 @@ public void testAddingWorker() throws Exception { // setup up props for the source connector Map props = defaultSourceConnectorProps(TOPIC_NAME); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, - "Connect workers did not start in time."); - // start a source connector IntStream.range(0, 4).forEachOrdered(i -> connect.configureConnector(CONNECTOR_NAME + i, props)); @@ -250,9 +238,6 @@ public void testRemovingWorker() throws Exception { // setup up props for the source connector Map props = defaultSourceConnectorProps(TOPIC_NAME); - connect.assertions().assertExactlyNumWorkersAreUp(NUM_WORKERS, - "Connect workers did not start in time."); - // start a source connector IntStream.range(0, 4).forEachOrdered(i -> connect.configureConnector(CONNECTOR_NAME + i, props)); @@ -276,9 +261,6 @@ public void testMultipleWorkersRejoining() throws Exception { // setup up props for the source connector Map props = defaultSourceConnectorProps(TOPIC_NAME); - connect.assertions().assertExactlyNumWorkersAreUp(NUM_WORKERS, - "Connect workers did not start in time."); - // start a source connector IntStream.range(0, 4).forEachOrdered(i -> connect.configureConnector(CONNECTOR_NAME + i, props)); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RestExtensionIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RestExtensionIntegrationTest.java index a0f993f2f6..112dd21981 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RestExtensionIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RestExtensionIntegrationTest.java @@ -77,9 +77,6 @@ public void testRestExtensionApi() throws InterruptedException { // start the clusters connect.start(); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, - "Initial group of workers did not start in time."); - WorkerHandle worker = connect.workers().stream() .findFirst() .orElseThrow(() -> new AssertionError("At least one worker handle should be available")); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/SessionedProtocolIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/SessionedProtocolIntegrationTest.java index 7ced24c82f..8f71033b79 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/SessionedProtocolIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/SessionedProtocolIntegrationTest.java @@ -19,9 +19,7 @@ import org.apache.kafka.connect.runtime.distributed.ConnectProtocolCompatibility; import org.apache.kafka.connect.storage.StringConverter; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; -import org.apache.kafka.connect.util.clusters.WorkerHandle; import org.apache.kafka.test.IntegrationTest; -import org.apache.kafka.test.TestUtils; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -98,10 +96,6 @@ public void ensureInternalEndpointIsSecured() throws Throwable { invalidSignatureHeaders.put(SIGNATURE_HEADER, "S2Fma2Flc3F1ZQ=="); invalidSignatureHeaders.put(SIGNATURE_ALGORITHM_HEADER, "HmacSHA256"); - TestUtils.waitForCondition( - () -> connect.workers().stream().allMatch(WorkerHandle::isRunning), - 30000L, "Timed out waiting for workers to start"); - // We haven't created the connector yet, but this should still return a 400 instead of a 404 // if the endpoint is secured log.info( @@ -120,9 +114,10 @@ public void ensureInternalEndpointIsSecured() throws Throwable { + "expecting 403 error response", connectorTasksEndpoint ); - TestUtils.waitForCondition( - () -> connect.requestPost(connectorTasksEndpoint, "[]", invalidSignatureHeaders).getStatus() == FORBIDDEN.getStatusCode(), - 30000L, "Timed out waiting for workers to start"); + assertEquals( + FORBIDDEN.getStatusCode(), + connect.requestPost(connectorTasksEndpoint, "[]", invalidSignatureHeaders).getStatus() + ); // Create the connector now // setup up props for the sink connector diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/SinkConnectorsIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/SinkConnectorsIntegrationTest.java index a8bfbb291f..f42addf396 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/SinkConnectorsIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/SinkConnectorsIntegrationTest.java @@ -83,7 +83,6 @@ public void setup() throws Exception { .brokerProps(brokerProps) .build(); connect.start(); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Initial group of workers did not start in time."); } @After @@ -209,7 +208,6 @@ public void testCooperativeConsumerPartitionAssignment() throws Exception { final Collection topics = Arrays.asList(topic1, topic2, topic3); Map connectorProps = baseSinkConnectorProps(String.join(",", topics)); - // Need an eager assignor here; round robin is as good as any connectorProps.put( CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX + PARTITION_ASSIGNMENT_STRATEGY_CONFIG, CooperativeStickyAssignor.class.getName()); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/SourceConnectorsIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/SourceConnectorsIntegrationTest.java index b35b072080..046e5a9ccf 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/SourceConnectorsIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/SourceConnectorsIntegrationTest.java @@ -103,8 +103,6 @@ public void testTopicsAreCreatedWhenAutoCreateTopicsIsEnabledAtTheBroker() throw // start the clusters connect.start(); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Initial group of workers did not start in time."); - Map fooProps = sourceConnectorPropsWithGroups(FOO_TOPIC); // start a source connector @@ -128,8 +126,6 @@ public void testTopicsAreCreatedWhenTopicCreationIsEnabled() throws InterruptedE // start the clusters connect.start(); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Initial group of workers did not start in time."); - Map fooProps = sourceConnectorPropsWithGroups(FOO_TOPIC); // start a source connector @@ -160,8 +156,6 @@ public void testSwitchingToTopicCreationEnabled() throws InterruptedException { connect.assertions().assertTopicSettings(BAR_TOPIC, DEFAULT_REPLICATION_FACTOR, DEFAULT_PARTITIONS, "Topic " + BAR_TOPIC + " does not have the expected settings"); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Initial group of workers did not start in time."); - Map barProps = defaultSourceConnectorProps(BAR_TOPIC); // start a source connector with topic creation properties connect.configureConnector(BAR_CONNECTOR, barProps); @@ -195,7 +189,7 @@ public void testSwitchingToTopicCreationEnabled() throws InterruptedException { workerProps.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(true)); IntStream.range(0, 3).forEach(i -> connect.addWorker()); - connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Initial group of workers did not start in time."); + connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Workers did not start in time after cluster was rolled."); connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(FOO_CONNECTOR, NUM_TASKS, "Connector tasks did not start in time."); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TransformationIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TransformationIntegrationTest.java index 02d8c7f71b..760684425d 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TransformationIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TransformationIntegrationTest.java @@ -115,8 +115,6 @@ public void close() { */ @Test public void testFilterOnTopicNameWithSinkConnector() throws Exception { - assertConnectReady(); - Map observedRecords = observeRecords(); // create test topics @@ -180,12 +178,6 @@ public void testFilterOnTopicNameWithSinkConnector() throws Exception { connect.deleteConnector(CONNECTOR_NAME); } - private void assertConnectReady() throws InterruptedException { - connect.assertions().assertExactlyNumBrokersAreUp(1, "Brokers did not start in time."); - connect.assertions().assertExactlyNumWorkersAreUp(NUM_WORKERS, "Worker did not start in time."); - log.info("Completed startup of {} Kafka brokers and {} Connect workers", 1, NUM_WORKERS); - } - private void assertConnectorRunning() throws InterruptedException { connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(CONNECTOR_NAME, NUM_TASKS, "Connector tasks did not start in time."); @@ -212,8 +204,6 @@ record -> observedRecords.compute(record.topic(), */ @Test public void testFilterOnTombstonesWithSinkConnector() throws Exception { - assertConnectReady(); - Map observedRecords = observeRecords(); // create test topics @@ -273,8 +263,6 @@ public void testFilterOnTombstonesWithSinkConnector() throws Exception { */ @Test public void testFilterOnHasHeaderKeyWithSourceConnectorAndTopicCreation() throws Exception { - assertConnectReady(); - // setup up props for the sink connector Map props = new HashMap<>(); props.put("name", CONNECTOR_NAME); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java index 62283a0277..da8fed5b66 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java @@ -47,6 +47,7 @@ import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; import org.apache.kafka.connect.runtime.rest.entities.ConnectorType; import org.apache.kafka.connect.runtime.rest.errors.BadRequestException; +import org.apache.kafka.connect.storage.AppliedConnectorConfig; import org.apache.kafka.connect.storage.ClusterConfigState; import org.apache.kafka.connect.storage.ConfigBackingStore; import org.apache.kafka.connect.storage.StatusBackingStore; @@ -84,12 +85,14 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.CALLS_REAL_METHODS; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -147,6 +150,7 @@ public class AbstractHerderTest { TASK_CONFIGS_MAP, Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONN1, new AppliedConnectorConfig(CONN1_CONFIG)), Collections.emptySet(), Collections.emptySet()); private static final ClusterConfigState SNAPSHOT_NO_TASKS = new ClusterConfigState( @@ -158,6 +162,7 @@ public class AbstractHerderTest { Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONN1, new AppliedConnectorConfig(CONN1_CONFIG)), Collections.emptySet(), Collections.emptySet()); @@ -1116,6 +1121,69 @@ public void testConnectorOffsets() throws Exception { assertEquals(offsets, cb.get(1000, TimeUnit.MILLISECONDS)); } + @Test + public void testTaskConfigComparison() { + ClusterConfigState snapshot = mock(ClusterConfigState.class); + + when(snapshot.taskCount(CONN1)).thenReturn(TASK_CONFIGS.size()); + TASK_CONFIGS_MAP.forEach((task, config) -> when(snapshot.rawTaskConfig(task)).thenReturn(config)); + // Same task configs, same number of tasks--no change + assertFalse(AbstractHerder.taskConfigsChanged(snapshot, CONN1, TASK_CONFIGS)); + + when(snapshot.taskCount(CONN1)).thenReturn(TASK_CONFIGS.size() + 1); + // Different number of tasks; should report a change + assertTrue(AbstractHerder.taskConfigsChanged(snapshot, CONN1, TASK_CONFIGS)); + + when(snapshot.taskCount(CONN1)).thenReturn(TASK_CONFIG.size()); + List> alteredTaskConfigs = new ArrayList<>(TASK_CONFIGS); + alteredTaskConfigs.set(alteredTaskConfigs.size() - 1, Collections.emptyMap()); + // Last task config is different; should report a change + assertTrue(AbstractHerder.taskConfigsChanged(snapshot, CONN1, alteredTaskConfigs)); + + // Make sure we used exclusively raw task configs and never attempted transformation, + // since otherwise failures in transformation could either cause an infinite loop of task + // config generation, or prevent any task configs from being published + verify(snapshot, never()).taskConfig(any()); + } + + @Test + public void testTaskConfigsChangedWhenAppliedConnectorConfigDiffers() { + assertFalse(AbstractHerder.taskConfigsChanged(SNAPSHOT, CONN1, TASK_CONFIGS)); + + ClusterConfigState snapshotWithNoAppliedConfig = new ClusterConfigState( + 1, + null, + Collections.singletonMap(CONN1, 3), + Collections.singletonMap(CONN1, CONN1_CONFIG), + Collections.singletonMap(CONN1, TargetState.STARTED), + TASK_CONFIGS_MAP, + Collections.emptyMap(), + Collections.emptyMap(), + Collections.emptyMap(), + Collections.emptySet(), + Collections.emptySet() + ); + assertTrue(AbstractHerder.taskConfigsChanged(snapshotWithNoAppliedConfig, CONN1, TASK_CONFIGS)); + + Map appliedConfig = new HashMap<>(CONN1_CONFIG); + String newTopicsProperty = appliedConfig.getOrDefault(SinkConnectorConfig.TOPICS_CONFIG, "foo") + ",newTopic"; + appliedConfig.put(SinkConnectorConfig.TOPICS_CONFIG, newTopicsProperty); + ClusterConfigState snapshotWithDifferentAppliedConfig = new ClusterConfigState( + 1, + null, + Collections.singletonMap(CONN1, 3), + Collections.singletonMap(CONN1, CONN1_CONFIG), + Collections.singletonMap(CONN1, TargetState.STARTED), + TASK_CONFIGS_MAP, + Collections.emptyMap(), + Collections.emptyMap(), + Collections.singletonMap(CONN1, new AppliedConnectorConfig(appliedConfig)), + Collections.emptySet(), + Collections.emptySet() + ); + assertTrue(AbstractHerder.taskConfigsChanged(snapshotWithDifferentAppliedConfig, CONN1, TASK_CONFIGS)); + } + protected void addConfigKey(Map keys, String name, String group) { ConfigDef configDef = new ConfigDef().define(name, ConfigDef.Type.STRING, null, null, ConfigDef.Importance.HIGH, "doc", group, 10, diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java index b51b84d1ac..4c5a04533e 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java @@ -73,6 +73,7 @@ import org.apache.kafka.connect.source.SourceConnector; import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.source.SourceTask; +import org.apache.kafka.connect.storage.AppliedConnectorConfig; import org.apache.kafka.connect.storage.CloseableOffsetStorageReader; import org.apache.kafka.connect.storage.ClusterConfigState; import org.apache.kafka.connect.storage.ConnectorOffsetBackingStore; @@ -634,6 +635,7 @@ public void testAddRemoveSourceTask() { Collections.singletonMap(TASK_ID, origProps), Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONNECTOR_ID, new AppliedConnectorConfig(connectorConfigs)), Collections.emptySet(), Collections.emptySet() ); @@ -689,6 +691,7 @@ public void testAddRemoveSinkTask() { Collections.singletonMap(TASK_ID, origProps), Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONNECTOR_ID, new AppliedConnectorConfig(connectorConfigs)), Collections.emptySet(), Collections.emptySet() ); @@ -759,6 +762,7 @@ public void testAddRemoveExactlyOnceSourceTask() { Collections.singletonMap(TASK_ID, origProps), Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONNECTOR_ID, new AppliedConnectorConfig(connectorConfigs)), Collections.emptySet(), Collections.emptySet() ); @@ -2728,6 +2732,7 @@ private void testStartTaskWithTooManyTaskConfigs(boolean enforced) { Collections.singletonMap(TASK_ID, origProps), Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(connName, new AppliedConnectorConfig(connectorConfigs)), Collections.emptySet(), Collections.emptySet() ); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTestUtils.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTestUtils.java index 084d865cc5..6101dc48c6 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTestUtils.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTestUtils.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.connect.runtime; +import org.apache.kafka.connect.storage.AppliedConnectorConfig; import org.apache.kafka.connect.storage.ClusterConfigState; import org.apache.kafka.connect.runtime.distributed.ExtendedAssignment; import org.apache.kafka.connect.runtime.distributed.ExtendedWorkerState; @@ -63,15 +64,22 @@ public static List newTasks(int start, int end) { public static ClusterConfigState clusterConfigState(long offset, int connectorNum, int taskNum) { + Map> connectorConfigs = connectorConfigs(1, connectorNum); + Map appliedConnectorConfigs = connectorConfigs.entrySet().stream() + .collect(Collectors.toMap( + Map.Entry::getKey, + e -> new AppliedConnectorConfig(e.getValue()) + )); return new ClusterConfigState( offset, null, connectorTaskCounts(1, connectorNum, taskNum), - connectorConfigs(1, connectorNum), + connectorConfigs, connectorTargetStates(1, connectorNum, TargetState.STARTED), taskConfigs(0, connectorNum, connectorNum * taskNum), Collections.emptyMap(), Collections.emptyMap(), + appliedConnectorConfigs, Collections.emptySet(), Collections.emptySet()); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java index f69f586bc9..f2b73e9699 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java @@ -64,6 +64,7 @@ import org.apache.kafka.connect.source.ExactlyOnceSupport; import org.apache.kafka.connect.source.SourceConnector; import org.apache.kafka.connect.source.SourceTask; +import org.apache.kafka.connect.storage.AppliedConnectorConfig; import org.apache.kafka.connect.storage.ClusterConfigState; import org.apache.kafka.connect.storage.ConfigBackingStore; import org.apache.kafka.connect.storage.StatusBackingStore; @@ -220,6 +221,7 @@ public class DistributedHerderTest { TASK_CONFIGS_MAP, Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONN1, new AppliedConnectorConfig(CONN1_CONFIG)), Collections.emptySet(), Collections.emptySet()); private static final ClusterConfigState SNAPSHOT_PAUSED_CONN1 = new ClusterConfigState( @@ -231,6 +233,7 @@ public class DistributedHerderTest { TASK_CONFIGS_MAP, Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONN1, new AppliedConnectorConfig(CONN1_CONFIG)), Collections.emptySet(), Collections.emptySet()); private static final ClusterConfigState SNAPSHOT_STOPPED_CONN1 = new ClusterConfigState( @@ -242,6 +245,7 @@ public class DistributedHerderTest { Collections.emptyMap(), // Stopped connectors should have an empty set of task configs Collections.singletonMap(CONN1, 3), Collections.singletonMap(CONN1, 10), + Collections.singletonMap(CONN1, new AppliedConnectorConfig(CONN1_CONFIG)), Collections.singleton(CONN1), Collections.emptySet()); @@ -254,6 +258,7 @@ public class DistributedHerderTest { Collections.emptyMap(), Collections.singletonMap(CONN1, 0), Collections.singletonMap(CONN1, 11), + Collections.singletonMap(CONN1, new AppliedConnectorConfig(CONN1_CONFIG)), Collections.emptySet(), Collections.emptySet()); private static final ClusterConfigState SNAPSHOT_UPDATED_CONN1_CONFIG = new ClusterConfigState( @@ -265,6 +270,7 @@ public class DistributedHerderTest { TASK_CONFIGS_MAP, Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONN1, new AppliedConnectorConfig(CONN1_CONFIG_UPDATED)), Collections.emptySet(), Collections.emptySet()); @@ -632,6 +638,7 @@ public void revokeAndReassign(boolean incompleteRebalance) throws TimeoutExcepti TASK_CONFIGS_MAP, Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONN1, new AppliedConnectorConfig(CONN1_CONFIG)), Collections.emptySet(), Collections.emptySet() ); @@ -1616,6 +1623,7 @@ public void testConnectorConfigUpdateFailedTransformation() throws Exception { TASK_CONFIGS_MAP, Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONN1, new AppliedConnectorConfig(CONN1_CONFIG)), Collections.emptySet(), Collections.emptySet(), configTransformer @@ -2220,6 +2228,7 @@ public void testAccessors() throws Exception { TASK_CONFIGS_MAP, Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONN1, new AppliedConnectorConfig(CONN1_CONFIG)), Collections.emptySet(), Collections.emptySet(), configTransformer); @@ -2351,6 +2360,7 @@ public void testPatchConnectorConfigNotFound() { Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap(), + Collections.emptyMap(), Collections.emptySet(), Collections.emptySet()); expectConfigRefreshAndSnapshot(clusterConfigState); @@ -2380,6 +2390,7 @@ public void testPatchConnectorConfigNotALeader() { Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap(), + Collections.emptyMap(), Collections.emptySet(), Collections.emptySet()); expectConfigRefreshAndSnapshot(originalSnapshot); @@ -2420,6 +2431,7 @@ public void testPatchConnectorConfig() throws Exception { Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap(), + Collections.emptyMap(), Collections.emptySet(), Collections.emptySet()); expectConfigRefreshAndSnapshot(originalSnapshot); @@ -2490,6 +2502,7 @@ public void testKeyRotationWhenWorkerBecomesLeader() throws Exception { TASK_CONFIGS_MAP, Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONN1, new AppliedConnectorConfig(CONN1_CONFIG)), Collections.emptySet(), Collections.emptySet()); expectConfigRefreshAndSnapshot(snapshotWithKey); @@ -2536,6 +2549,7 @@ public void testKeyRotationDisabledWhenWorkerBecomesFollower() throws Exception TASK_CONFIGS_MAP, Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONN1, new AppliedConnectorConfig(CONN1_CONFIG)), Collections.emptySet(), Collections.emptySet()); expectConfigRefreshAndSnapshot(snapshotWithKey); @@ -2737,6 +2751,7 @@ public void testFailedToReadBackNewlyWrittenSessionKey() throws Exception { TASK_CONFIGS_MAP, Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONN1, new AppliedConnectorConfig(CONN1_CONFIG)), Collections.emptySet(), Collections.emptySet()); @@ -3221,6 +3236,7 @@ public void testVerifyTaskGeneration() { TASK_CONFIGS_MAP, Collections.emptyMap(), taskConfigGenerations, + Collections.singletonMap(CONN1, new AppliedConnectorConfig(CONN1_CONFIG)), Collections.emptySet(), Collections.emptySet()); @@ -4138,6 +4154,15 @@ private ClusterConfigState exactlyOnceSnapshot( Map> connectorConfigs = connectors.stream() .collect(Collectors.toMap(Function.identity(), c -> CONN1_CONFIG)); + Map appliedConnectorConfigs = taskConfigs.keySet().stream() + .map(ConnectorTaskId::connector) + .distinct() + .filter(connectorConfigs::containsKey) + .collect(Collectors.toMap( + Function.identity(), + connector -> new AppliedConnectorConfig(connectorConfigs.get(connector)) + )); + return new ClusterConfigState( 1, sessionKey, @@ -4147,6 +4172,7 @@ private ClusterConfigState exactlyOnceSnapshot( taskConfigs, taskCountRecords, taskConfigGenerations, + appliedConnectorConfigs, pendingFencing, Collections.emptySet()); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/IncrementalCooperativeAssignorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/IncrementalCooperativeAssignorTest.java index 319bdc9f9f..48b7973fdc 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/IncrementalCooperativeAssignorTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/IncrementalCooperativeAssignorTest.java @@ -23,6 +23,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.runtime.TargetState; import org.apache.kafka.connect.runtime.distributed.WorkerCoordinator.ConnectorsAndTasks; +import org.apache.kafka.connect.storage.AppliedConnectorConfig; import org.apache.kafka.connect.util.ConnectUtils; import org.apache.kafka.connect.storage.ClusterConfigState; import org.apache.kafka.connect.util.ConnectorTaskId; @@ -1396,6 +1397,11 @@ private ClusterConfigState configState() { Function.identity(), connectorTaskId -> Collections.emptyMap() )); + Map appliedConnectorConfigs = connectorConfigs.entrySet().stream() + .collect(Collectors.toMap( + Map.Entry::getKey, + e -> new AppliedConnectorConfig(e.getValue()) + )); return new ClusterConfigState( CONFIG_OFFSET, null, @@ -1405,6 +1411,7 @@ private ClusterConfigState configState() { taskConfigs, Collections.emptyMap(), Collections.emptyMap(), + appliedConnectorConfigs, Collections.emptySet(), Collections.emptySet()); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java index 861e98bfeb..1911eb10b1 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java @@ -37,6 +37,7 @@ import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.runtime.TargetState; +import org.apache.kafka.connect.storage.AppliedConnectorConfig; import org.apache.kafka.connect.storage.ClusterConfigState; import org.apache.kafka.connect.storage.KafkaConfigBackingStore; import org.apache.kafka.connect.util.ConnectorTaskId; @@ -61,6 +62,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.stream.Collectors; import static org.apache.kafka.connect.runtime.distributed.ConnectProtocolCompatibility.COMPATIBLE; import static org.apache.kafka.connect.runtime.distributed.ConnectProtocolCompatibility.EAGER; @@ -171,6 +173,7 @@ public void setup() { Collections.singletonMap(taskId1x0, new HashMap<>()), Collections.emptyMap(), Collections.emptyMap(), + Collections.emptyMap(), Collections.emptySet(), Collections.emptySet() ); @@ -197,6 +200,7 @@ public void setup() { configState2TaskConfigs, Collections.emptyMap(), Collections.emptyMap(), + Collections.emptyMap(), Collections.emptySet(), Collections.emptySet() ); @@ -217,6 +221,11 @@ public void setup() { configStateSingleTaskConnectorsTaskConfigs.put(taskId1x0, new HashMap<>()); configStateSingleTaskConnectorsTaskConfigs.put(taskId2x0, new HashMap<>()); configStateSingleTaskConnectorsTaskConfigs.put(taskId3x0, new HashMap<>()); + Map appliedConnectorConfigs = configStateSingleTaskConnectorsConnectorConfigs.entrySet().stream() + .collect(Collectors.toMap( + Map.Entry::getKey, + e -> new AppliedConnectorConfig(e.getValue()) + )); configStateSingleTaskConnectors = new ClusterConfigState( 12L, null, @@ -226,6 +235,7 @@ public void setup() { configStateSingleTaskConnectorsTaskConfigs, Collections.emptyMap(), Collections.emptyMap(), + appliedConnectorConfigs, Collections.emptySet(), Collections.emptySet() ); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginsTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginsTest.java index 8a637beee5..a3d08ad036 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginsTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginsTest.java @@ -32,6 +32,7 @@ import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.config.provider.ConfigProvider; import org.apache.kafka.common.utils.LogCaptureAppender; +import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.components.Versioned; import org.apache.kafka.connect.connector.policy.AllConnectorClientConfigOverridePolicy; import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy; @@ -596,6 +597,25 @@ public void testServiceLoadWithPlugins() { } } + @Test + public void testAliasesInConverters() throws ClassNotFoundException { + ClassLoader connectorLoader = plugins.connectorLoader(TestPlugin.SAMPLING_CONNECTOR.className()); + try (LoaderSwap loaderSwap = plugins.withClassLoader(connectorLoader)) { + String configKey = "config.key"; + String alias = "SamplingConverter"; + assertTrue(TestPlugin.SAMPLING_CONVERTER.className().contains(alias)); + ConfigDef def = new ConfigDef().define(configKey, ConfigDef.Type.CLASS, ConfigDef.Importance.HIGH, "docstring"); + AbstractConfig config = new AbstractConfig(def, Collections.singletonMap(configKey, alias)); + + assertNotNull(config.getClass(configKey)); + assertNotNull(config.getConfiguredInstance(configKey, Converter.class)); + assertNotNull(plugins.newConverter(config, configKey, ClassLoaderUsage.CURRENT_CLASSLOADER)); + assertNotNull(plugins.newConverter(config, configKey, ClassLoaderUsage.PLUGINS)); + + assertNotNull(Utils.newInstance(alias, Converter.class)); + } + } + private void assertClassLoaderReadsVersionFromResource( TestPlugin parentResource, TestPlugin childResource, String className, String... expectedVersions) { URL[] systemPath = TestPlugins.pluginPath(parentResource) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/SynchronizationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/SynchronizationTest.java index e8ba9153bc..beb1921401 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/SynchronizationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/SynchronizationTest.java @@ -44,7 +44,9 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; +import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.runtime.WorkerConfig; +import org.apache.kafka.connect.storage.Converter; import org.junit.After; import org.junit.Before; import org.junit.Rule; @@ -328,7 +330,7 @@ public void testPluginClassLoaderDoesntHoldMonitorLock() synchronized (externalTestLock) { try { progress.await(null); - Class.forName(TestPlugins.TestPlugin.SAMPLING_CONVERTER.className(), true, connectorLoader); + Utils.loadClass(TestPlugins.TestPlugin.SAMPLING_CONVERTER.className(), Converter.class); } catch (ClassNotFoundException e) { throw new RuntimeException("Failed to load test plugin", e); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java index e8ab2add18..92ab6bd149 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java @@ -42,6 +42,7 @@ import org.apache.kafka.connect.runtime.distributed.SampleConnectorClientConfigOverridePolicy; import org.apache.kafka.connect.runtime.isolation.LoaderSwap; import org.apache.kafka.connect.runtime.rest.entities.Message; +import org.apache.kafka.connect.storage.AppliedConnectorConfig; import org.apache.kafka.connect.storage.ClusterConfigState; import org.apache.kafka.connect.runtime.isolation.PluginClassLoader; import org.apache.kafka.connect.runtime.isolation.Plugins; @@ -387,6 +388,7 @@ public void testRestartTask() throws Exception { Collections.singletonMap(taskId, taskConfig(SourceSink.SOURCE)), Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)), new HashSet<>(), new HashSet<>(), transformer); @@ -420,6 +422,7 @@ public void testRestartTaskFailureOnStart() throws Exception { Collections.singletonMap(new ConnectorTaskId(CONNECTOR_NAME, 0), taskConfig(SourceSink.SOURCE)), Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)), new HashSet<>(), new HashSet<>(), transformer); @@ -555,6 +558,7 @@ public void testRestartConnectorAndTasksOnlyTasks() throws Exception { Collections.singletonMap(taskId, taskConfig(SourceSink.SINK)), Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)), new HashSet<>(), new HashSet<>(), transformer); @@ -609,6 +613,7 @@ public void testRestartConnectorAndTasksBoth() throws Exception { Collections.singletonMap(taskId, taskConfig(SourceSink.SINK)), Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)), new HashSet<>(), new HashSet<>(), transformer); @@ -729,10 +734,14 @@ public void testPutConnectorConfig() throws Exception { return true; }).when(worker).startConnector(eq(CONNECTOR_NAME), capturedConfig.capture(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture()); - // Generate same task config, which should result in no additional action to restart tasks + ConnectorTaskId taskId = new ConnectorTaskId(CONNECTOR_NAME, 0); + // Generate same task config, but from different connector config, resulting + // in task restarts when(worker.connectorTaskConfigs(CONNECTOR_NAME, new SourceConnectorConfig(plugins, newConnConfig, true))) .thenReturn(singletonList(taskConfig(SourceSink.SOURCE))); - + doNothing().when(worker).stopAndAwaitTasks(Collections.singletonList(taskId)); + doNothing().when(statusBackingStore).put(new TaskStatus(taskId, TaskStatus.State.DESTROYED, WORKER_ID, 0)); + when(worker.startSourceTask(eq(taskId), any(), eq(newConnConfig), eq(taskConfig(SourceSink.SOURCE)), eq(herder), eq(TargetState.STARTED))).thenReturn(true); herder.putConnectorConfig(CONNECTOR_NAME, connConfig, false, createCallback); Herder.Created connectorInfo = createCallback.get(WAIT_TIME_MS, TimeUnit.MILLISECONDS); @@ -928,6 +937,8 @@ public void testModifyConnectorOffsetsUnknownConnector() { @Test public void testModifyConnectorOffsetsConnectorNotInStoppedState() { + Map connectorConfig = connectorConfig(SourceSink.SOURCE); + herder.configState = new ClusterConfigState( 10, null, @@ -937,6 +948,7 @@ public void testModifyConnectorOffsetsConnectorNotInStoppedState() { Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)), Collections.emptySet(), Collections.emptySet() ); @@ -963,6 +975,8 @@ public void testAlterConnectorOffsets() throws Exception { return null; }).when(worker).modifyConnectorOffsets(eq(CONNECTOR_NAME), eq(connectorConfig(SourceSink.SOURCE)), any(Map.class), workerCallbackCapture.capture()); + Map connectorConfig = connectorConfig(SourceSink.SOURCE); + herder.configState = new ClusterConfigState( 10, null, @@ -972,6 +986,7 @@ public void testAlterConnectorOffsets() throws Exception { Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)), Collections.emptySet(), Collections.emptySet() ); @@ -992,6 +1007,8 @@ public void testResetConnectorOffsets() throws Exception { return null; }).when(worker).modifyConnectorOffsets(eq(CONNECTOR_NAME), eq(connectorConfig(SourceSink.SOURCE)), isNull(), workerCallbackCapture.capture()); + Map connectorConfig = connectorConfig(SourceSink.SOURCE); + herder.configState = new ClusterConfigState( 10, null, @@ -1001,6 +1018,7 @@ public void testResetConnectorOffsets() throws Exception { Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)), Collections.emptySet(), Collections.emptySet() ); @@ -1071,6 +1089,7 @@ private void expectAdd(SourceSink sourceSink) { // And we should instantiate the tasks. For a sink task, we should see added properties for the input topic partitions + Map connectorConfig = connectorConfig(sourceSink); Map generatedTaskProps = taskConfig(sourceSink); when(worker.connectorTaskConfigs(CONNECTOR_NAME, connConfig)) @@ -1080,11 +1099,12 @@ private void expectAdd(SourceSink sourceSink) { -1, null, Collections.singletonMap(CONNECTOR_NAME, 1), - Collections.singletonMap(CONNECTOR_NAME, connectorConfig(sourceSink)), + Collections.singletonMap(CONNECTOR_NAME, connectorConfig), Collections.singletonMap(CONNECTOR_NAME, TargetState.STARTED), Collections.singletonMap(new ConnectorTaskId(CONNECTOR_NAME, 0), generatedTaskProps), Collections.emptyMap(), Collections.emptyMap(), + Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)), new HashSet<>(), new HashSet<>(), transformer); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreMockitoTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreMockitoTest.java index 6ebac34103..b41bd99b91 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreMockitoTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreMockitoTest.java @@ -170,7 +170,8 @@ public class KafkaConfigBackingStoreMockitoTest { .put("state.v2", "STOPPED"); private static final List CONNECTOR_TASK_COUNT_RECORD_STRUCTS = Arrays.asList( new Struct(KafkaConfigBackingStore.TASK_COUNT_RECORD_V0).put("task-count", 6), - new Struct(KafkaConfigBackingStore.TASK_COUNT_RECORD_V0).put("task-count", 9) + new Struct(KafkaConfigBackingStore.TASK_COUNT_RECORD_V0).put("task-count", 9), + new Struct(KafkaConfigBackingStore.TASK_COUNT_RECORD_V0).put("task-count", 2) ); // The exact format doesn't matter here since both conversions are mocked @@ -813,6 +814,56 @@ public void testRestoreZeroTasks() { verify(configLog).stop(); } + @Test + public void testRestoreCompactedDeletedConnector() { + // When a connector is deleted, we emit a tombstone record for its config (with key + // "connector-") and its target state (with key "target-state-"), but not + // for its task configs + // As a result, we need to carefully handle the case where task configs are present in + // the config topic for a connector, but there is no accompanying config for the + // connector itself + + int offset = 0; + List> existingRecords = Arrays.asList( + new ConsumerRecord<>(TOPIC, 0, offset++, 0L, TimestampType.CREATE_TIME, 0, 0, + TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, offset++, 0L, TimestampType.CREATE_TIME, 0, 0, + TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(1), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, offset++, 0L, TimestampType.CREATE_TIME, 0, 0, + COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(2), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, offset++, 0L, TimestampType.CREATE_TIME, 0, 0, + CONNECTOR_TASK_COUNT_RECORD_KEYS.get(0), CONFIGS_SERIALIZED.get(3), new RecordHeaders(), Optional.empty())); + LinkedHashMap deserialized = new LinkedHashMap<>(); + deserialized.put(CONFIGS_SERIALIZED.get(0), TASK_CONFIG_STRUCTS.get(0)); + deserialized.put(CONFIGS_SERIALIZED.get(1), TASK_CONFIG_STRUCTS.get(0)); + deserialized.put(CONFIGS_SERIALIZED.get(2), TASKS_COMMIT_STRUCT_TWO_TASK_CONNECTOR); + deserialized.put(CONFIGS_SERIALIZED.get(3), CONNECTOR_TASK_COUNT_RECORD_STRUCTS.get(2)); + logOffset = offset; + expectStart(existingRecords, deserialized); + when(configLog.partitionCount()).thenReturn(1); + + configStorage.setupAndCreateKafkaBasedLog(TOPIC, config); + verifyConfigure(); + configStorage.start(); + + // Should see no connectors and no task configs + ClusterConfigState configState = configStorage.snapshot(); + assertEquals(Collections.emptySet(), configState.connectors()); + assertEquals(0, configState.taskCount(CONNECTOR_1_NAME)); + assertNull(configState.rawTaskConfig(TASK_IDS.get(0))); + assertNull(configState.rawTaskConfig(TASK_IDS.get(1))); + + // Probe internal collections just to be sure + assertEquals(Collections.emptyMap(), configState.connectorConfigs); + assertEquals(Collections.emptyMap(), configState.taskConfigs); + assertEquals(Collections.emptyMap(), configState.connectorTaskCounts); + + // Exception: we still include task count records, for the unlikely-but-possible case + // where there are still zombie instances of the tasks for this long-deleted connector + // running somewhere on the cluster + assertEquals(2, (int) configState.taskCountRecord(CONNECTOR_1_NAME)); + } + @Test public void testRecordToRestartRequest() { ConsumerRecord record = new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, RESTART_CONNECTOR_KEYS.get(0), diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnect.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnect.java index a37de76d1a..e7e268425c 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnect.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnect.java @@ -118,7 +118,8 @@ protected EmbeddedConnect( }; /** - * Start the connect cluster and the embedded Kafka and Zookeeper cluster. + * Start the Connect cluster and the embedded Kafka and Zookeeper cluster, + * and wait for the Kafka and Connect clusters to become healthy. */ public void start() { if (maskExitProcedures) { @@ -132,6 +133,27 @@ public void start() { } catch (Exception e) { throw new ConnectException("Failed to start HTTP client", e); } + + try { + if (numBrokers > 0) { + assertions().assertExactlyNumBrokersAreUp( + numBrokers, + "Kafka cluster did not start in time" + ); + log.info("Completed startup of {} Kafka brokers", numBrokers); + } + + int numWorkers = workers().size(); + if (numWorkers > 0) { + assertions().assertExactlyNumWorkersAreUp( + numWorkers, + "Connect cluster did not start in time" + ); + log.info("Completed startup of {} Connect workers", numWorkers); + } + } catch (InterruptedException e) { + throw new RuntimeException("Interrupted while awaiting cluster startup", e); + } } /** diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java index f959d22537..fe97476a71 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java @@ -50,7 +50,6 @@ import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.errors.ConnectException; @@ -112,7 +111,7 @@ public class EmbeddedKafkaCluster { // Kafka Config private final KafkaServer[] brokers; private final Properties brokerConfig; - private final Time time = new MockTime(); + private final Time time = Time.SYSTEM; private final int[] currentBrokerPorts; private final String[] currentBrokerLogDirs; private final boolean hasListenerConfig; @@ -611,6 +610,19 @@ public ConsumerRecords consumeAll( return new ConsumerRecords<>(records); } + public long endOffset(TopicPartition topicPartition) throws TimeoutException, InterruptedException, ExecutionException { + try (Admin admin = createAdminClient()) { + Map offsets = Collections.singletonMap( + topicPartition, OffsetSpec.latest() + ); + return admin.listOffsets(offsets) + .partitionResult(topicPartition) + // Hardcode duration for now; if necessary, we can add a parameter for it later + .get(10, TimeUnit.SECONDS) + .offset(); + } + } + /** * List all the known partitions for the given {@link Collection} of topics * @param maxDurationMs the max duration to wait for while fetching metadata from Kafka (in milliseconds). diff --git a/connect/runtime/src/test/resources/META-INF/services/org.apache.kafka.connect.sink.SinkConnector b/connect/runtime/src/test/resources/META-INF/services/org.apache.kafka.connect.sink.SinkConnector index 4c26fece18..56e054ddbe 100644 --- a/connect/runtime/src/test/resources/META-INF/services/org.apache.kafka.connect.sink.SinkConnector +++ b/connect/runtime/src/test/resources/META-INF/services/org.apache.kafka.connect.sink.SinkConnector @@ -17,4 +17,5 @@ org.apache.kafka.connect.integration.BlockingConnectorTest$BlockingSinkConnector org.apache.kafka.connect.integration.BlockingConnectorTest$TaskInitializeBlockingSinkConnector org.apache.kafka.connect.integration.ErrantRecordSinkConnector org.apache.kafka.connect.integration.MonitorableSinkConnector -org.apache.kafka.connect.runtime.SampleSinkConnector \ No newline at end of file +org.apache.kafka.connect.runtime.SampleSinkConnector +org.apache.kafka.connect.integration.ConnectWorkerIntegrationTest$EmptyTaskConfigsConnector \ No newline at end of file diff --git a/core/src/main/java/kafka/automq/StorageUtil.java b/core/src/main/java/kafka/automq/StorageUtil.java index fbd8cd382a..71a055df8d 100644 --- a/core/src/main/java/kafka/automq/StorageUtil.java +++ b/core/src/main/java/kafka/automq/StorageUtil.java @@ -34,7 +34,7 @@ public static void formatStorage(Properties serverProps) { String metadataLog = new KafkaConfig(serverProps, false).metadataLogDir(); if (!Files.exists(Paths.get(metadataLog, "meta.properties"))) { String configFilePath = persistConfig(serverProps, metadataLog); - StorageTool.main(new String[] {"auto-format", "-t", clusterId, "-c=" + configFilePath}); + StorageTool.execute(new String[] {"format", "-t", clusterId, "-c=" + configFilePath}); } else { persistConfig(serverProps, metadataLog); } diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index e5bf9597ca..2e897e2836 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -17,12 +17,14 @@ package kafka.log.remote; import com.yammer.metrics.core.Gauge; +import com.yammer.metrics.core.Timer; import kafka.cluster.EndPoint; import kafka.cluster.Partition; import kafka.log.UnifiedLog; import kafka.log.remote.quota.RLMQuotaManager; import kafka.log.remote.quota.RLMQuotaManagerConfig; import kafka.server.BrokerTopicStats; +import kafka.server.KafkaConfig; import kafka.server.QuotaType; import kafka.server.StopPartition; import org.apache.kafka.common.KafkaException; @@ -33,6 +35,7 @@ import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.message.FetchResponseData; import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Quota; import org.apache.kafka.common.record.FileRecords; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Record; @@ -46,6 +49,7 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.server.common.CheckpointFile; import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.server.config.ServerConfigs; import org.apache.kafka.server.log.remote.metadata.storage.ClassLoaderAwareRemoteLogMetadataManager; @@ -61,7 +65,7 @@ import org.apache.kafka.server.log.remote.storage.RemoteStorageException; import org.apache.kafka.server.log.remote.storage.RemoteStorageManager; import org.apache.kafka.server.metrics.KafkaMetricsGroup; -import org.apache.kafka.storage.internals.checkpoint.InMemoryLeaderEpochCheckpoint; +import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile; import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache; import org.apache.kafka.storage.internals.log.AbortedTxn; import org.apache.kafka.storage.internals.log.EpochEntry; @@ -83,14 +87,19 @@ import scala.Option; import scala.collection.JavaConverters; +import java.io.BufferedWriter; +import java.io.ByteArrayOutputStream; import java.io.Closeable; import java.io.File; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStreamWriter; import java.lang.reflect.InvocationTargetException; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.nio.file.Path; import java.security.PrivilegedAction; +import java.time.Duration; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -117,6 +126,8 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.Function; @@ -126,6 +137,7 @@ import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX; import static org.apache.kafka.server.log.remote.storage.RemoteStorageMetrics.REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT_METRIC; +import static org.apache.kafka.server.log.remote.storage.RemoteStorageMetrics.REMOTE_LOG_READER_FETCH_RATE_AND_TIME_METRIC; import static org.apache.kafka.server.config.ServerLogConfigs.LOG_DIR_CONFIG; /** @@ -140,7 +152,7 @@ public class RemoteLogManager implements Closeable { private static final Logger LOGGER = LoggerFactory.getLogger(RemoteLogManager.class); private static final String REMOTE_LOG_READER_THREAD_NAME_PREFIX = "remote-log-reader"; - private final RemoteLogManagerConfig rlmConfig; + private final KafkaConfig config; private final int brokerId; private final String logDir; private final Time time; @@ -153,6 +165,8 @@ public class RemoteLogManager implements Closeable { private final RemoteLogMetadataManager remoteLogMetadataManager; + private final ReentrantLock copyQuotaManagerLock = new ReentrantLock(true); + private final Condition copyQuotaManagerLockCondition = copyQuotaManagerLock.newCondition(); private final RLMQuotaManager rlmCopyQuotaManager; private final RLMQuotaManager rlmFetchQuotaManager; @@ -174,11 +188,12 @@ public class RemoteLogManager implements Closeable { private boolean closed = false; private volatile boolean remoteLogManagerConfigured = false; + private final Timer remoteReadTimer; /** * Creates RemoteLogManager instance with the given arguments. * - * @param rlmConfig Configuration required for remote logging subsystem(tiered storage) at the broker level. + * @param config Configuration required for remote logging subsystem(tiered storage) at the broker level. * @param brokerId id of the current broker. * @param logDir directory of Kafka log segments. * @param time Time instance. @@ -188,7 +203,7 @@ public class RemoteLogManager implements Closeable { * @param brokerTopicStats BrokerTopicStats instance to update the respective metrics. * @param metrics Metrics instance */ - public RemoteLogManager(RemoteLogManagerConfig rlmConfig, + public RemoteLogManager(KafkaConfig config, int brokerId, String logDir, String clusterId, @@ -197,7 +212,7 @@ public RemoteLogManager(RemoteLogManagerConfig rlmConfig, BiConsumer updateRemoteLogStartOffset, BrokerTopicStats brokerTopicStats, Metrics metrics) throws IOException { - this.rlmConfig = rlmConfig; + this.config = config; this.brokerId = brokerId; this.logDir = logDir; this.clusterId = clusterId; @@ -212,16 +227,19 @@ public RemoteLogManager(RemoteLogManagerConfig rlmConfig, rlmCopyQuotaManager = createRLMCopyQuotaManager(); rlmFetchQuotaManager = createRLMFetchQuotaManager(); - indexCache = new RemoteIndexCache(rlmConfig.remoteLogIndexFileCacheTotalSizeBytes(), remoteLogStorageManager, logDir); + RemoteLogManagerConfig rlmConfig = config.remoteLogManagerConfig(); + indexCache = new RemoteIndexCache(config.remoteLogIndexFileCacheTotalSizeBytes(), remoteLogStorageManager, logDir); delayInMs = rlmConfig.remoteLogManagerTaskIntervalMs(); rlmScheduledThreadPool = new RLMScheduledThreadPool(rlmConfig.remoteLogManagerThreadPoolSize()); - metricsGroup.newGauge(REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT_METRIC.getName(), new Gauge() { + metricsGroup.newGauge(REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT_METRIC, new Gauge() { @Override public Double value() { return rlmScheduledThreadPool.getIdlePercent(); } }); + remoteReadTimer = metricsGroup.newTimer(REMOTE_LOG_READER_FETCH_RATE_AND_TIME_METRIC, + TimeUnit.MILLISECONDS, TimeUnit.SECONDS); remoteStorageReaderThreadPool = new RemoteStorageThreadPool( REMOTE_LOG_READER_THREAD_NAME_PREFIX, @@ -234,29 +252,53 @@ public void resizeCacheSize(long remoteLogIndexFileCacheSize) { indexCache.resizeCacheSize(remoteLogIndexFileCacheSize); } + public void updateCopyQuota(long quota) { + LOGGER.info("Updating remote copy quota to {} bytes per second", quota); + rlmCopyQuotaManager.updateQuota(new Quota(quota, true)); + } + + public void updateFetchQuota(long quota) { + LOGGER.info("Updating remote fetch quota to {} bytes per second", quota); + rlmFetchQuotaManager.updateQuota(new Quota(quota, true)); + } + private void removeMetrics() { - metricsGroup.removeMetric(REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT_METRIC.getName()); + metricsGroup.removeMetric(REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT_METRIC); + metricsGroup.removeMetric(REMOTE_LOG_READER_FETCH_RATE_AND_TIME_METRIC); remoteStorageReaderThreadPool.removeMetrics(); } + /** + * Returns the timeout for the RLM Tasks to wait for the quota to be available + */ + Duration quotaTimeout() { + return Duration.ofSeconds(1); + } + RLMQuotaManager createRLMCopyQuotaManager() { - return new RLMQuotaManager(copyQuotaManagerConfig(rlmConfig), metrics, QuotaType.RLMCopy$.MODULE$, + return new RLMQuotaManager(copyQuotaManagerConfig(config), metrics, QuotaType.RLMCopy$.MODULE$, "Tracking copy byte-rate for Remote Log Manager", time); } RLMQuotaManager createRLMFetchQuotaManager() { - return new RLMQuotaManager(fetchQuotaManagerConfig(rlmConfig), metrics, QuotaType.RLMFetch$.MODULE$, + return new RLMQuotaManager(fetchQuotaManagerConfig(config), metrics, QuotaType.RLMFetch$.MODULE$, "Tracking fetch byte-rate for Remote Log Manager", time); } - static RLMQuotaManagerConfig copyQuotaManagerConfig(RemoteLogManagerConfig rlmConfig) { - return new RLMQuotaManagerConfig(rlmConfig.remoteLogManagerCopyMaxBytesPerSecond(), + public boolean isRemoteLogFetchQuotaExceeded() { + return rlmFetchQuotaManager.isQuotaExceeded(); + } + + static RLMQuotaManagerConfig copyQuotaManagerConfig(KafkaConfig config) { + RemoteLogManagerConfig rlmConfig = config.remoteLogManagerConfig(); + return new RLMQuotaManagerConfig(config.remoteLogManagerCopyMaxBytesPerSecond(), rlmConfig.remoteLogManagerCopyNumQuotaSamples(), rlmConfig.remoteLogManagerCopyQuotaWindowSizeSeconds()); } - static RLMQuotaManagerConfig fetchQuotaManagerConfig(RemoteLogManagerConfig rlmConfig) { - return new RLMQuotaManagerConfig(rlmConfig.remoteLogManagerFetchMaxBytesPerSecond(), + static RLMQuotaManagerConfig fetchQuotaManagerConfig(KafkaConfig config) { + RemoteLogManagerConfig rlmConfig = config.remoteLogManagerConfig(); + return new RLMQuotaManagerConfig(config.remoteLogManagerFetchMaxBytesPerSecond(), rlmConfig.remoteLogManagerFetchNumQuotaSamples(), rlmConfig.remoteLogManagerFetchQuotaWindowSizeSeconds()); } @@ -273,6 +315,7 @@ private T createDelegate(ClassLoader classLoader, String className) { @SuppressWarnings("removal") RemoteStorageManager createRemoteStorageManager() { + RemoteLogManagerConfig rlmConfig = config.remoteLogManagerConfig(); return java.security.AccessController.doPrivileged(new PrivilegedAction() { private final String classPath = rlmConfig.remoteStorageManagerClassPath(); @@ -289,13 +332,14 @@ public RemoteStorageManager run() { } private void configureRSM() { - final Map rsmProps = new HashMap<>(rlmConfig.remoteStorageManagerProps()); + final Map rsmProps = new HashMap<>(config.remoteLogManagerConfig().remoteStorageManagerProps()); rsmProps.put(ServerConfigs.BROKER_ID_CONFIG, brokerId); remoteLogStorageManager.configure(rsmProps); } @SuppressWarnings("removal") RemoteLogMetadataManager createRemoteLogMetadataManager() { + RemoteLogManagerConfig rlmConfig = config.remoteLogManagerConfig(); return java.security.AccessController.doPrivileged(new PrivilegedAction() { private final String classPath = rlmConfig.remoteLogMetadataManagerClassPath(); @@ -322,7 +366,7 @@ private void configureRLMM() { rlmmProps.put(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "security.protocol", e.securityProtocol().name); }); // update the remoteLogMetadataProps here to override endpoint config if any - rlmmProps.putAll(rlmConfig.remoteLogMetadataManagerProps()); + rlmmProps.putAll(config.remoteLogManagerConfig().remoteLogMetadataManagerProps()); rlmmProps.put(ServerConfigs.BROKER_ID_CONFIG, brokerId); rlmmProps.put(LOG_DIR_CONFIG, logDir); @@ -374,7 +418,7 @@ public void onLeadershipChange(Set partitionsBecomeLeader, Map topicIds) { LOGGER.debug("Received leadership changes for leaders: {} and followers: {}", partitionsBecomeLeader, partitionsBecomeFollower); - if (this.rlmConfig.enableRemoteStorageSystem() && !isRemoteLogManagerConfigured()) { + if (config.remoteLogManagerConfig().isRemoteStorageSystemEnabled() && !isRemoteLogManagerConfigured()) { throw new KafkaException("RemoteLogManager is not configured when remote storage system is enabled"); } @@ -393,12 +437,15 @@ public void onLeadershipChange(Set partitionsBecomeLeader, leaderPartitions.forEach(this::cacheTopicPartitionIds); followerPartitions.forEach(this::cacheTopicPartitionIds); - followerPartitions.forEach(this::removeRemoteTopicPartitionMetrics); remoteLogMetadataManager.onPartitionLeadershipChanges(leaderPartitions, followerPartitions); followerPartitions.forEach(topicIdPartition -> doHandleLeaderOrFollowerPartitions(topicIdPartition, RLMTask::convertToFollower)); + // If this node was the previous leader for the partition, then the RLMTask might be running in the + // background thread and might emit metrics. So, removing the metrics after marking this node as follower. + followerPartitions.forEach(this::removeRemoteTopicPartitionMetrics); + leaderPartitionsWithLeaderEpoch.forEach((topicIdPartition, leaderEpoch) -> doHandleLeaderOrFollowerPartitions(topicIdPartition, rlmTask -> rlmTask.convertToLeader(leaderEpoch))); @@ -608,25 +655,32 @@ public boolean isCancelled() { } /** - * Returns the leader epoch checkpoint by truncating with the given start[exclusive] and end[inclusive] offset + * Returns the leader epoch entries within the range of the given start[exclusive] and end[inclusive] offset. + *

+ * Visible for testing. * * @param log The actual log from where to take the leader-epoch checkpoint - * @param startOffset The start offset of the checkpoint file (exclusive in the truncation). + * @param startOffset The start offset of the epoch entries (inclusive). * If start offset is 6, then it will retain an entry at offset 6. - * @param endOffset The end offset of the checkpoint file (inclusive in the truncation) + * @param endOffset The end offset of the epoch entries (exclusive) * If end offset is 100, then it will remove the entries greater than or equal to 100. - * @return the truncated leader epoch checkpoint + * @return the leader epoch entries */ - InMemoryLeaderEpochCheckpoint getLeaderEpochCheckpoint(UnifiedLog log, long startOffset, long endOffset) { - InMemoryLeaderEpochCheckpoint checkpoint = new InMemoryLeaderEpochCheckpoint(); + List getLeaderEpochEntries(UnifiedLog log, long startOffset, long endOffset) { if (log.leaderEpochCache().isDefined()) { - LeaderEpochFileCache cache = log.leaderEpochCache().get().writeTo(checkpoint); - if (startOffset >= 0) { - cache.truncateFromStart(startOffset); - } - cache.truncateFromEnd(endOffset); + return log.leaderEpochCache().get().epochEntriesInRange(startOffset, endOffset); + } else { + return Collections.emptyList(); + } + } + + // VisibleForTesting + RLMTask rlmTask(TopicIdPartition topicIdPartition) { + RLMTaskWithFuture task = leaderOrFollowerTasks.get(topicIdPartition); + if (task != null) { + return task.rlmTask; } - return checkpoint; + return null; } class RLMTask extends CancellableRunnable { @@ -750,6 +804,23 @@ public void copyLogSegmentsToRemote(UnifiedLog log) throws InterruptedException isCancelled(), isLeader()); return; } + + copyQuotaManagerLock.lock(); + try { + while (rlmCopyQuotaManager.isQuotaExceeded()) { + logger.debug("Quota exceeded for copying log segments, waiting for the quota to be available."); + // If the thread gets interrupted while waiting, the InterruptedException is thrown + // back to the caller. It's important to note that the task being executed is already + // cancelled before the executing thread is interrupted. The caller is responsible + // for handling the exception gracefully by checking if the task is already cancelled. + boolean ignored = copyQuotaManagerLockCondition.await(quotaTimeout().toMillis(), TimeUnit.MILLISECONDS); + } + rlmCopyQuotaManager.record(candidateLogSegment.logSegment.log().sizeInBytes()); + // Signal waiting threads to check the quota again + copyQuotaManagerLockCondition.signalAll(); + } finally { + copyQuotaManagerLock.unlock(); + } copyLogSegment(log, candidateLogSegment.logSegment, candidateLogSegment.nextSegmentOffset); } } @@ -784,7 +855,7 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, long nextSegment long endOffset = nextSegmentBaseOffset - 1; File producerStateSnapshotFile = log.producerStateManager().fetchSnapshot(nextSegmentBaseOffset).orElse(null); - List epochEntries = getLeaderEpochCheckpoint(log, segment.baseOffset(), nextSegmentBaseOffset).read(); + List epochEntries = getLeaderEpochEntries(log, segment.baseOffset(), nextSegmentBaseOffset); Map segmentLeaderEpochs = new HashMap<>(epochEntries.size()); epochEntries.forEach(entry -> segmentLeaderEpochs.put(entry.epoch, entry.startOffset)); @@ -794,7 +865,7 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, long nextSegment remoteLogMetadataManager.addRemoteLogSegmentMetadata(copySegmentStartedRlsm).get(); - ByteBuffer leaderEpochsIndex = getLeaderEpochCheckpoint(log, -1, nextSegmentBaseOffset).readAsByteBuffer(); + ByteBuffer leaderEpochsIndex = epochEntriesAsByteBuffer(getLeaderEpochEntries(log, -1, nextSegmentBaseOffset)); LogSegmentData segmentData = new LogSegmentData(logFile.toPath(), toPathIfExists(segment.offsetIndex().file()), toPathIfExists(segment.timeIndex().file()), Optional.ofNullable(toPathIfExists(segment.txnIndex().file())), producerStateSnapshotFile.toPath(), leaderEpochsIndex); @@ -840,11 +911,18 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, long nextSegment logger.info("Copied {} to remote storage with segment-id: {}", logFileName, copySegmentFinishedRlsm.remoteLogSegmentId()); long bytesLag = log.onlyLocalLogSegmentsSize() - log.activeSegment().size(); - String topic = topicIdPartition.topic(); - int partition = topicIdPartition.partition(); long segmentsLag = log.onlyLocalLogSegmentsCount(); - brokerTopicStats.recordRemoteCopyLagBytes(topic, partition, bytesLag); - brokerTopicStats.recordRemoteCopyLagSegments(topic, partition, segmentsLag); + recordLagStats(bytesLag, segmentsLag); + } + + // VisibleForTesting + void recordLagStats(long bytesLag, long segmentsLag) { + if (isLeader()) { + String topic = topicIdPartition.topic(); + int partition = topicIdPartition.partition(); + brokerTopicStats.recordRemoteCopyLagBytes(topic, partition, bytesLag); + brokerTopicStats.recordRemoteCopyLagSegments(topic, partition, segmentsLag); + } } private Path toPathIfExists(File file) { @@ -930,7 +1008,9 @@ private boolean isSegmentBreachedByRetentionSize(RemoteLogSegmentMetadata metada } } if (shouldDeleteSegment) { - logStartOffset = OptionalLong.of(metadata.endOffset() + 1); + if (!logStartOffset.isPresent() || logStartOffset.getAsLong() < metadata.endOffset() + 1) { + logStartOffset = OptionalLong.of(metadata.endOffset() + 1); + } logger.info("About to delete remote log segment {} due to retention size {} breach. Log size after deletion will be {}.", metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize); } @@ -947,7 +1027,9 @@ public boolean isSegmentBreachedByRetentionTime(RemoteLogSegmentMetadata metadat remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes()); // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals // are ascending with in an epoch. - logStartOffset = OptionalLong.of(metadata.endOffset() + 1); + if (!logStartOffset.isPresent() || logStartOffset.getAsLong() < metadata.endOffset() + 1) { + logStartOffset = OptionalLong.of(metadata.endOffset() + 1); + } logger.info("About to delete remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs); } @@ -1660,14 +1742,16 @@ long findLogStartOffset(TopicIdPartition topicIdPartition, UnifiedLog log) throw * @throws java.util.concurrent.RejectedExecutionException if the task cannot be accepted for execution (task queue is full) */ public Future asyncRead(RemoteStorageFetchInfo fetchInfo, Consumer callback) { - return remoteStorageReaderThreadPool.submit(new RemoteLogReader(fetchInfo, this, callback, brokerTopicStats)); + return remoteStorageReaderThreadPool.submit( + new RemoteLogReader(fetchInfo, this, callback, brokerTopicStats, rlmFetchQuotaManager, remoteReadTimer)); } void doHandleLeaderOrFollowerPartitions(TopicIdPartition topicPartition, Consumer convertToLeaderOrFollower) { + RemoteLogManagerConfig rlmConfig = config.remoteLogManagerConfig(); RLMTaskWithFuture rlmTaskWithFuture = leaderOrFollowerTasks.computeIfAbsent(topicPartition, topicIdPartition -> { - RLMTask task = new RLMTask(topicIdPartition, this.rlmConfig.remoteLogMetadataCustomMetadataMaxBytes()); + RLMTask task = new RLMTask(topicIdPartition, rlmConfig.remoteLogMetadataCustomMetadataMaxBytes()); // set this upfront when it is getting initialized instead of doing it after scheduling. convertToLeaderOrFollower.accept(task); LOGGER.info("Created a new task: {} and getting scheduled", task); @@ -1747,6 +1831,19 @@ private static void shutdownAndAwaitTermination(ExecutorService pool, String poo LOGGER.info("Shutting down of thread pool {} is completed", poolName); } + //Visible for testing + static ByteBuffer epochEntriesAsByteBuffer(List epochEntries) throws IOException { + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + try (BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(stream, StandardCharsets.UTF_8))) { + CheckpointFile.CheckpointWriteBuffer writeBuffer = + new CheckpointFile.CheckpointWriteBuffer<>(writer, 0, LeaderEpochCheckpointFile.FORMATTER); + writeBuffer.write(epochEntries); + writer.flush(); + } + + return ByteBuffer.wrap(stream.toByteArray()); + } + private void removeRemoteTopicPartitionMetrics(TopicIdPartition topicIdPartition) { String topic = topicIdPartition.topic(); if (!brokerTopicStats.isTopicStatsExisted(topicIdPartition.topic())) { diff --git a/core/src/main/java/kafka/log/remote/RemoteLogReader.java b/core/src/main/java/kafka/log/remote/RemoteLogReader.java index 5d24b2bbbd..c28677459e 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogReader.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogReader.java @@ -16,6 +16,8 @@ */ package kafka.log.remote; +import com.yammer.metrics.core.Timer; +import kafka.log.remote.quota.RLMQuotaManager; import kafka.server.BrokerTopicStats; import org.apache.kafka.common.errors.OffsetOutOfRangeException; import org.apache.kafka.common.utils.LogContext; @@ -34,17 +36,23 @@ public class RemoteLogReader implements Callable { private final RemoteLogManager rlm; private final BrokerTopicStats brokerTopicStats; private final Consumer callback; + private final RLMQuotaManager quotaManager; + private final Timer remoteReadTimer; public RemoteLogReader(RemoteStorageFetchInfo fetchInfo, RemoteLogManager rlm, Consumer callback, - BrokerTopicStats brokerTopicStats) { + BrokerTopicStats brokerTopicStats, + RLMQuotaManager quotaManager, + Timer remoteReadTimer) { this.fetchInfo = fetchInfo; this.rlm = rlm; this.brokerTopicStats = brokerTopicStats; this.callback = callback; this.brokerTopicStats.topicStats(fetchInfo.topicPartition.topic()).remoteFetchRequestRate().mark(); this.brokerTopicStats.allTopicsStats().remoteFetchRequestRate().mark(); + this.quotaManager = quotaManager; + this.remoteReadTimer = remoteReadTimer; logger = new LogContext() { @Override public String logPrefix() { @@ -58,8 +66,7 @@ public Void call() { RemoteLogReadResult result; try { logger.debug("Reading records from remote storage for topic partition {}", fetchInfo.topicPartition); - - FetchDataInfo fetchDataInfo = rlm.read(fetchInfo); + FetchDataInfo fetchDataInfo = remoteReadTimer.time(() -> rlm.read(fetchInfo)); brokerTopicStats.topicStats(fetchInfo.topicPartition.topic()).remoteFetchBytesRate().mark(fetchDataInfo.records.sizeInBytes()); brokerTopicStats.allTopicsStats().remoteFetchBytesRate().mark(fetchDataInfo.records.sizeInBytes()); result = new RemoteLogReadResult(Optional.of(fetchDataInfo), Optional.empty()); @@ -71,10 +78,9 @@ public Void call() { logger.error("Error occurred while reading the remote data for {}", fetchInfo.topicPartition, e); result = new RemoteLogReadResult(Optional.empty(), Optional.of(e)); } - logger.debug("Finished reading records from remote storage for topic partition {}", fetchInfo.topicPartition); + quotaManager.record(result.fetchDataInfo.map(fetchDataInfo -> fetchDataInfo.records.sizeInBytes()).orElse(0)); callback.accept(result); - return null; } } diff --git a/core/src/main/java/kafka/server/MetadataVersionConfigValidator.java b/core/src/main/java/kafka/server/MetadataVersionConfigValidator.java new file mode 100644 index 0000000000..a62987be20 --- /dev/null +++ b/core/src/main/java/kafka/server/MetadataVersionConfigValidator.java @@ -0,0 +1,69 @@ +/* + * 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. + */ + +package kafka.server; + +import org.apache.kafka.image.MetadataDelta; +import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.image.loader.LoaderManifest; +import org.apache.kafka.image.publisher.MetadataPublisher; +import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.fault.FaultHandler; + +public class MetadataVersionConfigValidator implements MetadataPublisher { + private final String name; + private final KafkaConfig config; + private final FaultHandler faultHandler; + + public MetadataVersionConfigValidator( + KafkaConfig config, + FaultHandler faultHandler + ) { + int id = config.brokerId(); + this.name = "MetadataVersionPublisher(id=" + id + ")"; + this.config = config; + this.faultHandler = faultHandler; + } + + @Override + public String name() { + return name; + } + + @Override + public void onMetadataUpdate( + MetadataDelta delta, + MetadataImage newImage, + LoaderManifest manifest + ) { + if (delta.featuresDelta() != null) { + if (delta.metadataVersionChanged().isPresent()) { + onMetadataVersionChanged(newImage.features().metadataVersion()); + } + } + } + + @SuppressWarnings("ThrowableNotThrown") + private void onMetadataVersionChanged(MetadataVersion metadataVersion) { + try { + this.config.validateWithMetadataVersion(metadataVersion); + } catch (Throwable t) { + this.faultHandler.handleFault( + "Broker configuration does not support the cluster MetadataVersion", t); + } + } +} diff --git a/core/src/main/java/kafka/server/NetworkUtils.java b/core/src/main/java/kafka/server/NetworkUtils.java index 5607f2623f..83093c19e1 100644 --- a/core/src/main/java/kafka/server/NetworkUtils.java +++ b/core/src/main/java/kafka/server/NetworkUtils.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.ApiVersions; import org.apache.kafka.clients.ManualMetadataUpdater; +import org.apache.kafka.clients.MetadataRecoveryStrategy; import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.common.Reconfigurable; import org.apache.kafka.common.metrics.Metrics; @@ -84,7 +85,8 @@ public static NetworkClient buildNetworkClient(String prefix, time, true, new ApiVersions(), - logContext + logContext, + MetadataRecoveryStrategy.NONE ); } } \ No newline at end of file diff --git a/core/src/main/java/kafka/server/builders/KafkaApisBuilder.java b/core/src/main/java/kafka/server/builders/KafkaApisBuilder.java index 6de5496339..f512ecb0d5 100644 --- a/core/src/main/java/kafka/server/builders/KafkaApisBuilder.java +++ b/core/src/main/java/kafka/server/builders/KafkaApisBuilder.java @@ -182,7 +182,7 @@ public KafkaApis build() { if (metrics == null) throw new RuntimeException("You must set metrics"); if (quotas == null) throw new RuntimeException("You must set quotas"); if (fetchManager == null) throw new RuntimeException("You must set fetchManager"); - if (brokerTopicStats == null) brokerTopicStats = new BrokerTopicStats(config.isRemoteLogStorageSystemEnabled()); + if (brokerTopicStats == null) brokerTopicStats = new BrokerTopicStats(config.remoteLogManagerConfig().isRemoteStorageSystemEnabled()); if (apiVersionManager == null) throw new RuntimeException("You must set apiVersionManager"); return new ElasticKafkaApis(requestChannel, diff --git a/core/src/main/java/kafka/server/builders/ReplicaManagerBuilder.java b/core/src/main/java/kafka/server/builders/ReplicaManagerBuilder.java index 82aa75909a..7cac33200d 100644 --- a/core/src/main/java/kafka/server/builders/ReplicaManagerBuilder.java +++ b/core/src/main/java/kafka/server/builders/ReplicaManagerBuilder.java @@ -185,7 +185,7 @@ public ReplicaManager build() { if (metadataCache == null) throw new RuntimeException("You must set metadataCache"); if (logDirFailureChannel == null) throw new RuntimeException("You must set logDirFailureChannel"); if (alterPartitionManager == null) throw new RuntimeException("You must set alterIsrManager"); - if (brokerTopicStats == null) brokerTopicStats = new BrokerTopicStats(config.isRemoteLogStorageSystemEnabled()); + if (brokerTopicStats == null) brokerTopicStats = new BrokerTopicStats(config.remoteLogManagerConfig().isRemoteStorageSystemEnabled()); // Initialize metrics in the end just before passing it to ReplicaManager to ensure ReplicaManager closes the // metrics correctly. There might be a resource leak if it is initialized and an exception occurs between // its initialization and creation of ReplicaManager. diff --git a/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala b/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala index de958b00ba..b14e86c6ce 100644 --- a/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala +++ b/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala @@ -26,7 +26,7 @@ import joptsimple.OptionSpec import kafka.utils.Implicits._ import kafka.utils.Logging import org.apache.kafka.common.utils.Utils -import org.apache.kafka.clients.{ApiVersions, ClientDnsLookup, ClientResponse, ClientUtils, CommonClientConfigs, Metadata, NetworkClient, NodeApiVersions} +import org.apache.kafka.clients.{ApiVersions, ClientDnsLookup, ClientResponse, ClientUtils, CommonClientConfigs, Metadata, MetadataRecoveryStrategy, NetworkClient, NodeApiVersions} import org.apache.kafka.clients.consumer.internals.{ConsumerNetworkClient, RequestFuture} import org.apache.kafka.common.config.ConfigDef.ValidString._ import org.apache.kafka.common.config.ConfigDef.{Importance, Type} @@ -310,7 +310,8 @@ object BrokerApiVersionsCommand { time, true, new ApiVersions, - logContext) + logContext, + MetadataRecoveryStrategy.NONE) val highLevelClient = new ConsumerNetworkClient( logContext, diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index 67d3f3963b..793b39538e 100755 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -164,7 +164,8 @@ class ControllerChannelManager(controllerEpoch: () => Int, time, false, new ApiVersions, - logContext + logContext, + MetadataRecoveryStrategy.NONE ) (networkClient, reconfigurableChannelBuilder) } diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala index 068dff4cca..44176d2276 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala @@ -96,7 +96,8 @@ object TransactionMarkerChannelManager { time, false, new ApiVersions, - logContext + logContext, + MetadataRecoveryStrategy.NONE ) new TransactionMarkerChannelManager(config, diff --git a/core/src/main/scala/kafka/log/LogLoader.scala b/core/src/main/scala/kafka/log/LogLoader.scala index b0f1fdd0e1..b3b0ec2c63 100644 --- a/core/src/main/scala/kafka/log/LogLoader.scala +++ b/core/src/main/scala/kafka/log/LogLoader.scala @@ -173,14 +173,14 @@ class LogLoader( } } - leaderEpochCache.ifPresent(_.truncateFromEnd(nextOffset)) + leaderEpochCache.ifPresent(_.truncateFromEndAsyncFlush(nextOffset)) val newLogStartOffset = if (isRemoteLogEnabled) { logStartOffsetCheckpoint } else { math.max(logStartOffsetCheckpoint, segments.firstSegment.get.baseOffset) } // The earliest leader epoch may not be flushed during a hard failure. Recover it here. - leaderEpochCache.ifPresent(_.truncateFromStart(logStartOffsetCheckpoint)) + leaderEpochCache.ifPresent(_.truncateFromStartAsyncFlush(logStartOffsetCheckpoint)) // Any segment loading or recovery code must not use producerStateManager, so that we can build the full state here // from scratch. diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 6dbae30e29..490062a841 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -1656,7 +1656,7 @@ object LogManager { keepPartitionMetadataFile: Boolean): LogManager = { val defaultProps = config.extractLogConfigMap - LogConfig.validateBrokerLogConfigValues(defaultProps, config.isRemoteLogStorageSystemEnabled) + LogConfig.validateBrokerLogConfigValues(defaultProps, config.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) val defaultLogConfig = new LogConfig(defaultProps) val cleanerConfig = LogCleaner.cleanerConfig(config) @@ -1680,7 +1680,7 @@ object LogManager { time = time, keepPartitionMetadataFile = keepPartitionMetadataFile, interBrokerProtocolVersion = config.interBrokerProtocolVersion, - remoteStorageSystemEnable = config.remoteLogManagerConfig.enableRemoteStorageSystem(), + remoteStorageSystemEnable = config.remoteLogManagerConfig.isRemoteStorageSystemEnabled(), initialTaskDelayMs = config.logInitialTaskDelayMs) } @@ -1698,11 +1698,16 @@ object LogManager { newTopicsImage: TopicsImage, log: UnifiedLog ): Boolean = { - val topicId = log.topicId.getOrElse { - throw new RuntimeException(s"The log dir $log does not have a topic ID, " + - "which is not allowed when running in KRaft mode.") + if (log.topicId.isEmpty) { + // Missing topic ID could result from storage failure or unclean shutdown after topic creation but before flushing + // data to the `partition.metadata` file. And before appending data to the log, the `partition.metadata` is always + // flushed to disk. So if the topic ID is missing, it mostly means no data was appended, and we can treat this as + // a stray log. + info(s"The topicId does not exist in $log, treat it as a stray log") + return true } + val topicId = log.topicId.get val partitionId = log.topicPartition.partition() Option(newTopicsImage.getPartition(topicId, partitionId)) match { case Some(partition) => diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index 9f753d78db..608f263477 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -42,7 +42,7 @@ import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.server.util.Scheduler import org.apache.kafka.storage.internals.checkpoint.{LeaderEpochCheckpointFile, PartitionMetadataFile} import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache -import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, BatchMetadata, CompletedTxn, FetchDataInfo, FetchIsolation, LastRecord, LeaderHwChange, LogAppendInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetMetadata, LogOffsetSnapshot, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogValidator, ProducerAppendInfo, ProducerStateManager, ProducerStateManagerConfig, RollParams, VerificationGuard} +import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, BatchMetadata, CompletedTxn, EpochEntry, FetchDataInfo, FetchIsolation, LastRecord, LeaderHwChange, LogAppendInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetMetadata, LogOffsetSnapshot, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogValidator, ProducerAppendInfo, ProducerStateManager, ProducerStateManagerConfig, RollParams, VerificationGuard} import java.io.{File, IOException} import java.nio.file.{Files, Path} @@ -151,9 +151,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, def localLogStartOffset(): Long = _localLogStartOffset // This is the offset(inclusive) until which segments are copied to the remote storage. - @volatile private[kafka] var _highestOffsetInRemoteStorage: Long = -1L - - def highestOffsetInRemoteStorage(): Long = _highestOffsetInRemoteStorage + @volatile private var highestOffsetInRemoteStorage: Long = -1L locally { def updateLocalLogStartOffset(offset: Long): Unit = { @@ -522,7 +520,8 @@ class UnifiedLog(@volatile var logStartOffset: Long, } private def initializeLeaderEpochCache(): Unit = lock synchronized { - leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, recordVersion, logIdent) + leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( + dir, topicPartition, logDirFailureChannel, recordVersion, logIdent, leaderEpochCache, scheduler) } private def updateHighWatermarkWithLogEndOffset(): Unit = { @@ -547,8 +546,8 @@ class UnifiedLog(@volatile var logStartOffset: Long, def updateHighestOffsetInRemoteStorage(offset: Long): Unit = { if (!remoteLogEnabled()) - warn(s"Unable to update the highest offset in remote storage with offset $offset since remote storage is not enabled. The existing highest offset is ${highestOffsetInRemoteStorage()}.") - else if (offset > highestOffsetInRemoteStorage()) _highestOffsetInRemoteStorage = offset + warn(s"Unable to update the highest offset in remote storage with offset $offset since remote storage is not enabled. The existing highest offset is $highestOffsetInRemoteStorage.") + else if (offset > highestOffsetInRemoteStorage) highestOffsetInRemoteStorage = offset } // Rebuild producer state until lastOffset. This method may be called from the recovery code path, and thus must be @@ -1021,7 +1020,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, updatedLogStartOffset = true updateLogStartOffset(newLogStartOffset) info(s"Incremented log start offset to $newLogStartOffset due to $reason") - leaderEpochCache.foreach(_.truncateFromStart(logStartOffset)) + leaderEpochCache.foreach(_.truncateFromStartAsyncFlush(logStartOffset)) producerStateManager.onLogStartOffsetIncremented(newLogStartOffset) maybeIncrementFirstUnstableOffset() } @@ -1279,6 +1278,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, if (config.messageFormatVersion.isLessThan(IBP_0_10_0_IV0) && targetTimestamp != ListOffsetsRequest.EARLIEST_TIMESTAMP && + targetTimestamp != ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP && targetTimestamp != ListOffsetsRequest.LATEST_TIMESTAMP) throw new UnsupportedForMessageFormatException(s"Cannot search offsets based on timestamp because message format version " + s"for partition $topicPartition is ${config.messageFormatVersion} which is earlier than the minimum " + @@ -1299,15 +1299,16 @@ class UnifiedLog(@volatile var logStartOffset: Long, } else if (targetTimestamp == ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP) { val curLocalLogStartOffset = localLogStartOffset() - val epochResult: Optional[Integer] = - if (leaderEpochCache.isDefined) { - val epochOpt = leaderEpochCache.get.epochForOffset(curLocalLogStartOffset) - if (epochOpt.isPresent) Optional.of(epochOpt.getAsInt) else Optional.empty() - } else { - Optional.empty() - } + val earliestLocalLogEpochEntry = leaderEpochCache.asJava.flatMap(cache => { + val epoch = cache.epochForOffset(curLocalLogStartOffset) + if (epoch.isPresent) cache.epochEntry(epoch.getAsInt) else Optional.empty[EpochEntry]() + }) - Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, curLocalLogStartOffset, epochResult)) + val epochOpt = if (earliestLocalLogEpochEntry.isPresent && earliestLocalLogEpochEntry.get().startOffset <= curLocalLogStartOffset) + Optional.of[Integer](earliestLocalLogEpochEntry.get().epoch) + else Optional.empty[Integer]() + + Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, curLocalLogStartOffset, epochOpt)) } else if (targetTimestamp == ListOffsetsRequest.LATEST_TIMESTAMP) { val epoch = leaderEpochCache match { case Some(cache) => @@ -1316,28 +1317,6 @@ class UnifiedLog(@volatile var logStartOffset: Long, case None => Optional.empty[Integer]() } Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, logEndOffset, epoch)) - } else if (targetTimestamp == ListOffsetsRequest.LATEST_TIERED_TIMESTAMP) { - if (remoteLogEnabled()) { - val curHighestRemoteOffset = highestOffsetInRemoteStorage() - - val epochResult: Optional[Integer] = - if (leaderEpochCache.isDefined) { - val epochOpt = leaderEpochCache.get.epochForOffset(curHighestRemoteOffset) - if (epochOpt.isPresent) { - Optional.of(epochOpt.getAsInt) - } else if (curHighestRemoteOffset == -1) { - Optional.of(RecordBatch.NO_PARTITION_LEADER_EPOCH) - } else { - Optional.empty() - } - } else { - Optional.empty() - } - - Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, curHighestRemoteOffset, epochResult)) - } else { - Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, -1L, Optional.of(-1))) - } } else if (targetTimestamp == ListOffsetsRequest.MAX_TIMESTAMP) { // Cache to avoid race conditions. `toBuffer` is faster than most alternatives and provides // constant time access while being safe to use with concurrent collections unlike `toArray`. @@ -1485,7 +1464,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, // 1. they are uploaded to the remote storage // 2. log-start-offset was incremented higher than the largest offset in the candidate segment if (remoteLogEnabled()) { - (upperBoundOffset > 0 && upperBoundOffset - 1 <= highestOffsetInRemoteStorage()) || + (upperBoundOffset > 0 && upperBoundOffset - 1 <= highestOffsetInRemoteStorage) || allowDeletionDueToLogStartOffsetIncremented } else { true @@ -1623,13 +1602,13 @@ class UnifiedLog(@volatile var logStartOffset: Long, * The log size in bytes for all segments that are only in local log but not yet in remote log. */ def onlyLocalLogSegmentsSize: Long = - UnifiedLog.sizeInBytes(logSegments.stream.filter(_.baseOffset >= highestOffsetInRemoteStorage()).collect(Collectors.toList[LogSegment])) + UnifiedLog.sizeInBytes(logSegments.stream.filter(_.baseOffset >= highestOffsetInRemoteStorage).collect(Collectors.toList[LogSegment])) /** * The number of segments that are only in local log but not yet in remote log. */ def onlyLocalLogSegmentsCount: Long = - logSegments.stream().filter(_.baseOffset >= highestOffsetInRemoteStorage()).count() + logSegments.stream().filter(_.baseOffset >= highestOffsetInRemoteStorage).count() /** * The offset of the next message that will be appended to the log @@ -1821,7 +1800,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, // and inserted the first start offset entry, but then failed to append any entries // before another leader was elected. lock synchronized { - leaderEpochCache.foreach(_.truncateFromEnd(logEndOffset)) + leaderEpochCache.foreach(_.truncateFromEndAsyncFlush(logEndOffset)) } false @@ -1834,7 +1813,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, } else { val deletedSegments = localLog.truncateTo(targetOffset) deleteProducerSnapshots(deletedSegments, asyncDelete = true) - leaderEpochCache.foreach(_.truncateFromEnd(targetOffset)) + leaderEpochCache.foreach(_.truncateFromEndAsyncFlush(targetOffset)) logStartOffset = math.min(targetOffset, logStartOffset) rebuildProducerState(targetOffset, producerStateManager) if (highWatermark >= localLog.logEndOffset) @@ -2044,12 +2023,17 @@ object UnifiedLog extends Logging { Files.createDirectories(dir.toPath) val topicPartition = UnifiedLog.parseTopicPartitionName(dir) val segments = new LogSegments(topicPartition) + // The created leaderEpochCache will be truncated by LogLoader if necessary + // so it is guaranteed that the epoch entries will be correct even when on-disk + // checkpoint was stale (due to async nature of LeaderEpochFileCache#truncateFromStart/End). val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( dir, topicPartition, logDirFailureChannel, config.recordVersion, - s"[UnifiedLog partition=$topicPartition, dir=${dir.getParent}] ") + s"[UnifiedLog partition=$topicPartition, dir=${dir.getParent}] ", + None, + scheduler) val producerStateManager = new ProducerStateManager(topicPartition, dir, maxTransactionTimeoutMs, producerStateManagerConfig, time) val isRemoteLogEnabled = UnifiedLog.isRemoteLogEnabled(remoteStorageSystemEnable, config, topicPartition.topic) @@ -2136,7 +2120,8 @@ object UnifiedLog extends Logging { } /** - * If the recordVersion is >= RecordVersion.V2, then create and return a LeaderEpochFileCache. + * If the recordVersion is >= RecordVersion.V2, create a new LeaderEpochFileCache instance. + * Loading the epoch entries from the backing checkpoint file or the provided currentCache if not empty. * Otherwise, the message format is considered incompatible and the existing LeaderEpoch file * is deleted. * @@ -2145,33 +2130,29 @@ object UnifiedLog extends Logging { * @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure * @param recordVersion The record version * @param logPrefix The logging prefix + * @param currentCache The current LeaderEpochFileCache instance (if any) + * @param scheduler The scheduler for executing asynchronous tasks * @return The new LeaderEpochFileCache instance (if created), none otherwise */ def maybeCreateLeaderEpochCache(dir: File, topicPartition: TopicPartition, logDirFailureChannel: LogDirFailureChannel, recordVersion: RecordVersion, - logPrefix: String): Option[LeaderEpochFileCache] = { + logPrefix: String, + currentCache: Option[LeaderEpochFileCache], + scheduler: Scheduler): Option[LeaderEpochFileCache] = { val leaderEpochFile = LeaderEpochCheckpointFile.newFile(dir) - def newLeaderEpochFileCache(): LeaderEpochFileCache = { - val checkpointFile = new LeaderEpochCheckpointFile(leaderEpochFile, logDirFailureChannel) - new LeaderEpochFileCache(topicPartition, checkpointFile) - } - if (recordVersion.precedes(RecordVersion.V2)) { - val currentCache = if (leaderEpochFile.exists()) - Some(newLeaderEpochFileCache()) - else - None - - if (currentCache.exists(_.nonEmpty)) + if (leaderEpochFile.exists()) { warn(s"${logPrefix}Deleting non-empty leader epoch cache due to incompatible message format $recordVersion") - + } Files.deleteIfExists(leaderEpochFile.toPath) None } else { - Some(newLeaderEpochFileCache()) + val checkpointFile = new LeaderEpochCheckpointFile(leaderEpochFile, logDirFailureChannel) + currentCache.map(_.withCheckpoint(checkpointFile)) + .orElse(Some(new LeaderEpochFileCache(topicPartition, checkpointFile, scheduler))) } } diff --git a/core/src/main/scala/kafka/log/streamaspect/ElasticLeaderEpochCheckpoint.java b/core/src/main/scala/kafka/log/streamaspect/ElasticLeaderEpochCheckpoint.java index dbd0eaa9b0..2ba6b4d2b1 100644 --- a/core/src/main/scala/kafka/log/streamaspect/ElasticLeaderEpochCheckpoint.java +++ b/core/src/main/scala/kafka/log/streamaspect/ElasticLeaderEpochCheckpoint.java @@ -16,11 +16,11 @@ import java.util.Collection; import java.util.List; import java.util.function.Consumer; -import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint; +import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile; import org.apache.kafka.storage.internals.log.EpochEntry; // TODO: better implementation, limit the partition meta logic in partition -class ElasticLeaderEpochCheckpoint implements LeaderEpochCheckpoint { +class ElasticLeaderEpochCheckpoint extends LeaderEpochCheckpointFile { private final ElasticLeaderEpochCheckpointMeta meta; private final Consumer saveFunc; @@ -31,11 +31,16 @@ public ElasticLeaderEpochCheckpoint(ElasticLeaderEpochCheckpointMeta meta, } @Override - public synchronized void write(Collection epochs, boolean sync) { + public synchronized void write(Collection epochs) { meta.setEntries(new ArrayList<>(epochs)); saveFunc.accept(meta); } + @Override + public synchronized void writeForTruncation(Collection epochs) { + write(epochs); + } + @Override public synchronized List read() { return meta.entries(); diff --git a/core/src/main/scala/kafka/log/streamaspect/ElasticUnifiedLog.scala b/core/src/main/scala/kafka/log/streamaspect/ElasticUnifiedLog.scala index e8394bb9b4..44d425bc13 100644 --- a/core/src/main/scala/kafka/log/streamaspect/ElasticUnifiedLog.scala +++ b/core/src/main/scala/kafka/log/streamaspect/ElasticUnifiedLog.scala @@ -281,7 +281,7 @@ object ElasticUnifiedLog extends Logging { } } } - val leaderEpochFileCache = ElasticUnifiedLog.maybeCreateLeaderEpochCache(topicPartition, config.recordVersion, new ElasticLeaderEpochCheckpoint(localLog.leaderEpochCheckpointMeta, localLog.saveLeaderEpochCheckpoint)) + val leaderEpochFileCache = ElasticUnifiedLog.maybeCreateLeaderEpochCache(topicPartition, config.recordVersion, new ElasticLeaderEpochCheckpoint(localLog.leaderEpochCheckpointMeta, localLog.saveLeaderEpochCheckpoint), scheduler) // The real logStartOffset should be set by loaded offsets from ElasticLogLoader. // Since the real value has been passed to localLog, we just pass it to ElasticUnifiedLog. val elasticUnifiedLog = new ElasticUnifiedLog(localLog.logStartOffset, @@ -324,9 +324,11 @@ object ElasticUnifiedLog extends Logging { */ private[log] def maybeCreateLeaderEpochCache(topicPartition: TopicPartition, recordVersion: RecordVersion, - leaderEpochCheckpoint: ElasticLeaderEpochCheckpoint): Option[LeaderEpochFileCache] = { + leaderEpochCheckpoint: ElasticLeaderEpochCheckpoint, + scheduler: Scheduler + ): Option[LeaderEpochFileCache] = { - def newLeaderEpochFileCache(): LeaderEpochFileCache = new LeaderEpochFileCache(topicPartition, leaderEpochCheckpoint) + def newLeaderEpochFileCache(): LeaderEpochFileCache = new LeaderEpochFileCache(topicPartition, leaderEpochCheckpoint, scheduler) if (recordVersion.precedes(RecordVersion.V2)) { None diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index 6bf8bd893b..65ef855640 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -23,14 +23,18 @@ import java.nio.file.Paths import java.util.OptionalInt import java.util.concurrent.CompletableFuture import java.util.{Map => JMap} +import java.util.{Collection => JCollection} import kafka.log.LogManager import kafka.log.UnifiedLog import kafka.server.KafkaConfig import kafka.utils.CoreUtils import kafka.utils.FileLock import kafka.utils.Logging -import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, NetworkClient} -import org.apache.kafka.common.{KafkaException, Node, TopicPartition, Uuid} +import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, MetadataRecoveryStrategy, NetworkClient} +import org.apache.kafka.common.KafkaException +import org.apache.kafka.common.Node +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.Uuid import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.{ChannelBuilders, ListenerName, NetworkReceive, Selectable, Selector} @@ -133,7 +137,7 @@ trait RaftManager[T] { def replicatedLog: ReplicatedLog - def voterNode(id: Int, listener: String): Option[Node] + def voterNode(id: Int, listener: ListenerName): Option[Node] } class KafkaRaftManager[T]( @@ -147,6 +151,7 @@ class KafkaRaftManager[T]( metrics: Metrics, threadNamePrefixOpt: Option[String], val controllerQuorumVotersFuture: CompletableFuture[JMap[Integer, InetSocketAddress]], + bootstrapServers: JCollection[InetSocketAddress], fatalFaultHandler: FaultHandler ) extends RaftManager[T] with Logging { @@ -185,7 +190,6 @@ class KafkaRaftManager[T]( def startup(): Unit = { client.initialize( controllerQuorumVotersFuture.get(), - config.controllerListenerNames.head, new FileQuorumStateStore(new File(dataDir, FileQuorumStateStore.DEFAULT_FILE_NAME)), metrics ) @@ -228,14 +232,15 @@ class KafkaRaftManager[T]( expirationService, logContext, clusterId, + bootstrapServers, raftConfig ) client } private def buildNetworkChannel(): KafkaNetworkChannel = { - val netClient = buildNetworkClient() - new KafkaNetworkChannel(time, netClient, config.quorumRequestTimeoutMs, threadNamePrefix) + val (listenerName, netClient) = buildNetworkClient() + new KafkaNetworkChannel(time, listenerName, netClient, config.quorumRequestTimeoutMs, threadNamePrefix) } private def createDataDir(): File = { @@ -254,7 +259,7 @@ class KafkaRaftManager[T]( ) } - private def buildNetworkClient(): NetworkClient = { + private def buildNetworkClient(): (ListenerName, NetworkClient) = { val controllerListenerName = new ListenerName(config.controllerListenerNames.head) val controllerSecurityProtocol = config.effectiveListenerSecurityProtocolMap.getOrElse( controllerListenerName, @@ -292,7 +297,7 @@ class KafkaRaftManager[T]( val reconnectBackoffMsMs = 500 val discoverBrokerVersions = true - new NetworkClient( + val networkClient = new NetworkClient( selector, new ManualMetadataUpdater(), clientId, @@ -307,15 +312,18 @@ class KafkaRaftManager[T]( time, discoverBrokerVersions, apiVersions, - logContext + logContext, + MetadataRecoveryStrategy.NONE ) + + (controllerListenerName, networkClient) } override def leaderAndEpoch: LeaderAndEpoch = { client.leaderAndEpoch } - override def voterNode(id: Int, listener: String): Option[Node] = { + override def voterNode(id: Int, listener: ListenerName): Option[Node] = { client.voterNode(id, listener).toScala } } diff --git a/core/src/main/scala/kafka/server/BrokerBlockingSender.scala b/core/src/main/scala/kafka/server/BrokerBlockingSender.scala index 7d9fb0512a..3cb692045b 100644 --- a/core/src/main/scala/kafka/server/BrokerBlockingSender.scala +++ b/core/src/main/scala/kafka/server/BrokerBlockingSender.scala @@ -96,7 +96,8 @@ class BrokerBlockingSender(sourceBroker: BrokerEndPoint, time, false, new ApiVersions, - logContext + logContext, + MetadataRecoveryStrategy.NONE ) (networkClient, reconfigurableChannelBuilder) } diff --git a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala index 5f3fdc8188..51bc16fb09 100644 --- a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala +++ b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala @@ -264,11 +264,11 @@ class BrokerLifecycleManager( new OfflineDirBrokerFailureEvent(directory)) } - def handleKraftJBODMetadataVersionUpdate(): Unit = { - eventQueue.append(new KraftJBODMetadataVersionUpdateEvent()) + def resendBrokerRegistrationUnlessZkMode(): Unit = { + eventQueue.append(new ResendBrokerRegistrationUnlessZkModeEvent()) } - private class KraftJBODMetadataVersionUpdateEvent extends EventQueue.Event { + private class ResendBrokerRegistrationUnlessZkModeEvent extends EventQueue.Event { override def run(): Unit = { if (!isZkBroker) { registered = false diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 343f792c20..e264e48652 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -41,6 +41,7 @@ import org.apache.kafka.coordinator.group.metrics.{GroupCoordinatorMetrics, Grou import org.apache.kafka.coordinator.group.{CoordinatorRecord, GroupCoordinator, GroupCoordinatorConfig, GroupCoordinatorService, CoordinatorRecordSerde} import org.apache.kafka.image.loader.MetadataLoader import org.apache.kafka.image.publisher.MetadataPublisher +import org.apache.kafka.image.publisher.BrokerRegistrationTracker import org.apache.kafka.metadata.{BrokerState, ListenerInfo, VersionRange} import org.apache.kafka.security.CredentialProvider import org.apache.kafka.server.{AssignmentsManager, ClientMetricsManager, NodeToControllerChannelManager} @@ -142,6 +143,8 @@ class BrokerServer( var brokerMetadataPublisher: BrokerMetadataPublisher = _ + var brokerRegistrationTracker: BrokerRegistrationTracker = _ + val brokerFeatures: BrokerFeatures = BrokerFeatures.createDefault(config.unstableFeatureVersionsEnabled) def kafkaYammerMetrics: KafkaYammerMetrics = KafkaYammerMetrics.INSTANCE @@ -193,7 +196,7 @@ class BrokerServer( kafkaScheduler.startup() /* register broker metrics */ - brokerTopicStats = new BrokerTopicStats(config.isRemoteLogStorageSystemEnabled) + brokerTopicStats = new BrokerTopicStats(config.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) quotaManagers = QuotaFactory.instantiate(config, metrics, time, s"broker-${config.nodeId}-") @@ -462,6 +465,7 @@ class BrokerServer( rlm.startup() } + metadataPublishers.add(new MetadataVersionConfigValidator(config, sharedServer.metadataPublishingFaultHandler)) brokerMetadataPublisher = new BrokerMetadataPublisher(config, metadataCache, logManager, @@ -499,6 +503,11 @@ class BrokerServer( lifecycleManager ) metadataPublishers.add(brokerMetadataPublisher) + brokerRegistrationTracker = new BrokerRegistrationTracker(config.brokerId, + logManager.directoryIdsSet.toList.asJava, + () => lifecycleManager.resendBrokerRegistrationUnlessZkMode()) + metadataPublishers.add(brokerRegistrationTracker) + // Register parts of the broker that can be reconfigured via dynamic configs. This needs to // be done before we publish the dynamic configs, so that we don't miss anything. @@ -586,6 +595,7 @@ class BrokerServer( val serde = new CoordinatorRecordSerde val groupCoordinatorConfig = new GroupCoordinatorConfig( config.groupCoordinatorNumThreads, + config.groupCoordinatorAppendLingerMs, config.consumerGroupSessionTimeoutMs, config.consumerGroupHeartbeatIntervalMs, config.consumerGroupMaxSize, @@ -635,8 +645,8 @@ class BrokerServer( } protected def createRemoteLogManager(): Option[RemoteLogManager] = { - if (config.remoteLogManagerConfig.enableRemoteStorageSystem()) { - Some(new RemoteLogManager(config.remoteLogManagerConfig, config.brokerId, config.logDirs.head, clusterId, time, + if (config.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) { + Some(new RemoteLogManager(config, config.brokerId, config.logDirs.head, clusterId, time, (tp: TopicPartition) => logManager.getLog(tp).asJava, (tp: TopicPartition, remoteLogStartOffset: java.lang.Long) => { logManager.getLog(tp).foreach { log => diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala index 5a96146b67..9728de3329 100644 --- a/core/src/main/scala/kafka/server/ConfigHandler.scala +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -70,7 +70,7 @@ class TopicConfigHandler(private val replicaManager: ReplicaManager, val logs = logManager.logsByTopic(topic) val wasRemoteLogEnabledBeforeUpdate = logs.exists(_.remoteLogEnabled()) - logManager.updateTopicConfig(topic, props, kafkaConfig.isRemoteLogStorageSystemEnabled) + logManager.updateTopicConfig(topic, props, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) maybeBootstrapRemoteLogComponents(topic, logs, wasRemoteLogEnabledBeforeUpdate) } diff --git a/core/src/main/scala/kafka/server/ControllerConfigurationValidator.scala b/core/src/main/scala/kafka/server/ControllerConfigurationValidator.scala index 15eb1eff04..b99065b573 100644 --- a/core/src/main/scala/kafka/server/ControllerConfigurationValidator.scala +++ b/core/src/main/scala/kafka/server/ControllerConfigurationValidator.scala @@ -107,7 +107,8 @@ class ControllerConfigurationValidator(kafkaConfig: KafkaConfig) extends Configu throw new InvalidConfigurationException("Null value not supported for topic configs: " + nullTopicConfigs.mkString(",")) } - LogConfig.validate(properties, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled) + LogConfig.validate(properties, kafkaConfig.extractLogConfigMap, + kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) case BROKER => validateBrokerName(resource.name()) case CLIENT_METRICS => val properties = new Properties() diff --git a/core/src/main/scala/kafka/server/DelayedRemoteFetch.scala b/core/src/main/scala/kafka/server/DelayedRemoteFetch.scala index 00d6afb89f..58a866aa4a 100644 --- a/core/src/main/scala/kafka/server/DelayedRemoteFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedRemoteFetch.scala @@ -35,12 +35,13 @@ import scala.collection._ class DelayedRemoteFetch(remoteFetchTask: Future[Void], remoteFetchResult: CompletableFuture[RemoteLogReadResult], remoteFetchInfo: RemoteStorageFetchInfo, + remoteFetchMaxWaitMs: Long, fetchPartitionStatus: Seq[(TopicIdPartition, FetchPartitionStatus)], fetchParams: FetchParams, localReadResults: Seq[(TopicIdPartition, LogReadResult)], replicaManager: ReplicaManager, responseCallback: Seq[(TopicIdPartition, FetchPartitionData)] => Unit) - extends DelayedOperation(fetchParams.maxWaitMs) { + extends DelayedOperation(remoteFetchMaxWaitMs) { if (fetchParams.isFromFollower) { throw new IllegalStateException(s"The follower should not invoke remote fetch. Fetch params are: $fetchParams") diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 154abe822a..613e93c84f 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -1170,36 +1170,57 @@ class DynamicRemoteLogConfig(server: KafkaBroker) extends BrokerReconfigurable w override def validateReconfiguration(newConfig: KafkaConfig): Unit = { newConfig.values.forEach { (k, v) => - if (reconfigurableConfigs.contains(k)) { - if (k.equals(RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP)) { - val newValue = v.asInstanceOf[Long] - val oldValue = getValue(server.config, k) - if (newValue != oldValue && newValue <= 0) { - val errorMsg = s"Dynamic remote log manager config update validation failed for $k=$v" - throw new ConfigException(s"$errorMsg, value should be at least 1") - } + if (RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP.equals(k) || + RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP.equals(k) || + RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP.equals(k)) { + val newValue = v.asInstanceOf[Long] + val oldValue = getValue(server.config, k) + if (newValue != oldValue && newValue <= 0) { + val errorMsg = s"Dynamic remote log manager config update validation failed for $k=$v" + throw new ConfigException(s"$errorMsg, value should be at least 1") } } } } override def reconfigure(oldConfig: KafkaConfig, newConfig: KafkaConfig): Unit = { - val oldValue = oldConfig.getLong(RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP) - val newValue = newConfig.getLong(RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP) - if (oldValue != newValue) { - val remoteLogManager = server.remoteLogManagerOpt - if (remoteLogManager.nonEmpty) { + def oldLongValue(k: String): Long = oldConfig.getLong(k) + def newLongValue(k: String): Long = newConfig.getLong(k) + + def isChangedLongValue(k : String): Boolean = oldLongValue(k) != newLongValue(k) + + val remoteLogManager = server.remoteLogManagerOpt + if (remoteLogManager.nonEmpty) { + if (isChangedLongValue(RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP)) { + val oldValue = oldLongValue(RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP) + val newValue = newLongValue(RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP) remoteLogManager.get.resizeCacheSize(newValue) info(s"Dynamic remote log manager config: ${RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP} updated, " + s"old value: $oldValue, new value: $newValue") } + if (isChangedLongValue(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP)) { + val oldValue = oldLongValue(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP) + val newValue = newLongValue(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP) + remoteLogManager.get.updateCopyQuota(newValue) + info(s"Dynamic remote log manager config: ${RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP} updated, " + + s"old value: $oldValue, new value: $newValue") + } + if (isChangedLongValue(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP)) { + val oldValue = oldLongValue(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP) + val newValue = newLongValue(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP) + remoteLogManager.get.updateFetchQuota(newValue) + info(s"Dynamic remote log manager config: ${RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP} updated, " + + s"old value: $oldValue, new value: $newValue") + } } } private def getValue(config: KafkaConfig, name: String): Long = { name match { - case RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP => - config.getLong(RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP) + case RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP | + RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP | + RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP => + config.getLong(name) case n => throw new IllegalStateException(s"Unexpected dynamic remote log manager config $n") } } @@ -1207,6 +1228,9 @@ class DynamicRemoteLogConfig(server: KafkaBroker) extends BrokerReconfigurable w object DynamicRemoteLogConfig { val ReconfigurableConfigs = Set( - RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP + RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP, + RemoteLogManagerConfig.REMOTE_FETCH_MAX_WAIT_MS_PROP, + RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP, + RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP ) } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 4a7952eff0..465a839b70 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -3835,6 +3835,7 @@ class KafkaApis(val requestChannel: RequestChannel, def handleConsumerGroupDescribe(request: RequestChannel.Request): CompletableFuture[Unit] = { val consumerGroupDescribeRequest = request.body[ConsumerGroupDescribeRequest] + val includeAuthorizedOperations = consumerGroupDescribeRequest.data.includeAuthorizedOperations if (!config.isNewGroupCoordinatorEnabled) { // The API is not supported by the "old" group coordinator (the default). If the @@ -3863,6 +3864,17 @@ class KafkaApis(val requestChannel: RequestChannel, if (exception != null) { requestHelper.sendMaybeThrottle(request, consumerGroupDescribeRequest.getErrorResponse(exception)) } else { + if (includeAuthorizedOperations) { + results.forEach { groupResult => + if (groupResult.errorCode == Errors.NONE.code) { + groupResult.setAuthorizedOperations(authHelper.authorizedOperations( + request, + new Resource(ResourceType.GROUP, groupResult.groupId) + )) + } + } + } + if (response.groups.isEmpty) { // If the response is empty, we can directly reuse the results. response.setGroups(results) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 2e2eb1116e..4a83cb73a0 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -22,7 +22,7 @@ import io.netty.util.internal.PlatformDependent import kafka.autobalancer.config.{AutoBalancerControllerConfig, AutoBalancerMetricsReporterConfig} import kafka.automq.AutoMQConfig -import java.{lang, util} +import java.util import java.util.concurrent.TimeUnit import java.util.{Collections, Properties} import kafka.cluster.EndPoint @@ -42,7 +42,7 @@ import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.group.ConsumerGroupMigrationPolicy import org.apache.kafka.coordinator.group.Group.GroupType import org.apache.kafka.coordinator.group.GroupCoordinatorConfig -import org.apache.kafka.coordinator.group.assignor.ConsumerGroupPartitionAssignor +import org.apache.kafka.coordinator.group.api.assignor.ConsumerGroupPartitionAssignor import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.raft.QuorumConfig @@ -283,6 +283,7 @@ object KafkaConfig { .define(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, LIST, GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DEFAULT, ValidList.in(Utils.enumOptions(classOf[GroupType]):_*), MEDIUM, GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DOC) .define(GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_CONFIG, INT, GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_DEFAULT, atLeast(1), MEDIUM, GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_DOC) + .define(GroupCoordinatorConfig.GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG, INT, GroupCoordinatorConfig.GROUP_COORDINATOR_APPEND_LINGER_MS_DEFAULT, atLeast(0), MEDIUM, GroupCoordinatorConfig.GROUP_COORDINATOR_APPEND_LINGER_MS_DOC) // Internal configuration used by integration and system tests. .defineInternal(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG, BOOLEAN, GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_DEFAULT, null, MEDIUM, GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_DOC) @@ -444,6 +445,7 @@ object KafkaConfig { /** ********* Raft Quorum Configuration *********/ .define(QuorumConfig.QUORUM_VOTERS_CONFIG, LIST, QuorumConfig.DEFAULT_QUORUM_VOTERS, new QuorumConfig.ControllerQuorumVotersValidator(), HIGH, QuorumConfig.QUORUM_VOTERS_DOC) + .defineInternal(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, LIST, QuorumConfig.DEFAULT_QUORUM_BOOTSTRAP_SERVERS, new QuorumConfig.ControllerQuorumBootstrapServersValidator(), HIGH, QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_DOC) .define(QuorumConfig.QUORUM_ELECTION_TIMEOUT_MS_CONFIG, INT, QuorumConfig.DEFAULT_QUORUM_ELECTION_TIMEOUT_MS, null, HIGH, QuorumConfig.QUORUM_ELECTION_TIMEOUT_MS_DOC) .define(QuorumConfig.QUORUM_FETCH_TIMEOUT_MS_CONFIG, INT, QuorumConfig.DEFAULT_QUORUM_FETCH_TIMEOUT_MS, null, HIGH, QuorumConfig.QUORUM_FETCH_TIMEOUT_MS_DOC) .define(QuorumConfig.QUORUM_ELECTION_BACKOFF_MAX_MS_CONFIG, INT, QuorumConfig.DEFAULT_QUORUM_ELECTION_BACKOFF_MAX_MS, null, HIGH, QuorumConfig.QUORUM_ELECTION_BACKOFF_MAX_MS_DOC) @@ -462,9 +464,12 @@ object KafkaConfig { // AutoMQ inject end /** ********* Remote Log Management Configuration *********/ - RemoteLogManagerConfig.CONFIG_DEF.configKeys().values().forEach(key => configDef.define(key)) + RemoteLogManagerConfig.configDef().configKeys().values().forEach(key => configDef.define(key)) + + // AutoMQ inject start AutoBalancerControllerConfig.CONFIG_DEF.configKeys().values().forEach(key => configDef.define(key)) AutoBalancerMetricsReporterConfig.CONFIG_DEF.configKeys().values().forEach(key => configDef.define(key)) + // AutoMQ inject end def configNames: Seq[String] = configDef.names.asScala.toBuffer.sorted private[server] def defaultValues: Map[String, _] = configDef.defaultValues.asScala @@ -599,7 +604,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val zkEnableSecureAcls: Boolean = getBoolean(ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG) val zkMaxInFlightRequests: Int = getInt(ZkConfigs.ZK_MAX_IN_FLIGHT_REQUESTS_CONFIG) - private val _remoteLogManagerConfig = new RemoteLogManagerConfig(this) + private val _remoteLogManagerConfig = new RemoteLogManagerConfig(props) def remoteLogManagerConfig = _remoteLogManagerConfig private def zkBooleanConfigOrSystemPropertyWithDefaultValue(propKey: String): Boolean = { @@ -957,6 +962,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val isNewGroupCoordinatorEnabled = getBoolean(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG) || groupCoordinatorRebalanceProtocols.contains(GroupType.CONSUMER) val groupCoordinatorNumThreads = getInt(GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_CONFIG) + val groupCoordinatorAppendLingerMs = getInt(GroupCoordinatorConfig.GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG) /** Consumer group configs */ val consumerGroupSessionTimeoutMs = getInt(GroupCoordinatorConfig.CONSUMER_GROUP_SESSION_TIMEOUT_MS_CONFIG) @@ -1065,6 +1071,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami /** ********* Raft Quorum Configuration *********/ val quorumVoters = getList(QuorumConfig.QUORUM_VOTERS_CONFIG) + val quorumBootstrapServers = getList(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG) val quorumElectionTimeoutMs = getInt(QuorumConfig.QUORUM_ELECTION_TIMEOUT_MS_CONFIG) val quorumFetchTimeoutMs = getInt(QuorumConfig.QUORUM_FETCH_TIMEOUT_MS_CONFIG) val quorumElectionBackoffMs = getInt(QuorumConfig.QUORUM_ELECTION_BACKOFF_MAX_MS_CONFIG) @@ -1315,12 +1322,18 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami def usesTopicId: Boolean = usesSelfManagedQuorum || interBrokerProtocolVersion.isTopicIdsSupported - - val isRemoteLogStorageSystemEnabled: lang.Boolean = getBoolean(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP) def logLocalRetentionBytes: java.lang.Long = getLong(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP) def logLocalRetentionMs: java.lang.Long = getLong(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP) + def remoteFetchMaxWaitMs = getInt(RemoteLogManagerConfig.REMOTE_FETCH_MAX_WAIT_MS_PROP) + + def remoteLogIndexFileCacheTotalSizeBytes: Long = getLong(RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP) + + def remoteLogManagerCopyMaxBytesPerSecond: Long = getLong(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP) + + def remoteLogManagerFetchMaxBytesPerSecond: Long = getLong(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP) + validateValues() @nowarn("cat=deprecation") @@ -1570,6 +1583,18 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami } } + /** + * Validate some configurations for new MetadataVersion. A new MetadataVersion can take place when + * a FeatureLevelRecord for "metadata.version" is read from the cluster metadata. + */ + def validateWithMetadataVersion(metadataVersion: MetadataVersion): Unit = { + if (processRoles.contains(ProcessRole.BrokerRole) && logDirs.size > 1) { + require(metadataVersion.isDirectoryAssignmentSupported, + s"Multiple log directories (aka JBOD) are not supported in the current MetadataVersion ${metadataVersion}. " + + s"Need ${MetadataVersion.IBP_3_7_IV2} or higher") + } + } + /** * Copy the subset of properties that are relevant to Logs. The individual properties * are listed here since the names are slightly different in each Config class... diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala index 821a1de24a..46d5cd9429 100644 --- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala +++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala @@ -114,6 +114,7 @@ class KafkaRaftServer( time, metrics, CompletableFuture.completedFuture(QuorumConfig.parseVoterConnections(config.quorumVoters)), + List().asJava, new StandardFaultHandlerFactory(), ) } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 15876d6d17..a11be84787 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -31,7 +31,7 @@ import kafka.server.metadata.{OffsetTrackingListener, ZkConfigRepository, ZkMeta import kafka.server.streamaspect.{ElasticKafkaApis, ElasticReplicaManager} import kafka.utils._ import kafka.zk.{AdminZkClient, BrokerInfo, KafkaZkClient} -import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, NetworkClient, NetworkClientUtils} +import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, MetadataRecoveryStrategy, NetworkClient, NetworkClientUtils} import org.apache.kafka.common.config.ConfigException import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.message.ApiMessageType.ListenerType @@ -71,9 +71,9 @@ import java.net.{InetAddress, SocketTimeoutException} import java.nio.file.{Files, Paths} import java.time.Duration import java.util -import java.util.{Optional, OptionalInt, OptionalLong} import java.util.concurrent._ import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} +import java.util.{Optional, OptionalInt, OptionalLong} import scala.collection.{Map, Seq} import scala.compat.java8.OptionConverters.RichOptionForJava8 import scala.jdk.CollectionConverters._ @@ -277,7 +277,7 @@ class KafkaServer( createCurrentControllerIdMetric() /* register broker metrics */ - _brokerTopicStats = new BrokerTopicStats(config.isRemoteLogStorageSystemEnabled) + _brokerTopicStats = new BrokerTopicStats(config.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) quotaManagers = QuotaFactory.instantiate(config, metrics, time, threadNamePrefix.getOrElse("")) KafkaBroker.notifyClusterListeners(clusterId, kafkaMetricsReporters ++ metrics.reporters.asScala) @@ -440,6 +440,7 @@ class KafkaServer( metrics, threadNamePrefix, CompletableFuture.completedFuture(quorumVoters), + List().asJava, fatalFaultHandler = new LoggingFaultHandler("raftManager", () => shutdown()) ) quorumControllerNodeProvider = RaftControllerNodeProvider(raftManager, config) @@ -690,8 +691,8 @@ class KafkaServer( } protected def createRemoteLogManager(): Option[RemoteLogManager] = { - if (config.remoteLogManagerConfig.enableRemoteStorageSystem()) { - Some(new RemoteLogManager(config.remoteLogManagerConfig, config.brokerId, config.logDirs.head, clusterId, time, + if (config.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) { + Some(new RemoteLogManager(config, config.brokerId, config.logDirs.head, clusterId, time, (tp: TopicPartition) => logManager.getLog(tp).asJava, (tp: TopicPartition, remoteLogStartOffset: java.lang.Long) => { logManager.getLog(tp).foreach { log => @@ -827,7 +828,8 @@ class KafkaServer( time, false, new ApiVersions, - logContext) + logContext, + MetadataRecoveryStrategy.NONE) } var shutdownSucceeded: Boolean = false diff --git a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala index 0017a5876a..6ce6e9e0a4 100644 --- a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala +++ b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala @@ -112,7 +112,7 @@ class RaftControllerNodeProvider( val saslMechanism: String ) extends ControllerNodeProvider with Logging { - private def idToNode(id: Int): Option[Node] = raftManager.voterNode(id, listenerName.value()) + private def idToNode(id: Int): Option[Node] = raftManager.voterNode(id, listenerName) override def getControllerInfo(): ControllerInformation = ControllerInformation(raftManager.leaderAndEpoch.leaderId.asScala.flatMap(idToNode), @@ -191,7 +191,8 @@ class NodeToControllerChannelManagerImpl( time, true, apiVersions, - logContext + logContext, + MetadataRecoveryStrategy.NONE ) } val threadName = s"${threadNamePrefix}to-controller-${channelName}-channel-manager" diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 169021ef77..8d929d5889 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -33,6 +33,7 @@ import kafka.zk.KafkaZkClient import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.message.DeleteRecordsResponseData.DeleteRecordsPartitionResult +import org.apache.kafka.common.message.DescribeLogDirsResponseData.DescribeLogDirsTopic import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.message.LeaderAndIsrResponseData.{LeaderAndIsrPartitionError, LeaderAndIsrTopicError} import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopic @@ -67,7 +68,7 @@ import java.util import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong} import java.util.concurrent.locks.Lock import java.util.concurrent.{ArrayBlockingQueue, CompletableFuture, Future, RejectedExecutionException, TimeUnit} -import java.util.{Optional, OptionalInt, OptionalLong} +import java.util.{Collections, Optional, OptionalInt, OptionalLong} import scala.collection.{Map, Seq, Set, mutable} import scala.compat.java8.OptionConverters._ import scala.jdk.CollectionConverters._ @@ -1298,9 +1299,9 @@ class ReplicaManager(val config: KafkaConfig, val fileStore = Files.getFileStore(file) val totalBytes = adjustForLargeFileSystems(fileStore.getTotalSpace) val usableBytes = adjustForLargeFileSystems(fileStore.getUsableSpace) - logsByDir.get(absolutePath) match { + val topicInfos = logsByDir.get(absolutePath) match { case Some(logs) => - val topicInfos = logs.groupBy(_.topicPartition.topic).map{case (topic, logs) => + logs.groupBy(_.topicPartition.topic).map { case (topic, logs) => new DescribeLogDirsResponseData.DescribeLogDirsTopic().setName(topic).setPartitions( logs.filter { log => partitions.contains(log.topicPartition) @@ -1311,17 +1312,19 @@ class ReplicaManager(val config: KafkaConfig, .setOffsetLag(getLogEndOffsetLag(log.topicPartition, log.logEndOffset, log.isFuture)) .setIsFutureKey(log.isFuture) }.toList.asJava) - }.toList.asJava - - new DescribeLogDirsResponseData.DescribeLogDirsResult().setLogDir(absolutePath) - .setErrorCode(Errors.NONE.code).setTopics(topicInfos) - .setTotalBytes(totalBytes).setUsableBytes(usableBytes) + }.filterNot(_.partitions().isEmpty).toList.asJava case None => - new DescribeLogDirsResponseData.DescribeLogDirsResult().setLogDir(absolutePath) - .setErrorCode(Errors.NONE.code) - .setTotalBytes(totalBytes).setUsableBytes(usableBytes) + Collections.emptyList[DescribeLogDirsTopic]() } + val describeLogDirsResult = new DescribeLogDirsResponseData.DescribeLogDirsResult() + .setLogDir(absolutePath).setTopics(topicInfos) + .setErrorCode(Errors.NONE.code) + .setTotalBytes(totalBytes).setUsableBytes(usableBytes) + if (!topicInfos.isEmpty) + describeLogDirsResult.setTopics(topicInfos) + describeLogDirsResult + } catch { case e: KafkaStorageException => warn("Unable to describe replica dirs for %s".format(absolutePath), e) @@ -1525,9 +1528,9 @@ class ReplicaManager(val config: KafkaConfig, return Some(createLogReadResult(e)) } - val remoteFetch = new DelayedRemoteFetch(remoteFetchTask, remoteFetchResult, remoteFetchInfo, + val remoteFetchMaxWaitMs = config.remoteFetchMaxWaitMs.toLong + val remoteFetch = new DelayedRemoteFetch(remoteFetchTask, remoteFetchResult, remoteFetchInfo, remoteFetchMaxWaitMs, fetchPartitionStatus, params, logReadResults, this, responseCallback) - delayedRemoteFetchPurgatory.tryCompleteElseWatch(remoteFetch, Seq(key)) None } @@ -1803,12 +1806,25 @@ class ReplicaManager(val config: KafkaConfig, createLogReadResult(highWatermark, leaderLogStartOffset, leaderLogEndOffset, new OffsetMovedToTieredStorageException("Given offset" + offset + " is moved to tiered storage")) } else { - // For consume fetch requests, create a dummy FetchDataInfo with the remote storage fetch information. - // For the first topic-partition that needs remote data, we will use this information to read the data in another thread. - val fetchDataInfo = - new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false, Optional.empty(), - Optional.of(new RemoteStorageFetchInfo(adjustedMaxBytes, minOneMessage, tp.topicPartition(), - fetchInfo, params.isolation, params.hardMaxBytesLimit()))) + val fetchDataInfo = if (remoteLogManager.get.isRemoteLogFetchQuotaExceeded) { + // We do not want to send an exception in a LogReadResult response (like we do in other cases when we send + // UnknownOffsetMetadata), because it is classified as an error in reading the data, and a response is + // immediately sent back to the client. Instead, we want to serve data for the other topic partitions of the + // fetch request via delayed fetch if required (when sending immediate response, we skip delayed fetch). + new FetchDataInfo( + LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, + MemoryRecords.EMPTY, + false, + Optional.empty(), + Optional.empty() + ) + } else { + // For consume fetch requests, create a dummy FetchDataInfo with the remote storage fetch information. + // For the first topic-partition that needs remote data, we will use this information to read the data in another thread. + new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY, false, Optional.empty(), + Optional.of(new RemoteStorageFetchInfo(adjustedMaxBytes, minOneMessage, tp.topicPartition(), + fetchInfo, params.isolation, params.hardMaxBytesLimit()))) + } LogReadResult(fetchDataInfo, divergingEpoch = None, diff --git a/core/src/main/scala/kafka/server/SharedServer.scala b/core/src/main/scala/kafka/server/SharedServer.scala index 4cac65a167..9c900ccd78 100644 --- a/core/src/main/scala/kafka/server/SharedServer.scala +++ b/core/src/main/scala/kafka/server/SharedServer.scala @@ -43,6 +43,7 @@ import java.util.Arrays import java.util.Optional import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.{CompletableFuture, TimeUnit} +import java.util.{Collection => JCollection} import java.util.{Map => JMap} @@ -96,6 +97,7 @@ class SharedServer( val time: Time, private val _metrics: Metrics, val controllerQuorumVotersFuture: CompletableFuture[JMap[Integer, InetSocketAddress]], + val bootstrapServers: JCollection[InetSocketAddress], val faultHandlerFactory: FaultHandlerFactory ) extends Logging { private val logContext: LogContext = new LogContext(s"[SharedServer id=${sharedServerConfig.nodeId}] ") @@ -277,6 +279,7 @@ class SharedServer( metrics, Some(s"kafka-${sharedServerConfig.nodeId}-raft"), // No dash expected at the end controllerQuorumVotersFuture, + bootstrapServers, raftManagerFaultHandler ) raftManager = _raftManager diff --git a/core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala b/core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala index 807f2a59d9..3712dc9425 100644 --- a/core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala +++ b/core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala @@ -91,7 +91,7 @@ class OffsetCheckpointFile(val file: File, logDirFailureChannel: LogDirFailureCh if (checkpoint == null) { elasticCheckpoint.write(offsets) } else { - checkpoint.write(list, true) + checkpoint.write(list) } } diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala index 7274c7e5af..0e9c693ce1 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala @@ -31,7 +31,6 @@ import org.apache.kafka.coordinator.group.GroupCoordinator import org.apache.kafka.image.loader.LoaderManifest import org.apache.kafka.image.publisher.MetadataPublisher import org.apache.kafka.image.{MetadataDelta, MetadataImage, TopicDelta, TopicsDelta} -import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.fault.FaultHandler import java.util @@ -133,21 +132,6 @@ class BrokerMetadataPublisher( debug(s"Publishing metadata at offset $highestOffsetAndEpoch with $metadataVersionLogMsg.") } - Option(delta.featuresDelta()).foreach { featuresDelta => - featuresDelta.metadataVersionChange().ifPresent{ metadataVersion => - info(s"Updating metadata.version to ${metadataVersion.featureLevel()} at offset $highestOffsetAndEpoch.") - val currentMetadataVersion = delta.image().features().metadataVersion() - if (currentMetadataVersion.isLessThan(MetadataVersion.IBP_3_7_IV2) && metadataVersion.isAtLeast(MetadataVersion.IBP_3_7_IV2)) { - info( - s"""Resending BrokerRegistration with existing incarnation-id to inform the - |controller about log directories in the broker following metadata update: - |previousMetadataVersion: ${delta.image().features().metadataVersion()} - |newMetadataVersion: $metadataVersion""".stripMargin.linesIterator.mkString(" ").trim) - brokerLifecycleManager.handleKraftJBODMetadataVersionUpdate() - } - } - } - // Apply topic deltas. Option(delta.topicsDelta()).foreach { topicsDelta => handleTopicsDelta(deltaName, topicsDelta, delta, newImage) diff --git a/core/src/main/scala/kafka/tools/StorageTool.scala b/core/src/main/scala/kafka/tools/StorageTool.scala index dd84ab43f9..999a12e8c1 100644 --- a/core/src/main/scala/kafka/tools/StorageTool.scala +++ b/core/src/main/scala/kafka/tools/StorageTool.scala @@ -46,93 +46,117 @@ import scala.jdk.CollectionConverters._ import scala.collection.mutable.ArrayBuffer object StorageTool extends Logging { + def main(args: Array[String]): Unit = { + var exitCode: Integer = 0 + var message: Option[String] = None try { - val namespace = parseArguments(args) - val command = namespace.getString("command") - val config = Option(namespace.getString("config")).flatMap( - p => Some(new KafkaConfig(Utils.loadProps(p)))) - command match { - case "info" => - val directories = configToLogDirectories(config.get) - val selfManagedMode = configToSelfManagedMode(config.get) - Exit.exit(infoCommand(System.out, selfManagedMode, directories)) - - case "format" | "auto-format" => - val directories = configToLogDirectories(config.get) - val clusterId = namespace.getString("cluster_id") - val metaProperties = new MetaProperties.Builder(). - setVersion(MetaPropertiesVersion.V1). - setClusterId(clusterId). - setNodeId(config.get.nodeId). - build() - val metadataRecords : ArrayBuffer[ApiMessageAndVersion] = ArrayBuffer() - val specifiedFeatures: util.List[String] = namespace.getList("feature") - val releaseVersionFlagSpecified = namespace.getString("release_version") != null - if (releaseVersionFlagSpecified && specifiedFeatures != null) { - throw new TerseFailure("Both --release-version and --feature were set. Only one of the two flags can be set.") - } - val featureNamesAndLevelsMap = featureNamesAndLevels(Option(specifiedFeatures).getOrElse(Collections.emptyList).asScala.toList) - val metadataVersion = getMetadataVersion(namespace, featureNamesAndLevelsMap, - Option(config.get.originals.get(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG)).map(_.toString)) - validateMetadataVersion(metadataVersion, config) - // Get all other features, validate, and create records for them - // Use latest default for features if --release-version is not specified - generateFeatureRecords( - metadataRecords, - metadataVersion, - featureNamesAndLevelsMap, - Features.PRODUCTION_FEATURES.asScala.toList, - config.get.unstableFeatureVersionsEnabled, - releaseVersionFlagSpecified - ) - getUserScramCredentialRecords(namespace).foreach(userScramCredentialRecords => { - if (!metadataVersion.isScramSupported) { - throw new TerseFailure(s"SCRAM is only supported in metadata.version ${MetadataVersion.IBP_3_5_IV2} or later.") - } - for (record <- userScramCredentialRecords) { - metadataRecords.append(new ApiMessageAndVersion(record, 0.toShort)) - } - }) - - val bootstrapMetadata = buildBootstrapMetadata(metadataVersion, Some(metadataRecords), "format command") - val ignoreFormatted = namespace.getBoolean("ignore_formatted") - if (!configToSelfManagedMode(config.get)) { - throw new TerseFailure("The kafka configuration file appears to be for " + - "a legacy cluster. Formatting is only supported for clusters in KRaft mode.") - } - - val commandResult = formatCommand(System.out, directories, metaProperties, bootstrapMetadata, - metadataVersion,ignoreFormatted) - if (command == "format") { - Exit.exit(commandResult) - } + exitCode = execute(args) + } catch { + case e: TerseFailure => + exitCode = 1 + message = Some(e.getMessage) + } + message.foreach(System.err.println) + Exit.exit(exitCode, message) + } - case "random-uuid" => - System.out.println(Uuid.randomUuid) - Exit.exit(0) + /** + * Executes the command according to the given arguments and returns the appropriate exit code. + * @param args The command line arguments + * @return The exit code + */ + def execute(args: Array[String]): Int = { + val namespace = parseArguments(args) + val command = namespace.getString("command") + val config = Option(namespace.getString("config")).flatMap( + p => Some(new KafkaConfig(Utils.loadProps(p)))) + command match { + case "info" => + val directories = configToLogDirectories(config.get) + val selfManagedMode = configToSelfManagedMode(config.get) + infoCommand(System.out, selfManagedMode, directories) + + case "format" => + runFormatCommand(namespace, config.get) + + case "random-uuid" => + System.out.println(Uuid.randomUuid) + 0 + case _ => + throw new RuntimeException(s"Unknown command $command") + } + } - case _ => - throw new RuntimeException(s"Unknown command $command") + /** + * Validates arguments, configuration, prepares bootstrap metadata and delegates to {{@link formatCommand}}. + * Visible for testing. + * @param namespace Arguments + * @param config The server configuration + * @return The exit code + */ + def runFormatCommand(namespace: Namespace, config: KafkaConfig) = { + val directories = configToLogDirectories(config) + val clusterId = namespace.getString("cluster_id") + val metaProperties = new MetaProperties.Builder(). + setVersion(MetaPropertiesVersion.V1). + setClusterId(clusterId). + setNodeId(config.nodeId). + build() + val metadataRecords : ArrayBuffer[ApiMessageAndVersion] = ArrayBuffer() + val specifiedFeatures: util.List[String] = namespace.getList("feature") + val releaseVersionFlagSpecified = namespace.getString("release_version") != null + if (releaseVersionFlagSpecified && specifiedFeatures != null) { + throw new TerseFailure("Both --release-version and --feature were set. Only one of the two flags can be set.") + } + val featureNamesAndLevelsMap = featureNamesAndLevels(Option(specifiedFeatures).getOrElse(Collections.emptyList).asScala.toList) + val metadataVersion = getMetadataVersion(namespace, featureNamesAndLevelsMap, + Option(config.originals.get(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG)).map(_.toString)) + validateMetadataVersion(metadataVersion, config) + // Get all other features, validate, and create records for them + // Use latest default for features if --release-version is not specified + generateFeatureRecords( + metadataRecords, + metadataVersion, + featureNamesAndLevelsMap, + Features.PRODUCTION_FEATURES.asScala.toList, + config.unstableFeatureVersionsEnabled, + releaseVersionFlagSpecified + ) + getUserScramCredentialRecords(namespace).foreach(userScramCredentialRecords => { + if (!metadataVersion.isScramSupported) { + throw new TerseFailure(s"SCRAM is only supported in metadata.version ${MetadataVersion.IBP_3_5_IV2} or later.") } - } catch { - case e: TerseFailure => - System.err.println(e.getMessage) - Exit.exit(1, Some(e.getMessage)) + for (record <- userScramCredentialRecords) { + metadataRecords.append(new ApiMessageAndVersion(record, 0.toShort)) + } + }) + val bootstrapMetadata = buildBootstrapMetadata(metadataVersion, Some(metadataRecords), "format command") + val ignoreFormatted = namespace.getBoolean("ignore_formatted") + if (!configToSelfManagedMode(config)) { + throw new TerseFailure("The kafka configuration file appears to be for " + + "a legacy cluster. Formatting is only supported for clusters in KRaft mode.") } + formatCommand(System.out, directories, metaProperties, bootstrapMetadata, + metadataVersion,ignoreFormatted) } - private def validateMetadataVersion(metadataVersion: MetadataVersion, config: Option[KafkaConfig]): Unit = { + private def validateMetadataVersion(metadataVersion: MetadataVersion, config: KafkaConfig): Unit = { if (!metadataVersion.isKRaftSupported) { throw new TerseFailure(s"Must specify a valid KRaft metadata.version of at least ${MetadataVersion.IBP_3_0_IV0}.") } if (!metadataVersion.isProduction) { - if (config.get.unstableFeatureVersionsEnabled) { + if (config.unstableFeatureVersionsEnabled) { System.out.println(s"WARNING: using pre-production metadata.version $metadataVersion.") } else { throw new TerseFailure(s"The metadata.version $metadataVersion is not ready for production use yet.") } } + try { + config.validateWithMetadataVersion(metadataVersion) + } catch { + case e: IllegalArgumentException => throw new TerseFailure(s"Invalid configuration for metadata version: ${e.getMessage}") + } } private[tools] def generateFeatureRecords(metadataRecords: ArrayBuffer[ApiMessageAndVersion], @@ -521,7 +545,6 @@ object StorageTool extends Logging { metaPropertiesEnsemble.verify(metaProperties.clusterId(), metaProperties.nodeId(), util.EnumSet.noneOf(classOf[VerificationFlag])) - System.out.println(s"metaPropertiesEnsemble=$metaPropertiesEnsemble") val copier = new MetaPropertiesEnsemble.Copier(metaPropertiesEnsemble) if (!(ignoreFormatted || copier.logDirProps().isEmpty)) { val firstLogDir = copier.logDirProps().keySet().iterator().next() diff --git a/core/src/main/scala/kafka/tools/TestRaftServer.scala b/core/src/main/scala/kafka/tools/TestRaftServer.scala index d357ad0bd5..0acae6c5dc 100644 --- a/core/src/main/scala/kafka/tools/TestRaftServer.scala +++ b/core/src/main/scala/kafka/tools/TestRaftServer.scala @@ -95,6 +95,7 @@ class TestRaftServer( metrics, Some(threadNamePrefix), CompletableFuture.completedFuture(QuorumConfig.parseVoterConnections(config.quorumVoters)), + QuorumConfig.parseBootstrapServers(config.quorumBootstrapServers), new ProcessTerminatingFaultHandler.Builder().build() ) diff --git a/core/src/main/scala/kafka/utils/CoreUtils.scala b/core/src/main/scala/kafka/utils/CoreUtils.scala index 8da7a4e7cc..8403f5245e 100755 --- a/core/src/main/scala/kafka/utils/CoreUtils.scala +++ b/core/src/main/scala/kafka/utils/CoreUtils.scala @@ -114,7 +114,7 @@ object CoreUtils { * Create an instance of the class with the given class name */ def createObject[T <: AnyRef](className: String, args: AnyRef*): T = { - val klass = Class.forName(className, true, Utils.getContextOrKafkaClassLoader).asInstanceOf[Class[T]] + val klass = Utils.loadClass(className, classOf[Object]).asInstanceOf[Class[T]] val constructor = klass.getConstructor(args.map(_.getClass): _*) constructor.newInstance(args: _*) } diff --git a/core/src/main/scala/kafka/zk/AdminZkClient.scala b/core/src/main/scala/kafka/zk/AdminZkClient.scala index efecfe854b..cd9153c07d 100644 --- a/core/src/main/scala/kafka/zk/AdminZkClient.scala +++ b/core/src/main/scala/kafka/zk/AdminZkClient.scala @@ -163,7 +163,7 @@ class AdminZkClient(zkClient: KafkaZkClient, LogConfig.validate(config, kafkaConfig.map(_.extractLogConfigMap).getOrElse(Collections.emptyMap()), - kafkaConfig.exists(_.isRemoteLogStorageSystemEnabled)) + kafkaConfig.exists(_.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) } private def writeTopicPartitionAssignment(topic: String, replicaAssignment: Map[Int, ReplicaAssignment], @@ -481,7 +481,7 @@ class AdminZkClient(zkClient: KafkaZkClient, // remove the topic overrides LogConfig.validate(configs, kafkaConfig.map(_.extractLogConfigMap).getOrElse(Collections.emptyMap()), - kafkaConfig.exists(_.isRemoteLogStorageSystemEnabled)) + kafkaConfig.exists(_.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) } /** diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index 72154c9fc2..eeac6bdbae 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -17,9 +17,11 @@ package kafka.log.remote; import com.yammer.metrics.core.Gauge; +import com.yammer.metrics.core.MetricName; import kafka.cluster.EndPoint; import kafka.cluster.Partition; import kafka.log.UnifiedLog; +import kafka.log.remote.quota.RLMQuotaManager; import kafka.log.remote.quota.RLMQuotaManagerConfig; import kafka.server.BrokerTopicStats; import kafka.server.KafkaConfig; @@ -29,7 +31,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.compress.Compression; -import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.errors.ReplicaNotAvailableException; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.network.ListenerName; @@ -60,14 +61,15 @@ import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType; import org.apache.kafka.server.metrics.KafkaMetricsGroup; import org.apache.kafka.server.metrics.KafkaYammerMetrics; -import org.apache.kafka.storage.internals.checkpoint.InMemoryLeaderEpochCheckpoint; -import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint; +import org.apache.kafka.server.util.MockScheduler; +import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile; import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache; import org.apache.kafka.storage.internals.log.EpochEntry; import org.apache.kafka.storage.internals.log.FetchDataInfo; import org.apache.kafka.storage.internals.log.FetchIsolation; import org.apache.kafka.storage.internals.log.LazyIndex; import org.apache.kafka.storage.internals.log.LogConfig; +import org.apache.kafka.storage.internals.log.LogDirFailureChannel; import org.apache.kafka.storage.internals.log.LogFileUtils; import org.apache.kafka.storage.internals.log.LogSegment; import org.apache.kafka.storage.internals.log.OffsetIndex; @@ -86,19 +88,24 @@ import org.mockito.InOrder; import org.mockito.MockedConstruction; import org.mockito.Mockito; +import org.opentest4j.AssertionFailedError; import scala.Option; import scala.collection.JavaConverters; +import java.io.BufferedReader; import java.io.ByteArrayInputStream; import java.io.File; import java.io.InputStream; import java.io.FileInputStream; import java.io.IOException; +import java.io.InputStreamReader; +import java.io.UncheckedIOException; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; +import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -128,9 +135,13 @@ import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND; import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_NUM; import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_SIZE_SECONDS; +import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND; +import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_NUM; +import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_SIZE_SECONDS; import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_MANAGER_CONFIG_PREFIX; import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.DEFAULT_REMOTE_STORAGE_MANAGER_CONFIG_PREFIX; import static org.apache.kafka.server.log.remote.storage.RemoteStorageMetrics.REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT_METRIC; +import static org.apache.kafka.server.log.remote.storage.RemoteStorageMetrics.REMOTE_LOG_READER_FETCH_RATE_AND_TIME_METRIC; import static org.apache.kafka.server.log.remote.storage.RemoteStorageMetrics.REMOTE_STORAGE_THREAD_POOL_METRICS; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -138,6 +149,7 @@ import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; @@ -145,6 +157,7 @@ import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doThrow; @@ -179,7 +192,8 @@ public class RemoteLogManagerTest { private final RemoteStorageManager remoteStorageManager = mock(RemoteStorageManager.class); private final RemoteLogMetadataManager remoteLogMetadataManager = mock(RemoteLogMetadataManager.class); - private RemoteLogManagerConfig remoteLogManagerConfig = null; + private final RLMQuotaManager rlmCopyQuotaManager = mock(RLMQuotaManager.class); + private KafkaConfig config; private BrokerTopicStats brokerTopicStats = null; private final Metrics metrics = new Metrics(time); @@ -194,34 +208,26 @@ public class RemoteLogManagerTest { private final EpochEntry epochEntry1 = new EpochEntry(1, 100); private final EpochEntry epochEntry2 = new EpochEntry(2, 200); private final List totalEpochEntries = Arrays.asList(epochEntry0, epochEntry1, epochEntry2); - private final LeaderEpochCheckpoint checkpoint = new LeaderEpochCheckpoint() { - List epochs = Collections.emptyList(); - - @Override - public void write(Collection epochs, boolean ignored) { - this.epochs = new ArrayList<>(epochs); - } - - @Override - public List read() { - return epochs; - } - }; + private LeaderEpochCheckpointFile checkpoint; private final AtomicLong currentLogStartOffset = new AtomicLong(0L); private UnifiedLog mockLog = mock(UnifiedLog.class); + private final MockScheduler scheduler = new MockScheduler(time); + @BeforeEach void setUp() throws Exception { + checkpoint = new LeaderEpochCheckpointFile(TestUtils.tempFile(), new LogDirFailureChannel(1)); topicIds.put(leaderTopicIdPartition.topicPartition().topic(), leaderTopicIdPartition.topicId()); topicIds.put(followerTopicIdPartition.topicPartition().topic(), followerTopicIdPartition.topicId()); Properties props = kafka.utils.TestUtils.createDummyBrokerConfig(); props.setProperty(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, "true"); props.setProperty(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_TASK_INTERVAL_MS_PROP, "100"); - remoteLogManagerConfig = createRLMConfig(props); - brokerTopicStats = new BrokerTopicStats(KafkaConfig.fromProps(props).isRemoteLogStorageSystemEnabled()); + createRLMConfig(props); + config = KafkaConfig.fromProps(props); + brokerTopicStats = new BrokerTopicStats(KafkaConfig.fromProps(props).remoteLogManagerConfig().isRemoteStorageSystemEnabled()); - remoteLogManager = new RemoteLogManager(remoteLogManagerConfig, brokerId, logDir, clusterId, time, + remoteLogManager = new RemoteLogManager(config, brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> currentLogStartOffset.set(offset), brokerTopicStats, metrics) { @@ -231,6 +237,12 @@ public RemoteStorageManager createRemoteStorageManager() { public RemoteLogMetadataManager createRemoteLogMetadataManager() { return remoteLogMetadataManager; } + public RLMQuotaManager createRLMCopyQuotaManager() { + return rlmCopyQuotaManager; + } + public Duration quotaTimeout() { + return Duration.ofMillis(100); + } @Override long findLogStartOffset(TopicIdPartition topicIdPartition, UnifiedLog log) { return 0L; @@ -250,13 +262,11 @@ void tearDown() { @Test void testGetLeaderEpochCheckpoint() { checkpoint.write(totalEpochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); - InMemoryLeaderEpochCheckpoint inMemoryCheckpoint = remoteLogManager.getLeaderEpochCheckpoint(mockLog, 0, 300); - assertEquals(totalEpochEntries, inMemoryCheckpoint.read()); + assertEquals(totalEpochEntries, remoteLogManager.getLeaderEpochEntries(mockLog, 0, 300)); - InMemoryLeaderEpochCheckpoint inMemoryCheckpoint2 = remoteLogManager.getLeaderEpochCheckpoint(mockLog, 100, 200); - List epochEntries = inMemoryCheckpoint2.read(); + List epochEntries = remoteLogManager.getLeaderEpochEntries(mockLog, 100, 200); assertEquals(1, epochEntries.size()); assertEquals(epochEntry1, epochEntries.get(0)); } @@ -268,7 +278,7 @@ void testFindHighestRemoteOffsetOnEmptyRemoteStorage() throws RemoteStorageExce new EpochEntry(1, 500) ); checkpoint.write(totalEpochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); TopicIdPartition tpId = new TopicIdPartition(Uuid.randomUuid(), tp); OffsetAndEpoch offsetAndEpoch = remoteLogManager.findHighestRemoteOffset(tpId, mockLog); @@ -282,7 +292,7 @@ void testFindHighestRemoteOffset() throws RemoteStorageException { new EpochEntry(1, 500) ); checkpoint.write(totalEpochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); TopicIdPartition tpId = new TopicIdPartition(Uuid.randomUuid(), tp); when(remoteLogMetadataManager.highestOffsetForEpoch(eq(tpId), anyInt())).thenAnswer(ans -> { @@ -305,7 +315,7 @@ void testFindHighestRemoteOffsetWithUncleanLeaderElection() throws RemoteStorage new EpochEntry(2, 300) ); checkpoint.write(totalEpochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); TopicIdPartition tpId = new TopicIdPartition(Uuid.randomUuid(), tp); when(remoteLogMetadataManager.highestOffsetForEpoch(eq(tpId), anyInt())).thenAnswer(ans -> { @@ -369,10 +379,13 @@ void testRemoteLogMetadataManagerWithEndpointConfig() { @Test void testRemoteLogMetadataManagerWithEndpointConfigOverridden() throws IOException { Properties props = new Properties(); + props.put("zookeeper.connect", kafka.utils.TestUtils.MockZkConnect()); // override common security.protocol by adding "RLMM prefix" and "remote log metadata common client prefix" props.put(DEFAULT_REMOTE_LOG_METADATA_MANAGER_CONFIG_PREFIX + REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "security.protocol", "SSL"); + createRLMConfig(props); + KafkaConfig config = KafkaConfig.fromProps(props); try (RemoteLogManager remoteLogManager = new RemoteLogManager( - createRLMConfig(props), + config, brokerId, logDir, clusterId, @@ -467,7 +480,7 @@ private void assertCopyExpectedLogSegmentsToRemote(long oldSegmentStartOffset, // leader epoch preparation checkpoint.write(totalEpochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())).thenReturn(Optional.of(-1L)); @@ -581,7 +594,7 @@ void testCustomMetadataSizeExceedsLimit() throws Exception { // leader epoch preparation checkpoint.write(totalEpochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())).thenReturn(Optional.of(-1L)); @@ -681,7 +694,7 @@ void testRemoteLogManagerTasksAvgIdlePercentAndMetadataCountMetrics() throws Exc // leader epoch preparation checkpoint.write(totalEpochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())).thenReturn(Optional.of(0L)); @@ -800,7 +813,7 @@ void testRemoteLogTaskUpdateRemoteLogSegmentMetadataAfterLogDirChanged() throws // leader epoch preparation checkpoint.write(totalEpochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())) .thenReturn(Optional.of(0L)) @@ -914,7 +927,7 @@ void testRemoteLogManagerRemoteMetrics() throws Exception { // leader epoch preparation checkpoint.write(totalEpochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())).thenReturn(Optional.of(0L)); @@ -1064,7 +1077,7 @@ void testMetricsUpdateOnCopyLogSegmentsFailure() throws Exception { // leader epoch preparation checkpoint.write(totalEpochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())).thenReturn(Optional.of(0L)); @@ -1137,7 +1150,7 @@ void testCopyLogSegmentsToRemoteShouldNotCopySegmentForFollower() throws Excepti // leader epoch preparation checkpoint.write(totalEpochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())).thenReturn(Optional.of(0L)); @@ -1173,7 +1186,7 @@ void testRLMTaskDoesNotUploadSegmentsWhenRemoteLogMetadataManagerIsNotInitialize // leader epoch preparation checkpoint.write(totalEpochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); // Throw a retryable exception so indicate that the remote log metadata manager is not initialized yet @@ -1255,16 +1268,14 @@ private void verifyLogSegmentData(LogSegmentData logSegmentData, assertEquals(tempFile.getAbsolutePath(), logSegmentData.logSegment().toAbsolutePath().toString()); assertEquals(mockProducerSnapshotIndex.getAbsolutePath(), logSegmentData.producerSnapshotIndex().toAbsolutePath().toString()); - InMemoryLeaderEpochCheckpoint inMemoryLeaderEpochCheckpoint = new InMemoryLeaderEpochCheckpoint(); - inMemoryLeaderEpochCheckpoint.write(expectedLeaderEpoch); - assertEquals(inMemoryLeaderEpochCheckpoint.readAsByteBuffer(), logSegmentData.leaderEpochIndex()); + assertEquals(RemoteLogManager.epochEntriesAsByteBuffer(expectedLeaderEpoch), logSegmentData.leaderEpochIndex()); } @Test void testGetClassLoaderAwareRemoteStorageManager() throws Exception { ClassLoaderAwareRemoteStorageManager rsmManager = mock(ClassLoaderAwareRemoteStorageManager.class); try (RemoteLogManager remoteLogManager = - new RemoteLogManager(remoteLogManagerConfig, brokerId, logDir, clusterId, time, + new RemoteLogManager(config, brokerId, logDir, clusterId, time, t -> Optional.empty(), (topicPartition, offset) -> { }, brokerTopicStats, metrics) { @@ -1376,7 +1387,7 @@ void testFindOffsetByTimestamp() throws IOException, RemoteStorageException { TreeMap validSegmentEpochs = new TreeMap<>(); validSegmentEpochs.put(targetLeaderEpoch, startOffset); - LeaderEpochFileCache leaderEpochFileCache = new LeaderEpochFileCache(tp, checkpoint); + LeaderEpochFileCache leaderEpochFileCache = new LeaderEpochFileCache(tp, checkpoint, scheduler); leaderEpochFileCache.assign(4, 99L); leaderEpochFileCache.assign(5, 99L); leaderEpochFileCache.assign(targetLeaderEpoch, startOffset); @@ -1411,7 +1422,7 @@ void testFindOffsetByTimestampWithInvalidEpochSegments() throws IOException, Rem validSegmentEpochs.put(targetLeaderEpoch - 1, startOffset - 1); // invalid epochs not aligning with leader epoch cache validSegmentEpochs.put(targetLeaderEpoch, startOffset); - LeaderEpochFileCache leaderEpochFileCache = new LeaderEpochFileCache(tp, checkpoint); + LeaderEpochFileCache leaderEpochFileCache = new LeaderEpochFileCache(tp, checkpoint, scheduler); leaderEpochFileCache.assign(4, 99L); leaderEpochFileCache.assign(5, 99L); leaderEpochFileCache.assign(targetLeaderEpoch, startOffset); @@ -1442,7 +1453,7 @@ void testFindOffsetByTimestampWithSegmentNotReady() throws IOException, RemoteSt TreeMap validSegmentEpochs = new TreeMap<>(); validSegmentEpochs.put(targetLeaderEpoch, startOffset); - LeaderEpochFileCache leaderEpochFileCache = new LeaderEpochFileCache(tp, checkpoint); + LeaderEpochFileCache leaderEpochFileCache = new LeaderEpochFileCache(tp, checkpoint, scheduler); leaderEpochFileCache.assign(4, 99L); leaderEpochFileCache.assign(5, 99L); leaderEpochFileCache.assign(targetLeaderEpoch, startOffset); @@ -1526,7 +1537,7 @@ void testIdempotentClose() throws IOException { public void testRemoveMetricsOnClose() throws IOException { MockedConstruction mockMetricsGroupCtor = mockConstruction(KafkaMetricsGroup.class); try { - RemoteLogManager remoteLogManager = new RemoteLogManager(remoteLogManagerConfig, brokerId, logDir, clusterId, + RemoteLogManager remoteLogManager = new RemoteLogManager(config, brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, brokerTopicStats, metrics) { public RemoteStorageManager createRemoteStorageManager() { return remoteStorageManager; @@ -1542,10 +1553,13 @@ public RemoteLogMetadataManager createRemoteLogMetadataManager() { KafkaMetricsGroup mockRlmMetricsGroup = mockMetricsGroupCtor.constructed().get(0); KafkaMetricsGroup mockThreadPoolMetricsGroup = mockMetricsGroupCtor.constructed().get(1); - List remoteLogManagerMetricNames = Collections.singletonList(REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT_METRIC.getName()); + List remoteLogManagerMetricNames = Arrays.asList( + REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT_METRIC, + REMOTE_LOG_READER_FETCH_RATE_AND_TIME_METRIC); Set remoteStorageThreadPoolMetricNames = REMOTE_STORAGE_THREAD_POOL_METRICS; - verify(mockRlmMetricsGroup, times(remoteLogManagerMetricNames.size())).newGauge(anyString(), any()); + verify(mockRlmMetricsGroup, times(1)).newGauge(any(MetricName.class), any()); + verify(mockRlmMetricsGroup, times(1)).newTimer(any(MetricName.class), any(), any()); // Verify that the RemoteLogManager metrics are removed remoteLogManagerMetricNames.forEach(metricName -> verify(mockRlmMetricsGroup).removeMetric(metricName)); @@ -1899,7 +1913,7 @@ public void testFindLogStartOffset() throws RemoteStorageException, IOException epochEntries.add(new EpochEntry(2, 550L)); checkpoint.write(epochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); long timestamp = time.milliseconds(); @@ -1918,7 +1932,7 @@ public void testFindLogStartOffset() throws RemoteStorageException, IOException else return Collections.emptyIterator(); }); - try (RemoteLogManager remoteLogManager = new RemoteLogManager(remoteLogManagerConfig, brokerId, logDir, clusterId, time, + try (RemoteLogManager remoteLogManager = new RemoteLogManager(config, brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, brokerTopicStats, metrics) { @@ -1937,13 +1951,13 @@ public void testFindLogStartOffsetFallbackToLocalLogStartOffsetWhenRemoteIsEmpty epochEntries.add(new EpochEntry(2, 550L)); checkpoint.write(epochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); when(mockLog.localLogStartOffset()).thenReturn(250L); when(remoteLogMetadataManager.listRemoteLogSegments(eq(leaderTopicIdPartition), anyInt())) .thenReturn(Collections.emptyIterator()); - try (RemoteLogManager remoteLogManager = new RemoteLogManager(remoteLogManagerConfig, brokerId, logDir, clusterId, time, + try (RemoteLogManager remoteLogManager = new RemoteLogManager(config, brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, brokerTopicStats, metrics) { @@ -1962,7 +1976,7 @@ public void testLogStartOffsetUpdatedOnStartup() throws RemoteStorageException, epochEntries.add(new EpochEntry(2, 550L)); checkpoint.write(epochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); RemoteLogSegmentMetadata metadata = mock(RemoteLogSegmentMetadata.class); @@ -1977,7 +1991,7 @@ public void testLogStartOffsetUpdatedOnStartup() throws RemoteStorageException, }); AtomicLong logStartOffset = new AtomicLong(0); - try (RemoteLogManager remoteLogManager = new RemoteLogManager(remoteLogManagerConfig, brokerId, logDir, clusterId, time, + try (RemoteLogManager remoteLogManager = new RemoteLogManager(config, brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> logStartOffset.set(offset), brokerTopicStats, metrics) { @@ -2005,7 +2019,7 @@ public void testDeletionOnRetentionBreachedSegments(long retentionSize, List epochEntries = Collections.singletonList(epochEntry0); checkpoint.write(epochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition()); @@ -2045,6 +2059,75 @@ public void testDeletionOnRetentionBreachedSegments(long retentionSize, assertEquals(0, brokerTopicStats.allTopicsStats().failedRemoteDeleteRequestRate().count()); } + @ParameterizedTest(name = "testDeletionOnOverlappingRetentionBreachedSegments retentionSize={0} retentionMs={1}") + @CsvSource(value = {"0, -1", "-1, 0"}) + public void testDeletionOnOverlappingRetentionBreachedSegments(long retentionSize, + long retentionMs) + throws RemoteStorageException, ExecutionException, InterruptedException { + Map logProps = new HashMap<>(); + logProps.put("retention.bytes", retentionSize); + logProps.put("retention.ms", retentionMs); + LogConfig mockLogConfig = new LogConfig(logProps); + when(mockLog.config()).thenReturn(mockLogConfig); + + List epochEntries = Collections.singletonList(epochEntry0); + checkpoint.write(epochEntries); + LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); + when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + + when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition()); + when(mockLog.logEndOffset()).thenReturn(200L); + + RemoteLogSegmentMetadata metadata1 = listRemoteLogSegmentMetadata(leaderTopicIdPartition, 1, 100, 1024, + epochEntries, RemoteLogSegmentState.COPY_SEGMENT_FINISHED) + .get(0); + // overlapping segment + RemoteLogSegmentMetadata metadata2 = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(leaderTopicIdPartition, Uuid.randomUuid()), + metadata1.startOffset(), metadata1.endOffset() + 5, metadata1.maxTimestampMs(), + metadata1.brokerId() + 1, metadata1.eventTimestampMs(), metadata1.segmentSizeInBytes() + 128, + metadata1.customMetadata(), metadata1.state(), metadata1.segmentLeaderEpochs()); + + // When there are overlapping/duplicate segments, the RemoteLogMetadataManager#listRemoteLogSegments + // returns the segments in order of (valid ++ unreferenced) segments: + // (eg) B0 uploaded segment S0 with offsets 0-100 and B1 uploaded segment S1 with offsets 0-200. + // We will mark the segment S0 as duplicate and add it to unreferencedSegmentIds. + // The order of segments returned by listRemoteLogSegments will be S1, S0. + // While computing the next-log-start-offset, taking the max of deleted segment's end-offset + 1. + List metadataList = new ArrayList<>(); + metadataList.add(metadata2); + metadataList.add(metadata1); + + when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition)) + .thenReturn(metadataList.iterator()); + when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition, 0)) + .thenAnswer(ans -> metadataList.iterator()); + when(remoteLogMetadataManager.updateRemoteLogSegmentMetadata(any(RemoteLogSegmentMetadataUpdate.class))) + .thenReturn(CompletableFuture.runAsync(() -> { })); + + // Verify the metrics for remote deletes and for failures is zero before attempt to delete segments + assertEquals(0, brokerTopicStats.topicStats(leaderTopicIdPartition.topic()).remoteDeleteRequestRate().count()); + assertEquals(0, brokerTopicStats.topicStats(leaderTopicIdPartition.topic()).failedRemoteDeleteRequestRate().count()); + // Verify aggregate metrics + assertEquals(0, brokerTopicStats.allTopicsStats().remoteDeleteRequestRate().count()); + assertEquals(0, brokerTopicStats.allTopicsStats().failedRemoteDeleteRequestRate().count()); + + RemoteLogManager.RLMTask task = remoteLogManager.new RLMTask(leaderTopicIdPartition, 128); + task.convertToLeader(0); + task.cleanupExpiredRemoteLogSegments(); + + assertEquals(metadata2.endOffset() + 1, currentLogStartOffset.get()); + verify(remoteStorageManager).deleteLogSegmentData(metadataList.get(0)); + verify(remoteStorageManager).deleteLogSegmentData(metadataList.get(1)); + + // Verify the metric for remote delete is updated correctly + assertEquals(2, brokerTopicStats.topicStats(leaderTopicIdPartition.topic()).remoteDeleteRequestRate().count()); + // Verify we did not report any failure for remote deletes + assertEquals(0, brokerTopicStats.topicStats(leaderTopicIdPartition.topic()).failedRemoteDeleteRequestRate().count()); + // Verify aggregate metrics + assertEquals(2, brokerTopicStats.allTopicsStats().remoteDeleteRequestRate().count()); + assertEquals(0, brokerTopicStats.allTopicsStats().failedRemoteDeleteRequestRate().count()); + } + @ParameterizedTest(name = "testRemoteDeleteLagsOnRetentionBreachedSegments retentionSize={0} retentionMs={1}") @CsvSource(value = {"0, -1", "-1, 0"}) public void testRemoteDeleteLagsOnRetentionBreachedSegments(long retentionSize, @@ -2058,7 +2141,7 @@ public void testRemoteDeleteLagsOnRetentionBreachedSegments(long retentionSize, List epochEntries = Collections.singletonList(epochEntry0); checkpoint.write(epochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition()); @@ -2129,7 +2212,7 @@ public void testDeleteRetentionMsBeingCancelledBeforeSecondDelete() throws Remot .thenAnswer(ans -> metadataList.iterator()); checkpoint.write(epochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); Map logProps = new HashMap<>(); @@ -2191,7 +2274,7 @@ public void testFailedDeleteExpiredSegments(long retentionSize, List epochEntries = Collections.singletonList(epochEntry0); checkpoint.write(epochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition()); @@ -2244,7 +2327,7 @@ public void testDeleteLogSegmentDueToRetentionSizeBreach(int segmentCount, new EpochEntry(4, 100L) ); checkpoint.write(epochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); int currentLeaderEpoch = epochEntries.get(epochEntries.size() - 1).epoch; long localLogSegmentsSize = 512L; @@ -2282,7 +2365,7 @@ public void testDeleteLogSegmentDueToRetentionTimeBreach(int segmentCount, new EpochEntry(4, 100L) ); checkpoint.write(epochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); int currentLeaderEpoch = epochEntries.get(epochEntries.size() - 1).epoch; long localLogSegmentsSize = 512L; @@ -2340,7 +2423,7 @@ private void verifyDeleteLogSegment(List segmentMetada @Test public void testDeleteRetentionMsOnExpiredSegment() throws RemoteStorageException, IOException { AtomicLong logStartOffset = new AtomicLong(0); - try (RemoteLogManager remoteLogManager = new RemoteLogManager(remoteLogManagerConfig, brokerId, logDir, clusterId, time, + try (RemoteLogManager remoteLogManager = new RemoteLogManager(config, brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> logStartOffset.set(offset), brokerTopicStats, metrics) { @@ -2369,7 +2452,7 @@ public RemoteLogMetadataManager createRemoteLogMetadataManager() { .thenReturn(remoteLogSegmentMetadatas.iterator()); checkpoint.write(epochEntries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint); + LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); Map logProps = new HashMap<>(); @@ -2444,11 +2527,17 @@ private List listRemoteLogSegmentMetadataByTime(TopicI private Map truncateAndGetLeaderEpochs(List entries, Long startOffset, Long endOffset) { - InMemoryLeaderEpochCheckpoint myCheckpoint = new InMemoryLeaderEpochCheckpoint(); + LeaderEpochCheckpointFile myCheckpoint; + try { + myCheckpoint = new LeaderEpochCheckpointFile( + TestUtils.tempFile(), new LogDirFailureChannel(1)); + } catch (IOException e) { + throw new UncheckedIOException(e); + } myCheckpoint.write(entries); - LeaderEpochFileCache cache = new LeaderEpochFileCache(null, myCheckpoint); - cache.truncateFromStart(startOffset); - cache.truncateFromEnd(endOffset); + LeaderEpochFileCache cache = new LeaderEpochFileCache(null, myCheckpoint, scheduler); + cache.truncateFromStartAsyncFlush(startOffset); + cache.truncateFromEndAsyncFlush(endOffset); return myCheckpoint.read().stream().collect(Collectors.toMap(e -> e.epoch, e -> e.startOffset)); } @@ -2476,7 +2565,7 @@ public void testReadForMissingFirstBatchInRemote() throws RemoteStorageException ); try (RemoteLogManager remoteLogManager = new RemoteLogManager( - remoteLogManagerConfig, + config, brokerId, logDir, clusterId, @@ -2549,7 +2638,7 @@ public void testReadForFirstBatchMoreThanMaxFetchBytes(boolean minOneMessage) th ); try (RemoteLogManager remoteLogManager = new RemoteLogManager( - remoteLogManagerConfig, + config, brokerId, logDir, clusterId, @@ -2634,7 +2723,7 @@ public void testReadForFirstBatchInLogCompaction() throws RemoteStorageException try (RemoteLogManager remoteLogManager = new RemoteLogManager( - remoteLogManagerConfig, + config, brokerId, logDir, clusterId, @@ -2675,47 +2764,292 @@ int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, l } } - + @Test public void testCopyQuotaManagerConfig() { Properties defaultProps = new Properties(); - RemoteLogManagerConfig defaultRlmConfig = createRLMConfig(defaultProps); + defaultProps.put("zookeeper.connect", kafka.utils.TestUtils.MockZkConnect()); + createRLMConfig(defaultProps); + KafkaConfig defaultRlmConfig = KafkaConfig.fromProps(defaultProps); RLMQuotaManagerConfig defaultConfig = RemoteLogManager.copyQuotaManagerConfig(defaultRlmConfig); assertEquals(DEFAULT_REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND, defaultConfig.quotaBytesPerSecond()); assertEquals(DEFAULT_REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_NUM, defaultConfig.numQuotaSamples()); assertEquals(DEFAULT_REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_SIZE_SECONDS, defaultConfig.quotaWindowSizeSeconds()); Properties customProps = new Properties(); + customProps.put("zookeeper.connect", kafka.utils.TestUtils.MockZkConnect()); customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP, 100); customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_NUM_PROP, 31); customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_SIZE_SECONDS_PROP, 1); - RemoteLogManagerConfig rlmConfig = createRLMConfig(customProps); - RLMQuotaManagerConfig rlmCopyQuotaManagerConfig = RemoteLogManager.copyQuotaManagerConfig(rlmConfig); + createRLMConfig(customProps); + KafkaConfig config = KafkaConfig.fromProps(customProps); + + RLMQuotaManagerConfig rlmCopyQuotaManagerConfig = RemoteLogManager.copyQuotaManagerConfig(config); assertEquals(100L, rlmCopyQuotaManagerConfig.quotaBytesPerSecond()); assertEquals(31, rlmCopyQuotaManagerConfig.numQuotaSamples()); assertEquals(1, rlmCopyQuotaManagerConfig.quotaWindowSizeSeconds()); } - + @Test public void testFetchQuotaManagerConfig() { Properties defaultProps = new Properties(); - RemoteLogManagerConfig defaultRlmConfig = createRLMConfig(defaultProps); + defaultProps.put("zookeeper.connect", kafka.utils.TestUtils.MockZkConnect()); + createRLMConfig(defaultProps); + KafkaConfig defaultRlmConfig = KafkaConfig.fromProps(defaultProps); + RLMQuotaManagerConfig defaultConfig = RemoteLogManager.fetchQuotaManagerConfig(defaultRlmConfig); - assertEquals(DEFAULT_REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND, defaultConfig.quotaBytesPerSecond()); - assertEquals(DEFAULT_REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_NUM, defaultConfig.numQuotaSamples()); - assertEquals(DEFAULT_REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_SIZE_SECONDS, defaultConfig.quotaWindowSizeSeconds()); + assertEquals(DEFAULT_REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND, defaultConfig.quotaBytesPerSecond()); + assertEquals(DEFAULT_REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_NUM, defaultConfig.numQuotaSamples()); + assertEquals(DEFAULT_REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_SIZE_SECONDS, defaultConfig.quotaWindowSizeSeconds()); Properties customProps = new Properties(); + customProps.put("zookeeper.connect", kafka.utils.TestUtils.MockZkConnect()); customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP, 100); customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_NUM_PROP, 31); customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_SIZE_SECONDS_PROP, 1); - RemoteLogManagerConfig rlmConfig = createRLMConfig(customProps); + createRLMConfig(customProps); + KafkaConfig rlmConfig = KafkaConfig.fromProps(customProps); RLMQuotaManagerConfig rlmFetchQuotaManagerConfig = RemoteLogManager.fetchQuotaManagerConfig(rlmConfig); assertEquals(100L, rlmFetchQuotaManagerConfig.quotaBytesPerSecond()); assertEquals(31, rlmFetchQuotaManagerConfig.numQuotaSamples()); assertEquals(1, rlmFetchQuotaManagerConfig.quotaWindowSizeSeconds()); } + @Test + public void testEpochEntriesAsByteBuffer() throws Exception { + int expectedEpoch = 0; + long expectedStartOffset = 1L; + int expectedVersion = 0; + List epochs = Arrays.asList(new EpochEntry(expectedEpoch, expectedStartOffset)); + ByteBuffer buffer = RemoteLogManager.epochEntriesAsByteBuffer(epochs); + BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(new ByteArrayInputStream(buffer.array()), StandardCharsets.UTF_8)); + + assertEquals(String.valueOf(expectedVersion), bufferedReader.readLine()); + assertEquals(String.valueOf(epochs.size()), bufferedReader.readLine()); + assertEquals(expectedEpoch + " " + expectedStartOffset, bufferedReader.readLine()); + } + + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testCopyQuota(boolean quotaExceeded) throws Exception { + RemoteLogManager.RLMTask task = setupRLMTask(quotaExceeded); + + if (quotaExceeded) { + // Verify that the copy operation times out, since no segments can be copied due to quota being exceeded + assertThrows(AssertionFailedError.class, () -> assertTimeoutPreemptively(Duration.ofMillis(200), () -> task.copyLogSegmentsToRemote(mockLog))); + + // Verify the highest offset in remote storage is updated only once + ArgumentCaptor capture = ArgumentCaptor.forClass(Long.class); + verify(mockLog, times(1)).updateHighestOffsetInRemoteStorage(capture.capture()); + // Verify the highest offset in remote storage was -1L before the copy started + assertEquals(-1L, capture.getValue()); + } else { + // Verify the copy operation completes within the timeout, since it does not need to wait for quota availability + assertTimeoutPreemptively(Duration.ofMillis(100), () -> task.copyLogSegmentsToRemote(mockLog)); + + // Verify quota check was performed + verify(rlmCopyQuotaManager, times(1)).isQuotaExceeded(); + // Verify bytes to copy was recorded with the quota manager + verify(rlmCopyQuotaManager, times(1)).record(10); + + // Verify the highest offset in remote storage is updated + ArgumentCaptor capture = ArgumentCaptor.forClass(Long.class); + verify(mockLog, times(2)).updateHighestOffsetInRemoteStorage(capture.capture()); + List capturedValues = capture.getAllValues(); + // Verify the highest offset in remote storage was -1L before the copy + assertEquals(-1L, capturedValues.get(0).longValue()); + // Verify it was updated to 149L after the copy + assertEquals(149L, capturedValues.get(1).longValue()); + } + } + + @Test + public void testRLMShutdownDuringQuotaExceededScenario() throws Exception { + remoteLogManager.startup(); + setupRLMTask(true); + remoteLogManager.onLeadershipChange( + Collections.singleton(mockPartition(leaderTopicIdPartition)), Collections.emptySet(), topicIds); + // Ensure the copy operation is waiting for quota to be available + TestUtils.waitForCondition(() -> { + verify(rlmCopyQuotaManager, atLeast(1)).isQuotaExceeded(); + return true; + }, "Quota exceeded check did not happen"); + // Verify RLM is able to shut down + assertTimeoutPreemptively(Duration.ofMillis(100), () -> remoteLogManager.close()); + } + + // helper method to set up a RemoteLogManager.RLMTask for testing copy quota behaviour + private RemoteLogManager.RLMTask setupRLMTask(boolean quotaExceeded) throws RemoteStorageException, IOException { + long oldSegmentStartOffset = 0L; + long nextSegmentStartOffset = 150L; + + when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition()); + + // leader epoch preparation + checkpoint.write(totalEpochEntries); + LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); + when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.parentDir()).thenReturn("dir1"); + when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())).thenReturn(Optional.of(0L)); + + // create 2 log segments, with 0 and 150 as log start offset + LogSegment oldSegment = mock(LogSegment.class); + LogSegment activeSegment = mock(LogSegment.class); + + File tempFile = TestUtils.tempFile(); + FileRecords fileRecords = mock(FileRecords.class); + when(fileRecords.file()).thenReturn(tempFile); + when(fileRecords.sizeInBytes()).thenReturn(10); + + // Set up the segment that is eligible for copy + when(oldSegment.log()).thenReturn(fileRecords); + when(oldSegment.baseOffset()).thenReturn(oldSegmentStartOffset); + when(oldSegment.readNextOffset()).thenReturn(nextSegmentStartOffset); + + // set up the active segment + when(activeSegment.baseOffset()).thenReturn(nextSegmentStartOffset); + + when(mockLog.activeSegment()).thenReturn(activeSegment); + when(mockLog.logStartOffset()).thenReturn(oldSegmentStartOffset); + when(mockLog.logSegments(anyLong(), anyLong())).thenReturn(JavaConverters.collectionAsScalaIterable(Arrays.asList(oldSegment, activeSegment))); + + File mockProducerSnapshotIndex = TestUtils.tempFile(); + ProducerStateManager mockStateManager = mock(ProducerStateManager.class); + when(mockStateManager.fetchSnapshot(anyLong())).thenReturn(Optional.of(mockProducerSnapshotIndex)); + + when(mockLog.producerStateManager()).thenReturn(mockStateManager); + when(mockLog.lastStableOffset()).thenReturn(250L); + + File tempDir = TestUtils.tempDirectory(); + OffsetIndex idx = LazyIndex.forOffset(LogFileUtils.offsetIndexFile(tempDir, oldSegmentStartOffset, ""), oldSegmentStartOffset, 1000).get(); + TimeIndex timeIdx = LazyIndex.forTime(LogFileUtils.timeIndexFile(tempDir, oldSegmentStartOffset, ""), oldSegmentStartOffset, 1500).get(); + File txnFile = UnifiedLog.transactionIndexFile(tempDir, oldSegmentStartOffset, ""); + txnFile.createNewFile(); + TransactionIndex txnIndex = new TransactionIndex(oldSegmentStartOffset, txnFile); + when(oldSegment.timeIndex()).thenReturn(timeIdx); + when(oldSegment.offsetIndex()).thenReturn(idx); + when(oldSegment.txnIndex()).thenReturn(txnIndex); + + CompletableFuture dummyFuture = new CompletableFuture<>(); + dummyFuture.complete(null); + when(remoteLogMetadataManager.addRemoteLogSegmentMetadata(any(RemoteLogSegmentMetadata.class))).thenReturn(dummyFuture); + when(remoteLogMetadataManager.updateRemoteLogSegmentMetadata(any(RemoteLogSegmentMetadataUpdate.class))).thenReturn(dummyFuture); + when(remoteStorageManager.copyLogSegmentData(any(RemoteLogSegmentMetadata.class), any(LogSegmentData.class))).thenReturn(Optional.empty()); + + when(rlmCopyQuotaManager.isQuotaExceeded()).thenReturn(quotaExceeded); + doNothing().when(rlmCopyQuotaManager).record(anyInt()); + + RemoteLogManager.RLMTask task = remoteLogManager.new RLMTask(leaderTopicIdPartition, 128); + task.convertToLeader(2); + return task; + } + + @Test + public void testCopyThrottling() throws Exception { + long oldestSegmentStartOffset = 0L; + + when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition()); + + // leader epoch preparation + checkpoint.write(totalEpochEntries); + LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); + when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())).thenReturn(Optional.of(0L)); + + // create 3 log segments + LogSegment segmentToCopy = mock(LogSegment.class); + LogSegment segmentToThrottle = mock(LogSegment.class); + LogSegment activeSegment = mock(LogSegment.class); + + File tempFile = TestUtils.tempFile(); + FileRecords fileRecords = mock(FileRecords.class); + when(fileRecords.file()).thenReturn(tempFile); + when(fileRecords.sizeInBytes()).thenReturn(10); + + // set up the segment that will be copied + when(segmentToCopy.log()).thenReturn(fileRecords); + when(segmentToCopy.baseOffset()).thenReturn(oldestSegmentStartOffset); + when(segmentToCopy.readNextOffset()).thenReturn(100L); + + // set up the segment that will not be copied because of hitting quota + when(segmentToThrottle.log()).thenReturn(fileRecords); + when(segmentToThrottle.baseOffset()).thenReturn(100L); + when(segmentToThrottle.readNextOffset()).thenReturn(150L); + + // set up the active segment + when(activeSegment.log()).thenReturn(fileRecords); + when(activeSegment.baseOffset()).thenReturn(150L); + + when(mockLog.activeSegment()).thenReturn(activeSegment); + when(mockLog.logStartOffset()).thenReturn(oldestSegmentStartOffset); + when(mockLog.logSegments(anyLong(), anyLong())).thenReturn(JavaConverters.collectionAsScalaIterable(Arrays.asList(segmentToCopy, segmentToThrottle, activeSegment))); + + File mockProducerSnapshotIndex = TestUtils.tempFile(); + ProducerStateManager mockStateManager = mock(ProducerStateManager.class); + when(mockStateManager.fetchSnapshot(anyLong())).thenReturn(Optional.of(mockProducerSnapshotIndex)); + + when(mockLog.producerStateManager()).thenReturn(mockStateManager); + when(mockLog.lastStableOffset()).thenReturn(250L); + + File tempDir = TestUtils.tempDirectory(); + OffsetIndex idx = LazyIndex.forOffset(LogFileUtils.offsetIndexFile(tempDir, oldestSegmentStartOffset, ""), oldestSegmentStartOffset, 1000).get(); + TimeIndex timeIdx = LazyIndex.forTime(LogFileUtils.timeIndexFile(tempDir, oldestSegmentStartOffset, ""), oldestSegmentStartOffset, 1500).get(); + File txnFile = UnifiedLog.transactionIndexFile(tempDir, oldestSegmentStartOffset, ""); + txnFile.createNewFile(); + TransactionIndex txnIndex = new TransactionIndex(oldestSegmentStartOffset, txnFile); + when(segmentToCopy.timeIndex()).thenReturn(timeIdx); + when(segmentToCopy.offsetIndex()).thenReturn(idx); + when(segmentToCopy.txnIndex()).thenReturn(txnIndex); + + CompletableFuture dummyFuture = new CompletableFuture<>(); + dummyFuture.complete(null); + when(remoteLogMetadataManager.addRemoteLogSegmentMetadata(any(RemoteLogSegmentMetadata.class))).thenReturn(dummyFuture); + when(remoteLogMetadataManager.updateRemoteLogSegmentMetadata(any(RemoteLogSegmentMetadataUpdate.class))).thenReturn(dummyFuture); + when(remoteStorageManager.copyLogSegmentData(any(RemoteLogSegmentMetadata.class), any(LogSegmentData.class))).thenReturn(Optional.empty()); + + // After the first call, isQuotaExceeded should return true + when(rlmCopyQuotaManager.isQuotaExceeded()).thenReturn(false, true); + doNothing().when(rlmCopyQuotaManager).record(anyInt()); + + RemoteLogManager.RLMTask task = remoteLogManager.new RLMTask(leaderTopicIdPartition, 128); + task.convertToLeader(2); + + // Verify that the copy operation times out, since the second segment cannot be copied due to quota being exceeded + assertThrows(AssertionFailedError.class, () -> assertTimeoutPreemptively(Duration.ofMillis(200), () -> task.copyLogSegmentsToRemote(mockLog))); + + // Verify the highest offset in remote storage is updated corresponding to the only segment that was copied + ArgumentCaptor capture = ArgumentCaptor.forClass(Long.class); + verify(mockLog, times(2)).updateHighestOffsetInRemoteStorage(capture.capture()); + List capturedValues = capture.getAllValues(); + // Verify the highest offset in remote storage was -1L before the copy + assertEquals(-1L, capturedValues.get(0).longValue()); + // Verify it was updated to 99L after the copy + assertEquals(99L, capturedValues.get(1).longValue()); + } + + @Test + public void testTierLagResetsToZeroOnBecomingFollower() { + remoteLogManager.startup(); + remoteLogManager.onLeadershipChange( + Collections.singleton(mockPartition(leaderTopicIdPartition)), Collections.emptySet(), topicIds); + RemoteLogManager.RLMTask rlmTask = remoteLogManager.rlmTask(leaderTopicIdPartition); + assertNotNull(rlmTask); + rlmTask.recordLagStats(1024, 2); + assertEquals(1024, brokerTopicStats.topicStats(leaderTopicIdPartition.topic()).remoteCopyLagBytes()); + assertEquals(2, brokerTopicStats.topicStats(leaderTopicIdPartition.topic()).remoteCopyLagSegments()); + // The same node becomes follower now which was the previous leader + remoteLogManager.onLeadershipChange(Collections.emptySet(), + Collections.singleton(mockPartition(leaderTopicIdPartition)), topicIds); + assertEquals(0, brokerTopicStats.topicStats(leaderTopicIdPartition.topic()).remoteCopyLagBytes()); + assertEquals(0, brokerTopicStats.topicStats(leaderTopicIdPartition.topic()).remoteCopyLagSegments()); + + // If the old task emits the tier-lag stats, then it should be discarded + rlmTask.recordLagStats(2048, 4); + assertEquals(0, brokerTopicStats.topicStats(leaderTopicIdPartition.topic()).remoteCopyLagBytes()); + assertEquals(0, brokerTopicStats.topicStats(leaderTopicIdPartition.topic()).remoteCopyLagSegments()); + } + private Partition mockPartition(TopicIdPartition topicIdPartition) { TopicPartition tp = topicIdPartition.topicPartition(); Partition partition = mock(Partition.class); @@ -2739,8 +3073,7 @@ private RemoteLogManagerConfig createRLMConfig(Properties props) { props.put(DEFAULT_REMOTE_LOG_METADATA_MANAGER_CONFIG_PREFIX + remoteLogMetadataConsumerTestProp, remoteLogMetadataConsumerTestVal); props.put(DEFAULT_REMOTE_LOG_METADATA_MANAGER_CONFIG_PREFIX + remoteLogMetadataProducerTestProp, remoteLogMetadataProducerTestVal); - AbstractConfig config = new AbstractConfig(RemoteLogManagerConfig.CONFIG_DEF, props); - return new RemoteLogManagerConfig(config); + return new RemoteLogManagerConfig(props); } } diff --git a/core/src/test/java/kafka/log/remote/RemoteLogReaderTest.java b/core/src/test/java/kafka/log/remote/RemoteLogReaderTest.java index bff58364b9..2d82f78ec5 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogReaderTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogReaderTest.java @@ -16,6 +16,8 @@ */ package kafka.log.remote; +import com.yammer.metrics.core.Timer; +import kafka.log.remote.quota.RLMQuotaManager; import kafka.server.BrokerTopicStats; import kafka.utils.TestUtils; import org.apache.kafka.common.TopicPartition; @@ -30,6 +32,7 @@ import org.mockito.ArgumentCaptor; import java.io.IOException; +import java.util.concurrent.Callable; import java.util.function.Consumer; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -45,13 +48,16 @@ public class RemoteLogReaderTest { public static final String TOPIC = "test"; RemoteLogManager mockRLM = mock(RemoteLogManager.class); BrokerTopicStats brokerTopicStats = null; + RLMQuotaManager mockQuotaManager = mock(RLMQuotaManager.class); LogOffsetMetadata logOffsetMetadata = new LogOffsetMetadata(100); Records records = mock(Records.class); + Timer timer = mock(Timer.class); @BeforeEach - public void setUp() { + public void setUp() throws Exception { TestUtils.clearYammerMetrics(); brokerTopicStats = new BrokerTopicStats(true); + when(timer.time(any(Callable.class))).thenAnswer(ans -> ans.getArgument(0, Callable.class).call()); } @Test @@ -62,7 +68,8 @@ public void testRemoteLogReaderWithoutError() throws RemoteStorageException, IOE Consumer callback = mock(Consumer.class); RemoteStorageFetchInfo remoteStorageFetchInfo = new RemoteStorageFetchInfo(0, false, new TopicPartition(TOPIC, 0), null, null, false); - RemoteLogReader remoteLogReader = new RemoteLogReader(remoteStorageFetchInfo, mockRLM, callback, brokerTopicStats); + RemoteLogReader remoteLogReader = + new RemoteLogReader(remoteStorageFetchInfo, mockRLM, callback, brokerTopicStats, mockQuotaManager, timer); remoteLogReader.call(); // verify the callback did get invoked with the expected remoteLogReadResult @@ -73,6 +80,11 @@ public void testRemoteLogReaderWithoutError() throws RemoteStorageException, IOE assertTrue(actualRemoteLogReadResult.fetchDataInfo.isPresent()); assertEquals(fetchDataInfo, actualRemoteLogReadResult.fetchDataInfo.get()); + // verify the record method on quota manager was called with the expected value + ArgumentCaptor recordedArg = ArgumentCaptor.forClass(Double.class); + verify(mockQuotaManager, times(1)).record(recordedArg.capture()); + assertEquals(100, recordedArg.getValue()); + // Verify metrics for remote reads are updated correctly assertEquals(1, brokerTopicStats.topicStats(TOPIC).remoteFetchRequestRate().count()); assertEquals(100, brokerTopicStats.topicStats(TOPIC).remoteFetchBytesRate().count()); @@ -89,7 +101,8 @@ public void testRemoteLogReaderWithError() throws RemoteStorageException, IOExce Consumer callback = mock(Consumer.class); RemoteStorageFetchInfo remoteStorageFetchInfo = new RemoteStorageFetchInfo(0, false, new TopicPartition(TOPIC, 0), null, null, false); - RemoteLogReader remoteLogReader = new RemoteLogReader(remoteStorageFetchInfo, mockRLM, callback, brokerTopicStats); + RemoteLogReader remoteLogReader = + new RemoteLogReader(remoteStorageFetchInfo, mockRLM, callback, brokerTopicStats, mockQuotaManager, timer); remoteLogReader.call(); // verify the callback did get invoked with the expected remoteLogReadResult @@ -99,6 +112,11 @@ public void testRemoteLogReaderWithError() throws RemoteStorageException, IOExce assertTrue(actualRemoteLogReadResult.error.isPresent()); assertFalse(actualRemoteLogReadResult.fetchDataInfo.isPresent()); + // verify the record method on quota manager was called with the expected value + ArgumentCaptor recordedArg = ArgumentCaptor.forClass(Double.class); + verify(mockQuotaManager, times(1)).record(recordedArg.capture()); + assertEquals(0, recordedArg.getValue()); + // Verify metrics for remote reads are updated correctly assertEquals(1, brokerTopicStats.topicStats(TOPIC).remoteFetchRequestRate().count()); assertEquals(0, brokerTopicStats.topicStats(TOPIC).remoteFetchBytesRate().count()); diff --git a/core/src/test/java/kafka/log/streamaspect/ElasticLogSegmentTest.java b/core/src/test/java/kafka/log/streamaspect/ElasticLogSegmentTest.java index dcf3c57aee..ab186fce48 100644 --- a/core/src/test/java/kafka/log/streamaspect/ElasticLogSegmentTest.java +++ b/core/src/test/java/kafka/log/streamaspect/ElasticLogSegmentTest.java @@ -12,10 +12,9 @@ import java.io.File; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.HashMap; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Optional; @@ -35,7 +34,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.coordinator.transaction.TransactionLogConfigs; -import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint; +import org.apache.kafka.server.util.MockScheduler; import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache; import org.apache.kafka.storage.internals.log.EpochEntry; import org.apache.kafka.storage.internals.log.FetchDataInfo; @@ -258,21 +257,9 @@ public void testRecoverTransactionIndex() throws IOException { public void testRecoveryRebuildsEpochCache() throws IOException { ElasticLogSegment seg = createOrLoadSegment(0); - LeaderEpochCheckpoint checkpoint = new LeaderEpochCheckpoint() { - private List epochs = new LinkedList<>(); - - @Override - public void write(Collection epochs, boolean sync) { - this.epochs = new LinkedList<>(epochs); - } - - @Override - public List read() { - return epochs; - } - }; - - LeaderEpochFileCache cache = new LeaderEpochFileCache(topicPartition, checkpoint); + ElasticLeaderEpochCheckpoint checkpoint = new ElasticLeaderEpochCheckpoint(new ElasticLeaderEpochCheckpointMeta(0, new ArrayList<>()), meta -> { + }); + LeaderEpochFileCache cache = new LeaderEpochFileCache(topicPartition, checkpoint, new MockScheduler(Time.SYSTEM)); seg.append(105L, RecordBatch.NO_TIMESTAMP, 104L, MemoryRecords.withRecords(104L, Compression.NONE, 0, new SimpleRecord("a".getBytes()), new SimpleRecord("b".getBytes()))); diff --git a/core/src/test/java/kafka/server/LogManagerIntegrationTest.java b/core/src/test/java/kafka/server/LogManagerIntegrationTest.java new file mode 100644 index 0000000000..709454becc --- /dev/null +++ b/core/src/test/java/kafka/server/LogManagerIntegrationTest.java @@ -0,0 +1,137 @@ +/* + * 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. + */ +package kafka.server; + +import kafka.test.ClusterInstance; +import kafka.test.annotation.ClusterTest; +import kafka.test.annotation.Type; +import kafka.test.junit.ClusterTestExtensions; +import kafka.test.junit.RaftClusterInvocationContext; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.TopicPartitionInfo; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.storage.internals.checkpoint.PartitionMetadataFile; +import org.apache.kafka.test.TestUtils; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.ExecutionException; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +@ExtendWith(value = ClusterTestExtensions.class) +@Tag("integration") +public class LogManagerIntegrationTest { + private final ClusterInstance cluster; + + public LogManagerIntegrationTest(ClusterInstance cluster) { + this.cluster = cluster; + } + + @ClusterTest(types = {Type.KRAFT, Type.CO_KRAFT}, brokers = 3) + public void testRestartBrokerNoErrorIfMissingPartitionMetadata() throws IOException, ExecutionException, InterruptedException { + RaftClusterInvocationContext.RaftClusterInstance raftInstance = + (RaftClusterInvocationContext.RaftClusterInstance) cluster; + + try (Admin admin = cluster.createAdminClient()) { + admin.createTopics(Collections.singletonList(new NewTopic("foo", 1, (short) 3))).all().get(); + } + cluster.waitForTopic("foo", 1); + + Optional partitionMetadataFile = Optional.ofNullable( + raftInstance.getUnderlying().brokers().get(0).logManager() + .getLog(new TopicPartition("foo", 0), false).get() + .partitionMetadataFile().getOrElse(null)); + assertTrue(partitionMetadataFile.isPresent()); + + raftInstance.getUnderlying().brokers().get(0).shutdown(); + try (Admin admin = cluster.createAdminClient()) { + TestUtils.waitForCondition(() -> { + List partitionInfos = admin.describeTopics(Collections.singletonList("foo")) + .topicNameValues().get("foo").get().partitions(); + return partitionInfos.get(0).isr().size() == 2; + }, "isr size is not shrink to 2"); + } + + // delete partition.metadata file here to simulate the scenario that partition.metadata not flush to disk yet + partitionMetadataFile.get().delete(); + assertFalse(partitionMetadataFile.get().exists()); + raftInstance.getUnderlying().brokers().get(0).startup(); + // make sure there is no error during load logs + assertDoesNotThrow(() -> raftInstance.getUnderlying().fatalFaultHandler().maybeRethrowFirstException()); + try (Admin admin = cluster.createAdminClient()) { + TestUtils.waitForCondition(() -> { + List partitionInfos = admin.describeTopics(Collections.singletonList("foo")) + .topicNameValues().get("foo").get().partitions(); + return partitionInfos.get(0).isr().size() == 3; + }, "isr size is not expand to 3"); + } + + // make sure topic still work fine + Map producerConfigs = new HashMap<>(); + producerConfigs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); + producerConfigs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + producerConfigs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + + try (Producer producer = new KafkaProducer<>(producerConfigs)) { + producer.send(new ProducerRecord<>("foo", 0, null, "bar")).get(); + producer.flush(); + } + + Map consumerConfigs = new HashMap<>(); + consumerConfigs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); + consumerConfigs.put(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString()); + consumerConfigs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + consumerConfigs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + + try (Consumer consumer = new KafkaConsumer<>(consumerConfigs)) { + consumer.assign(Collections.singletonList(new TopicPartition("foo", 0))); + consumer.seekToBeginning(Collections.singletonList(new TopicPartition("foo", 0))); + List values = new ArrayList<>(); + ConsumerRecords records = consumer.poll(Duration.ofMinutes(1)); + for (ConsumerRecord record : records) { + values.add(record.value()); + } + assertEquals(1, values.size()); + assertEquals("bar", values.get(0)); + } + } +} diff --git a/core/src/test/java/kafka/server/MetadataVersionConfigValidatorTest.java b/core/src/test/java/kafka/server/MetadataVersionConfigValidatorTest.java new file mode 100644 index 0000000000..168c329834 --- /dev/null +++ b/core/src/test/java/kafka/server/MetadataVersionConfigValidatorTest.java @@ -0,0 +1,102 @@ +/* + * 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. + */ + +package kafka.server; + +import org.apache.kafka.common.metadata.FeatureLevelRecord; +import org.apache.kafka.image.MetadataDelta; +import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.image.MetadataProvenance; +import org.apache.kafka.image.loader.LogDeltaManifest; +import org.apache.kafka.raft.LeaderAndEpoch; +import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.fault.FaultHandler; +import org.junit.jupiter.api.Test; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.BDDMockito.willAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; + +public class MetadataVersionConfigValidatorTest { + + private static final LogDeltaManifest TEST_MANIFEST = LogDeltaManifest.newBuilder() + .provenance(MetadataProvenance.EMPTY) + .leaderAndEpoch(LeaderAndEpoch.UNKNOWN) + .numBatches(1) + .elapsedNs(90) + .numBytes(88) + .build(); + public static final MetadataProvenance TEST_PROVENANCE = + new MetadataProvenance(50, 3, 8000); + + void testWith(MetadataVersion metadataVersion, KafkaConfig config, FaultHandler faultHandler) throws Exception { + try (MetadataVersionConfigValidator validator = new MetadataVersionConfigValidator(config, faultHandler)) { + MetadataDelta delta = new MetadataDelta.Builder() + .setImage(MetadataImage.EMPTY) + .build(); + if (metadataVersion != null) { + delta.replay(new FeatureLevelRecord(). + setName(MetadataVersion.FEATURE_NAME). + setFeatureLevel(metadataVersion.featureLevel())); + } + MetadataImage image = delta.apply(TEST_PROVENANCE); + + validator.onMetadataUpdate(delta, image, TEST_MANIFEST); + } + } + + @Test + void testValidatesConfigOnMetadataChange() throws Exception { + MetadataVersion metadataVersion = MetadataVersion.IBP_3_7_IV2; + KafkaConfig config = mock(KafkaConfig.class); + FaultHandler faultHandler = mock(FaultHandler.class); + + when(config.brokerId()).thenReturn(8); + + testWith(metadataVersion, config, faultHandler); + + verify(config, times(1)).validateWithMetadataVersion(eq(metadataVersion)); + verifyNoMoreInteractions(faultHandler); + } + + @SuppressWarnings("ThrowableNotThrown") + @Test + void testInvokesFaultHandlerOnException() throws Exception { + MetadataVersion metadataVersion = MetadataVersion.IBP_3_7_IV2; + Exception exception = new Exception(); + KafkaConfig config = mock(KafkaConfig.class); + FaultHandler faultHandler = mock(FaultHandler.class); + + when(faultHandler.handleFault(any(), any())).thenReturn(new RuntimeException("returned exception")); + when(config.brokerId()).thenReturn(8); + willAnswer(invocation -> { + throw exception; + }).given(config).validateWithMetadataVersion(eq(metadataVersion)); + + testWith(metadataVersion, config, faultHandler); + + verify(config, times(1)).validateWithMetadataVersion(eq(metadataVersion)); + verify(faultHandler, times(1)).handleFault( + eq("Broker configuration does not support the cluster MetadataVersion"), + eq(exception)); + } +} diff --git a/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java b/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java index cdbb942e6d..56a2a9e27d 100644 --- a/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java +++ b/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java @@ -185,7 +185,7 @@ public void testNoAutoStart() { @ClusterTest public void testDefaults(ClusterInstance clusterInstance) { - Assertions.assertEquals(MetadataVersion.IBP_3_8_IV0, clusterInstance.config().metadataVersion()); + Assertions.assertEquals(MetadataVersion.latestTesting(), clusterInstance.config().metadataVersion()); } @ClusterTests({ diff --git a/core/src/test/java/kafka/test/annotation/ClusterTest.java b/core/src/test/java/kafka/test/annotation/ClusterTest.java index 9364ce690e..1fcfdbeeef 100644 --- a/core/src/test/java/kafka/test/annotation/ClusterTest.java +++ b/core/src/test/java/kafka/test/annotation/ClusterTest.java @@ -40,7 +40,7 @@ AutoStart autoStart() default AutoStart.DEFAULT; SecurityProtocol securityProtocol() default SecurityProtocol.PLAINTEXT; String listener() default ""; - MetadataVersion metadataVersion() default MetadataVersion.IBP_3_8_IV0; + MetadataVersion metadataVersion() default MetadataVersion.IBP_3_9_IV0; ClusterConfigProperty[] serverProperties() default {}; // users can add tags that they want to display in test String[] tags() default {}; diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java index 5365652a5f..94d94dc717 100644 --- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java +++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java @@ -239,12 +239,15 @@ public KafkaClusterTestKit build() throws Exception { ThreadUtils.createThreadFactory("kafka-cluster-test-kit-executor-%d", false)); for (ControllerNode node : nodes.controllerNodes().values()) { setupNodeDirectories(baseDirectory, node.metadataDirectory(), Collections.emptyList()); - SharedServer sharedServer = new SharedServer(createNodeConfig(node), - node.initialMetaPropertiesEnsemble(), - Time.SYSTEM, - new Metrics(), - connectFutureManager.future, - faultHandlerFactory); + SharedServer sharedServer = new SharedServer( + createNodeConfig(node), + node.initialMetaPropertiesEnsemble(), + Time.SYSTEM, + new Metrics(), + connectFutureManager.future, + Collections.emptyList(), + faultHandlerFactory + ); ControllerServer controller = null; try { controller = new ControllerServer( @@ -267,13 +270,18 @@ public KafkaClusterTestKit build() throws Exception { jointServers.put(node.id(), sharedServer); } for (BrokerNode node : nodes.brokerNodes().values()) { - SharedServer sharedServer = jointServers.computeIfAbsent(node.id(), - id -> new SharedServer(createNodeConfig(node), + SharedServer sharedServer = jointServers.computeIfAbsent( + node.id(), + id -> new SharedServer( + createNodeConfig(node), node.initialMetaPropertiesEnsemble(), Time.SYSTEM, new Metrics(), connectFutureManager.future, - faultHandlerFactory)); + Collections.emptyList(), + faultHandlerFactory + ) + ); BrokerServer broker = null; try { broker = new BrokerServer(sharedServer); diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index c854828041..f396239012 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -21,6 +21,5 @@ log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n log4j.logger.kafka=INFO log4j.logger.org.apache.kafka=INFO - # zkclient can be verbose, during debugging it is common to adjust it separately log4j.logger.org.apache.zookeeper=WARN \ No newline at end of file diff --git a/core/src/test/scala/integration/kafka/admin/AdminFenceProducersIntegrationTest.scala b/core/src/test/scala/integration/kafka/admin/AdminFenceProducersIntegrationTest.scala new file mode 100644 index 0000000000..cbfeee5098 --- /dev/null +++ b/core/src/test/scala/integration/kafka/admin/AdminFenceProducersIntegrationTest.scala @@ -0,0 +1,142 @@ +/* + * 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. + */ + +package integration.kafka.admin + +import kafka.api.IntegrationTestHarness +import org.apache.kafka.clients.admin._ +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} +import org.apache.kafka.common.errors.{InvalidProducerEpochException, ProducerFencedException} +import org.apache.kafka.common.utils.Utils +import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} +import org.apache.kafka.server.config.ServerLogConfigs +import org.junit.jupiter.api.Assertions.{assertInstanceOf, assertThrows, assertTrue, fail} +import org.junit.jupiter.api.{AfterEach, BeforeEach, Tag, TestInfo} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource + +import java.util.concurrent.ExecutionException +import java.util.{Collections, Properties} +import scala.collection.Seq + +@Tag("integration") +class AdminFenceProducersIntegrationTest extends IntegrationTestHarness { + override def brokerCount = 1 + + private val topicName = "mytopic" + private val txnId = "mytxnid" + private val record = new ProducerRecord[Array[Byte], Array[Byte]](topicName, null, new Array[Byte](1)) + + private var adminClient: Admin = _ + private var producer: KafkaProducer[Array[Byte], Array[Byte]] = _ + + @BeforeEach + override def setUp(testInfo: TestInfo): Unit = { + super.setUp(testInfo) + + val producerProps = new Properties + producerProps.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, txnId) + producerProps.put(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG, "2000") + producer = createProducer(configOverrides = producerProps) + adminClient = createAdminClient() + createTopic(topicName) + } + + def overridingProps(): Properties = { + val props = new Properties() + props.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString) + // Set a smaller value for the number of partitions for speed + props.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 1.toString) + props.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, 1.toString) + props.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG, 1.toString) + props.put(TransactionStateManagerConfigs.TRANSACTIONS_ABORT_TIMED_OUT_TRANSACTION_CLEANUP_INTERVAL_MS_CONFIG, "2000") + props + } + + override protected def modifyConfigs(props: Seq[Properties]): Unit = { + props.foreach(p => p.putAll(overridingProps())) + } + + override protected def kraftControllerConfigs(): Seq[Properties] = { + Seq(overridingProps()) + } + + @AfterEach + override def tearDown(): Unit = { + Utils.closeQuietly(adminClient, "AdminFenceProducersIntegrationTest") + Utils.closeQuietly(producer, "AdminFenceProducersIntegrationTest") + super.tearDown() + } + + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testFenceAfterProducerCommit(quorum: String): Unit = { + producer.initTransactions() + producer.beginTransaction() + producer.send(record).get() + producer.commitTransaction() + + adminClient.fenceProducers(Collections.singletonList(txnId)).all().get() + + producer.beginTransaction() + try { + producer.send(record).get() + fail("expected ProducerFencedException") + } catch { + case _: ProducerFencedException => //ok + case ee: ExecutionException => + assertInstanceOf(classOf[ProducerFencedException], ee.getCause) //ok + case e: Exception => + throw e + } + + assertThrows(classOf[ProducerFencedException], () => producer.commitTransaction()) + } + + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testFenceBeforeProducerCommit(quorum: String): Unit = { + producer.initTransactions() + producer.beginTransaction() + producer.send(record).get() + + adminClient.fenceProducers(Collections.singletonList(txnId)).all().get() + + try { + producer.send(record).get() + fail("expected Exception") + } catch { + case ee: ExecutionException => + assertTrue(ee.getCause.isInstanceOf[ProducerFencedException] || + ee.getCause.isInstanceOf[InvalidProducerEpochException], + "Unexpected ExecutionException cause " + ee.getCause) + case e: Exception => + throw e + } + + try { + producer.commitTransaction() + fail("expected Exception") + } catch { + case _: ProducerFencedException => + case _: InvalidProducerEpochException => + case e: Exception => + throw e + } + } +} + diff --git a/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala b/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala index d242ea105e..d29f05b36b 100644 --- a/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala @@ -23,6 +23,7 @@ import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.common.record.TimestampType import org.apache.kafka.common.TopicPartition +import org.apache.kafka.test.{TestUtils => JTestUtils} import kafka.utils.TestUtils import kafka.server.BaseRequestTest import org.junit.jupiter.api.Assertions._ @@ -90,12 +91,14 @@ abstract class AbstractConsumerTest extends BaseRequestTest { s"The current assignment is ${consumer.assignment()}") } - def awaitNonEmptyRecords[K, V](consumer: Consumer[K, V], partition: TopicPartition): ConsumerRecords[K, V] = { + def awaitNonEmptyRecords[K, V](consumer: Consumer[K, V], + partition: TopicPartition, + pollTimeoutMs: Long = 100): ConsumerRecords[K, V] = { TestUtils.pollRecordsUntilTrue(consumer, (polledRecords: ConsumerRecords[K, V]) => { if (polledRecords.records(partition).asScala.nonEmpty) return polledRecords false - }, s"Consumer did not consume any messages for partition $partition before timeout.") + }, s"Consumer did not consume any messages for partition $partition before timeout.", JTestUtils.DEFAULT_MAX_WAIT_MS, pollTimeoutMs) throw new IllegalStateException("Should have timed out before reaching here") } diff --git a/core/src/test/scala/integration/kafka/api/AdminClientRebootstrapTest.scala b/core/src/test/scala/integration/kafka/api/AdminClientRebootstrapTest.scala new file mode 100644 index 0000000000..70b514f199 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/AdminClientRebootstrapTest.scala @@ -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. + */ +package kafka.api + +import org.junit.jupiter.api.Test + +class AdminClientRebootstrapTest extends RebootstrapTest { + @Test + def testRebootstrap(): Unit = { + server1.shutdown() + server1.awaitShutdown() + + val adminClient = createAdminClient(configOverrides = clientOverrides) + + // Only the server 0 is available for the admin client during the bootstrap. + adminClient.listTopics().names().get() + + server0.shutdown() + server0.awaitShutdown() + server1.startup() + + // The server 0, originally cached during the bootstrap, is offline. + // However, the server 1 from the bootstrap list is online. + // Should be able to list topics again. + adminClient.listTopics().names().get() + + server1.shutdown() + server1.awaitShutdown() + server0.startup() + + // The same situation, but the server 1 has gone and server 0 is back. + adminClient.listTopics().names().get() + } +} diff --git a/core/src/test/scala/integration/kafka/api/ConsumerRebootstrapTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerRebootstrapTest.scala new file mode 100644 index 0000000000..9979e3eb91 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/ConsumerRebootstrapTest.scala @@ -0,0 +1,87 @@ +/* + * 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. + */ +package kafka.api + +import kafka.utils.TestUtils +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} +import org.junit.jupiter.api.Test + +import java.util.Collections + +class ConsumerRebootstrapTest extends RebootstrapTest { + @Test + def testRebootstrap(): Unit = { + sendRecords(10, 0) + + TestUtils.waitUntilTrue( + () => server0.logManager.logsByTopic(tp.topic()).head.logEndOffset == server1.logManager.logsByTopic(tp.topic()).head.logEndOffset, + "Timeout waiting for records to be replicated" + ) + + server1.shutdown() + server1.awaitShutdown() + + val consumer = createConsumer(configOverrides = clientOverrides) + + // Only the server 0 is available for the consumer during the bootstrap. + consumer.assign(Collections.singleton(tp)) + + consumeAndVerifyRecords(consumer, 10, 0) + + // Bring back the server 1 and shut down 0. + server1.startup() + + TestUtils.waitUntilTrue( + () => server0.logManager.logsByTopic(tp.topic()).head.logEndOffset == server1.logManager.logsByTopic(tp.topic()).head.logEndOffset, + "Timeout waiting for records to be replicated" + ) + + server0.shutdown() + server0.awaitShutdown() + sendRecords(10, 10) + + // The server 0, originally cached during the bootstrap, is offline. + // However, the server 1 from the bootstrap list is online. + // Should be able to consume records. + consumeAndVerifyRecords(consumer, 10, 10, startingKeyAndValueIndex = 10, startingTimestamp = 10) + + // Bring back the server 0 and shut down 1. + server0.startup() + + TestUtils.waitUntilTrue( + () => server0.logManager.logsByTopic(tp.topic()).head.logEndOffset == server1.logManager.logsByTopic(tp.topic()).head.logEndOffset, + "Timeout waiting for records to be replicated" + ) + + server1.shutdown() + server1.awaitShutdown() + sendRecords(10, 20) + + // The same situation, but the server 1 has gone and server 0 is back. + consumeAndVerifyRecords(consumer, 10, 20, startingKeyAndValueIndex = 20, startingTimestamp = 20) + } + + private def sendRecords(numRecords: Int, from: Int): Unit = { + val producer: KafkaProducer[Array[Byte], Array[Byte]] = createProducer() + (from until (numRecords + from)).foreach { i => + val record = new ProducerRecord(tp.topic(), tp.partition(), i.toLong, s"key $i".getBytes, s"value $i".getBytes) + producer.send(record) + } + producer.flush() + producer.close() + } +} diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index e1f549c4e5..e89c68670b 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -62,7 +62,6 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness { } override def generateConfigs: Seq[KafkaConfig] = { - val cfgs = TestUtils.createBrokerConfigs(brokerCount, zkConnectOrNull, interBrokerSecurityProtocol = Some(securityProtocol), trustStoreFile = trustStoreFile, saslProperties = serverSaslProperties, logDirCount = logDirCount) configureListeners(cfgs) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index ffe10c19c4..2a21873ed2 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -204,6 +204,21 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { assertFutureExceptionTypeEquals(results.get(nonExistingTopicId), classOf[UnknownTopicIdException]) } + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testDescribeTopicsWithNames(quorum: String): Unit = { + client = createAdminClient + + val existingTopic = "existing-topic" + client.createTopics(Seq(existingTopic).map(new NewTopic(_, 1, 1.toShort)).asJava).all.get() + waitForTopics(client, Seq(existingTopic), List()) + ensureConsistentKRaftMetadata() + + val existingTopicId = brokers.head.metadataCache.getTopicId(existingTopic) + val results = client.describeTopics(TopicCollection.ofTopicNames(Seq(existingTopic).asJava)).topicNameValues() + assertEquals(existingTopicId, results.get(existingTopic).get.topicId()) + } + @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testDescribeCluster(quorum: String): Unit = { diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerPollTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerPollTest.scala index daed397e43..83a325fcc5 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerPollTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerPollTest.scala @@ -12,7 +12,7 @@ */ package kafka.api -import kafka.utils.TestInfoUtils +import kafka.utils.{TestInfoUtils, TestUtils} import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.{MetricName, TopicPartition} import org.apache.kafka.common.utils.Utils @@ -238,6 +238,33 @@ class PlaintextConsumerPollTest extends AbstractConsumerTest { runMultiConsumerSessionTimeoutTest(true) } + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testPollEventuallyReturnsRecordsWithZeroTimeout(quorum: String, groupProtocol: String): Unit = { + val numMessages = 100 + val producer = createProducer() + sendRecords(producer, numMessages, tp) + + val consumer = createConsumer() + consumer.subscribe(Set(topic).asJava) + val records = awaitNonEmptyRecords(consumer, tp, 0L) + assertEquals(numMessages, records.count()) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testNoOffsetForPartitionExceptionOnPollZero(quorum: String, groupProtocol: String): Unit = { + this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") + val consumer = createConsumer(configOverrides = this.consumerConfig) + + consumer.assign(List(tp).asJava) + + // continuous poll should eventually fail because there is no offset reset strategy set (fail only when resetting positions after coordinator is known) + TestUtils.tryUntilNoAssertionError() { + assertThrows(classOf[NoOffsetForPartitionException], () => consumer.poll(Duration.ZERO)) + } + } + def runMultiConsumerSessionTimeoutTest(closeConsumer: Boolean): Unit = { // use consumers defined in this class plus one additional consumer // Use topic defined in this class + one additional topic diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 7d32d1bd3e..b4617af150 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -21,12 +21,12 @@ import kafka.utils.{TestInfoUtils, TestUtils} import org.apache.kafka.clients.admin.{NewPartitions, NewTopic} import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} -import org.apache.kafka.common.{KafkaException, MetricName, TopicPartition} import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.{InvalidGroupIdException, InvalidTopicException, TimeoutException, WakeupException} import org.apache.kafka.common.header.Headers import org.apache.kafka.common.record.{CompressionType, TimestampType} import org.apache.kafka.common.serialization._ +import org.apache.kafka.common.{KafkaException, MetricName, TopicPartition} import org.apache.kafka.test.{MockConsumerInterceptor, MockProducerInterceptor} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Timeout diff --git a/core/src/test/scala/integration/kafka/api/ProducerRebootstrapTest.scala b/core/src/test/scala/integration/kafka/api/ProducerRebootstrapTest.scala new file mode 100644 index 0000000000..3cb40b6a0c --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/ProducerRebootstrapTest.scala @@ -0,0 +1,52 @@ +/* + * 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. + */ +package kafka.api + +import org.apache.kafka.clients.producer.ProducerRecord +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.Test + +class ProducerRebootstrapTest extends RebootstrapTest { + @Test + def testRebootstrap(): Unit = { + server1.shutdown() + server1.awaitShutdown() + + val producer = createProducer(configOverrides = clientOverrides) + + // Only the server 0 is available for the producer during the bootstrap. + producer.send(new ProducerRecord(topic, part, "key 0".getBytes, "value 0".getBytes)).get() + + server0.shutdown() + server0.awaitShutdown() + server1.startup() + + // The server 0, originally cached during the bootstrap, is offline. + // However, the server 1 from the bootstrap list is online. + // Should be able to produce records. + val recordMetadata1 = producer.send(new ProducerRecord(topic, part, "key 1".getBytes, "value 1".getBytes)).get() + assertEquals(0, recordMetadata1.offset()) + + server1.shutdown() + server1.awaitShutdown() + server0.startup() + + // The same situation, but the server 1 has gone and server 0 is back. + val recordMetadata2 = producer.send(new ProducerRecord(topic, part, "key 1".getBytes, "value 1".getBytes)).get() + assertEquals(1, recordMetadata2.offset()) + } +} diff --git a/core/src/test/scala/integration/kafka/api/RebootstrapTest.scala b/core/src/test/scala/integration/kafka/api/RebootstrapTest.scala new file mode 100644 index 0000000000..b3b044ebcd --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/RebootstrapTest.scala @@ -0,0 +1,52 @@ +/* + * 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. + */ +package kafka.api + +import kafka.server.{KafkaConfig, KafkaServer} +import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.common.config.TopicConfig +import org.apache.kafka.coordinator.group.GroupCoordinatorConfig + +import java.util.Properties + +abstract class RebootstrapTest extends AbstractConsumerTest { + override def brokerCount: Int = 2 + + def server0: KafkaServer = serverForId(0).get + def server1: KafkaServer = serverForId(1).get + + override def generateConfigs: Seq[KafkaConfig] = { + val overridingProps = new Properties() + overridingProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, brokerCount.toString) + overridingProps.put(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, "true") + + // In this test, fixed ports are necessary, because brokers must have the + // same port after the restart. + FixedPortTestUtils.createBrokerConfigs(brokerCount, zkConnect, enableControlledShutdown = false) + .map(KafkaConfig.fromProps(_, overridingProps)) + } + + def clientOverrides: Properties = { + val overrides = new Properties() + overrides.put(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG, "5000") + overrides.put(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG, "5000") + overrides.put(CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG, "1000") + overrides.put(CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_CONFIG, "1000") + overrides.put(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG, "rebootstrap") + overrides + } +} diff --git a/core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala b/core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala index c35385bcbc..ea1ffaf0b1 100644 --- a/core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala +++ b/core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala @@ -40,6 +40,7 @@ class DelayedRemoteFetchTest { private val fetchOffset = 500L private val logStartOffset = 0L private val currentLeaderEpoch = Optional.of[Integer](10) + private val remoteFetchMaxWaitMs = 500 private val fetchStatus = FetchPartitionStatus( startOffsetMetadata = new LogOffsetMetadata(fetchOffset), @@ -64,8 +65,8 @@ class DelayedRemoteFetchTest { val leaderLogStartOffset = 10 val logReadInfo = buildReadResult(Errors.NONE, highWatermark, leaderLogStartOffset) - val delayedRemoteFetch = new DelayedRemoteFetch(null, future, fetchInfo, Seq(topicIdPartition -> fetchStatus), fetchParams, - Seq(topicIdPartition -> logReadInfo), replicaManager, callback) + val delayedRemoteFetch = new DelayedRemoteFetch(null, future, fetchInfo, remoteFetchMaxWaitMs, + Seq(topicIdPartition -> fetchStatus), fetchParams, Seq(topicIdPartition -> logReadInfo), replicaManager, callback) when(replicaManager.getPartitionOrException(topicIdPartition.topicPartition)) .thenReturn(mock(classOf[Partition])) @@ -100,8 +101,8 @@ class DelayedRemoteFetchTest { val leaderLogStartOffset = 10 val logReadInfo = buildReadResult(Errors.NONE, highWatermark, leaderLogStartOffset) val fetchParams = buildFetchParams(replicaId = 1, maxWaitMs = 500) - assertThrows(classOf[IllegalStateException], () => new DelayedRemoteFetch(null, future, fetchInfo, Seq(topicIdPartition -> fetchStatus), fetchParams, - Seq(topicIdPartition -> logReadInfo), replicaManager, callback)) + assertThrows(classOf[IllegalStateException], () => new DelayedRemoteFetch(null, future, fetchInfo, remoteFetchMaxWaitMs, + Seq(topicIdPartition -> fetchStatus), fetchParams, Seq(topicIdPartition -> logReadInfo), replicaManager, callback)) } @Test @@ -124,8 +125,8 @@ class DelayedRemoteFetchTest { val logReadInfo = buildReadResult(Errors.NONE) - val delayedRemoteFetch = new DelayedRemoteFetch(null, future, fetchInfo, Seq(topicIdPartition -> fetchStatus), fetchParams, - Seq(topicIdPartition -> logReadInfo), replicaManager, callback) + val delayedRemoteFetch = new DelayedRemoteFetch(null, future, fetchInfo, remoteFetchMaxWaitMs, + Seq(topicIdPartition -> fetchStatus), fetchParams, Seq(topicIdPartition -> logReadInfo), replicaManager, callback) // delayed remote fetch should still be able to complete assertTrue(delayedRemoteFetch.tryComplete()) @@ -155,8 +156,8 @@ class DelayedRemoteFetchTest { // build a read result with error val logReadInfo = buildReadResult(Errors.FENCED_LEADER_EPOCH) - val delayedRemoteFetch = new DelayedRemoteFetch(null, future, fetchInfo, Seq(topicIdPartition -> fetchStatus), fetchParams, - Seq(topicIdPartition -> logReadInfo), replicaManager, callback) + val delayedRemoteFetch = new DelayedRemoteFetch(null, future, fetchInfo, remoteFetchMaxWaitMs, + Seq(topicIdPartition -> fetchStatus), fetchParams, Seq(topicIdPartition -> logReadInfo), replicaManager, callback) assertTrue(delayedRemoteFetch.tryComplete()) assertTrue(delayedRemoteFetch.isCompleted) @@ -184,8 +185,8 @@ class DelayedRemoteFetchTest { val fetchInfo: RemoteStorageFetchInfo = new RemoteStorageFetchInfo(0, false, topicIdPartition.topicPartition(), null, null, false) val logReadInfo = buildReadResult(Errors.NONE, highWatermark, leaderLogStartOffset) - val delayedRemoteFetch = new DelayedRemoteFetch(remoteFetchTask, future, fetchInfo, Seq(topicIdPartition -> fetchStatus), fetchParams, - Seq(topicIdPartition -> logReadInfo), replicaManager, callback) + val delayedRemoteFetch = new DelayedRemoteFetch(remoteFetchTask, future, fetchInfo, remoteFetchMaxWaitMs, + Seq(topicIdPartition -> fetchStatus), fetchParams, Seq(topicIdPartition -> logReadInfo), replicaManager, callback) when(replicaManager.getPartitionOrException(topicIdPartition.topicPartition)) .thenReturn(mock(classOf[Partition])) diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala index fbb59dd90f..3d62cb9948 100755 --- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala +++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala @@ -124,12 +124,15 @@ class KRaftQuorumImplementation( metaPropertiesEnsemble.verify(Optional.of(clusterId), OptionalInt.of(config.nodeId), util.EnumSet.of(REQUIRE_AT_LEAST_ONE_VALID, REQUIRE_METADATA_LOG_DIR)) - val sharedServer = new SharedServer(config, + val sharedServer = new SharedServer( + config, metaPropertiesEnsemble, time, new Metrics(), controllerQuorumVotersFuture, - faultHandlerFactory) + controllerQuorumVotersFuture.get().values(), + faultHandlerFactory + ) var broker: BrokerServer = null try { broker = new BrokerServer(sharedServer) @@ -362,12 +365,15 @@ abstract class QuorumTestHarness extends Logging { metaPropertiesEnsemble.verify(Optional.of(metaProperties.clusterId().get()), OptionalInt.of(nodeId), util.EnumSet.of(REQUIRE_AT_LEAST_ONE_VALID, REQUIRE_METADATA_LOG_DIR)) - val sharedServer = new SharedServer(config, + val sharedServer = new SharedServer( + config, metaPropertiesEnsemble, Time.SYSTEM, new Metrics(), controllerQuorumVotersFuture, - faultHandlerFactory) + Collections.emptyList(), + faultHandlerFactory + ) var controllerServer: ControllerServer = null try { controllerServer = new ControllerServer( diff --git a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala index 5fd32d8f4b..827ec559cd 100644 --- a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala @@ -72,7 +72,9 @@ object ZkMigrationIntegrationTest { MetadataVersion.IBP_3_7_IV1, MetadataVersion.IBP_3_7_IV2, MetadataVersion.IBP_3_7_IV4, - MetadataVersion.IBP_3_8_IV0 + MetadataVersion.IBP_3_8_IV0, + MetadataVersion.IBP_3_9_IV0 + // Note: ZK Migration is not supported in Apache Kafka 4.0 and beyond. ).map { mv => val serverProperties = new util.HashMap[String, String]() serverProperties.put("inter.broker.listener.name", "EXTERNAL") @@ -491,7 +493,7 @@ class ZkMigrationIntegrationTest { } } - @ClusterTest(types = Array(Type.ZK), brokers = 3, metadataVersion = MetadataVersion.IBP_3_8_IV0, serverProperties = Array( + @ClusterTest(types = Array(Type.ZK), brokers = 3, serverProperties = Array( new ClusterConfigProperty(key = "inter.broker.listener.name", value = "EXTERNAL"), new ClusterConfigProperty(key = "listeners", value = "PLAINTEXT://localhost:0,EXTERNAL://localhost:0"), new ClusterConfigProperty(key = "advertised.listeners", value = "PLAINTEXT://localhost:0,EXTERNAL://localhost:0"), @@ -513,7 +515,7 @@ class ZkMigrationIntegrationTest { val clusterId = zkCluster.clusterId() val kraftCluster = new KafkaClusterTestKit.Builder( new TestKitNodes.Builder(). - setBootstrapMetadataVersion(MetadataVersion.IBP_3_8_IV0). + setBootstrapMetadataVersion(MetadataVersion.IBP_3_9_IV0). setClusterId(Uuid.fromString(clusterId)). setNumBrokerNodes(0). setNumControllerNodes(1).build()) diff --git a/core/src/test/scala/kafka/log/streamaspect/ElasticUnifiedLogTest.scala b/core/src/test/scala/kafka/log/streamaspect/ElasticUnifiedLogTest.scala index 1da0985897..adeebeb084 100644 --- a/core/src/test/scala/kafka/log/streamaspect/ElasticUnifiedLogTest.scala +++ b/core/src/test/scala/kafka/log/streamaspect/ElasticUnifiedLogTest.scala @@ -573,10 +573,6 @@ class ElasticUnifiedLogTest extends UnifiedLogTest { assertNull(readInfo) } - override def testFetchLatestTieredTimestampWithRemoteStorage(): Unit = { - // AutoMQ embedded tiered storage in S3Stream - } - override def testIncrementLocalLogStartOffsetAfterLocalLogDeletion(): Unit = { // AutoMQ embedded tiered storage in S3Stream } diff --git a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala index 457326cd19..3a1fc2e4bd 100644 --- a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala @@ -86,7 +86,7 @@ class KafkaConfigTest { @Test def testBrokerRoleNodeIdValidation(): Unit = { - // Ensure that validation is happening at startup to check that brokers do not use their node.id as a voter in controller.quorum.voters + // Ensure that validation is happening at startup to check that brokers do not use their node.id as a voter in controller.quorum.voters val propertiesFile = new Properties propertiesFile.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") propertiesFile.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") @@ -102,7 +102,7 @@ class KafkaConfigTest { @Test def testControllerRoleNodeIdValidation(): Unit = { - // Ensure that validation is happening at startup to check that controllers use their node.id as a voter in controller.quorum.voters + // Ensure that validation is happening at startup to check that controllers use their node.id as a voter in controller.quorum.voters val propertiesFile = new Properties propertiesFile.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller") propertiesFile.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index 32ddfc6418..2e9bc06897 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -299,7 +299,8 @@ class PartitionLockTest extends Logging { val log = super.createLog(isNew, isFutureReplica, offsetCheckpoints, None, None) val logDirFailureChannel = new LogDirFailureChannel(1) val segments = new LogSegments(log.topicPartition) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.recordVersion, "") + val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( + log.dir, log.topicPartition, logDirFailureChannel, log.config.recordVersion, "", None, mockTime.scheduler) val maxTransactionTimeout = 5 * 60 * 1000 val producerStateManagerConfig = new ProducerStateManagerConfig(TransactionLogConfigs.PRODUCER_ID_EXPIRATION_MS_DEFAULT, false) val producerStateManager = new ProducerStateManager( diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index 2134dcfaaa..6dc6cc2a3c 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -434,7 +434,8 @@ class PartitionTest extends AbstractPartitionTest { val log = super.createLog(isNew, isFutureReplica, offsetCheckpoints, None, None) val logDirFailureChannel = new LogDirFailureChannel(1) val segments = new LogSegments(log.topicPartition) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.recordVersion, "") + val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( + log.dir, log.topicPartition, logDirFailureChannel, log.config.recordVersion, "", None, time.scheduler) val maxTransactionTimeoutMs = 5 * 60 * 1000 val producerStateManagerConfig = new ProducerStateManagerConfig(TransactionLogConfigs.PRODUCER_ID_EXPIRATION_MS_DEFAULT, true) val producerStateManager = new ProducerStateManager( diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index f17c724066..bdbcac462b 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -108,7 +108,8 @@ class LogCleanerManagerTest extends Logging { val maxTransactionTimeoutMs = 5 * 60 * 1000 val producerIdExpirationCheckIntervalMs = TransactionLogConfigs.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT val segments = new LogSegments(tp) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(tpDir, topicPartition, logDirFailureChannel, config.recordVersion, "") + val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( + tpDir, topicPartition, logDirFailureChannel, config.recordVersion, "", None, time.scheduler) val producerStateManager = new ProducerStateManager(topicPartition, tpDir, maxTransactionTimeoutMs, producerStateManagerConfig, time) val offsets = new LogLoader( tpDir, diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 9af898d273..fb7180fe48 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -94,7 +94,7 @@ class LogCleanerTest extends Logging { val mockMetricsGroup = mockMetricsGroupCtor.constructed.get(0) val numMetricsRegistered = LogCleaner.MetricNames.size verify(mockMetricsGroup, times(numMetricsRegistered)).newGauge(anyString(), any()) - + // verify that each metric in `LogCleaner` is removed LogCleaner.MetricNames.foreach(verify(mockMetricsGroup).removeMetric(_)) @@ -169,7 +169,8 @@ class LogCleanerTest extends Logging { val maxTransactionTimeoutMs = 5 * 60 * 1000 val producerIdExpirationCheckIntervalMs = TransactionLogConfigs.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT val logSegments = new LogSegments(topicPartition) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, config.recordVersion, "") + val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( + dir, topicPartition, logDirFailureChannel, config.recordVersion, "", None, time.scheduler) val producerStateManager = new ProducerStateManager(topicPartition, dir, maxTransactionTimeoutMs, producerStateManagerConfig, time) val offsets = new LogLoader( diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index ed91c936ed..bd1ee39a48 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -22,6 +22,7 @@ import kafka.utils.TestUtils import org.apache.kafka.common.config.ConfigDef.Importance.MEDIUM import org.apache.kafka.common.config.ConfigDef.Type.INT import org.apache.kafka.common.config.{ConfigException, SslConfigs, TopicConfig} +import org.apache.kafka.server.common.MetadataVersion import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -297,7 +298,7 @@ class LogConfigTest { props.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, localRetentionMs.toString) props.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, localRetentionBytes.toString) assertThrows(classOf[ConfigException], - () => LogConfig.validate(props, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled)) + () => LogConfig.validate(props, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) } @Test @@ -309,17 +310,17 @@ class LogConfigTest { val logProps = new Properties() logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE) logProps.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") - LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled) + LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) assertThrows(classOf[ConfigException], - () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled)) + () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "delete,compact") assertThrows(classOf[ConfigException], - () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled)) + () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact,delete") assertThrows(classOf[ConfigException], - () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled)) + () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) } @ParameterizedTest(name = "testEnableRemoteLogStorage with sysRemoteStorageEnabled: {0}") @@ -332,10 +333,10 @@ class LogConfigTest { val logProps = new Properties() logProps.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") if (sysRemoteStorageEnabled) { - LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled) + LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) } else { val message = assertThrows(classOf[ConfigException], - () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled)) + () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) assertTrue(message.getMessage.contains("Tiered Storage functionality is disabled in the broker")) } } @@ -355,10 +356,10 @@ class LogConfigTest { logProps.put(TopicConfig.RETENTION_MS_CONFIG, "500") if (sysRemoteStorageEnabled) { val message = assertThrows(classOf[ConfigException], - () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled)) + () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) assertTrue(message.getMessage.contains(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG)) } else { - LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled) + LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) } } @@ -377,10 +378,10 @@ class LogConfigTest { logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, "128") if (sysRemoteStorageEnabled) { val message = assertThrows(classOf[ConfigException], - () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled)) + () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) assertTrue(message.getMessage.contains(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG)) } else { - LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled) + LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) } } @@ -395,10 +396,10 @@ class LogConfigTest { if (sysRemoteStorageEnabled) { val message = assertThrows(classOf[ConfigException], - () => LogConfig.validateBrokerLogConfigValues(kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled)) + () => LogConfig.validateBrokerLogConfigValues(kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) assertTrue(message.getMessage.contains(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG)) } else { - LogConfig.validateBrokerLogConfigValues(kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled) + LogConfig.validateBrokerLogConfigValues(kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) } } @@ -419,4 +420,21 @@ class LogConfigTest { assertEquals(oneDayInMillis, logProps.get(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG)) assertEquals(oneDayInMillis, logProps.get(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG)) } + + @Test + def testValidateWithMetadataVersionJbodSupport(): Unit = { + def validate(metadataVersion: MetadataVersion, jbodConfig: Boolean): Unit = + KafkaConfig.fromProps( + TestUtils.createBrokerConfig(nodeId = 0, zkConnect = null, logDirCount = if (jbodConfig) 2 else 1) + ).validateWithMetadataVersion(metadataVersion) + + validate(MetadataVersion.IBP_3_6_IV2, jbodConfig = false) + validate(MetadataVersion.IBP_3_7_IV0, jbodConfig = false) + validate(MetadataVersion.IBP_3_7_IV2, jbodConfig = false) + assertThrows(classOf[IllegalArgumentException], () => + validate(MetadataVersion.IBP_3_6_IV2, jbodConfig = true)) + assertThrows(classOf[IllegalArgumentException], () => + validate(MetadataVersion.IBP_3_7_IV0, jbodConfig = true)) + validate(MetadataVersion.IBP_3_7_IV2, jbodConfig = true) + } } diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index 1a781a93ea..5287ef67e8 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -128,7 +128,7 @@ class LogLoaderTest { logDirFailureChannel = logDirFailureChannel, time = time, keepPartitionMetadataFile = config.usesTopicId, - remoteStorageSystemEnable = config.remoteLogManagerConfig.enableRemoteStorageSystem(), + remoteStorageSystemEnable = config.remoteLogManagerConfig.isRemoteStorageSystemEnabled(), initialTaskDelayMs = config.logInitialTaskDelayMs) { override def loadLog(logDir: File, hadCleanShutdown: Boolean, recoveryPoints: Map[TopicPartition, Long], @@ -154,7 +154,8 @@ class LogLoaderTest { val logStartOffset = logStartOffsets.getOrElse(topicPartition, 0L) val logDirFailureChannel: LogDirFailureChannel = new LogDirFailureChannel(1) val segments = new LogSegments(topicPartition) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.recordVersion, "") + val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( + logDir, topicPartition, logDirFailureChannel, config.recordVersion, "", None, time.scheduler) val producerStateManager = new ProducerStateManager(topicPartition, logDir, this.maxTransactionTimeoutMs, this.producerStateManagerConfig, time) val logLoader = new LogLoader(logDir, topicPartition, config, time.scheduler, time, @@ -367,7 +368,8 @@ class LogLoaderTest { super.add(wrapper) } } - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, logConfig.recordVersion, "") + val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( + logDir, topicPartition, logDirFailureChannel, logConfig.recordVersion, "", None, mockTime.scheduler) val producerStateManager = new ProducerStateManager(topicPartition, logDir, maxTransactionTimeoutMs, producerStateManagerConfig, mockTime) val logLoader = new LogLoader( @@ -431,7 +433,8 @@ class LogLoaderTest { val logDirFailureChannel: LogDirFailureChannel = new LogDirFailureChannel(1) val config = new LogConfig(new Properties()) val segments = new LogSegments(topicPartition) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.recordVersion, "") + val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( + logDir, topicPartition, logDirFailureChannel, config.recordVersion, "", None, mockTime.scheduler) val offsets = new LogLoader( logDir, topicPartition, @@ -540,7 +543,8 @@ class LogLoaderTest { val config = new LogConfig(logProps) val logDirFailureChannel = null val segments = new LogSegments(topicPartition) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.recordVersion, "") + val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( + logDir, topicPartition, logDirFailureChannel, config.recordVersion, "", None, mockTime.scheduler) val offsets = new LogLoader( logDir, topicPartition, @@ -594,7 +598,8 @@ class LogLoaderTest { val config = new LogConfig(logProps) val logDirFailureChannel = null val segments = new LogSegments(topicPartition) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.recordVersion, "") + val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( + logDir, topicPartition, logDirFailureChannel, config.recordVersion, "", None, mockTime.scheduler) val offsets = new LogLoader( logDir, topicPartition, @@ -647,7 +652,8 @@ class LogLoaderTest { val config = new LogConfig(logProps) val logDirFailureChannel = null val segments = new LogSegments(topicPartition) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.recordVersion, "") + val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( + logDir, topicPartition, logDirFailureChannel, config.recordVersion, "", None, mockTime.scheduler) val offsets = new LogLoader( logDir, topicPartition, @@ -1387,7 +1393,7 @@ class LogLoaderTest { assertEquals(java.util.Arrays.asList(new EpochEntry(1, 0), new EpochEntry(2, 1), new EpochEntry(3, 3)), leaderEpochCache.epochEntries) // deliberately remove some of the epoch entries - leaderEpochCache.truncateFromEnd(2) + leaderEpochCache.truncateFromEndAsyncFlush(2) assertNotEquals(java.util.Arrays.asList(new EpochEntry(1, 0), new EpochEntry(2, 1), new EpochEntry(3, 3)), leaderEpochCache.epochEntries) log.close() @@ -1789,7 +1795,8 @@ class LogLoaderTest { log.logSegments.forEach(segment => segments.add(segment)) assertEquals(5, segments.firstSegment.get.baseOffset) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, logConfig.recordVersion, "") + val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( + logDir, topicPartition, logDirFailureChannel, logConfig.recordVersion, "", None, mockTime.scheduler) val offsets = new LogLoader( logDir, topicPartition, diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 13ce4d28e9..c70aed6c9a 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -46,7 +46,7 @@ import java.nio.file.Files import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, Future} import java.util.{Collections, Optional, OptionalLong, Properties} import org.apache.kafka.server.metrics.KafkaYammerMetrics -import org.apache.kafka.server.util.{KafkaScheduler, MockTime} +import org.apache.kafka.server.util.{KafkaScheduler, MockTime, Scheduler} import org.apache.kafka.storage.internals.log.{CleanerConfig, FetchDataInfo, FetchIsolation, LogConfig, LogDirFailureChannel, LogStartOffsetIncrementReason, ProducerStateManagerConfig, RemoteIndexCache} import org.apache.kafka.storage.internals.checkpoint.CleanShutdownFileHandler import org.junit.jupiter.api.function.Executable @@ -965,7 +965,7 @@ class LogManagerTest { maxTransactionTimeoutMs = 5 * 60 * 1000, producerStateManagerConfig = new ProducerStateManagerConfig(5 * 60 * 1000, false), producerIdExpirationCheckIntervalMs = TransactionLogConfigs.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT, - scheduler = mockTime.scheduler, + scheduler = mock(classOf[Scheduler]), time = mockTime, brokerTopicStats = mockBrokerTopicStats, logDirFailureChannel = mock(classOf[LogDirFailureChannel]), @@ -1286,6 +1286,13 @@ class LogManagerTest { onDisk.foreach(log => assertEquals(expectedStrays.contains(log.topicPartition), LogManager.isStrayKraftReplica(0, image, log))) } + @Test + def testIsStrayKraftMissingTopicId(): Unit = { + val log = Mockito.mock(classOf[UnifiedLog]) + Mockito.when(log.topicId).thenReturn(Option.empty) + assertTrue(LogManager.isStrayKraftReplica(0, topicsImage(Seq()), log)) + } + @Test def testFindStrayReplicasInEmptyLAIR(): Unit = { val onDisk = Seq(foo0, foo1, bar0, bar1, baz0, baz1, baz2, quux0) diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala index 59a3002992..a70b972375 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala @@ -24,7 +24,8 @@ import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{MockTime, Time, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfigs -import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint +import org.apache.kafka.server.util.MockScheduler +import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache import org.apache.kafka.storage.internals.log._ import org.junit.jupiter.api.Assertions._ @@ -33,7 +34,6 @@ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{CsvSource, ValueSource} import java.io.{File, RandomAccessFile} -import java.util import java.util.{Optional, OptionalLong} import scala.collection._ import scala.jdk.CollectionConverters._ @@ -431,17 +431,9 @@ class LogSegmentTest { def testRecoveryRebuildsEpochCache(): Unit = { val seg = createSegment(0) - val checkpoint: LeaderEpochCheckpoint = new LeaderEpochCheckpoint { - private var epochs = Seq.empty[EpochEntry] + val checkpoint: LeaderEpochCheckpointFile = new LeaderEpochCheckpointFile(TestUtils.tempFile(), new LogDirFailureChannel(1)) - override def write(epochs: util.Collection[EpochEntry], ignored: Boolean): Unit = { - this.epochs = epochs.asScala.toSeq - } - - override def read(): java.util.List[EpochEntry] = this.epochs.asJava - } - - val cache = new LeaderEpochFileCache(topicPartition, checkpoint) + val cache = new LeaderEpochFileCache(topicPartition, checkpoint, new MockScheduler(new MockTime())) seg.append(105L, RecordBatch.NO_TIMESTAMP, 104L, MemoryRecords.withRecords(104L, Compression.NONE, 0, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes))) diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 4d7ffe45a2..dfd3020f51 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -2136,92 +2136,6 @@ class UnifiedLogTest { log.fetchOffsetByTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP, Some(remoteLogManager))) } - @Test - def testFetchLatestTieredTimestampNoRemoteStorage(): Unit = { - val logConfig = LogTestUtils.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1) - val log = createLog(logDir, logConfig) - - assertEquals(Some(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, -1, Optional.of(-1))), - log.fetchOffsetByTimestamp(ListOffsetsRequest.LATEST_TIERED_TIMESTAMP)) - - val firstTimestamp = mockTime.milliseconds - val leaderEpoch = 0 - log.appendAsLeader(TestUtils.singletonRecords( - value = TestUtils.randomBytes(10), - timestamp = firstTimestamp), - leaderEpoch = leaderEpoch) - - val secondTimestamp = firstTimestamp + 1 - log.appendAsLeader(TestUtils.singletonRecords( - value = TestUtils.randomBytes(10), - timestamp = secondTimestamp), - leaderEpoch = leaderEpoch) - - assertEquals(Some(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, -1, Optional.of(-1))), - log.fetchOffsetByTimestamp(ListOffsetsRequest.LATEST_TIERED_TIMESTAMP)) - } - - @Test - def testFetchLatestTieredTimestampWithRemoteStorage(): Unit = { - val remoteLogManager = mock(classOf[RemoteLogManager]) - val logConfig = LogTestUtils.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1, - remoteLogStorageEnable = true) - val log = createLog(logDir, logConfig, remoteStorageSystemEnable = true, remoteLogManager = Some(remoteLogManager)) - when(remoteLogManager.findOffsetByTimestamp(log.topicPartition, 0, 0, log.leaderEpochCache.get)) - .thenReturn(Optional.empty[TimestampAndOffset]()) - - assertEquals(None, log.fetchOffsetByTimestamp(0L, Some(remoteLogManager))) - assertEquals(Some(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 0, Optional.empty())), - log.fetchOffsetByTimestamp(ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP, Some(remoteLogManager))) - - val firstTimestamp = mockTime.milliseconds - val firstLeaderEpoch = 0 - log.appendAsLeader(TestUtils.singletonRecords( - value = TestUtils.randomBytes(10), - timestamp = firstTimestamp), - leaderEpoch = firstLeaderEpoch) - - val secondTimestamp = firstTimestamp + 1 - val secondLeaderEpoch = 1 - log.appendAsLeader(TestUtils.singletonRecords( - value = TestUtils.randomBytes(10), - timestamp = secondTimestamp), - leaderEpoch = secondLeaderEpoch) - - when(remoteLogManager.findOffsetByTimestamp(ArgumentMatchers.eq(log.topicPartition), - anyLong(), anyLong(), ArgumentMatchers.eq(log.leaderEpochCache.get))) - .thenAnswer(ans => { - val timestamp = ans.getArgument(1).asInstanceOf[Long] - Optional.of(timestamp) - .filter(_ == firstTimestamp) - .map[TimestampAndOffset](x => new TimestampAndOffset(x, 0L, Optional.of(firstLeaderEpoch))) - }) - log._localLogStartOffset = 1 - log._highestOffsetInRemoteStorage = 0 - - // In the assertions below we test that offset 0 (first timestamp) is in remote and offset 1 (second timestamp) is in local storage. - assertEquals(Some(new TimestampAndOffset(firstTimestamp, 0L, Optional.of(firstLeaderEpoch))), - log.fetchOffsetByTimestamp(firstTimestamp, Some(remoteLogManager))) - assertEquals(Some(new TimestampAndOffset(secondTimestamp, 1L, Optional.of(secondLeaderEpoch))), - log.fetchOffsetByTimestamp(secondTimestamp, Some(remoteLogManager))) - - assertEquals(Some(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 0L, Optional.of(firstLeaderEpoch))), - log.fetchOffsetByTimestamp(ListOffsetsRequest.EARLIEST_TIMESTAMP, Some(remoteLogManager))) - assertEquals(Some(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 0L, Optional.of(firstLeaderEpoch))), - log.fetchOffsetByTimestamp(ListOffsetsRequest.LATEST_TIERED_TIMESTAMP, Some(remoteLogManager))) - assertEquals(Some(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 1L, Optional.of(secondLeaderEpoch))), - log.fetchOffsetByTimestamp(ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP, Some(remoteLogManager))) - assertEquals(Some(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 2L, Optional.of(secondLeaderEpoch))), - log.fetchOffsetByTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP, Some(remoteLogManager))) - - // The cache can be updated directly after a leader change. - // The new latest offset should reflect the updated epoch. - log.maybeAssignEpochStartOffset(2, 2L) - - assertEquals(Some(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 2L, Optional.of(2))), - log.fetchOffsetByTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP, Some(remoteLogManager))) - } - /** * Test the Log truncate operations */ diff --git a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala index 3416ffe65b..da9d29304e 100644 --- a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala +++ b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala @@ -118,6 +118,7 @@ class RaftManagerTest { new Metrics(Time.SYSTEM), Option.empty, CompletableFuture.completedFuture(QuorumConfig.parseVoterConnections(config.quorumVoters)), + QuorumConfig.parseBootstrapServers(config.quorumBootstrapServers), mock(classOf[FaultHandler]) ) } diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala index 116cef5146..9ccce20cf6 100644 --- a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala @@ -47,7 +47,7 @@ object ApiVersionsRequestTest { List(ClusterConfig.defaultBuilder() .setTypes(java.util.Collections.singleton(Type.ZK)) .setServerProperties(serverProperties) - .setMetadataVersion(MetadataVersion.IBP_3_8_IV0) + .setMetadataVersion(MetadataVersion.latestTesting()) .build()).asJava } @@ -67,7 +67,7 @@ object ApiVersionsRequestTest { serverProperties.put("unstable.feature.versions.enable", "false") List(ClusterConfig.defaultBuilder() .setTypes(java.util.Collections.singleton(Type.ZK)) - .setMetadataVersion(MetadataVersion.IBP_3_7_IV4) + .setMetadataVersion(MetadataVersion.latestProduction()) .build()).asJava } @@ -83,7 +83,7 @@ object ApiVersionsRequestTest { class ApiVersionsRequestTest(cluster: ClusterInstance) extends AbstractApiVersionsRequestTest(cluster) { @ClusterTemplate("testApiVersionsRequestTemplate") - @ClusterTest(types = Array(Type.KRAFT, Type.CO_KRAFT), metadataVersion = MetadataVersion.IBP_3_8_IV0, serverProperties = Array( + @ClusterTest(types = Array(Type.KRAFT, Type.CO_KRAFT), serverProperties = Array( new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "false"), new ClusterConfigProperty(key = "unstable.feature.versions.enable", value = "true") )) @@ -108,7 +108,7 @@ class ApiVersionsRequestTest(cluster: ClusterInstance) extends AbstractApiVersio def testApiVersionsRequestThroughControlPlaneListener(): Unit = { val request = new ApiVersionsRequest.Builder().build() val apiVersionsResponse = sendApiVersionsRequest(request, cluster.controlPlaneListenerName().get()) - validateApiVersionsResponse(apiVersionsResponse, cluster.controlPlaneListenerName().get()) + validateApiVersionsResponse(apiVersionsResponse, cluster.controlPlaneListenerName().get(), true) } @ClusterTest(types = Array(Type.KRAFT)) @@ -131,22 +131,25 @@ class ApiVersionsRequestTest(cluster: ClusterInstance) extends AbstractApiVersio assertEquals(ApiKeys.API_VERSIONS.latestVersion(), apiVersion.maxVersion()) } + // Use the latest production MV for this test @ClusterTemplate("testApiVersionsRequestValidationV0Template") - @ClusterTest(types = Array(Type.KRAFT, Type.CO_KRAFT), metadataVersion = MetadataVersion.IBP_3_7_IV4, serverProperties = Array( + @ClusterTest(types = Array(Type.KRAFT, Type.CO_KRAFT), metadataVersion = MetadataVersion.IBP_3_8_IV0, serverProperties = Array( new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "false"), new ClusterConfigProperty(key = "unstable.feature.versions.enable", value = "false"), )) def testApiVersionsRequestValidationV0(): Unit = { val apiVersionsRequest = new ApiVersionsRequest.Builder().build(0.asInstanceOf[Short]) val apiVersionsResponse = sendApiVersionsRequest(apiVersionsRequest, cluster.clientListener()) - validateApiVersionsResponse(apiVersionsResponse, apiVersion = 0) + validateApiVersionsResponse(apiVersionsResponse, apiVersion = 0, + enableUnstableLastVersion = !"false".equals( + cluster.config().serverProperties().get("unstable.api.versions.enable"))) } @ClusterTemplate("zkApiVersionsRequest") def testApiVersionsRequestValidationV0ThroughControlPlaneListener(): Unit = { val apiVersionsRequest = new ApiVersionsRequest.Builder().build(0.asInstanceOf[Short]) val apiVersionsResponse = sendApiVersionsRequest(apiVersionsRequest, cluster.controlPlaneListenerName().get()) - validateApiVersionsResponse(apiVersionsResponse, cluster.controlPlaneListenerName().get()) + validateApiVersionsResponse(apiVersionsResponse, cluster.controlPlaneListenerName().get(), true) } @ClusterTest(types = Array(Type.KRAFT)) diff --git a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala index 34f9d139a0..b0162dc635 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala @@ -285,7 +285,7 @@ class BrokerLifecycleManagerTest { assertEquals(1000L, manager.brokerEpoch) // Trigger JBOD MV update - manager.handleKraftJBODMetadataVersionUpdate() + manager.resendBrokerRegistrationUnlessZkMode() // Accept new registration, response sets epoch to 1200 nextRegistrationRequest(1200L) diff --git a/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestsTest.scala b/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestsTest.scala index b2e4e0f85f..74b7ca34cf 100644 --- a/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestsTest.scala +++ b/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestsTest.scala @@ -16,7 +16,6 @@ */ package kafka.server -import kafka.server.GroupCoordinatorBaseRequestTest import kafka.test.ClusterInstance import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} import kafka.test.junit.ClusterTestExtensions @@ -26,10 +25,14 @@ import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData.{Assign import org.apache.kafka.common.message.{ConsumerGroupDescribeRequestData, ConsumerGroupDescribeResponseData, ConsumerGroupHeartbeatResponseData} import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{ConsumerGroupDescribeRequest, ConsumerGroupDescribeResponse} +import org.apache.kafka.common.resource.ResourceType +import org.apache.kafka.common.utils.Utils +import org.apache.kafka.security.authorizer.AclEntry import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.extension.ExtendWith import org.junit.jupiter.api.{Tag, Timeout} +import java.lang.{Byte => JByte} import scala.jdk.CollectionConverters._ @Timeout(120) @@ -80,6 +83,9 @@ class ConsumerGroupDescribeRequestsTest(cluster: ClusterInstance) extends GroupC val timeoutMs = 5 * 60 * 1000 val clientId = "client-id" val clientHost = "/127.0.0.1" + val authorizedOperationsInt = Utils.to32BitField( + AclEntry.supportedOperations(ResourceType.GROUP).asScala + .map(_.code.asInstanceOf[JByte]).asJava) // Add first group with one member. var grp1Member1Response: ConsumerGroupHeartbeatResponseData = null @@ -126,6 +132,7 @@ class ConsumerGroupDescribeRequestsTest(cluster: ClusterInstance) extends GroupC .setGroupEpoch(1) .setAssignmentEpoch(1) .setAssignorName("uniform") + .setAuthorizedOperations(authorizedOperationsInt) .setMembers(List( new ConsumerGroupDescribeResponseData.Member() .setMemberId(grp1Member1Response.memberId) @@ -141,6 +148,7 @@ class ConsumerGroupDescribeRequestsTest(cluster: ClusterInstance) extends GroupC .setGroupEpoch(grp2Member2Response.memberEpoch) .setAssignmentEpoch(grp2Member2Response.memberEpoch) .setAssignorName("range") + .setAuthorizedOperations(authorizedOperationsInt) .setMembers(List( new ConsumerGroupDescribeResponseData.Member() .setMemberId(grp2Member2Response.memberId) @@ -183,7 +191,8 @@ class ConsumerGroupDescribeRequestsTest(cluster: ClusterInstance) extends GroupC val actual = consumerGroupDescribe( groupIds = List("grp-1", "grp-2"), - version = version.toShort + includeAuthorizedOperations = true, + version = version.toShort, ) assertEquals(expected, actual) diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 44381d7ad3..18dcdb1992 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -48,7 +48,7 @@ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import org.mockito.ArgumentMatchers.anyString import org.mockito.{ArgumentCaptor, ArgumentMatchers, Mockito} -import org.mockito.Mockito.{mock, when} +import org.mockito.Mockito.{mock, verify, verifyNoMoreInteractions, when} import scala.annotation.nowarn import scala.jdk.CollectionConverters._ @@ -793,6 +793,39 @@ class DynamicBrokerConfigTest { verifyIncorrectLogLocalRetentionProps(2000L, 1000L, -1, 100) } + @Test + def testDynamicRemoteFetchMaxWaitMsConfig(): Unit = { + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + val config = KafkaConfig(props) + val kafkaBroker = mock(classOf[KafkaBroker]) + when(kafkaBroker.config).thenReturn(config) + assertEquals(500, config.remoteFetchMaxWaitMs) + + val dynamicRemoteLogConfig = new DynamicRemoteLogConfig(kafkaBroker) + config.dynamicConfig.initialize(None, None) + config.dynamicConfig.addBrokerReconfigurable(dynamicRemoteLogConfig) + + val newProps = new Properties() + newProps.put(RemoteLogManagerConfig.REMOTE_FETCH_MAX_WAIT_MS_PROP, "30000") + // update default config + config.dynamicConfig.validate(newProps, perBrokerConfig = false) + config.dynamicConfig.updateDefaultConfig(newProps) + assertEquals(30000, config.remoteFetchMaxWaitMs) + + // update per broker config + newProps.put(RemoteLogManagerConfig.REMOTE_FETCH_MAX_WAIT_MS_PROP, "10000") + config.dynamicConfig.validate(newProps, perBrokerConfig = true) + config.dynamicConfig.updateBrokerConfig(0, newProps) + assertEquals(10000, config.remoteFetchMaxWaitMs) + + // invalid values + for (maxWaitMs <- Seq(-1, 0)) { + newProps.put(RemoteLogManagerConfig.REMOTE_FETCH_MAX_WAIT_MS_PROP, maxWaitMs.toString) + assertThrows(classOf[ConfigException], () => config.dynamicConfig.validate(newProps, perBrokerConfig = true)) + assertThrows(classOf[ConfigException], () => config.dynamicConfig.validate(newProps, perBrokerConfig = false)) + } + } + @Test def testUpdateDynamicRemoteLogManagerConfig(): Unit = { val origProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) @@ -818,6 +851,117 @@ class DynamicBrokerConfigTest { Mockito.verifyNoMoreInteractions(remoteLogManagerMockOpt.get) } + @Test + def testRemoteLogManagerCopyQuotaUpdates(): Unit = { + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 9092) + val config = KafkaConfig.fromProps(props) + val serverMock: KafkaServer = mock(classOf[KafkaServer]) + val remoteLogManager = mock(classOf[RemoteLogManager]) + + Mockito.when(serverMock.config).thenReturn(config) + Mockito.when(serverMock.remoteLogManagerOpt).thenReturn(Some(remoteLogManager)) + + config.dynamicConfig.initialize(None, None) + config.dynamicConfig.addBrokerReconfigurable(new DynamicRemoteLogConfig(serverMock)) + + assertEquals(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND, + config.remoteLogManagerCopyMaxBytesPerSecond) + + // Update default config + props.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP, "100") + config.dynamicConfig.updateDefaultConfig(props) + assertEquals(100, config.remoteLogManagerCopyMaxBytesPerSecond) + verify(remoteLogManager).updateCopyQuota(100) + + // Update per broker config + props.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP, "200") + config.dynamicConfig.updateBrokerConfig(0, props) + assertEquals(200, config.remoteLogManagerCopyMaxBytesPerSecond) + verify(remoteLogManager).updateCopyQuota(200) + + verifyNoMoreInteractions(remoteLogManager) + } + + @Test + def testRemoteLogManagerFetchQuotaUpdates(): Unit = { + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 9092) + val config = KafkaConfig.fromProps(props) + val serverMock: KafkaServer = mock(classOf[KafkaServer]) + val remoteLogManager = mock(classOf[RemoteLogManager]) + + Mockito.when(serverMock.config).thenReturn(config) + Mockito.when(serverMock.remoteLogManagerOpt).thenReturn(Some(remoteLogManager)) + + config.dynamicConfig.initialize(None, None) + config.dynamicConfig.addBrokerReconfigurable(new DynamicRemoteLogConfig(serverMock)) + + assertEquals(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND, + config.remoteLogManagerFetchMaxBytesPerSecond) + + // Update default config + props.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP, "100") + config.dynamicConfig.updateDefaultConfig(props) + assertEquals(100, config.remoteLogManagerFetchMaxBytesPerSecond) + verify(remoteLogManager).updateFetchQuota(100) + + // Update per broker config + props.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP, "200") + config.dynamicConfig.updateBrokerConfig(0, props) + assertEquals(200, config.remoteLogManagerFetchMaxBytesPerSecond) + verify(remoteLogManager).updateFetchQuota(200) + + verifyNoMoreInteractions(remoteLogManager) + } + + @Test + def testRemoteLogManagerMultipleConfigUpdates(): Unit = { + val indexFileCacheSizeProp = RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP + val copyQuotaProp = RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP + val fetchQuotaProp = RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP + + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 9092) + val config = KafkaConfig.fromProps(props) + val serverMock: KafkaServer = mock(classOf[KafkaServer]) + val remoteLogManager = Mockito.mock(classOf[RemoteLogManager]) + + Mockito.when(serverMock.config).thenReturn(config) + Mockito.when(serverMock.remoteLogManagerOpt).thenReturn(Some(remoteLogManager)) + + config.dynamicConfig.initialize(None, None) + config.dynamicConfig.addBrokerReconfigurable(new DynamicRemoteLogConfig(serverMock)) + + // Default values + assertEquals(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES, config.remoteLogIndexFileCacheTotalSizeBytes) + assertEquals(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND, config.remoteLogManagerCopyMaxBytesPerSecond) + assertEquals(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND, config.remoteLogManagerFetchMaxBytesPerSecond) + + // Update default config + props.put(indexFileCacheSizeProp, "4") + props.put(copyQuotaProp, "100") + props.put(fetchQuotaProp, "200") + config.dynamicConfig.updateDefaultConfig(props) + assertEquals(4, config.remoteLogIndexFileCacheTotalSizeBytes) + assertEquals(100, config.remoteLogManagerCopyMaxBytesPerSecond) + assertEquals(200, config.remoteLogManagerFetchMaxBytesPerSecond) + verify(remoteLogManager).resizeCacheSize(4) + verify(remoteLogManager).updateCopyQuota(100) + verify(remoteLogManager).updateFetchQuota(200) + + // Update per broker config + props.put(indexFileCacheSizeProp, "8") + props.put(copyQuotaProp, "200") + props.put(fetchQuotaProp, "400") + config.dynamicConfig.updateBrokerConfig(0, props) + assertEquals(8, config.remoteLogIndexFileCacheTotalSizeBytes) + assertEquals(200, config.remoteLogManagerCopyMaxBytesPerSecond) + assertEquals(400, config.remoteLogManagerFetchMaxBytesPerSecond) + verify(remoteLogManager).resizeCacheSize(8) + verify(remoteLogManager).updateCopyQuota(200) + verify(remoteLogManager).updateFetchQuota(400) + + verifyNoMoreInteractions(remoteLogManager) + } + def verifyIncorrectLogLocalRetentionProps(logLocalRetentionMs: Long, retentionMs: Long, logLocalRetentionBytes: Long, diff --git a/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala index 847bdf3225..9fad21476e 100644 --- a/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala @@ -421,10 +421,13 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) { protected def consumerGroupDescribe( groupIds: List[String], + includeAuthorizedOperations: Boolean, version: Short = ApiKeys.CONSUMER_GROUP_DESCRIBE.latestVersion(isUnstableApiEnabled) ): List[ConsumerGroupDescribeResponseData.DescribedGroup] = { val consumerGroupDescribeRequest = new ConsumerGroupDescribeRequest.Builder( - new ConsumerGroupDescribeRequestData().setGroupIds(groupIds.asJava) + new ConsumerGroupDescribeRequestData() + .setGroupIds(groupIds.asJava) + .setIncludeAuthorizedOperations(includeAuthorizedOperations) ).build(version) val consumerGroupDescribeResponse = connectAndReceive[ConsumerGroupDescribeResponse](consumerGroupDescribeRequest) diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 668a723259..ec755dcca0 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -77,6 +77,7 @@ import org.apache.kafka.common.utils.{ProducerIdAndEpoch, SecurityUtils, Utils} import org.apache.kafka.coordinator.group.{GroupCoordinator, GroupCoordinatorConfig} import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.raft.QuorumConfig +import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.server.ClientMetricsManager import org.apache.kafka.server.authorizer.{Action, AuthorizationResult, Authorizer} import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_2_IV0, IBP_2_2_IV1} @@ -93,6 +94,7 @@ import org.mockito.ArgumentMatchers._ import org.mockito.Mockito._ import org.mockito.{ArgumentCaptor, ArgumentMatchers, Mockito} +import java.lang.{Byte => JByte} import java.net.InetAddress import java.nio.charset.StandardCharsets import java.time.Duration @@ -7084,10 +7086,12 @@ class KafkaApisTest extends Logging { assertEquals(Errors.GROUP_AUTHORIZATION_FAILED.code, response.data.errorCode) } - @Test - def testConsumerGroupDescribe(): Unit = { + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testConsumerGroupDescribe(includeAuthorizedOperations: Boolean): Unit = { val groupIds = List("group-id-0", "group-id-1", "group-id-2").asJava val consumerGroupDescribeRequestData = new ConsumerGroupDescribeRequestData() + .setIncludeAuthorizedOperations(includeAuthorizedOperations) consumerGroupDescribeRequestData.groupIds.addAll(groupIds) val requestChannelRequest = buildRequest(new ConsumerGroupDescribeRequest.Builder(consumerGroupDescribeRequestData, true).build()) @@ -7101,15 +7105,27 @@ class KafkaApisTest extends Logging { ) kafkaApis.handle(requestChannelRequest, RequestLocal.NoCaching) - val describedGroups = List( + future.complete(List( new DescribedGroup().setGroupId(groupIds.get(0)), new DescribedGroup().setGroupId(groupIds.get(1)), new DescribedGroup().setGroupId(groupIds.get(2)) - ).asJava + ).asJava) - future.complete(describedGroups) + var authorizedOperationsInt = Int.MinValue; + if (includeAuthorizedOperations) { + authorizedOperationsInt = Utils.to32BitField( + AclEntry.supportedOperations(ResourceType.GROUP).asScala + .map(_.code.asInstanceOf[JByte]).asJava) + } + + // Can't reuse the above list here because we would not test the implementation in KafkaApis then + val describedGroups = List( + new DescribedGroup().setGroupId(groupIds.get(0)), + new DescribedGroup().setGroupId(groupIds.get(1)), + new DescribedGroup().setGroupId(groupIds.get(2)) + ).map(group => group.setAuthorizedOperations(authorizedOperationsInt)) val expectedConsumerGroupDescribeResponseData = new ConsumerGroupDescribeResponseData() - .setGroups(describedGroups) + .setGroups(describedGroups.asJava) val response = verifyNoThrottling[ConsumerGroupDescribeResponse](requestChannelRequest) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 933f514df5..266b64560f 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -19,7 +19,7 @@ package kafka.server import java.net.InetSocketAddress import java.util -import java.util.{Collections, Properties} +import java.util.{Arrays, Collections, Properties} import kafka.cluster.EndPoint import kafka.security.authorizer.AclAuthorizer import kafka.utils.TestUtils.assertBadConfigContainingMessage @@ -1032,6 +1032,7 @@ class KafkaConfigTest { // Raft Quorum Configs case QuorumConfig.QUORUM_VOTERS_CONFIG => // ignore string + case QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG => // ignore string case QuorumConfig.QUORUM_ELECTION_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case QuorumConfig.QUORUM_FETCH_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case QuorumConfig.QUORUM_ELECTION_BACKOFF_MAX_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") @@ -1402,6 +1403,23 @@ class KafkaConfigTest { assertEquals(expectedVoters, addresses) } + @Test + def testParseQuorumBootstrapServers(): Unit = { + val expected = Arrays.asList( + InetSocketAddress.createUnresolved("kafka1", 9092), + InetSocketAddress.createUnresolved("kafka2", 9092) + ) + + val props = TestUtils.createBrokerConfig(0, null) + props.setProperty(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, "kafka1:9092,kafka2:9092") + + val addresses = QuorumConfig.parseBootstrapServers( + KafkaConfig.fromProps(props).quorumBootstrapServers + ) + + assertEquals(expected, addresses) + } + @Test def testAcceptsLargeNodeIdForRaftBasedCase(): Unit = { // Generation of Broker IDs is not supported when using Raft-based controller quorums, diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala index 7791c6c942..2dab34f7d4 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala @@ -88,7 +88,7 @@ class OffsetCommitRequestTest(cluster: ClusterInstance) extends GroupCoordinator topic = "foo", partition = 0, offset = 100L, - expectedError = Errors.NONE, + expectedError = if (useNewProtocol && version < 9) Errors.UNSUPPORTED_VERSION else Errors.NONE, version = version.toShort ) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 128e694b45..bb0f699cbb 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -18,7 +18,7 @@ package kafka.server import com.google.common.util.concurrent.MoreExecutors -import com.yammer.metrics.core.{Gauge, Meter} +import com.yammer.metrics.core.{Gauge, Meter, Timer} import kafka.api._ import kafka.cluster.PartitionTest.MockPartitionListener import kafka.cluster.{BrokerEndPoint, Partition} @@ -34,7 +34,7 @@ import kafka.zk.KafkaZkClient import org.apache.kafka.clients.FetchSessionHandler import org.apache.kafka.common.{DirectoryId, IsolationLevel, Node, TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.compress.Compression -import org.apache.kafka.common.config.{AbstractConfig, TopicConfig} +import org.apache.kafka.common.config.{TopicConfig} import org.apache.kafka.common.errors.{InvalidPidMappingException, KafkaStorageException} import org.apache.kafka.common.message.LeaderAndIsrRequestData import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState @@ -2917,7 +2917,8 @@ class ReplicaManagerTest { val maxTransactionTimeoutMs = 30000 val maxProducerIdExpirationMs = 30000 val segments = new LogSegments(tp) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(logDir, tp, mockLogDirFailureChannel, logConfig.recordVersion, "") + val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( + logDir, tp, mockLogDirFailureChannel, logConfig.recordVersion, "", None, time.scheduler) val producerStateManager = new ProducerStateManager(tp, logDir, maxTransactionTimeoutMs, new ProducerStateManagerConfig(maxProducerIdExpirationMs, true), time) val offsets = new LogLoader( @@ -3424,7 +3425,7 @@ class ReplicaManagerTest { when(mockLog.remoteLogEnabled()).thenReturn(enableRemoteStorage) when(mockLog.remoteStorageSystemEnable).thenReturn(enableRemoteStorage) val aliveBrokers = aliveBrokerIds.map(brokerId => new Node(brokerId, s"host$brokerId", brokerId)) - brokerTopicStats = new BrokerTopicStats(KafkaConfig.fromProps(props).remoteLogManagerConfig.enableRemoteStorageSystem) + brokerTopicStats = new BrokerTopicStats(KafkaConfig.fromProps(props).remoteLogManagerConfig.isRemoteStorageSystemEnabled) val metadataCache: MetadataCache = mock(classOf[MetadataCache]) when(metadataCache.topicIdInfo()).thenReturn((topicIds.asJava, topicNames.asJava)) @@ -4130,12 +4131,11 @@ class ReplicaManagerTest { props.put(RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP, classOf[NoOpRemoteLogMetadataManager].getName) // set log reader threads number to 2 props.put(RemoteLogManagerConfig.REMOTE_LOG_READER_THREADS_PROP, 2.toString) - val config = new AbstractConfig(RemoteLogManagerConfig.CONFIG_DEF, props) - val remoteLogManagerConfig = new RemoteLogManagerConfig(config) + val config = KafkaConfig.fromProps(props) val mockLog = mock(classOf[UnifiedLog]) - val brokerTopicStats = new BrokerTopicStats(KafkaConfig.fromProps(props).isRemoteLogStorageSystemEnabled) + val brokerTopicStats = new BrokerTopicStats(config.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) val remoteLogManager = new RemoteLogManager( - remoteLogManagerConfig, + config, 0, TestUtils.tempRelativeDir("data").getAbsolutePath, "clusterId", @@ -4173,14 +4173,17 @@ class ReplicaManagerTest { val params = new FetchParams(ApiKeys.FETCH.latestVersion, replicaId, 1, 1000, 10, 100, FetchIsolation.LOG_END, None.asJava) val fetchOffset = 1 + val responseLatch = new CountDownLatch(5) def fetchCallback(responseStatus: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = { assertEquals(1, responseStatus.size) assertEquals(tidp0, responseStatus.toMap.keySet.head) + responseLatch.countDown() } assertEquals(1.0, yammerMetricValue("RemoteLogReaderAvgIdlePercent").asInstanceOf[Double]) assertEquals(0, yammerMetricValue("RemoteLogReaderTaskQueueSize").asInstanceOf[Int]) + assertEquals(0L, yammerMetricValue("RemoteLogReaderFetchRateAndTimeMs").asInstanceOf[Long]) // our thread number is 2 val queueLatch = new CountDownLatch(2) @@ -4205,6 +4208,8 @@ class ReplicaManagerTest { assertEquals(3, yammerMetricValue("RemoteLogReaderTaskQueueSize").asInstanceOf[Int]) // unlock all tasks doneLatch.countDown() + responseLatch.await(5000, TimeUnit.MILLISECONDS) + assertEquals(5L, yammerMetricValue("RemoteLogReaderFetchRateAndTimeMs").asInstanceOf[Long]) } finally { Utils.tryAll(util.Arrays.asList[Callable[Void]]( () => { @@ -4216,6 +4221,8 @@ class ReplicaManagerTest { null } )) + val allMetrics = KafkaYammerMetrics.defaultRegistry.allMetrics.asScala + assertFalse(allMetrics.exists { case (n, _) => n.getMBeanName.endsWith("RemoteLogReaderFetchRateAndTimeMs") }) } } @@ -4230,12 +4237,11 @@ class ReplicaManagerTest { props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, true.toString) props.put(RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP, classOf[NoOpRemoteStorageManager].getName) props.put(RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP, classOf[NoOpRemoteLogMetadataManager].getName) - val config = new AbstractConfig(RemoteLogManagerConfig.CONFIG_DEF, props) - val remoteLogManagerConfig = new RemoteLogManagerConfig(config) + val config = KafkaConfig.fromProps(props) val dummyLog = mock(classOf[UnifiedLog]) - val brokerTopicStats = new BrokerTopicStats(KafkaConfig.fromProps(props).isRemoteLogStorageSystemEnabled) + val brokerTopicStats = new BrokerTopicStats(config.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) val remoteLogManager = new RemoteLogManager( - remoteLogManagerConfig, + config, 0, TestUtils.tempRelativeDir("data").getAbsolutePath, "clusterId", @@ -4330,6 +4336,7 @@ class ReplicaManagerTest { metric match { case m: Gauge[_] => m.value case m: Meter => m.count() + case m: Timer => m.count() case m => fail(s"Unexpected broker metric of class ${m.getClass}") } } @@ -6008,6 +6015,7 @@ class ReplicaManagerTest { .setPartitionId(0) .setTopicId(FOO_UUID) .setReplicas(util.Arrays.asList(localId, localId + 1, localId + 2)) + .setDirectories(util.Arrays.asList(Uuid.fromString("fKgQ2axkQiuzt4ANqKbPkQ"), DirectoryId.UNASSIGNED, DirectoryId.UNASSIGNED)) .setIsr(util.Arrays.asList(localId, localId + 1)) ) followerMetadataImage = imageFromTopics(followerTopicsDelta.apply()) @@ -6491,6 +6499,39 @@ class ReplicaManagerTest { assertEquals(Errors.NONE.code, response.errorCode) assertTrue(response.totalBytes > 0) assertTrue(response.usableBytes >= 0) + assertFalse(response.topics().isEmpty) + response.topics().forEach(t => assertFalse(t.partitions().isEmpty)) + } + } finally { + replicaManager.shutdown(checkpointHW = false) + } + } + + @Test + def testDescribeLogDirsWithoutAnyPartitionTopic(): Unit = { + val noneTopic = "none-topic" + val topicPartition = 0 + val topicId = Uuid.randomUuid() + val followerBrokerId = 0 + val leaderBrokerId = 1 + val leaderEpoch = 1 + val leaderEpochIncrement = 2 + val countDownLatch = new CountDownLatch(1) + val offsetFromLeader = 5 + + // Prepare the mocked components for the test + val (replicaManager, mockLogMgr) = prepareReplicaManagerAndLogManager(new MockTimer(time), + topicPartition, leaderEpoch + leaderEpochIncrement, followerBrokerId, leaderBrokerId, countDownLatch, + expectTruncation = false, localLogOffset = Some(10), offsetFromLeader = offsetFromLeader, topicId = Some(topicId)) + + try { + val responses = replicaManager.describeLogDirs(Set(new TopicPartition(noneTopic, topicPartition))) + assertEquals(mockLogMgr.liveLogDirs.size, responses.size) + responses.foreach { response => + assertEquals(Errors.NONE.code, response.errorCode) + assertTrue(response.totalBytes > 0) + assertTrue(response.usableBytes >= 0) + assertTrue(response.topics().isEmpty) } } finally { replicaManager.shutdown(checkpointHW = false) @@ -6530,7 +6571,7 @@ class ReplicaManagerTest { partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None) val leaderAndIsr = LeaderAndIsr(0, 1, List(0, 1), LeaderRecoveryState.RECOVERED, LeaderAndIsr.InitialPartitionEpoch) - val becomeLeaderRequest = makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), leaderAndIsr) + val becomeLeaderRequest = makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), leaderAndIsr) replicaManager.becomeLeaderOrFollower(1, becomeLeaderRequest, (_, _) => ()) verifyRLMOnLeadershipChange(Collections.singleton(partition), Collections.emptySet()) @@ -6711,6 +6752,79 @@ class ReplicaManagerTest { )) } } + + @Test + def testRemoteReadQuotaExceeded(): Unit = { + when(mockRemoteLogManager.isRemoteLogFetchQuotaExceeded).thenReturn(true) + + val tp0 = new TopicPartition(topic, 0) + val tpId0 = new TopicIdPartition(topicId, tp0) + val fetch: Seq[(TopicIdPartition, LogReadResult)] = readFromLogWithOffsetOutOfRange(tp0) + + assertEquals(1, fetch.size) + assertEquals(tpId0, fetch.head._1) + val fetchInfo = fetch.head._2.info + assertEquals(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, fetchInfo.fetchOffsetMetadata) + assertFalse(fetchInfo.records.records().iterator().hasNext) + assertFalse(fetchInfo.firstEntryIncomplete) + assertFalse(fetchInfo.abortedTransactions.isPresent) + assertFalse(fetchInfo.delayedRemoteStorageFetch.isPresent) + } + + @Test + def testRemoteReadQuotaNotExceeded(): Unit = { + when(mockRemoteLogManager.isRemoteLogFetchQuotaExceeded).thenReturn(false) + + val tp0 = new TopicPartition(topic, 0) + val tpId0 = new TopicIdPartition(topicId, tp0) + val fetch: Seq[(TopicIdPartition, LogReadResult)] = readFromLogWithOffsetOutOfRange(tp0) + + assertEquals(1, fetch.size) + assertEquals(tpId0, fetch.head._1) + val fetchInfo = fetch.head._2.info + assertEquals(1L, fetchInfo.fetchOffsetMetadata.messageOffset) + assertEquals(UnifiedLog.UnknownOffset, fetchInfo.fetchOffsetMetadata.segmentBaseOffset) + assertEquals(-1, fetchInfo.fetchOffsetMetadata.relativePositionInSegment) + assertEquals(MemoryRecords.EMPTY, fetchInfo.records) + assertTrue(fetchInfo.delayedRemoteStorageFetch.isPresent) + } + + private def readFromLogWithOffsetOutOfRange(tp: TopicPartition): Seq[(TopicIdPartition, LogReadResult)] = { + val replicaManager = setupReplicaManagerWithMockedPurgatories(new MockTimer(time), aliveBrokerIds = Seq(0, 1, 2), enableRemoteStorage = true, shouldMockLog = true) + try { + val offsetCheckpoints = new LazyOffsetCheckpoints(replicaManager.highWatermarkCheckpoints) + replicaManager.createPartition(tp).createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints = offsetCheckpoints, None) + val partition0Replicas = Seq[Integer](0, 1).asJava + val topicIds = Map(tp.topic -> topicId).asJava + val leaderEpoch = 0 + val leaderAndIsrRequest = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, + Seq( + new LeaderAndIsrPartitionState() + .setTopicName(tp.topic) + .setPartitionIndex(tp.partition) + .setControllerEpoch(0) + .setLeader(0) + .setLeaderEpoch(0) + .setIsr(partition0Replicas) + .setPartitionEpoch(0) + .setReplicas(partition0Replicas) + .setIsNew(true) + ).asJava, + topicIds, + Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() + replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest, (_, _) => ()) + + val params = new FetchParams(ApiKeys.FETCH.latestVersion, -1, 1, 1000, 0, 100, FetchIsolation.HIGH_WATERMARK, None.asJava) + replicaManager.readFromLog( + params, + Seq(new TopicIdPartition(topicId, 0, topic) -> new PartitionData(topicId, 1, 0, 100000, Optional.of[Integer](leaderEpoch), Optional.of(leaderEpoch))), + UnboundedQuota, + readFromPurgatory = false) + } finally { + replicaManager.shutdown(checkpointHW = false) + } + } + // Some threads are closed, but the state didn't reflect in the JVM immediately, so add some wait time for it private def assertNoNonDaemonThreadsWithWaiting(threadNamePrefix: String, waitTimeMs: Long = 500L): Unit = { var nonDemonThreads: mutable.Set[Thread] = mutable.Set.empty[Thread] diff --git a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala index 8b4bb93a66..4133aba5b4 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala @@ -299,7 +299,7 @@ class ReplicationQuotasTest extends QuorumTestHarness { features.add(new BrokerRegistrationRequestData.Feature() .setName(MetadataVersion.FEATURE_NAME) .setMinSupportedVersion(MetadataVersion.IBP_3_0_IV1.featureLevel()) - .setMaxSupportedVersion(MetadataVersion.IBP_3_8_IV0.featureLevel())) + .setMaxSupportedVersion(MetadataVersion.latestTesting().featureLevel())) controllerServer.controller.registerBroker( ControllerRequestContextUtil.ANONYMOUS_CONTEXT, new BrokerRegistrationRequestData() diff --git a/core/src/test/scala/unit/kafka/server/checkpoints/InMemoryLeaderEpochCheckpointTest.scala b/core/src/test/scala/unit/kafka/server/checkpoints/InMemoryLeaderEpochCheckpointTest.scala deleted file mode 100644 index 3af126f5c5..0000000000 --- a/core/src/test/scala/unit/kafka/server/checkpoints/InMemoryLeaderEpochCheckpointTest.scala +++ /dev/null @@ -1,58 +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. - */ -package kafka.server.checkpoints - -import org.apache.kafka.storage.internals.checkpoint.InMemoryLeaderEpochCheckpoint -import org.apache.kafka.storage.internals.log.EpochEntry -import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.Test - -import java.io.{BufferedReader, ByteArrayInputStream, InputStreamReader} -import java.nio.charset.StandardCharsets - -class InMemoryLeaderEpochCheckpointTest { - - @Test - def shouldAppendNewEntry(): Unit = { - val checkpoint = new InMemoryLeaderEpochCheckpoint() - val epochs = java.util.Arrays.asList(new EpochEntry(0, 1L), new EpochEntry(1, 2L), new EpochEntry(2, 3L)) - checkpoint.write(epochs) - assertEquals(epochs, checkpoint.read()) - - val epochs2 = java.util.Arrays.asList(new EpochEntry(3, 4L), new EpochEntry(4, 5L)) - checkpoint.write(epochs2) - - assertEquals(epochs2, checkpoint.read()) - } - - @Test - def testReadAsByteBuffer(): Unit = { - val checkpoint = new InMemoryLeaderEpochCheckpoint() - val expectedEpoch = 0 - val expectedStartOffset = 1L - val expectedVersion = 0 - val epochs = java.util.Arrays.asList(new EpochEntry(expectedEpoch, expectedStartOffset)) - checkpoint.write(epochs) - assertEquals(epochs, checkpoint.read()) - val buffer = checkpoint.readAsByteBuffer() - - val bufferedReader = new BufferedReader(new InputStreamReader(new ByteArrayInputStream(buffer.array()), StandardCharsets.UTF_8)) - assertEquals(expectedVersion.toString, bufferedReader.readLine()) - assertEquals(epochs.size().toString, bufferedReader.readLine()) - assertEquals(s"$expectedEpoch $expectedStartOffset", bufferedReader.readLine()) - } -} diff --git a/core/src/test/scala/unit/kafka/server/checkpoints/OffsetCheckpointFileWithFailureHandlerTest.scala b/core/src/test/scala/unit/kafka/server/checkpoints/OffsetCheckpointFileWithFailureHandlerTest.scala index a7e370d7f4..7808cedb07 100644 --- a/core/src/test/scala/unit/kafka/server/checkpoints/OffsetCheckpointFileWithFailureHandlerTest.scala +++ b/core/src/test/scala/unit/kafka/server/checkpoints/OffsetCheckpointFileWithFailureHandlerTest.scala @@ -19,12 +19,13 @@ package kafka.server.checkpoints import kafka.utils.{Logging, TestUtils} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.KafkaStorageException -import org.apache.kafka.storage.internals.checkpoint.CheckpointFileWithFailureHandler -import org.apache.kafka.storage.internals.log.LogDirFailureChannel +import org.apache.kafka.storage.internals.checkpoint.{CheckpointFileWithFailureHandler, LeaderEpochCheckpointFile} +import org.apache.kafka.storage.internals.log.{EpochEntry, LogDirFailureChannel} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import org.mockito.Mockito +import java.io.File import java.util.Collections import scala.collection.Map @@ -97,7 +98,7 @@ class OffsetCheckpointFileWithFailureHandlerTest extends Logging { val logDirFailureChannel = new LogDirFailureChannel(10) val checkpointFile = new CheckpointFileWithFailureHandler(file, OffsetCheckpointFile.CurrentVersion + 1, OffsetCheckpointFile.Formatter, logDirFailureChannel, file.getParent) - checkpointFile.write(Collections.singletonList(new TopicPartition("foo", 5) -> 10L), true) + checkpointFile.write(Collections.singletonList(new TopicPartition("foo", 5) -> 10L)) assertThrows(classOf[KafkaStorageException], () => new OffsetCheckpointFile(checkpointFile.file, logDirFailureChannel).read()) } @@ -133,4 +134,15 @@ class OffsetCheckpointFileWithFailureHandlerTest extends Logging { assertThrows(classOf[IllegalArgumentException], () => lazyCheckpoints.fetch("/invalid/kafka-logs", new TopicPartition("foo", 0))) } + @Test + def testWriteIfDirExistsShouldNotThrowWhenDirNotExists(): Unit = { + val dir = TestUtils.tempDir() + val file = dir.toPath.resolve("test-checkpoint").toFile + val logDirFailureChannel = new LogDirFailureChannel(10) + val checkpointFile = new CheckpointFileWithFailureHandler(file, 0, + LeaderEpochCheckpointFile.FORMATTER, logDirFailureChannel, file.getParent) + + dir.renameTo(new File(dir.getAbsolutePath + "-renamed")) + checkpointFile.writeIfDirExists(Collections.singletonList(new EpochEntry(1, 42))) + } } diff --git a/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochFileCacheTest.scala b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochFileCacheTest.scala index 05041f3970..6f6d0bdbda 100644 --- a/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochFileCacheTest.scala +++ b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochFileCacheTest.scala @@ -20,15 +20,15 @@ package kafka.server.epoch import kafka.utils.TestUtils import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET} -import org.apache.kafka.storage.internals.checkpoint.{LeaderEpochCheckpoint, LeaderEpochCheckpointFile} +import org.apache.kafka.server.util.MockTime +import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache import org.apache.kafka.storage.internals.log.{EpochEntry, LogDirFailureChannel} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import java.io.File -import java.util.{Collections, OptionalInt, Optional} -import scala.collection.Seq +import java.util.{Collections, Optional, OptionalInt} import scala.jdk.CollectionConverters._ /** @@ -36,13 +36,10 @@ import scala.jdk.CollectionConverters._ */ class LeaderEpochFileCacheTest { val tp = new TopicPartition("TestTopic", 5) - private val checkpoint: LeaderEpochCheckpoint = new LeaderEpochCheckpoint { - private var epochs: Seq[EpochEntry] = Seq() - override def write(epochs: java.util.Collection[EpochEntry], ignored: Boolean): Unit = this.epochs = epochs.asScala.toSeq - override def read(): java.util.List[EpochEntry] = this.epochs.asJava - } + val mockTime = new MockTime() + private val checkpoint: LeaderEpochCheckpointFile = new LeaderEpochCheckpointFile(TestUtils.tempFile(), new LogDirFailureChannel(1)) - private val cache = new LeaderEpochFileCache(tp, checkpoint) + private val cache = new LeaderEpochFileCache(tp, checkpoint, mockTime.scheduler) @Test def testPreviousEpoch(): Unit = { @@ -57,7 +54,7 @@ class LeaderEpochFileCacheTest { cache.assign(10, 20) assertEquals(OptionalInt.of(4), cache.previousEpoch) - cache.truncateFromEnd(18) + cache.truncateFromEndAsyncFlush(18) assertEquals(OptionalInt.of(2), cache.previousEpoch) } @@ -245,12 +242,12 @@ class LeaderEpochFileCacheTest { val checkpoint = new LeaderEpochCheckpointFile(new File(checkpointPath), new LogDirFailureChannel(1)) //Given - val cache = new LeaderEpochFileCache(tp, checkpoint) + val cache = new LeaderEpochFileCache(tp, checkpoint, new MockTime().scheduler) cache.assign(2, 6) //When val checkpoint2 = new LeaderEpochCheckpointFile(new File(checkpointPath), new LogDirFailureChannel(1)) - val cache2 = new LeaderEpochFileCache(tp, checkpoint2) + val cache2 = new LeaderEpochFileCache(tp, checkpoint2, new MockTime().scheduler) //Then assertEquals(1, cache2.epochEntries.size) @@ -387,7 +384,7 @@ class LeaderEpochFileCacheTest { cache.assign(4, 11) //When clear latest on epoch boundary - cache.truncateFromEnd(8) + cache.truncateFromEndAsyncFlush(8) //Then should remove two latest epochs (remove is inclusive) assertEquals(java.util.Arrays.asList(new EpochEntry(2, 6)), cache.epochEntries) @@ -401,7 +398,7 @@ class LeaderEpochFileCacheTest { cache.assign(4, 11) //When reset to offset ON epoch boundary - cache.truncateFromStart(8) + cache.truncateFromStartAsyncFlush(8) //Then should preserve (3, 8) assertEquals(java.util.Arrays.asList(new EpochEntry(3, 8), new EpochEntry(4, 11)), cache.epochEntries) @@ -415,7 +412,7 @@ class LeaderEpochFileCacheTest { cache.assign(4, 11) //When reset to offset BETWEEN epoch boundaries - cache.truncateFromStart(9) + cache.truncateFromStartAsyncFlush(9) //Then we should retain epoch 3, but update it's offset to 9 as 8 has been removed assertEquals(java.util.Arrays.asList(new EpochEntry(3, 9), new EpochEntry(4, 11)), cache.epochEntries) @@ -429,7 +426,7 @@ class LeaderEpochFileCacheTest { cache.assign(4, 11) //When reset to offset before first epoch offset - cache.truncateFromStart(1) + cache.truncateFromStartAsyncFlush(1) //Then nothing should change assertEquals(java.util.Arrays.asList(new EpochEntry(2, 6),new EpochEntry(3, 8), new EpochEntry(4, 11)), cache.epochEntries) @@ -443,7 +440,7 @@ class LeaderEpochFileCacheTest { cache.assign(4, 11) //When reset to offset on earliest epoch boundary - cache.truncateFromStart(6) + cache.truncateFromStartAsyncFlush(6) //Then nothing should change assertEquals(java.util.Arrays.asList(new EpochEntry(2, 6),new EpochEntry(3, 8), new EpochEntry(4, 11)), cache.epochEntries) @@ -457,7 +454,7 @@ class LeaderEpochFileCacheTest { cache.assign(4, 11) //When - cache.truncateFromStart(11) + cache.truncateFromStartAsyncFlush(11) //Then retain the last assertEquals(Collections.singletonList(new EpochEntry(4, 11)), cache.epochEntries) @@ -471,7 +468,7 @@ class LeaderEpochFileCacheTest { cache.assign(4, 11) //When we clear from a position between offset 8 & offset 11 - cache.truncateFromStart(9) + cache.truncateFromStartAsyncFlush(9) //Then we should update the middle epoch entry's offset assertEquals(java.util.Arrays.asList(new EpochEntry(3, 9), new EpochEntry(4, 11)), cache.epochEntries) @@ -485,7 +482,7 @@ class LeaderEpochFileCacheTest { cache.assign(2, 10) //When we clear from a position between offset 0 & offset 7 - cache.truncateFromStart(5) + cache.truncateFromStartAsyncFlush(5) //Then we should keep epoch 0 but update the offset appropriately assertEquals(java.util.Arrays.asList(new EpochEntry(0,5), new EpochEntry(1, 7), new EpochEntry(2, 10)), @@ -500,7 +497,7 @@ class LeaderEpochFileCacheTest { cache.assign(4, 11) //When reset to offset beyond last epoch - cache.truncateFromStart(15) + cache.truncateFromStartAsyncFlush(15) //Then update the last assertEquals(Collections.singletonList(new EpochEntry(4, 15)), cache.epochEntries) @@ -514,7 +511,7 @@ class LeaderEpochFileCacheTest { cache.assign(4, 11) //When reset to offset BETWEEN epoch boundaries - cache.truncateFromEnd( 9) + cache.truncateFromEndAsyncFlush( 9) //Then should keep the preceding epochs assertEquals(OptionalInt.of(3), cache.latestEpoch) @@ -543,7 +540,7 @@ class LeaderEpochFileCacheTest { cache.assign(4, 11) //When reset to offset on epoch boundary - cache.truncateFromStart(UNDEFINED_EPOCH_OFFSET) + cache.truncateFromStartAsyncFlush(UNDEFINED_EPOCH_OFFSET) //Then should do nothing assertEquals(3, cache.epochEntries.size) @@ -557,7 +554,7 @@ class LeaderEpochFileCacheTest { cache.assign(4, 11) //When reset to offset on epoch boundary - cache.truncateFromEnd(UNDEFINED_EPOCH_OFFSET) + cache.truncateFromEndAsyncFlush(UNDEFINED_EPOCH_OFFSET) //Then should do nothing assertEquals(3, cache.epochEntries.size) @@ -578,13 +575,13 @@ class LeaderEpochFileCacheTest { @Test def shouldClearEarliestOnEmptyCache(): Unit = { //Then - cache.truncateFromStart(7) + cache.truncateFromStartAsyncFlush(7) } @Test def shouldClearLatestOnEmptyCache(): Unit = { //Then - cache.truncateFromEnd(7) + cache.truncateFromEndAsyncFlush(7) } @Test @@ -600,7 +597,7 @@ class LeaderEpochFileCacheTest { cache.assign(10, 20) assertEquals(OptionalInt.of(4), cache.previousEpoch(10)) - cache.truncateFromEnd(18) + cache.truncateFromEndAsyncFlush(18) assertEquals(OptionalInt.of(2), cache.previousEpoch(cache.latestEpoch.getAsInt)) } @@ -617,7 +614,7 @@ class LeaderEpochFileCacheTest { cache.assign(10, 20) assertEquals(Optional.of(new EpochEntry(4, 15)), cache.previousEntry(10)) - cache.truncateFromEnd(18) + cache.truncateFromEndAsyncFlush(18) assertEquals(Optional.of(new EpochEntry(2, 10)), cache.previousEntry(cache.latestEpoch.getAsInt)) } @@ -658,4 +655,15 @@ class LeaderEpochFileCacheTest { assertEquals(OptionalInt.empty(), cache.epochForOffset(5)) } + @Test + def shouldWriteCheckpointOnTruncation(): Unit = { + cache.assign(2, 6) + cache.assign(3, 8) + cache.assign(4, 11) + + cache.truncateFromEndAsyncFlush(11) + cache.truncateFromStartAsyncFlush(8) + + assertEquals(List(new EpochEntry(3, 8)).asJava, checkpoint.read()) + } } diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala index a57d99cfd2..b4f3aa9a95 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala @@ -31,7 +31,6 @@ import org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, ConfigEntry, NewTopic} import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.config.ConfigResource.Type.BROKER -import org.apache.kafka.common.metadata.FeatureLevelRecord import org.apache.kafka.common.utils.Exit import org.apache.kafka.coordinator.group.GroupCoordinator import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataImageTest, MetadataProvenance} @@ -44,7 +43,7 @@ import org.junit.jupiter.api.Assertions.{assertEquals, assertNotNull, assertTrue import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.ArgumentMatchers.any import org.mockito.Mockito -import org.mockito.Mockito.{clearInvocations, doThrow, mock, times, verify, verifyNoInteractions} +import org.mockito.Mockito.{doThrow, mock, verify} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer @@ -226,103 +225,4 @@ class BrokerMetadataPublisherTest { verify(groupCoordinator).onNewMetadataImage(image, delta) } - - @Test - def testMetadataVersionUpdateToIBP_3_7_IV2OrAboveTriggersBrokerReRegistration(): Unit = { - val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, "")) - val metadataCache = new KRaftMetadataCache(0) - val logManager = mock(classOf[LogManager]) - val replicaManager = mock(classOf[ReplicaManager]) - val groupCoordinator = mock(classOf[GroupCoordinator]) - val faultHandler = mock(classOf[FaultHandler]) - val brokerLifecycleManager = mock(classOf[BrokerLifecycleManager]) - - val metadataPublisher = new BrokerMetadataPublisher( - config, - metadataCache, - logManager, - replicaManager, - groupCoordinator, - mock(classOf[TransactionCoordinator]), - mock(classOf[DynamicConfigPublisher]), - mock(classOf[DynamicClientQuotaPublisher]), - mock(classOf[ScramPublisher]), - mock(classOf[DelegationTokenPublisher]), - mock(classOf[AclPublisher]), - faultHandler, - faultHandler, - brokerLifecycleManager, - partitionOpCallbackExecutor = MoreExecutors.newDirectExecutorService() - ) - - var image = MetadataImage.EMPTY - var delta = new MetadataDelta.Builder() - .setImage(image) - .build() - - // We first upgrade metadata version to 3_6_IV2 - delta.replay(new FeatureLevelRecord(). - setName(MetadataVersion.FEATURE_NAME). - setFeatureLevel(MetadataVersion.IBP_3_6_IV2.featureLevel())) - var newImage = delta.apply(new MetadataProvenance(100, 4, 2000)) - - metadataPublisher.onMetadataUpdate(delta, newImage, - LogDeltaManifest.newBuilder() - .provenance(MetadataProvenance.EMPTY) - .leaderAndEpoch(LeaderAndEpoch.UNKNOWN) - .numBatches(1) - .elapsedNs(100) - .numBytes(42) - .build()) - - // This should NOT trigger broker reregistration - verifyNoInteractions(brokerLifecycleManager) - - // We then upgrade to IBP_3_7_IV2 - image = newImage - delta = new MetadataDelta.Builder() - .setImage(image) - .build() - delta.replay(new FeatureLevelRecord(). - setName(MetadataVersion.FEATURE_NAME). - setFeatureLevel(MetadataVersion.IBP_3_7_IV2.featureLevel())) - newImage = delta.apply(new MetadataProvenance(100, 4, 2000)) - - metadataPublisher.onMetadataUpdate(delta, newImage, - LogDeltaManifest.newBuilder() - .provenance(MetadataProvenance.EMPTY) - .leaderAndEpoch(LeaderAndEpoch.UNKNOWN) - .numBatches(1) - .elapsedNs(100) - .numBytes(42) - .build()) - - // This SHOULD trigger a broker registration - verify(brokerLifecycleManager, times(1)).handleKraftJBODMetadataVersionUpdate() - clearInvocations(brokerLifecycleManager) - - // Finally upgrade to IBP_3_8_IV0 - image = newImage - delta = new MetadataDelta.Builder() - .setImage(image) - .build() - delta.replay(new FeatureLevelRecord(). - setName(MetadataVersion.FEATURE_NAME). - setFeatureLevel(MetadataVersion.IBP_3_8_IV0.featureLevel())) - newImage = delta.apply(new MetadataProvenance(200, 4, 3000)) - - metadataPublisher.onMetadataUpdate(delta, newImage, - LogDeltaManifest.newBuilder() - .provenance(MetadataProvenance.EMPTY) - .leaderAndEpoch(LeaderAndEpoch.UNKNOWN) - .numBatches(1) - .elapsedNs(100) - .numBytes(42) - .build()) - - // This should NOT trigger broker reregistration - verify(brokerLifecycleManager, times(0)).handleKraftJBODMetadataVersionUpdate() - - metadataPublisher.close() - } } diff --git a/core/src/test/scala/unit/kafka/server/streamaspect/ElasticReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/streamaspect/ElasticReplicaManagerTest.scala index 1287cbe091..bc8b1e2284 100644 --- a/core/src/test/scala/unit/kafka/server/streamaspect/ElasticReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/streamaspect/ElasticReplicaManagerTest.scala @@ -323,7 +323,7 @@ class ElasticReplicaManagerTest extends ReplicaManagerTest { val maxTransactionTimeoutMs = 30000 val maxProducerIdExpirationMs = 30000 val segments = new LogSegments(tp) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(logDir, tp, mockLogDirFailureChannel, logConfig.recordVersion, "") + val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(logDir, tp, mockLogDirFailureChannel, logConfig.recordVersion, "", None, mockScheduler) val producerStateManager = new ProducerStateManager(tp, logDir, maxTransactionTimeoutMs, new ProducerStateManagerConfig(maxProducerIdExpirationMs, true), time) val offsets = new LogLoader( diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index b8764f5fae..c625dc6e96 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -22,8 +22,8 @@ import java.nio.ByteBuffer import java.util import java.util.Collections import java.util.Optional -import java.util.Arrays import java.util.Properties +import java.util.stream.IntStream import kafka.log.{LogTestUtils, UnifiedLog} import kafka.raft.{KafkaMetadataLog, MetadataLogConfig} import kafka.server.{BrokerTopicStats, KafkaRaftServer} @@ -338,7 +338,7 @@ class DumpLogSegmentsTest { .setLastContainedLogTimestamp(lastContainedLogTimestamp) .setRawSnapshotWriter(metadataLog.createNewSnapshot(new OffsetAndEpoch(0, 0)).get) .setKraftVersion(1) - .setVoterSet(Optional.of(VoterSetTest.voterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true)))) + .setVoterSet(Optional.of(VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true)))) .build(MetadataRecordSerde.INSTANCE) ) { snapshotWriter => snapshotWriter.append(metadataRecords.asJava) diff --git a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala index fb9361fc3a..e0b4807f38 100644 --- a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala +++ b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala @@ -33,11 +33,12 @@ import org.apache.kafka.server.common.{ApiMessageAndVersion, Features, MetadataV import org.apache.kafka.common.metadata.{FeatureLevelRecord, UserScramCredentialRecord} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} import org.apache.kafka.raft.QuorumConfig -import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertThrows, assertTrue} import org.junit.jupiter.api.{Test, Timeout} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{EnumSource, ValueSource} +import org.mockito.Mockito import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -215,7 +216,6 @@ Found problem: assertEquals(0, runFormatCommand(stream, availableDirs)) val actual = stream.toString().split("\\r?\\n") val expect = availableDirs.map("Formatting %s".format(_)) - assertEquals(availableDirs.size, actual.size) expect.foreach(dir => { assertEquals(1, actual.count(_.startsWith(dir))) }) @@ -648,5 +648,37 @@ Found problem: assertEquals(1, exitStatus) } } + + @Test + def testFormatValidatesConfigForMetadataVersion(): Unit = { + val config = Mockito.spy(new KafkaConfig(TestUtils.createBrokerConfig(10, null))) + val args = Array("format", + "-c", "dummy.properties", + "-t", "XcZZOzUqS4yHOjhMQB6JLQ", + "--release-version", MetadataVersion.LATEST_PRODUCTION.toString) + val exitCode = StorageTool.runFormatCommand(StorageTool.parseArguments(args), config) + Mockito.verify(config, Mockito.times(1)).validateWithMetadataVersion(MetadataVersion.LATEST_PRODUCTION) + assertEquals(0, exitCode) + } + + @Test + def testJbodSupportValidation(): Unit = { + def formatWith(logDirCount: Int, metadataVersion: MetadataVersion): Integer = { + val properties = TestUtils.createBrokerConfig(10, null, logDirCount = logDirCount) + properties.remove(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG) + val configFile = TestUtils.tempPropertiesFile(properties.asScala.toMap).toPath.toString + StorageTool.execute(Array("format", + "-c", configFile, + "-t", "XcZZOzUqS4yHOjhMQB6JLQ", + "--release-version", metadataVersion.toString)) + } + + assertEquals(0, formatWith(1, MetadataVersion.IBP_3_6_IV2)) + assertEquals("Invalid configuration for metadata version: " + + "requirement failed: Multiple log directories (aka JBOD) are not supported in the current MetadataVersion 3.6-IV2. Need 3.7-IV2 or higher", + assertThrows(classOf[TerseFailure], () => formatWith(2, MetadataVersion.IBP_3_6_IV2)).getMessage) + assertEquals(0, formatWith(1, MetadataVersion.IBP_3_7_IV2)) + assertEquals(0, formatWith(2, MetadataVersion.IBP_3_7_IV2)) + } } diff --git a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala index d25fdb1b4e..6280318af5 100644 --- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala +++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala @@ -139,7 +139,8 @@ class SchedulerTest { val topicPartition = UnifiedLog.parseTopicPartitionName(logDir) val logDirFailureChannel = new LogDirFailureChannel(10) val segments = new LogSegments(topicPartition) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, logConfig.recordVersion, "") + val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( + logDir, topicPartition, logDirFailureChannel, logConfig.recordVersion, "", None, mockTime.scheduler) val producerStateManager = new ProducerStateManager(topicPartition, logDir, maxTransactionTimeoutMs, new ProducerStateManagerConfig(maxProducerIdExpirationMs, false), mockTime) val offsets = new LogLoader( diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index f2bb7608a5..fefc1fee23 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -909,9 +909,10 @@ object TestUtils extends Logging { def pollRecordsUntilTrue[K, V](consumer: Consumer[K, V], action: ConsumerRecords[K, V] => Boolean, msg: => String, - waitTimeMs: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Unit = { + waitTimeMs: Long = JTestUtils.DEFAULT_MAX_WAIT_MS, + pollTimeoutMs: Long = 100): Unit = { waitUntilTrue(() => { - val records = consumer.poll(Duration.ofMillis(100)) + val records = consumer.poll(Duration.ofMillis(pollTimeoutMs)) action(records) }, msg = msg, pause = 0L, waitTimeMs = waitTimeMs) } diff --git a/docker/README.md b/docker/README.md index 54d15c04fe..5c46ef954e 100644 --- a/docker/README.md +++ b/docker/README.md @@ -4,8 +4,10 @@ Docker Images Introduction ------------ -This directory contains scripts to build, test, push and promote docker image for kafka. -All of the steps can be either performed locally or by using Github Actions. +This directory contains: +- Scripts to build, test, push and promote docker images for apache kafka - both `jvm` based and graalvm based `native` kafka +- Scripts to build, test the JVM-based Docker Official Image, and generate a PR template for the same to be raised under the Docker official images repo. + All of the steps can be either performed locally or by using Github Actions. Github Actions -------------- @@ -27,28 +29,42 @@ Building image and running tests using github actions - kafka-url - This is the url to download kafka tarball from. For example kafka tarball url from (https://archive.apache.org/dist/kafka). For building RC image this will be an RC tarball url. -- image-type - This is the type of image that we intend to build. This will be dropdown menu type selection in the workflow. `jvm` image type is for official docker image (to be hosted on apache/kafka) as described in [KIP-975](https://cwiki.apache.org/confluence/display/KAFKA/KIP-975%3A+Docker+Image+for+Apache+Kafka) +- image-type - This is the type of image that we intend to build. This will be dropdown menu type selection in the workflow. + - `jvm` image type is for official docker image (to be hosted on apache/kafka) as described in [KIP-975](https://cwiki.apache.org/confluence/display/KAFKA/KIP-975%3A+Docker+Image+for+Apache+Kafka) + - `native` image type is for graalvm based `native` kafka docker image (to be hosted on apache/kafka-native) as described in [KIP-974](https://cwiki.apache.org/confluence/display/KAFKA/KIP-974%3A+Docker+Image+for+GraalVM+based+Native+Kafka+Broker#KIP974:DockerImageforGraalVMbasedNativeKafkaBroker-ImageNaming) -- Example:- +- Example(jvm):- To build and test a jvm image type ensuring kafka to be containerised should be https://archive.apache.org/dist/kafka/3.6.0/kafka_2.13-3.6.0.tgz (it is recommended to use scala 2.13 binary tarball), following inputs in github actions workflow are recommended. ``` image_type: jvm kafka_url: https://archive.apache.org/dist/kafka/3.6.0/kafka_2.13-3.6.0.tgz ``` - +- Example(native):- + To build and test a native image type ensuring kafka to be containerised should be https://archive.apache.org/dist/kafka/3.8.0/kafka_2.13-3.8.0.tgz (it is recommended to use scala 2.13 binary tarball), following inputs in github actions workflow are recommended. +``` +image_type: native +kafka_url: https://archive.apache.org/dist/kafka/3.8.0/kafka_2.13-3.8.0.tgz +``` Creating a Release Candidate using github actions ------------------------------------------------- - This is the recommended way to push an RC docker image. - Go to `Build and Push Release Candidate Docker Image` Github Actions Workflow. - Choose the `image_type` and provide `kafka_url` that needs to be containerised in the `rc_docker_image` that will be pushed to github. -- Example:- +- Example(jvm):- If you want to push a jvm image which contains kafka from https://archive.apache.org/dist/kafka/3.6.0/kafka_2.13-3.6.0.tgz to dockerhub under the namespace apache, repo name as kafka and image tag as 3.6.0-rc1 then following values need to be added in Github Actions Workflow:- ``` image_type: jvm kafka_url: https://archive.apache.org/dist/kafka/3.6.0/kafka_2.13-3.6.0.tgz rc_docker_image: apache/kafka:3.6.0-rc0 ``` +- Example(native):- + If you want to push a native image which contains kafka from https://archive.apache.org/dist/kafka/3.8.0/kafka_2.13-3.8.0.tgz to dockerhub under the namespace apache, repo name as kafka-native and image tag as 3.8.0-rc0 then following values need to be added in Github Actions Workflow:- +``` +image_type: native +kafka_url: https://archive.apache.org/dist/kafka/3.8.0/kafka_2.13-3.8.0.tgz +rc_docker_image: apache/kafka-native:3.8.0-rc0 +``` Promoting a Release Candidate using github actions -------------------------------------------------- @@ -56,12 +72,18 @@ Promoting a Release Candidate using github actions - This is the recommended way to promote an RC docker image. - Go to `Promote Release Candidate Docker Image` Github Actions Workflow. - Choose the RC docker image (`rc_docker_image`) that you want to promote and where it needs to be pushed to (`promoted_docker_image`), i.e. the final docker image release. -- Example:- +- Example(jvm):- If you want to promote apache/kafka:3.6.0-rc0 RC docker image to apache/kafka:3.6.0 then following parameters can be provided to the workflow. ``` rc_docker_image: apache/kafka:3.6.0-rc0 promoted_docker_image: apache/kafka:3.6.0 ``` +- Example(native):- + If you want to promote apache/kafka-native:3.8.0-rc0 RC docker image to apache/kafka-native:3.8.0 then following parameters can be provided to the workflow. +``` +rc_docker_image: apache/kafka-native:3.8.0-rc0 +promoted_docker_image: apache/kafka-native:3.8.0 +``` Cron job for checking CVEs in supported docker images ----------------------------------------------------- @@ -100,10 +122,14 @@ Building image and running tests locally - Sanity tests for the docker image are present in test/docker_sanity_test.py. - By default image will be built and tested, but if you only want to build the image, pass `--build` (or `-b`) flag and if you only want to test the given image pass `--test` (or `-t`) flag. - An html test report will be generated after the tests are executed containing the results. -- Example :- To build and test an image named test under kafka namespace with 3.6.0 tag and jvm image type ensuring kafka to be containerised should be https://downloads.apache.org/kafka/3.6.0/kafka_2.13-3.6.0.tgz (it is recommended to use scala 2.13 binary tarball), following command can be used +- Example(jvm) :- To build and test an image named test under kafka namespace with 3.6.0 tag and jvm image type ensuring kafka to be containerised should be https://downloads.apache.org/kafka/3.6.0/kafka_2.13-3.6.0.tgz (it is recommended to use scala 2.13 binary tarball), following command can be used ``` python docker_build_test.py kafka/test --image-tag=3.6.0 --image-type=jvm --kafka-url=https://archive.apache.org/dist/kafka/3.6.0/kafka_2.13-3.6.0.tgz ``` +- Example(native) :- To build and test an image named test under kafka namespace with 3.8.0 tag and native image type ensuring kafka to be containerised should be https://downloads.apache.org/kafka/3.8.0/kafka_2.13-3.8.0.tgz (it is recommended to use scala 2.13 binary tarball), following command can be used +``` +python docker_build_test.py kafka/test --image-tag=3.8.0 --image-type=native --kafka-url=https://archive.apache.org/dist/kafka/3.8.0/kafka_2.13-3.8.0.tgz +``` Creating a Release Candidate ---------------------------- @@ -111,11 +137,17 @@ Creating a Release Candidate - `docker_release.py` script builds a multi-architecture image and pushes it to provided docker registry. - Ensure you are logged in to the docker registry before triggering the script. - kafka binary tarball url along with image name (in the format `//:`) and type is needed to build the image. For detailed usage description check `python docker_release.py --help`. -- Example:- To push an image named test under kafka dockerhub namespace with 3.6.0 tag and jvm image type ensuring kafka to be containerised should be https://archive.apache.org/dist/kafka/3.6.0/kafka_2.13-3.6.0.tgz (it is recommended to use scala 2.13 binary tarball), following command can be used. (Make sure you have push access to the docker repo) +- Example(jvm):- To push an image named test under kafka dockerhub namespace with 3.6.0 tag and jvm image type ensuring kafka to be containerised should be https://archive.apache.org/dist/kafka/3.6.0/kafka_2.13-3.6.0.tgz (it is recommended to use scala 2.13 binary tarball), following command can be used. (Make sure you have push access to the docker repo) ``` # kafka/test is an example repo. Please replace with the docker hub repo you have push access to. -python docker_release.py kafka/test:3.6.0 --kafka-url https://archive.apache.org/dist/kafka/3.6.0/kafka_2.13-3.6.0.tgz +python docker_release.py kafka/test:3.6.0 --kafka-url --image-type=jvm https://archive.apache.org/dist/kafka/3.6.0/kafka_2.13-3.6.0.tgz +``` +- Example(native):- To push an image named test under kafka-native dockerhub namespace with 3.8.0 tag and native image type ensuring kafka to be containerised should be https://archive.apache.org/dist/kafka/3.8.0/kafka_2.13-3.8.0.tgz (it is recommended to use scala 2.13 binary tarball), following command can be used. (Make sure you have push access to the docker repo) +``` +# kafka-native/test is an example repo. Please replace with the docker hub repo you have push access to. + +python docker_release.py kafka-native/test:3.8.0 --kafka-url --image-type=native https://archive.apache.org/dist/kafka/3.8.0/kafka_2.13-3.8.0.tgz ``` - Please note that we use docker buildx for preparing the multi-architecture image and pushing it to docker registry. It's possible to encounter build failures because of buildx. Please retry the command in case some buildx related error occurs. @@ -124,14 +156,60 @@ Promoting a Release Candidate ----------------------------- - It's not recommended to promote the docker image locally, as we have github actions doing it in a convenient way, but if needed following command can be used to promote a Release Candidate image. -- Example:- If you want to promote RC image apache/kafka:3.6.0-rc0 to apache/kafka:3.6.0, following command can be used +- Example(jvm):- If you want to promote RC image apache/kafka:3.6.0-rc0 to apache/kafka:3.6.0, following command can be used ``` # Ensure docker buildx is enabled in your system and you have access to apache/kafka docker buildx imagetools create --tag apache/kafka:3.6.0 apache/kafka:3.6.0-rc0 ``` +- Example(native):- If you want to promote RC image apache/kafka-native:3.8.0-rc0 to apache/kafka-native:3.8.0, following command can be used + +``` +# Ensure docker buildx is enabled in your system and you have access to apache/kafka-native +docker buildx imagetools create --tag apache/kafka-native:3.8.0 apache/kafka-native:3.8.0-rc0 +``` Using the image in a docker container ------------------------------------- Please check [this](./examples/README.md) for usage guide of the docker image. + +Releasing the Docker Official Image +----------------------------------- + +- This is the recommended way to release docker official image. +- Ensure these steps are being run for a particular version, only once the AK release process for that version has been completed. + +- Provide the image type and kafka version to `Docker Prepare Docker Official Image Source` workflow. It will generate a artifact containing the static Dockerfile and assets for that specific version. Download the same from the workflow. + +``` +image_type: jvm +kafka_version: 3.7.0 +``` + +- Run the `docker/extract_docker_official_image_artifact.py` script, by providing it the path to the downloaded artifact. This will create a new directory under `docker/docker_official_images/kafka_version`. + +``` +python extract_docker_official_image_artifact.py --path_to_downloaded_artifact=path/to/downloaded/artifact +``` + +- If there any versions for which Docker Official Images should not be supported, remove the corresponding directories under `docker/docker_official_images`. +- Commit these changes to AK trunk. + +- Provide the image type and kafka version to `Docker Official Image Build Test` workflow. It will generate a test report and CVE report that can be shared with the community. + +``` +image_type: jvm +kafka_version: 3.7.0 +``` + +- Run the `docker/generate_kafka_pr_template.py` script from trunk, by providing it the image type. Update the existing entry, and raise a new PR in [Docker Hub's Docker Official Repo](https://github.com/docker-library/official-images/tree/master/library/kafka) by using this new PR template. + +``` +python generate_kafka_pr_template.py --image-type=jvm +``` + +- kafka-version - This is the version to create the Docker official images static Dockerfile and assets for, as well as the version to build and test the Docker official image for. +- image-type - This is the type of image that we intend to build. This will be dropdown menu type selection in the workflow. `jvm` image type is for official docker image (to be hosted on apache/kafka) as described in [KIP-975](https://cwiki.apache.org/confluence/display/KAFKA/KIP-975%3A+Docker+Image+for+Apache+Kafka). + - **NOTE:** As of now [KIP-1028](https://cwiki.apache.org/confluence/display/KAFKA/KIP-1028%3A+Docker+Official+Image+for+Apache+Kafka) only aims to release JVM based Docker Official Images and not GraalVM based native Apache Kafka docker image. + diff --git a/docker/docker_official_image_build_test.py b/docker/docker_official_image_build_test.py index 6ffe25ee0b..3da68854c2 100644 --- a/docker/docker_official_image_build_test.py +++ b/docker/docker_official_image_build_test.py @@ -76,7 +76,7 @@ def build_docker_official_image(image, tag, kafka_version, image_type): parser.add_argument("--test", "-t", action="store_true", dest="test_only", default=False, help="Only run the tests, don't build the image") args = parser.parse_args() - kafka_url = f"https://downloads.apache.org/kafka/{args.kafka_version}/kafka_2.13-{args.kafka_version}.tgz" + kafka_url = f"https://archive.apache.org/dist/kafka/{args.kafka_version}/kafka_2.13-{args.kafka_version}.tgz" if args.build_only or not (args.build_only or args.test_only): if args.kafka_version: build_docker_official_image(args.image, args.tag, args.kafka_version, args.image_type) diff --git a/docker/docker_official_images/3.7.0/jvm/Dockerfile b/docker/docker_official_images/3.7.0/jvm/Dockerfile new file mode 100755 index 0000000000..905e2f2149 --- /dev/null +++ b/docker/docker_official_images/3.7.0/jvm/Dockerfile @@ -0,0 +1,95 @@ +############################################################################### +# 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 eclipse-temurin:21-jre-alpine AS build-jsa + +USER root + +# Get Kafka from https://archive.apache.org/dist/kafka, url passed as env var, for version 3.7.0 +ENV kafka_url https://archive.apache.org/dist/kafka/3.7.0/kafka_2.13-3.7.0.tgz + +COPY jsa_launch /etc/kafka/docker/jsa_launch + +RUN set -eux ; \ + apk update ; \ + apk upgrade ; \ + apk add --no-cache wget gcompat gpg gpg-agent procps bash; \ + mkdir opt/kafka; \ + wget -nv -O kafka.tgz "$kafka_url"; \ + wget -nv -O kafka.tgz.asc "$kafka_url.asc"; \ + tar xfz kafka.tgz -C /opt/kafka --strip-components 1; \ + wget -nv -O KEYS https://downloads.apache.org/kafka/KEYS; \ + gpg --import KEYS; \ + gpg --batch --verify kafka.tgz.asc kafka.tgz + +# Generate jsa files using dynamic CDS for kafka server start command and kafka storage format command +RUN /etc/kafka/docker/jsa_launch + + +FROM eclipse-temurin:21-jre-alpine + +# exposed ports +EXPOSE 9092 + +USER root + +# Get Kafka from https://archive.apache.org/dist/kafka, url passed as env var, for version 3.7.0 +ENV kafka_url https://archive.apache.org/dist/kafka/3.7.0/kafka_2.13-3.7.0.tgz +ENV build_date 2024-06-11 + + +LABEL org.label-schema.name="kafka" \ + org.label-schema.description="Apache Kafka" \ + org.label-schema.build-date="${build_date}" \ + org.label-schema.vcs-url="https://github.com/apache/kafka" \ + maintainer="Apache Kafka" + +RUN set -eux ; \ + apk update ; \ + apk upgrade ; \ + apk add --no-cache wget gcompat gpg gpg-agent procps bash; \ + mkdir opt/kafka; \ + wget -nv -O kafka.tgz "$kafka_url"; \ + wget -nv -O kafka.tgz.asc "$kafka_url.asc"; \ + tar xfz kafka.tgz -C /opt/kafka --strip-components 1; \ + wget -nv -O KEYS https://downloads.apache.org/kafka/KEYS; \ + gpg --import KEYS; \ + gpg --batch --verify kafka.tgz.asc kafka.tgz; \ + mkdir -p /var/lib/kafka/data /etc/kafka/secrets; \ + mkdir -p /etc/kafka/docker /usr/logs /mnt/shared/config; \ + adduser -h /home/appuser -D --shell /bin/bash appuser; \ + chown appuser:appuser -R /usr/logs /opt/kafka /mnt/shared/config; \ + chown appuser:root -R /var/lib/kafka /etc/kafka/secrets /etc/kafka; \ + chmod -R ug+w /etc/kafka /var/lib/kafka /etc/kafka/secrets; \ + cp /opt/kafka/config/log4j.properties /etc/kafka/docker/log4j.properties; \ + cp /opt/kafka/config/tools-log4j.properties /etc/kafka/docker/tools-log4j.properties; \ + cp /opt/kafka/config/kraft/server.properties /etc/kafka/docker/server.properties; \ + rm kafka.tgz kafka.tgz.asc KEYS; \ + apk del wget gpg gpg-agent; \ + apk cache clean; + +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 +COPY --chown=appuser:appuser launch /etc/kafka/docker/launch + +USER appuser + +VOLUME ["/etc/kafka/secrets", "/var/lib/kafka/data", "/mnt/shared/config"] + +CMD ["/etc/kafka/docker/run"] diff --git a/docker/docker_official_images/3.7.0/jvm/jsa_launch b/docker/docker_official_images/3.7.0/jvm/jsa_launch new file mode 100755 index 0000000000..904fbbee42 --- /dev/null +++ b/docker/docker_official_images/3.7.0/jvm/jsa_launch @@ -0,0 +1,49 @@ +#!/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. + +KAFKA_CLUSTER_ID="$(opt/kafka/bin/kafka-storage.sh random-uuid)" +TOPIC="test-topic" + +KAFKA_JVM_PERFORMANCE_OPTS="-XX:ArchiveClassesAtExit=storage.jsa" opt/kafka/bin/kafka-storage.sh format -t $KAFKA_CLUSTER_ID -c opt/kafka/config/kraft/server.properties + +KAFKA_JVM_PERFORMANCE_OPTS="-XX:ArchiveClassesAtExit=kafka.jsa" opt/kafka/bin/kafka-server-start.sh opt/kafka/config/kraft/server.properties & + +check_timeout() { + if [ $TIMEOUT -eq 0 ]; then + echo "Server startup timed out" + exit 1 + fi + echo "Check will timeout in $(( TIMEOUT-- )) seconds" + sleep 1 +} + +opt/kafka/bin/kafka-topics.sh --create --topic $TOPIC --bootstrap-server localhost:9092 +[ $? -eq 0 ] || exit 1 + +echo "test" | opt/kafka/bin/kafka-console-producer.sh --topic $TOPIC --bootstrap-server localhost:9092 +[ $? -eq 0 ] || exit 1 + +opt/kafka/bin/kafka-console-consumer.sh --topic $TOPIC --from-beginning --bootstrap-server localhost:9092 --max-messages 1 --timeout-ms 20000 +[ $? -eq 0 ] || exit 1 + +opt/kafka/bin/kafka-server-stop.sh + +# Wait until jsa file is generated +TIMEOUT=20 +until [ -f /kafka.jsa ] +do + check_timeout +done \ No newline at end of file diff --git a/docker/docker_official_images/3.7.0/jvm/launch b/docker/docker_official_images/3.7.0/jvm/launch new file mode 100755 index 0000000000..6c4ca1d2e0 --- /dev/null +++ b/docker/docker_official_images/3.7.0/jvm/launch @@ -0,0 +1,68 @@ +#!/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. + +# Override this section from the script to include the com.sun.management.jmxremote.rmi.port property. +if [ -z "${KAFKA_JMX_OPTS-}" ]; then + export KAFKA_JMX_OPTS="-Dcom.sun.management.jmxremote=true \ + -Dcom.sun.management.jmxremote.authenticate=false \ + -Dcom.sun.management.jmxremote.ssl=false " +fi + +# The JMX client needs to be able to connect to java.rmi.server.hostname. +# The default for bridged n/w is the bridged IP so you will only be able to connect from another docker container. +# For host n/w, this is the IP that the hostname on the host resolves to. + +# If you have more than one n/w configured, hostname -i gives you all the IPs, +# the default is to pick the first IP (or network). +export KAFKA_JMX_HOSTNAME=${KAFKA_JMX_HOSTNAME:-$(hostname -i | cut -d" " -f1)} + +if [ "${KAFKA_JMX_PORT-}" ]; then + # This ensures that the "if" section for JMX_PORT in kafka launch script does not trigger. + export JMX_PORT=$KAFKA_JMX_PORT + export KAFKA_JMX_OPTS="${KAFKA_JMX_OPTS-} -Djava.rmi.server.hostname=$KAFKA_JMX_HOSTNAME \ + -Dcom.sun.management.jmxremote.local.only=false \ + -Dcom.sun.management.jmxremote.rmi.port=$JMX_PORT \ + -Dcom.sun.management.jmxremote.port=$JMX_PORT" +fi + +# Make a temp env variable to store user provided performance otps +if [ -z "${KAFKA_JVM_PERFORMANCE_OPTS-}" ]; then + export TEMP_KAFKA_JVM_PERFORMANCE_OPTS="" +else + export TEMP_KAFKA_JVM_PERFORMANCE_OPTS="$KAFKA_JVM_PERFORMANCE_OPTS" +fi + +# We will first use CDS for storage to format storage +export KAFKA_JVM_PERFORMANCE_OPTS="${KAFKA_JVM_PERFORMANCE_OPTS-} -XX:SharedArchiveFile=/opt/kafka/storage.jsa" + +echo "===> Using provided cluster id $CLUSTER_ID ..." + +# Invoke the docker wrapper to setup property files and format storage +result=$(/opt/kafka/bin/kafka-run-class.sh kafka.docker.KafkaDockerWrapper setup \ + --default-configs-dir /etc/kafka/docker \ + --mounted-configs-dir /mnt/shared/config \ + --final-configs-dir /opt/kafka/config 2>&1) || \ + echo $result | grep -i "already formatted" || \ + { echo $result && (exit 1) } + +# Using temp env variable to get rid of storage CDS command +export KAFKA_JVM_PERFORMANCE_OPTS="$TEMP_KAFKA_JVM_PERFORMANCE_OPTS" + +# Now we will use CDS for kafka to start kafka server +export KAFKA_JVM_PERFORMANCE_OPTS="$KAFKA_JVM_PERFORMANCE_OPTS -XX:SharedArchiveFile=/opt/kafka/kafka.jsa" + +# Start kafka broker +exec /opt/kafka/bin/kafka-server-start.sh /opt/kafka/config/server.properties diff --git a/docker/docker_official_images/3.7.0/jvm/resources/common-scripts/bash-config b/docker/docker_official_images/3.7.0/jvm/resources/common-scripts/bash-config new file mode 100755 index 0000000000..b6971610ef --- /dev/null +++ b/docker/docker_official_images/3.7.0/jvm/resources/common-scripts/bash-config @@ -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. + +set -o nounset \ + -o errexit + +# Trace may expose passwords/credentials by printing them to stdout, so turn on with care. +if [ "${TRACE:-}" == "true" ]; then + set -o verbose \ + -o xtrace +fi diff --git a/docker/docker_official_images/3.7.0/jvm/resources/common-scripts/configure b/docker/docker_official_images/3.7.0/jvm/resources/common-scripts/configure new file mode 100755 index 0000000000..deb82aedfc --- /dev/null +++ b/docker/docker_official_images/3.7.0/jvm/resources/common-scripts/configure @@ -0,0 +1,121 @@ +#!/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. + +ensure() { + if [[ -z "${!1}" ]]; then + echo "$1 environment variable not set" + exit 1 + fi +} + +path() { + if [[ $2 == "writable" ]]; then + if [[ ! -w "$1" ]]; then + echo "$1 file not writable" + exit 1 + fi + elif [[ $2 == "existence" ]]; then + if [[ ! -e "$1" ]]; then + echo "$1 file does not exist" + exit 1 + fi + fi +} + +# unset KAFKA_ADVERTISED_LISTENERS from ENV in KRaft mode when running as controller only +if [[ -n "${KAFKA_PROCESS_ROLES-}" ]] +then + echo "Running in KRaft mode..." + ensure CLUSTER_ID + if [[ $KAFKA_PROCESS_ROLES == "controller" ]] + then + if [[ -n "${KAFKA_ADVERTISED_LISTENERS-}" ]] + then + echo "KAFKA_ADVERTISED_LISTENERS is not supported on a KRaft controller." + exit 1 + else + # Unset in case env variable is set with empty value + unset KAFKA_ADVERTISED_LISTENERS + fi + fi +fi + +# By default, LISTENERS is derived from ADVERTISED_LISTENERS by replacing +# hosts with 0.0.0.0. This is good default as it ensures that the broker +# process listens on all ports. +if [[ -z "${KAFKA_LISTENERS-}" ]] && ( [[ -z "${KAFKA_PROCESS_ROLES-}" ]] || [[ $KAFKA_PROCESS_ROLES != "controller" ]] ) && [[ -n "${KAFKA_ADVERTISED_LISTENERS-}" ]] +then + export KAFKA_LISTENERS + KAFKA_LISTENERS=$(echo "$KAFKA_ADVERTISED_LISTENERS" | sed -e 's|://[^:]*:|://0.0.0.0:|g') +fi + +path /opt/kafka/config/ writable + +# Set if ADVERTISED_LISTENERS has SSL:// or SASL_SSL:// endpoints. +if [[ -n "${KAFKA_ADVERTISED_LISTENERS-}" ]] && [[ $KAFKA_ADVERTISED_LISTENERS == *"SSL://"* ]] +then + echo "SSL is enabled." + + ensure KAFKA_SSL_KEYSTORE_FILENAME + export KAFKA_SSL_KEYSTORE_LOCATION="/etc/kafka/secrets/$KAFKA_SSL_KEYSTORE_FILENAME" + path "$KAFKA_SSL_KEYSTORE_LOCATION" existence + + ensure KAFKA_SSL_KEY_CREDENTIALS + KAFKA_SSL_KEY_CREDENTIALS_LOCATION="/etc/kafka/secrets/$KAFKA_SSL_KEY_CREDENTIALS" + path "$KAFKA_SSL_KEY_CREDENTIALS_LOCATION" existence + export KAFKA_SSL_KEY_PASSWORD + KAFKA_SSL_KEY_PASSWORD=$(cat "$KAFKA_SSL_KEY_CREDENTIALS_LOCATION") + + ensure KAFKA_SSL_KEYSTORE_CREDENTIALS + KAFKA_SSL_KEYSTORE_CREDENTIALS_LOCATION="/etc/kafka/secrets/$KAFKA_SSL_KEYSTORE_CREDENTIALS" + path "$KAFKA_SSL_KEYSTORE_CREDENTIALS_LOCATION" existence + export KAFKA_SSL_KEYSTORE_PASSWORD + KAFKA_SSL_KEYSTORE_PASSWORD=$(cat "$KAFKA_SSL_KEYSTORE_CREDENTIALS_LOCATION") + + if [[ -n "${KAFKA_SSL_CLIENT_AUTH-}" ]] && ( [[ $KAFKA_SSL_CLIENT_AUTH == *"required"* ]] || [[ $KAFKA_SSL_CLIENT_AUTH == *"requested"* ]] ) + then + ensure KAFKA_SSL_TRUSTSTORE_FILENAME + export KAFKA_SSL_TRUSTSTORE_LOCATION="/etc/kafka/secrets/$KAFKA_SSL_TRUSTSTORE_FILENAME" + path "$KAFKA_SSL_TRUSTSTORE_LOCATION" existence + + ensure KAFKA_SSL_TRUSTSTORE_CREDENTIALS + KAFKA_SSL_TRUSTSTORE_CREDENTIALS_LOCATION="/etc/kafka/secrets/$KAFKA_SSL_TRUSTSTORE_CREDENTIALS" + path "$KAFKA_SSL_TRUSTSTORE_CREDENTIALS_LOCATION" existence + export KAFKA_SSL_TRUSTSTORE_PASSWORD + KAFKA_SSL_TRUSTSTORE_PASSWORD=$(cat "$KAFKA_SSL_TRUSTSTORE_CREDENTIALS_LOCATION") + fi +fi + +# Set if KAFKA_ADVERTISED_LISTENERS has SASL_PLAINTEXT:// or SASL_SSL:// endpoints. +if [[ -n "${KAFKA_ADVERTISED_LISTENERS-}" ]] && [[ $KAFKA_ADVERTISED_LISTENERS =~ .*SASL_.*://.* ]] +then + echo "SASL" is enabled. + + ensure KAFKA_OPTS + + if [[ ! $KAFKA_OPTS == *"java.security.auth.login.config"* ]] + then + echo "KAFKA_OPTS should contain 'java.security.auth.login.config' property." + fi +fi + +if [[ -n "${KAFKA_JMX_OPTS-}" ]] +then + if [[ ! $KAFKA_JMX_OPTS == *"com.sun.management.jmxremote.rmi.port"* ]] + then + echo "KAFKA_OPTS should contain 'com.sun.management.jmxremote.rmi.port' property. It is required for accessing the JMX metrics externally." + fi +fi diff --git a/docker/docker_official_images/3.7.0/jvm/resources/common-scripts/configureDefaults b/docker/docker_official_images/3.7.0/jvm/resources/common-scripts/configureDefaults new file mode 100755 index 0000000000..14d28548a8 --- /dev/null +++ b/docker/docker_official_images/3.7.0/jvm/resources/common-scripts/configureDefaults @@ -0,0 +1,28 @@ +#!/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. + +declare -A env_defaults +env_defaults=( +# Replace CLUSTER_ID with a unique base64 UUID using "bin/kafka-storage.sh random-uuid" + ["CLUSTER_ID"]="5L6g3nShT-eMCtK--X86sw" +) + +for key in "${!env_defaults[@]}"; do + if [[ -z "${!key:-}" ]]; then + echo ${key} not set. Setting it to default value: \"${env_defaults[$key]}\" + export "$key"="${env_defaults[$key]}" + fi +done diff --git a/docker/docker_official_images/3.7.0/jvm/resources/common-scripts/run b/docker/docker_official_images/3.7.0/jvm/resources/common-scripts/run new file mode 100755 index 0000000000..678aeb63a0 --- /dev/null +++ b/docker/docker_official_images/3.7.0/jvm/resources/common-scripts/run @@ -0,0 +1,38 @@ +#!/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. + +. /etc/kafka/docker/bash-config + +# Set environment values if they exist as arguments +if [ $# -ne 0 ]; then + echo "===> Overriding env params with args ..." + for var in "$@" + do + export "$var" + done +fi + +echo "===> User" +id + +echo "===> Setting default values of environment variables if not already set." +. /etc/kafka/docker/configureDefaults + +echo "===> Configuring ..." +. /etc/kafka/docker/configure + +echo "===> Launching ... " +. /etc/kafka/docker/launch diff --git a/docker/examples/README.md b/docker/examples/README.md index 130b6c22b3..8f4fdce436 100644 --- a/docker/examples/README.md +++ b/docker/examples/README.md @@ -58,7 +58,7 @@ Running in SSL mode - Recommended way to run in ssl mode is by mounting secrets on `/etc/kafka/secrets` in docker container and providing configs following through environment variables (`KAFKA_SSL_KEYSTORE_FILENAME`, `KAFKA_SSL_KEYSTORE_CREDENTIALS`, `KAFKA_SSL_KEY_CREDENTIALS`, `KAFKA_SSL_TRUSTSTORE_FILENAME` and `KAFKA_SSL_TRUSTSTORE_CREDENTIALS`) to let the docker image scripts extract passwords and populate correct paths in server.properties. - Please ensure appropriate `KAFKA_ADVERTISED_LISTENERS` are provided through environment variables to enable SSL mode in Kafka server, i.e. it should contain an `SSL` listener. - Alternatively property file input can be used to provide ssl properties. -- Make sure you set location of truststore and keystore correctly when using file input. See example for file input in `jvm/single-node/file-input` for better clarity. +- Make sure you set location of truststore and keystore correctly when using file input. See example for file input in `docker-compose-files/single-node/file-input` for better clarity. - Note that advertised.listeners property needs to be provided along with SSL properties in file input and cannot be provided through environment variable separately. - In conclusion, ssl properties with advertised.listeners should be treated as a group and provided in file input or environment variables in it's entirety. - In case ssl properties are provided both through file input and environment variables, environment variable properties will override the file input properties, just as mentioned in the beginning of this section. @@ -66,7 +66,15 @@ Running in SSL mode Examples -------- -- `jvm` directory contains docker compose files for some example configs to run `apache/kafka` docker image. +- `docker-compose-files` directory contains docker compose files for some example configs to run `apache/kafka` OR `apache/kafka-native` docker image. +- Pass the `IMAGE` variable with the Docker Compose file to specify which Docker image to use for bringing up the containers. +``` +# to bring up containers using apache/kafka docker image +IMAGE=apache/kafka:latest + +# to bring up containers using apache/kafka-native docker image +IMAGE=apache/kafka-native:latest +``` - Run the commands from root of the repository. - Checkout `single-node` examples for quick small examples to play around with. - `cluster` contains multi node examples, for `combined` mode as well as `isolated` mode. @@ -77,7 +85,7 @@ Single Node ----------- - These examples are for understanding various ways inputs can be provided and kafka can be configured in docker container. -- Examples are present inside `jvm/single-node` directory. +- Examples are present inside `docker-compose-files/single-node` directory. - Plaintext:- - This is the simplest compose file. - We are using environment variables purely for providing configs. @@ -88,7 +96,12 @@ Single Node - To run the example:- ``` # Run from root of the repo - $ docker compose -f docker/examples/jvm/single-node/plaintext/docker-compose.yml up + + # JVM based Apache Kafka Docker Image + $ IMAGE=apache/kafka:latest docker compose -f docker/examples/docker-compose-files/single-node/plaintext/docker-compose.yml up + + # GraalVM based Native Apache Kafka Docker Image + $ IMAGE=apache/kafka-native:latest docker compose -f docker/examples/docker-compose-files/single-node/plaintext/docker-compose.yml up ``` - To produce messages using client scripts:- ``` @@ -102,7 +115,12 @@ Single Node - To run the example:- ``` # Run from root of the repo - $ docker compose -f docker/examples/jvm/single-node/ssl/docker-compose.yml up + + # JVM based Apache Kafka Docker Image + $ IMAGE=apache/kafka:latest docker compose -f docker/examples/docker-compose-files/single-node/ssl/docker-compose.yml up + + # GraalVM based Native Apache Kafka Docker Image + $ IMAGE=apache/kafka-native:latest docker compose -f docker/examples/docker-compose-files/single-node/ssl/docker-compose.yml up ``` - To produce messages using client scripts (Ensure that java version >= 17):- ``` @@ -117,7 +135,12 @@ Single Node - To run the example:- ``` # Run from root of the repo - $ docker compose -f docker/examples/jvm/single-node/file-input/docker-compose.yml up + + # JVM based Apache Kafka Docker Image + $ IMAGE=apache/kafka:latest docker compose -f docker/examples/docker-compose-files/single-node/file-input/docker-compose.yml up + + # GraalVM based Native Apache Kafka Docker Image + $ IMAGE=apache/kafka-native:latest docker compose -f docker/examples/docker-compose-files/single-node/file-input/docker-compose.yml up ``` - To produce messages using client scripts (Ensure that java version >= 17):- ``` @@ -130,7 +153,7 @@ Multi Node Cluster - These examples are for real world usecases where multiple nodes of kafka are required. - Combined:- - - Examples are present in `jvm/cluster/combined` directory. + - Examples are present in `docker-compose-files/cluster/combined` directory. - Plaintext:- - Each broker must expose a unique port to host machine. - For example broker-1, broker2 and broker3 are listening on port 9092, they're exposing it to the host via ports 29092, 39092 and 49092 respectively. @@ -143,7 +166,12 @@ Multi Node Cluster - To run the example:- ``` # Run from root of the repo - $ docker compose -f docker/examples/jvm/cluster/combined/plaintext/docker-compose.yml up + + # JVM based Apache Kafka Docker Image + $ IMAGE=apache/kafka:latest docker compose -f docker/examples/docker-compose-files/cluster/combined/plaintext/docker-compose.yml up + + # GraalVM based Native Apache Kafka Docker Image + $ IMAGE=apache/kafka-native:latest docker compose -f docker/examples/docker-compose-files/cluster/combined/plaintext/docker-compose.yml up ``` - To access using client script:- ``` @@ -157,7 +185,12 @@ Multi Node Cluster - To run the example:- ``` # Run from root of the repo - $ docker compose -f docker/examples/jvm/cluster/combined/ssl/docker-compose.yml up + + # JVM based Apache Kafka Docker Image + $ IMAGE=apache/kafka:latest docker compose -f docker/examples/docker-compose-files/cluster/combined/ssl/docker-compose.yml up + + # GraalVM based Native Apache Kafka Docker Image + $ IMAGE=apache/kafka-native:latest docker compose -f docker/examples/docker-compose-files/cluster/combined/ssl/docker-compose.yml up ``` - To produce messages using client scripts (Ensure that java version >= 17):- ``` @@ -165,7 +198,7 @@ Multi Node Cluster $ bin/kafka-console-producer.sh --topic test --bootstrap-server localhost:29093 --producer.config ./docker/examples/fixtures/client-secrets/client-ssl.properties ``` - Isolated:- - - Examples are present in `jvm/cluster/isolated` directory. + - Examples are present in `docker-compose-files/cluster/isolated` directory. - Plaintext:- - Here controllers and brokers are configured separately. - It's a good practice to define that brokers depend on controllers. @@ -173,7 +206,12 @@ Multi Node Cluster - To run the example:- ``` # Run from root of the repo - $ docker compose -f docker/examples/jvm/cluster/isolated/plaintext/docker-compose.yml up + + # JVM based Apache Kafka Docker Image + $ IMAGE=apache/kafka:latest docker compose -f docker/examples/docker-compose-files/cluster/isolated/plaintext/docker-compose.yml up + + # GraalVM based Native Apache Kafka Docker Image + $ IMAGE=apache/kafka-native:latest docker compose -f docker/examples/docker-compose-files/cluster/isolated/plaintext/docker-compose.yml up ``` - To access using client script:- ``` @@ -186,7 +224,12 @@ Multi Node Cluster - To run the example:- ``` # Run from root of the repo - $ docker compose -f docker/examples/jvm/cluster/isolated/ssl/docker-compose.yml up + + # JVM based Apache Kafka Docker Image + $ IMAGE=apache/kafka:latest docker compose -f docker/examples/docker-compose-files/cluster/isolated/ssl/docker-compose.yml up + + # GraalVM based Native Apache Kafka Docker Image + $ IMAGE=apache/kafka-native:latest docker compose -f docker/examples/docker-compose-files/cluster/isolated/ssl/docker-compose.yml up ``` - To produce messages using client scripts (Ensure that java version >= 17):- ``` diff --git a/docker/examples/jvm/cluster/combined/plaintext/docker-compose.yml b/docker/examples/docker-compose-files/cluster/combined/plaintext/docker-compose.yml similarity index 97% rename from docker/examples/jvm/cluster/combined/plaintext/docker-compose.yml rename to docker/examples/docker-compose-files/cluster/combined/plaintext/docker-compose.yml index 0ba5f00606..6541f212d2 100644 --- a/docker/examples/jvm/cluster/combined/plaintext/docker-compose.yml +++ b/docker/examples/docker-compose-files/cluster/combined/plaintext/docker-compose.yml @@ -17,7 +17,7 @@ version: '2' services: kafka-1: - image: apache/kafka:latest + image: ${IMAGE} hostname: kafka-1 container_name: kafka-1 ports: @@ -39,7 +39,7 @@ services: KAFKA_LOG_DIRS: '/tmp/kraft-combined-logs' kafka-2: - image: apache/kafka:latest + image: ${IMAGE} hostname: kafka-2 container_name: kafka-2 ports: @@ -61,7 +61,7 @@ services: KAFKA_LOG_DIRS: '/tmp/kraft-combined-logs' kafka-3: - image: apache/kafka:latest + image: ${IMAGE} hostname: kafka-3 container_name: kafka-3 ports: diff --git a/docker/examples/jvm/cluster/combined/ssl/docker-compose.yml b/docker/examples/docker-compose-files/cluster/combined/ssl/docker-compose.yml similarity index 98% rename from docker/examples/jvm/cluster/combined/ssl/docker-compose.yml rename to docker/examples/docker-compose-files/cluster/combined/ssl/docker-compose.yml index 338116df4a..da97eb815a 100644 --- a/docker/examples/jvm/cluster/combined/ssl/docker-compose.yml +++ b/docker/examples/docker-compose-files/cluster/combined/ssl/docker-compose.yml @@ -17,7 +17,7 @@ version: '2' services: kafka-1: - image: apache/kafka:latest + image: ${IMAGE} hostname: kafka-1 container_name: kafka-1 ports: @@ -49,7 +49,7 @@ services: KAFKA_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM: "" kafka-2: - image: apache/kafka:latest + image: ${IMAGE} hostname: kafka-2 container_name: kafka-2 ports: @@ -81,7 +81,7 @@ services: KAFKA_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM: "" kafka-3: - image: apache/kafka:latest + image: ${IMAGE} hostname: kafka-3 container_name: kafka-3 ports: diff --git a/docker/examples/jvm/cluster/isolated/plaintext/docker-compose.yml b/docker/examples/docker-compose-files/cluster/isolated/plaintext/docker-compose.yml similarity index 96% rename from docker/examples/jvm/cluster/isolated/plaintext/docker-compose.yml rename to docker/examples/docker-compose-files/cluster/isolated/plaintext/docker-compose.yml index 35fc56d6bb..edf71567eb 100644 --- a/docker/examples/jvm/cluster/isolated/plaintext/docker-compose.yml +++ b/docker/examples/docker-compose-files/cluster/isolated/plaintext/docker-compose.yml @@ -17,7 +17,7 @@ version: '2' services: controller-1: - image: apache/kafka:latest + image: ${IMAGE} environment: KAFKA_NODE_ID: 1 KAFKA_PROCESS_ROLES: 'controller' @@ -33,7 +33,7 @@ services: KAFKA_LOG_DIRS: '/tmp/kraft-combined-logs' controller-2: - image: apache/kafka:latest + image: ${IMAGE} environment: KAFKA_NODE_ID: 2 KAFKA_PROCESS_ROLES: 'controller' @@ -49,7 +49,7 @@ services: KAFKA_LOG_DIRS: '/tmp/kraft-combined-logs' controller-3: - image: apache/kafka:latest + image: ${IMAGE} environment: KAFKA_NODE_ID: 3 KAFKA_PROCESS_ROLES: 'controller' @@ -65,7 +65,7 @@ services: KAFKA_LOG_DIRS: '/tmp/kraft-combined-logs' kafka-1: - image: apache/kafka:latest + image: ${IMAGE} ports: - 29092:9092 hostname: kafka-1 @@ -91,7 +91,7 @@ services: - controller-3 kafka-2: - image: apache/kafka:latest + image: ${IMAGE} ports: - 39092:9092 hostname: kafka-2 @@ -117,7 +117,7 @@ services: - controller-3 kafka-3: - image: apache/kafka:latest + image: ${IMAGE} ports: - 49092:9092 hostname: kafka-3 diff --git a/docker/examples/jvm/cluster/isolated/ssl/docker-compose.yml b/docker/examples/docker-compose-files/cluster/isolated/ssl/docker-compose.yml similarity index 97% rename from docker/examples/jvm/cluster/isolated/ssl/docker-compose.yml rename to docker/examples/docker-compose-files/cluster/isolated/ssl/docker-compose.yml index 85bfa3b17d..e3590508fe 100644 --- a/docker/examples/jvm/cluster/isolated/ssl/docker-compose.yml +++ b/docker/examples/docker-compose-files/cluster/isolated/ssl/docker-compose.yml @@ -17,7 +17,7 @@ version: '2' services: controller-1: - image: apache/kafka:latest + image: ${IMAGE} environment: KAFKA_NODE_ID: 1 KAFKA_PROCESS_ROLES: 'controller' @@ -33,7 +33,7 @@ services: KAFKA_LOG_DIRS: '/tmp/kraft-combined-logs' controller-2: - image: apache/kafka:latest + image: ${IMAGE} environment: KAFKA_NODE_ID: 2 KAFKA_PROCESS_ROLES: 'controller' @@ -49,7 +49,7 @@ services: KAFKA_LOG_DIRS: '/tmp/kraft-combined-logs' controller-3: - image: apache/kafka:latest + image: ${IMAGE} environment: KAFKA_NODE_ID: 3 KAFKA_PROCESS_ROLES: 'controller' @@ -65,7 +65,7 @@ services: KAFKA_LOG_DIRS: '/tmp/kraft-combined-logs' kafka-1: - image: apache/kafka:latest + image: ${IMAGE} hostname: kafka-1 container_name: kafka-1 ports: @@ -101,7 +101,7 @@ services: - controller-3 kafka-2: - image: apache/kafka:latest + image: ${IMAGE} hostname: kafka-2 container_name: kafka-2 ports: @@ -137,7 +137,7 @@ services: - controller-3 kafka-3: - image: apache/kafka:latest + image: ${IMAGE} hostname: kafka-3 container_name: kafka-3 ports: diff --git a/docker/examples/jvm/single-node/file-input/docker-compose.yml b/docker/examples/docker-compose-files/single-node/file-input/docker-compose.yml similarity index 97% rename from docker/examples/jvm/single-node/file-input/docker-compose.yml rename to docker/examples/docker-compose-files/single-node/file-input/docker-compose.yml index 051905ba51..34a7264d3e 100644 --- a/docker/examples/jvm/single-node/file-input/docker-compose.yml +++ b/docker/examples/docker-compose-files/single-node/file-input/docker-compose.yml @@ -17,7 +17,7 @@ version: '2' services: broker: - image: apache/kafka:latest + image: ${IMAGE} hostname: broker container_name: broker ports: diff --git a/docker/examples/jvm/single-node/plaintext/docker-compose.yml b/docker/examples/docker-compose-files/single-node/plaintext/docker-compose.yml similarity index 98% rename from docker/examples/jvm/single-node/plaintext/docker-compose.yml rename to docker/examples/docker-compose-files/single-node/plaintext/docker-compose.yml index 087c3afad8..fc6527521e 100644 --- a/docker/examples/jvm/single-node/plaintext/docker-compose.yml +++ b/docker/examples/docker-compose-files/single-node/plaintext/docker-compose.yml @@ -17,7 +17,7 @@ version: '2' services: broker: - image: apache/kafka:latest + image: ${IMAGE} hostname: broker container_name: broker ports: diff --git a/docker/examples/jvm/single-node/ssl/docker-compose.yml b/docker/examples/docker-compose-files/single-node/ssl/docker-compose.yml similarity index 98% rename from docker/examples/jvm/single-node/ssl/docker-compose.yml rename to docker/examples/docker-compose-files/single-node/ssl/docker-compose.yml index d152fafe1a..7539d44018 100644 --- a/docker/examples/jvm/single-node/ssl/docker-compose.yml +++ b/docker/examples/docker-compose-files/single-node/ssl/docker-compose.yml @@ -17,7 +17,7 @@ version: '2' services: broker: - image: apache/kafka:latest + image: ${IMAGE} hostname: broker container_name: broker ports: diff --git a/docker/native/README.md b/docker/native/README.md new file mode 100644 index 0000000000..0700dce346 --- /dev/null +++ b/docker/native/README.md @@ -0,0 +1,31 @@ +# GraalVM Based Native Apache Kafka Docker Image + +## Introduction +- The Native Apache Kafka Docker Image can launch brokers with sub-second startup time and minimal memory footprint by leveraging native Kafka executable. +- The native Kafka executable is built by compiling Apache Kafka code ahead-of-time using the [GraalVM native-image tool](https://www.graalvm.org/jdk21/reference-manual/native-image/). +- This image is experimental and intended for local development and testing purposes only; it is not recommended for production use. +- This is introduced with [KIP-974](https://cwiki.apache.org/confluence/display/KAFKA/KIP-974%3A+Docker+Image+for+GraalVM+based+Native+Kafka+Broker). + +## Native-Image reachability metadata +The native-image tool performs static analysis while building a native binary to determine the dynamic features(the dynamic language features of the JVM, including reflection and resource handling, compute the dynamically-accessed program elements such as invoked methods or resource URLs at runtime), but it cannot always exhaustively predict all uses. +To ensure inclusion of these elements into the native binary, reachability metadata should be provided to the native-image builder. +These metadata configs are provided in the `native-image-configs` directory. + +## How to generate reachability metadata +GraalVM provide an option to create these configs automatically by running the application normally with the [native-image agent](https://www.graalvm.org/latest/reference-manual/native-image/metadata/AutomaticMetadataCollection/) attached. +The configs in the `native-image-configs` directory are generated by running existing Apache Kafka System Tests(using GraalVM JIT and attaching the native-image agent to the process) as they are quite exhaustive. + +## Limitations of Native Apache Kafka executable +- Dynamic Features: For any new dynamic feature added or modified in Apache Kafka, the corresponding metadata configurations must be added or updated in the `native-image-configs`. As of now these configs are static. +- Runtime Jars: Native Kafka Executable does not support capabilities that require a new runtime jar provided by the user, as the jar's information is not available during build time. In such cases, a new native Kafka binary must be built, including that jar in the classpath during the build. +- Garbage Collection: This implementation uses the community edition of GraalVM, which supports only `serial` GC. Therefore, Native Apache Kafka supports `serial` GC and not `G1` GC. + +## How to use this image in docker container +Check out [this](../examples/README.md) guide. + +## How to build, test and release this docker image +Check out [this](../README.md) guide. + +## How to run system tests on native Apache Kafka +Check out [this](../../tests/README.md#running-tests-using-docker) guide. + diff --git a/docker/prepare_docker_official_image_source.py b/docker/prepare_docker_official_image_source.py index a39915c9e5..25d57c53e0 100644 --- a/docker/prepare_docker_official_image_source.py +++ b/docker/prepare_docker_official_image_source.py @@ -36,14 +36,18 @@ from distutils.dir_util import copy_tree import os import shutil +import re -def remove_args_and_hardcode_values(file_path, kafka_url): +def remove_args_and_hardcode_values(file_path, kafka_version, kafka_url): with open(file_path, 'r') as file: filedata = file.read() filedata = filedata.replace("ARG kafka_url", f"ENV kafka_url {kafka_url}") filedata = filedata.replace( "ARG build_date", f"ENV build_date {str(date.today())}") + original_comment = re.compile(r"# Get kafka from https://archive.apache.org/dist/kafka and pass the url through build arguments") + updated_comment = f"# Get Kafka from https://archive.apache.org/dist/kafka, url passed as env var, for version {kafka_version}" + filedata = original_comment.sub(updated_comment, filedata) with open(file_path, 'w') as file: file.write(filedata) @@ -55,7 +59,7 @@ def remove_args_and_hardcode_values(file_path, kafka_url): parser.add_argument("--kafka-version", "-v", dest="kafka_version", help="Kafka version for which the source for docker official image is to be built") args = parser.parse_args() - kafka_url = f"https://downloads.apache.org/kafka/{args.kafka_version}/kafka_2.13-{args.kafka_version}.tgz" + kafka_url = f"https://archive.apache.org/dist/kafka/{args.kafka_version}/kafka_2.13-{args.kafka_version}.tgz" current_dir = os.path.dirname(os.path.realpath(__file__)) new_dir = os.path.join( current_dir, f'docker_official_images', args.kafka_version) @@ -65,4 +69,4 @@ def remove_args_and_hardcode_values(file_path, kafka_url): copy_tree(os.path.join(current_dir, args.image_type), os.path.join(new_dir, args.kafka_version, args.image_type)) copy_tree(os.path.join(current_dir, 'resources'), os.path.join(new_dir, args.kafka_version, args.image_type, 'resources')) remove_args_and_hardcode_values( - os.path.join(new_dir, args.kafka_version, args.image_type, 'Dockerfile'), kafka_url) + os.path.join(new_dir, args.kafka_version, args.image_type, 'Dockerfile'), args.kafka_version, kafka_url) diff --git a/docs/docker.html b/docs/docker.html index bbedb10284..1f626ed060 100644 --- a/docs/docker.html +++ b/docs/docker.html @@ -20,33 +20,36 @@