From 5d6936a4992b77ef68da216a7c2dbf1f8c9f909e Mon Sep 17 00:00:00 2001 From: Gaurav Narula Date: Wed, 28 Feb 2024 09:37:58 +0000 Subject: [PATCH 001/521] KAFKA-16305: Avoid optimisation in handshakeUnwrap (#15434) Performs additional unwrap during handshake after data from client is processed to support openssl, which needs the extra unwrap to complete handshake. Reviewers: Ismael Juma , Rajini Sivaram --- .../common/network/SslTransportLayer.java | 7 ++- .../common/network/SslTransportLayerTest.java | 60 +++++++++++++++++++ 2 files changed, 64 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java index 904c5216a4..da80e363a9 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java @@ -498,13 +498,14 @@ private SSLEngineResult handshakeWrap(boolean doWrite) throws IOException { } /** - * Perform handshake unwrap + * Perform handshake unwrap. + * Visible for testing. * @param doRead boolean If true, read more from the socket channel * @param ignoreHandshakeStatus If true, continue to unwrap if data available regardless of handshake status * @return SSLEngineResult * @throws IOException */ - private SSLEngineResult handshakeUnwrap(boolean doRead, boolean ignoreHandshakeStatus) throws IOException { + SSLEngineResult handshakeUnwrap(boolean doRead, boolean ignoreHandshakeStatus) throws IOException { log.trace("SSLHandshake handshakeUnwrap {}", channelId); SSLEngineResult result; int read = 0; @@ -526,7 +527,7 @@ private SSLEngineResult handshakeUnwrap(boolean doRead, boolean ignoreHandshakeS handshakeStatus == HandshakeStatus.NEED_UNWRAP) || (ignoreHandshakeStatus && netReadBuffer.position() != position); log.trace("SSLHandshake handshakeUnwrap: handshakeStatus {} status {}", handshakeStatus, result.getStatus()); - } while (netReadBuffer.position() != 0 && cont); + } while (cont); // Throw EOF exception for failed read after processing already received data // so that handshake failures are reported correctly diff --git a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java index d92f4facb3..8b00bcdb95 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java @@ -36,6 +36,7 @@ import org.apache.kafka.test.TestSslUtils; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtensionContext; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; @@ -51,6 +52,7 @@ import java.nio.channels.Channels; import java.nio.channels.SelectionKey; import java.nio.channels.SocketChannel; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -65,13 +67,20 @@ import javax.net.ssl.SSLContext; import javax.net.ssl.SSLEngine; +import javax.net.ssl.SSLEngineResult; import javax.net.ssl.SSLParameters; +import javax.net.ssl.SSLSession; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assumptions.assumeTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; /** * Tests for the SSL transport layer. These use a test harness that runs a simple socket server that echos back responses. @@ -1467,4 +1476,55 @@ int updateAndGet(int actualSize, boolean update) { } } } + + /** + * SSLEngine implementations may transition from NEED_UNWRAP to NEED_UNWRAP + * even after reading all the data from the socket. This test ensures we + * continue unwrapping and not break early. + * Please refer KAFKA-16305 + * for more information. + */ + @Test + public void testHandshakeUnwrapContinuesUnwrappingOnNeedUnwrapAfterAllBytesRead() throws IOException { + // Given + byte[] data = "ClientHello?".getBytes(StandardCharsets.UTF_8); + + SSLEngine sslEngine = mock(SSLEngine.class); + SocketChannel socketChannel = mock(SocketChannel.class); + SelectionKey selectionKey = mock(SelectionKey.class); + when(selectionKey.channel()).thenReturn(socketChannel); + SSLSession sslSession = mock(SSLSession.class); + SslTransportLayer sslTransportLayer = new SslTransportLayer( + "test-channel", + selectionKey, + sslEngine, + mock(ChannelMetadataRegistry.class) + ); + + when(sslEngine.getSession()).thenReturn(sslSession); + when(sslSession.getPacketBufferSize()).thenReturn(data.length * 2); + sslTransportLayer.startHandshake(); // to initialize the buffers + + ByteBuffer netReadBuffer = sslTransportLayer.netReadBuffer(); + netReadBuffer.clear(); + ByteBuffer appReadBuffer = sslTransportLayer.appReadBuffer(); + when(socketChannel.read(any(ByteBuffer.class))).then(invocation -> { + ((ByteBuffer) invocation.getArgument(0)).put(data); + return data.length; + }); + + when(sslEngine.unwrap(netReadBuffer, appReadBuffer)) + .thenAnswer(invocation -> { + netReadBuffer.flip(); + return new SSLEngineResult(SSLEngineResult.Status.OK, SSLEngineResult.HandshakeStatus.NEED_UNWRAP, data.length, 0); + }).thenReturn(new SSLEngineResult(SSLEngineResult.Status.OK, SSLEngineResult.HandshakeStatus.NEED_WRAP, 0, 0)); + + // When + SSLEngineResult result = sslTransportLayer.handshakeUnwrap(true, false); + + // Then + verify(sslEngine, times(2)).unwrap(netReadBuffer, appReadBuffer); + assertEquals(SSLEngineResult.Status.OK, result.getStatus()); + assertEquals(SSLEngineResult.HandshakeStatus.NEED_WRAP, result.getHandshakeStatus()); + } } From 53c41aca7ba9469a0145023112f5fad254da4fa8 Mon Sep 17 00:00:00 2001 From: Philip Nee Date: Wed, 28 Feb 2024 01:52:01 -0800 Subject: [PATCH 002/521] KAFKA-16116: Rebalance Metrics for AsyncKafkaConsumer (#15339) Adding the following rebalance metrics to the consumer: rebalance-latency-avg rebalance-latency-max rebalance-latency-total rebalance-rate-per-hour rebalance-total failed-rebalance-rate-per-hour failed-rebalance-total Due to the difference in protocol, we need to redefine when rebalance starts and ends. Start of Rebalance: Current: Right before sending out JoinGroup ConsumerGroup: When the client receives assignments from the HB End of Rebalance - Successful Case: Current: Receiving SyncGroup request after transitioning to "COMPLETING_REBALANCE" ConsumerGroup: After completing reconciliation and right before sending out "Ack" heartbeat End of Rebalance - Failed Case: Current: Any failure in the JoinGroup/SyncGroup response ConsumerGroup: Failure in the heartbeat Note: Afterall, we try to be consistent with the current protocol. Rebalances start and end with sending and receiving network requests. Failures in network requests signify the user failures in rebalance. And it is entirely possible to have multiple failures before having a successful one. Reviewers: Lucas Brutschy --- .../internals/HeartbeatRequestManager.java | 3 +- .../consumer/internals/MembershipManager.java | 7 +- .../internals/MembershipManagerImpl.java | 62 ++++- .../consumer/internals/RequestManagers.java | 3 +- .../metrics/RebalanceMetricsManager.java | 114 +++++++++ .../internals/ConsumerTestBuilder.java | 26 +- .../HeartbeatRequestManagerTest.java | 8 +- .../internals/MembershipManagerImplTest.java | 240 +++++++++++++++--- 8 files changed, 408 insertions(+), 55 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/metrics/RebalanceMetricsManager.java 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 550e5b92eb..d551dbe250 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 @@ -327,7 +327,7 @@ private void onResponse(final ConsumerGroupHeartbeatResponse response, long curr heartbeatRequestState.updateHeartbeatIntervalMs(response.data().heartbeatIntervalMs()); heartbeatRequestState.onSuccessfulAttempt(currentTimeMs); heartbeatRequestState.resetTimer(); - membershipManager.onHeartbeatResponseReceived(response.data()); + membershipManager.onHeartbeatSuccess(response.data()); maybeSendGroupMetadataUpdateEvent(); return; } @@ -357,6 +357,7 @@ private void onErrorResponse(final ConsumerGroupHeartbeatResponse response, this.heartbeatState.reset(); this.heartbeatRequestState.onFailedAttempt(currentTimeMs); + membershipManager.onHeartbeatFailure(); switch (error) { case NOT_COORDINATOR: diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java index a9c23d7b4d..f0a641d140 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java @@ -68,7 +68,12 @@ public interface MembershipManager extends RequestManager { * * @param response Heartbeat response to extract member info and errors from. */ - void onHeartbeatResponseReceived(ConsumerGroupHeartbeatResponseData response); + void onHeartbeatSuccess(ConsumerGroupHeartbeatResponseData response); + + /** + * Notify the member that an error heartbeat response was received. + */ + void onHeartbeatFailure(); /** * Update state when a heartbeat is sent out. This will transition out of the states that end 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 a9b0f3b94d..6f3947eea4 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 @@ -27,11 +27,13 @@ import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; +import org.apache.kafka.clients.consumer.internals.metrics.RebalanceMetricsManager; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; +import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest; import org.apache.kafka.common.telemetry.internals.ClientTelemetryProvider; @@ -263,6 +265,11 @@ public class MembershipManagerImpl implements MembershipManager { * when the timer is reset, only when it completes releasing its assignment. */ private CompletableFuture staleMemberAssignmentRelease; + + /* + * Measures successful rebalance latency and number of failed rebalances. + */ + private final RebalanceMetricsManager metricsManager; private final Time time; @@ -284,7 +291,35 @@ public MembershipManagerImpl(String groupId, LogContext logContext, Optional clientTelemetryReporter, BackgroundEventHandler backgroundEventHandler, - Time time) { + Time time, + Metrics metrics) { + this(groupId, + groupInstanceId, + rebalanceTimeoutMs, + serverAssignor, + subscriptions, + commitRequestManager, + metadata, + logContext, + clientTelemetryReporter, + backgroundEventHandler, + time, + new RebalanceMetricsManager(metrics)); + } + + // Visible for testing + MembershipManagerImpl(String groupId, + Optional groupInstanceId, + int rebalanceTimeoutMs, + Optional serverAssignor, + SubscriptionState subscriptions, + CommitRequestManager commitRequestManager, + ConsumerMetadata metadata, + LogContext logContext, + Optional clientTelemetryReporter, + BackgroundEventHandler backgroundEventHandler, + Time time, + RebalanceMetricsManager metricsManager) { this.groupId = groupId; this.state = MemberState.UNSUBSCRIBED; this.serverAssignor = serverAssignor; @@ -301,6 +336,7 @@ public MembershipManagerImpl(String groupId, this.rebalanceTimeoutMs = rebalanceTimeoutMs; this.backgroundEventHandler = backgroundEventHandler; this.time = time; + this.metricsManager = metricsManager; } /** @@ -314,10 +350,27 @@ private void transitionTo(MemberState nextState) { throw new IllegalStateException(String.format("Invalid state transition from %s to %s", state, nextState)); } + + if (isCompletingRebalance(state, nextState)) { + metricsManager.recordRebalanceEnded(time.milliseconds()); + } + if (isStartingRebalance(state, nextState)) { + metricsManager.recordRebalanceStarted(time.milliseconds()); + } + log.trace("Member {} with epoch {} transitioned from {} to {}.", memberId, memberEpoch, state, nextState); this.state = nextState; } + private static boolean isCompletingRebalance(MemberState currentState, MemberState nextState) { + return currentState == MemberState.RECONCILING && + (nextState == MemberState.STABLE || nextState == MemberState.ACKNOWLEDGING); + } + + private static boolean isStartingRebalance(MemberState currentState, MemberState nextState) { + return currentState != MemberState.RECONCILING && nextState == MemberState.RECONCILING; + } + /** * {@inheritDoc} */ @@ -354,7 +407,7 @@ public int memberEpoch() { * {@inheritDoc} */ @Override - public void onHeartbeatResponseReceived(ConsumerGroupHeartbeatResponseData response) { + public void onHeartbeatSuccess(ConsumerGroupHeartbeatResponseData response) { if (response.errorCode() != Errors.NONE.code()) { String errorMessage = String.format( "Unexpected error in Heartbeat response. Expected no error, but received: %s", @@ -403,6 +456,11 @@ public void onHeartbeatResponseReceived(ConsumerGroupHeartbeatResponseData respo } } + @Override + public void onHeartbeatFailure() { + metricsManager.maybeRecordRebalanceFailed(); + } + /** * @return True if the consumer is not a member of the group. */ 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 2d90a3ad70..0b4c043d4a 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 @@ -189,7 +189,8 @@ protected RequestManagers create() { logContext, clientTelemetryReporter, backgroundEventHandler, - time); + time, + metrics); membershipManager.registerStateListener(commit); heartbeatRequestManager = new HeartbeatRequestManager( logContext, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/metrics/RebalanceMetricsManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/metrics/RebalanceMetricsManager.java new file mode 100644 index 0000000000..a255487f37 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/metrics/RebalanceMetricsManager.java @@ -0,0 +1,114 @@ +/* + * 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.metrics; + +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.Measurable; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Avg; +import org.apache.kafka.common.metrics.stats.CumulativeCount; +import org.apache.kafka.common.metrics.stats.CumulativeSum; +import org.apache.kafka.common.metrics.stats.Max; +import org.apache.kafka.common.metrics.stats.Rate; +import org.apache.kafka.common.metrics.stats.WindowedCount; + +import java.util.concurrent.TimeUnit; + +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.COORDINATOR_METRICS_SUFFIX; + +public class RebalanceMetricsManager { + private final Sensor successfulRebalanceSensor; + private final Sensor failedRebalanceSensor; + private final String metricGroupName; + + public final MetricName rebalanceLatencyAvg; + public final MetricName rebalanceLatencyMax; + public final MetricName rebalanceLatencyTotal; + public final MetricName rebalanceTotal; + public final MetricName rebalanceRatePerHour; + public final MetricName lastRebalanceSecondsAgo; + public final MetricName failedRebalanceTotal; + public final MetricName failedRebalanceRate; + private long lastRebalanceEndMs = -1L; + private long lastRebalanceStartMs = -1L; + + public RebalanceMetricsManager(Metrics metrics) { + metricGroupName = CONSUMER_METRIC_GROUP_PREFIX + COORDINATOR_METRICS_SUFFIX; + + rebalanceLatencyAvg = createMetric(metrics, "rebalance-latency-avg", + "The average time taken for a group to complete a rebalance"); + rebalanceLatencyMax = createMetric(metrics, "rebalance-latency-max", + "The max time taken for a group to complete a rebalance"); + rebalanceLatencyTotal = createMetric(metrics, "rebalance-latency-total", + "The total number of milliseconds spent in rebalances"); + rebalanceTotal = createMetric(metrics, "rebalance-total", + "The total number of rebalance events"); + rebalanceRatePerHour = createMetric(metrics, "rebalance-rate-per-hour", + "The number of rebalance events per hour"); + failedRebalanceTotal = createMetric(metrics, "failed-rebalance-total", + "The total number of failed rebalance events"); + failedRebalanceRate = createMetric(metrics, "failed-rebalance-rate-per-hour", + "The number of failed rebalance events per hour"); + + successfulRebalanceSensor = metrics.sensor("rebalance-latency"); + successfulRebalanceSensor.add(rebalanceLatencyAvg, new Avg()); + successfulRebalanceSensor.add(rebalanceLatencyMax, new Max()); + successfulRebalanceSensor.add(rebalanceLatencyTotal, new CumulativeSum()); + successfulRebalanceSensor.add(rebalanceTotal, new CumulativeCount()); + successfulRebalanceSensor.add(rebalanceRatePerHour, new Rate(TimeUnit.HOURS, new WindowedCount())); + + failedRebalanceSensor = metrics.sensor("failed-rebalance"); + failedRebalanceSensor.add(failedRebalanceTotal, new CumulativeSum()); + failedRebalanceSensor.add(failedRebalanceRate, new Rate(TimeUnit.HOURS, new WindowedCount())); + + Measurable lastRebalance = (config, now) -> { + if (lastRebalanceEndMs == -1L) + return -1d; + else + return TimeUnit.SECONDS.convert(now - lastRebalanceEndMs, TimeUnit.MILLISECONDS); + }; + lastRebalanceSecondsAgo = createMetric(metrics, + "last-rebalance-seconds-ago", + "The number of seconds since the last rebalance event"); + metrics.addMetric(lastRebalanceSecondsAgo, lastRebalance); + } + + private MetricName createMetric(Metrics metrics, String name, String description) { + return metrics.metricName(name, metricGroupName, description); + } + + public void recordRebalanceStarted(long nowMs) { + lastRebalanceStartMs = nowMs; + } + + public void recordRebalanceEnded(long nowMs) { + lastRebalanceEndMs = nowMs; + successfulRebalanceSensor.record(nowMs - lastRebalanceStartMs); + } + + public void maybeRecordRebalanceFailed() { + if (lastRebalanceStartMs <= lastRebalanceEndMs) + return; + failedRebalanceSensor.record(); + } + + public boolean rebalanceStarted() { + return lastRebalanceStartMs > lastRebalanceEndMs; + } +} \ No newline at end of file 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 d6ae629506..d62d3d8a35 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 @@ -26,6 +26,7 @@ import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; import org.apache.kafka.clients.consumer.internals.metrics.RebalanceCallbackMetricsManager; +import org.apache.kafka.clients.consumer.internals.metrics.RebalanceMetricsManager; import org.apache.kafka.common.internals.ClusterResourceListeners; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.requests.MetadataResponse; @@ -196,18 +197,19 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA gi.groupInstanceId, metrics)); MembershipManager mm = spy( - new MembershipManagerImpl( - gi.groupId, - gi.groupInstanceId, - groupRebalanceConfig.rebalanceTimeoutMs, - gi.serverAssignor, - subscriptions, - commit, - metadata, - logContext, - Optional.empty(), - backgroundEventHandler, - time + new MembershipManagerImpl( + gi.groupId, + gi.groupInstanceId, + groupRebalanceConfig.rebalanceTimeoutMs, + gi.serverAssignor, + subscriptions, + commit, + metadata, + logContext, + Optional.empty(), + backgroundEventHandler, + time, + mock(RebalanceMetricsManager.class) ) ); HeartbeatRequestManager.HeartbeatState heartbeatState = spy(new HeartbeatRequestManager.HeartbeatState( 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 5cf5b9e2d9..4d4492bcb4 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 @@ -306,7 +306,7 @@ public void testValidateConsumerGroupHeartbeatRequest(final short version) { new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() .setMemberId(memberId) .setMemberEpoch(memberEpoch)); - membershipManager.onHeartbeatResponseReceived(result.data()); + membershipManager.onHeartbeatSuccess(result.data()); // Create a ConsumerHeartbeatRequest and verify the payload NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); @@ -441,7 +441,7 @@ public void testHeartbeatResponseOnErrorHandling(final Errors error, final boole switch (error) { case NONE: verify(backgroundEventHandler).add(any(GroupMetadataUpdateEvent.class)); - verify(membershipManager, times(2)).onHeartbeatResponseReceived(mockResponse.data()); + verify(membershipManager, times(2)).onHeartbeatSuccess(mockResponse.data()); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, heartbeatRequestState.nextHeartbeatMs(time.milliseconds())); break; @@ -547,7 +547,7 @@ public void testHeartbeatState() { .setMemberEpoch(1) .setAssignment(assignmentTopic1)); when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, "topic1")); - membershipManager.onHeartbeatResponseReceived(rs1.data()); + membershipManager.onHeartbeatSuccess(rs1.data()); // We remain in RECONCILING state, as the assignment will be reconciled on the next poll assertEquals(MemberState.RECONCILING, membershipManager.state()); @@ -712,7 +712,7 @@ private void mockStableMember() { .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) .setMemberId(memberId) .setMemberEpoch(memberEpoch)); - membershipManager.onHeartbeatResponseReceived(rs1.data()); + membershipManager.onHeartbeatSuccess(rs1.data()); assertEquals(MemberState.STABLE, membershipManager.state()); } 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 3294068b07..8a0fcf8575 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 @@ -16,12 +16,15 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; import org.apache.kafka.clients.consumer.internals.metrics.RebalanceCallbackMetricsManager; +import org.apache.kafka.clients.consumer.internals.metrics.RebalanceMetricsManager; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; @@ -102,6 +105,8 @@ public class MembershipManagerImplTest { private BlockingQueue backgroundEventQueue; private BackgroundEventHandler backgroundEventHandler; private Time time; + private Metrics metrics; + private RebalanceMetricsManager rebalanceMetricsManager; @BeforeEach public void setup() { @@ -111,7 +116,9 @@ public void setup() { commitRequestManager = testBuilder.commitRequestManager.orElseThrow(IllegalStateException::new); backgroundEventQueue = testBuilder.backgroundEventQueue; backgroundEventHandler = testBuilder.backgroundEventHandler; - time = testBuilder.time; + time = new MockTime(0); + metrics = new Metrics(time); + rebalanceMetricsManager = new RebalanceMetricsManager(metrics); } @AfterEach @@ -135,15 +142,16 @@ private MembershipManagerImpl createMembershipManager(String groupInstanceId) { return spy(new MembershipManagerImpl( GROUP_ID, Optional.ofNullable(groupInstanceId), REBALANCE_TIMEOUT, Optional.empty(), subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), - backgroundEventHandler, time)); + backgroundEventHandler, time, rebalanceMetricsManager)); } private MembershipManagerImpl createMembershipManagerJoiningGroup(String groupInstanceId, String serverAssignor) { - MembershipManagerImpl manager = new MembershipManagerImpl( + MembershipManagerImpl manager = spy(new MembershipManagerImpl( GROUP_ID, Optional.ofNullable(groupInstanceId), REBALANCE_TIMEOUT, Optional.ofNullable(serverAssignor), subscriptionState, commitRequestManager, - metadata, logContext, Optional.empty(), backgroundEventHandler, time); + metadata, logContext, Optional.empty(), backgroundEventHandler, time, + rebalanceMetricsManager)); manager.transitionToJoining(); return manager; } @@ -160,7 +168,6 @@ public void testMembershipManagerServerAssignor() { @Test public void testMembershipManagerInitSupportsEmptyGroupInstanceId() { createMembershipManagerJoiningGroup(); - createMembershipManagerJoiningGroup(null, null); } @Test @@ -169,7 +176,7 @@ public void testMembershipManagerRegistersForClusterMetadataUpdatesOnFirstJoin() MembershipManagerImpl manager = new MembershipManagerImpl( GROUP_ID, Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), - backgroundEventHandler, time); + backgroundEventHandler, time, rebalanceMetricsManager); manager.transitionToJoining(); clearInvocations(metadata); @@ -200,12 +207,12 @@ public void testTransitionToReconcilingOnlyIfAssignmentReceived() { ConsumerGroupHeartbeatResponse responseWithoutAssignment = createConsumerGroupHeartbeatResponse(null); - membershipManager.onHeartbeatResponseReceived(responseWithoutAssignment.data()); + membershipManager.onHeartbeatSuccess(responseWithoutAssignment.data()); assertNotEquals(MemberState.RECONCILING, membershipManager.state()); ConsumerGroupHeartbeatResponse responseWithAssignment = createConsumerGroupHeartbeatResponse(createAssignment(true)); - membershipManager.onHeartbeatResponseReceived(responseWithAssignment.data()); + membershipManager.onHeartbeatSuccess(responseWithAssignment.data()); assertEquals(MemberState.RECONCILING, membershipManager.state()); } @@ -213,7 +220,7 @@ public void testTransitionToReconcilingOnlyIfAssignmentReceived() { public void testMemberIdAndEpochResetOnFencedMembers() { MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(null); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); assertEquals(MemberState.STABLE, membershipManager.state()); assertEquals(MEMBER_ID, membershipManager.memberId()); assertEquals(MEMBER_EPOCH, membershipManager.memberEpoch()); @@ -230,7 +237,7 @@ public void testTransitionToFatal() { MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(null); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); assertEquals(MemberState.STABLE, membershipManager.state()); assertEquals(MEMBER_ID, membershipManager.memberId()); assertEquals(MEMBER_EPOCH, membershipManager.memberEpoch()); @@ -246,7 +253,7 @@ public void testTransitionToFailedWhenTryingToJoin() { MembershipManagerImpl membershipManager = new MembershipManagerImpl( GROUP_ID, Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), - backgroundEventHandler, time); + backgroundEventHandler, time, rebalanceMetricsManager); assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); membershipManager.transitionToJoining(); @@ -299,7 +306,7 @@ public void testListenersGetNotifiedOfMemberEpochUpdatesOnlyIfItChanges() { membershipManager.registerStateListener(listener); int epoch = 5; - membershipManager.onHeartbeatResponseReceived(new ConsumerGroupHeartbeatResponseData() + membershipManager.onHeartbeatSuccess(new ConsumerGroupHeartbeatResponseData() .setErrorCode(Errors.NONE.code()) .setMemberId(MEMBER_ID) .setMemberEpoch(epoch)); @@ -307,7 +314,7 @@ public void testListenersGetNotifiedOfMemberEpochUpdatesOnlyIfItChanges() { verify(listener).onMemberEpochUpdated(Optional.of(epoch), Optional.of(MEMBER_ID)); clearInvocations(listener); - membershipManager.onHeartbeatResponseReceived(new ConsumerGroupHeartbeatResponseData() + membershipManager.onHeartbeatSuccess(new ConsumerGroupHeartbeatResponseData() .setErrorCode(Errors.NONE.code()) .setMemberId(MEMBER_ID) .setMemberEpoch(epoch)); @@ -316,7 +323,7 @@ public void testListenersGetNotifiedOfMemberEpochUpdatesOnlyIfItChanges() { private void mockStableMember(MembershipManagerImpl membershipManager) { ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(null); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); assertEquals(MemberState.STABLE, membershipManager.state()); assertEquals(MEMBER_ID, membershipManager.memberId()); assertEquals(MEMBER_EPOCH, membershipManager.memberEpoch()); @@ -709,7 +716,7 @@ public void testIgnoreHeartbeatWhenLeavingGroup() { CompletableFuture leaveResult = membershipManager.leaveGroup(); - membershipManager.onHeartbeatResponseReceived(createConsumerGroupHeartbeatResponse(createAssignment(true)).data()); + membershipManager.onHeartbeatSuccess(createConsumerGroupHeartbeatResponse(createAssignment(true)).data()); assertEquals(MemberState.LEAVING, membershipManager.state()); assertEquals(-1, membershipManager.memberEpoch()); @@ -726,7 +733,7 @@ public void testIgnoreHeartbeatResponseWhenNotInGroup(MemberState state) { when(membershipManager.state()).thenReturn(state); ConsumerGroupHeartbeatResponseData responseData = mock(ConsumerGroupHeartbeatResponseData.class); - membershipManager.onHeartbeatResponseReceived(responseData); + membershipManager.onHeartbeatSuccess(responseData); assertEquals(state, membershipManager.state()); verify(responseData, never()).memberId(); @@ -861,7 +868,7 @@ public void testFatalFailureWhenStateIsUnjoined() { public void testFatalFailureWhenStateIsStable() { MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(null); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); assertEquals(MemberState.STABLE, membershipManager.state()); testStateUpdateOnFatalFailure(membershipManager); @@ -930,9 +937,9 @@ public void testUpdateStateFailsOnResponsesWithErrors() { // Updating state with a heartbeat response containing errors cannot be performed and // should fail. ConsumerGroupHeartbeatResponse unknownMemberResponse = - createConsumerGroupHeartbeatResponseWithError(); + createConsumerGroupHeartbeatResponseWithError(Errors.UNKNOWN_MEMBER_ID); assertThrows(IllegalArgumentException.class, - () -> membershipManager.onHeartbeatResponseReceived(unknownMemberResponse.data())); + () -> membershipManager.onHeartbeatSuccess(unknownMemberResponse.data())); } /** @@ -1099,7 +1106,7 @@ public void testMemberKeepsUnresolvedAssignmentWaitingForMetadataUntilResolved() // Target assignment received again with the same unresolved topic. Client should keep it // as unresolved. clearInvocations(subscriptionState); - membershipManager.onHeartbeatResponseReceived(createConsumerGroupHeartbeatResponse(assignment).data()); + membershipManager.onHeartbeatSuccess(createConsumerGroupHeartbeatResponse(assignment).data()); assertEquals(MemberState.RECONCILING, membershipManager.state()); assertEquals(Collections.singleton(topic2), membershipManager.topicsAwaitingReconciliation()); verify(subscriptionState, never()).assignFromSubscribed(anyCollection()); @@ -1173,6 +1180,9 @@ public void testReconciliationSkippedWhenSameAssignmentReceived() { verify(subscriptionState, never()).assignFromSubscribed(anyCollection()); assertEquals(MemberState.STABLE, membershipManager.state()); + + assertEquals(1.0d, getMetricValue(metrics, rebalanceMetricsManager.rebalanceTotal)); + assertEquals(0.0d, getMetricValue(metrics, rebalanceMetricsManager.failedRebalanceTotal)); } @Test @@ -1376,7 +1386,6 @@ public void testOnSubscriptionUpdatedTransitionsToJoiningOnlyIfNotInGroup() { @Test public void testListenerCallbacksBasic() { - // Step 1: set up mocks MembershipManagerImpl membershipManager = createMemberInStableState(); CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener(); ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker(); @@ -1685,7 +1694,7 @@ public void testTransitionToLeavingWhileJoiningDueToStaleMember() { public void testTransitionToLeavingWhileStableDueToStaleMember() { MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(null); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); doNothing().when(subscriptionState).assignFromSubscribed(any()); assertEquals(MemberState.STABLE, membershipManager.state()); assertLeaveGroupDueToExpiredPollAndTransitionToStale(membershipManager); @@ -1832,8 +1841,8 @@ private ConsumerRebalanceListenerInvoker consumerRebalanceListenerInvoker() { return new ConsumerRebalanceListenerInvoker( new LogContext(), subscriptionState, - new MockTime(1), - new RebalanceCallbackMetricsManager(new Metrics()) + time, + new RebalanceCallbackMetricsManager(new Metrics(time)) ); } @@ -1930,7 +1939,7 @@ private void assertStaleMemberLeavesGroupAndClearsAssignment(MembershipManagerIm @Test public void testMemberJoiningTransitionsToStableWhenReceivingEmptyAssignment() { - MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(null); + MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); assertEquals(MemberState.JOINING, membershipManager.state()); receiveEmptyAssignment(membershipManager); @@ -1940,6 +1949,144 @@ public void testMemberJoiningTransitionsToStableWhenReceivingEmptyAssignment() { assertEquals(MemberState.STABLE, membershipManager.state()); } + @Test + public void testMetricsWhenHeartbeatFailed() { + MembershipManagerImpl membershipManager = createMemberInStableState(); + membershipManager.onHeartbeatFailure(); + + // Not expecting rebalance failures without assignments being reconciled + assertEquals(0.0d, getMetricValue(metrics, rebalanceMetricsManager.rebalanceTotal)); + assertEquals(0.0d, getMetricValue(metrics, rebalanceMetricsManager.failedRebalanceTotal)); + } + + @Test + public void testRebalanceMetricsOnSuccessfulRebalance() { + MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); + ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(null); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); + mockOwnedPartition(membershipManager, Uuid.randomUuid(), "topic1"); + + CompletableFuture commitResult = mockRevocationNoCallbacks(true); + + receiveEmptyAssignment(membershipManager); + long reconciliationDurationMs = 1234; + time.sleep(reconciliationDurationMs); + + membershipManager.poll(time.milliseconds()); + // Complete commit request to complete the callback invocation + commitResult.complete(null); + + assertEquals((double) reconciliationDurationMs, getMetricValue(metrics, rebalanceMetricsManager.rebalanceLatencyTotal)); + assertEquals((double) reconciliationDurationMs, getMetricValue(metrics, rebalanceMetricsManager.rebalanceLatencyAvg)); + assertEquals((double) reconciliationDurationMs, getMetricValue(metrics, rebalanceMetricsManager.rebalanceLatencyMax)); + assertEquals(1d, getMetricValue(metrics, rebalanceMetricsManager.rebalanceTotal)); + assertEquals(120d, 1d, (double) getMetricValue(metrics, rebalanceMetricsManager.rebalanceRatePerHour)); + assertEquals(0d, getMetricValue(metrics, rebalanceMetricsManager.failedRebalanceRate)); + assertEquals(0d, getMetricValue(metrics, rebalanceMetricsManager.failedRebalanceTotal)); + assertEquals(0d, getMetricValue(metrics, rebalanceMetricsManager.lastRebalanceSecondsAgo)); + } + + @Test + public void testRebalanceMetricsForMultipleReconcilations() { + MembershipManagerImpl membershipManager = createMemberInStableState(); + ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker(); + + String topicName = "topic1"; + Uuid topicId = Uuid.randomUuid(); + + SleepyRebalanceListener listener = new SleepyRebalanceListener(1453, time); + when(subscriptionState.assignedPartitions()).thenReturn(Collections.emptySet()); + when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); + when(subscriptionState.rebalanceListener()).thenReturn(Optional.of(listener)); + doNothing().when(subscriptionState).markPendingRevocation(anySet()); + when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, topicName)); + + when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, topicName)); + receiveAssignment(topicId, Arrays.asList(0, 1), membershipManager); + + membershipManager.poll(time.milliseconds()); + + // assign partitions + performCallback( + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, + topicPartitions(topicName, 0, 1), + true + ); + + long firstRebalanaceTimesMs = listener.sleepMs; + listener.reset(); + + // ack + membershipManager.onHeartbeatRequestSent(); + + // revoke all + when(subscriptionState.assignedPartitions()).thenReturn(topicPartitions(topicName, 0, 1)); + receiveAssignment(topicId, Collections.singletonList(2), membershipManager); + + membershipManager.poll(time.milliseconds()); + + performCallback( + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, + topicPartitions(topicName, 0, 1), + true + ); + + // assign new partition 2 + performCallback( + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, + topicPartitions(topicName, 2), + true + ); + membershipManager.onHeartbeatRequestSent(); + + long secondRebalanceMs = listener.sleepMs; + long total = firstRebalanaceTimesMs + secondRebalanceMs; + double avg = total / 2.0d; + long max = Math.max(firstRebalanaceTimesMs, secondRebalanceMs); + assertEquals((double) total, getMetricValue(metrics, rebalanceMetricsManager.rebalanceLatencyTotal)); + assertEquals(avg, (double) getMetricValue(metrics, rebalanceMetricsManager.rebalanceLatencyAvg), 1d); + assertEquals((double) max, getMetricValue(metrics, rebalanceMetricsManager.rebalanceLatencyMax)); + assertEquals(2d, getMetricValue(metrics, rebalanceMetricsManager.rebalanceTotal)); + // rate is not tested because it is subject to Rate implementation + assertEquals(0d, getMetricValue(metrics, rebalanceMetricsManager.failedRebalanceRate)); + assertEquals(0d, getMetricValue(metrics, rebalanceMetricsManager.failedRebalanceTotal)); + assertEquals(0d, getMetricValue(metrics, rebalanceMetricsManager.lastRebalanceSecondsAgo)); + + } + + @Test + public void testRebalanceMetricsOnFailedRebalance() { + MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); + ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(null); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); + + Uuid topicId = Uuid.randomUuid(); + + receiveAssignment(topicId, Arrays.asList(0, 1), membershipManager); + + // sleep for an arbitrary amount + time.sleep(2300); + + assertTrue(rebalanceMetricsManager.rebalanceStarted()); + membershipManager.onHeartbeatFailure(); + + assertEquals((double) 0, getMetricValue(metrics, rebalanceMetricsManager.rebalanceLatencyTotal)); + assertEquals(0d, getMetricValue(metrics, rebalanceMetricsManager.rebalanceTotal)); + assertEquals(120d, getMetricValue(metrics, rebalanceMetricsManager.failedRebalanceRate)); + assertEquals(1d, getMetricValue(metrics, rebalanceMetricsManager.failedRebalanceTotal)); + assertEquals(-1d, getMetricValue(metrics, rebalanceMetricsManager.lastRebalanceSecondsAgo)); + } + + private Object getMetricValue(Metrics metrics, MetricName name) { + return metrics.metrics().get(name).metricValue(); + } + private MembershipManagerImpl mockMemberSuccessfullyReceivesAndAcksAssignment( Uuid topicId, String topicName, List partitions) { MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); @@ -2119,7 +2266,7 @@ private MembershipManagerImpl mockJoinAndReceiveAssignment(boolean expectSubscri when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); when(subscriptionState.rebalanceListener()).thenReturn(Optional.empty()).thenReturn(Optional.empty()); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); membershipManager.poll(time.milliseconds()); if (expectSubscriptionUpdated) { @@ -2136,9 +2283,9 @@ private MembershipManagerImpl createMemberInStableState() { } private MembershipManagerImpl createMemberInStableState(String groupInstanceId) { - MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(groupInstanceId); + MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(groupInstanceId, null); ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(null); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); assertEquals(MemberState.STABLE, membershipManager.state()); return membershipManager; } @@ -2150,7 +2297,7 @@ private void receiveAssignment(Map> topicIdPartitionLis .setTopicId(tp.getKey()) .setPartitions(new ArrayList<>(tp.getValue()))).collect(Collectors.toList())); ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(targetAssignment); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); } private void receiveAssignment(Uuid topicId, List partitions, MembershipManager membershipManager) { @@ -2160,7 +2307,7 @@ private void receiveAssignment(Uuid topicId, List partitions, Membershi .setTopicId(topicId) .setPartitions(partitions))); ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(targetAssignment); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); } private void receiveAssignmentAfterRejoin(Uuid topicId, List partitions, MembershipManager membershipManager) { @@ -2171,7 +2318,7 @@ private void receiveAssignmentAfterRejoin(Uuid topicId, List partitions .setPartitions(partitions))); ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponseWithBumpedEpoch(targetAssignment); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); } private void receiveEmptyAssignment(MembershipManager membershipManager) { @@ -2179,7 +2326,7 @@ private void receiveEmptyAssignment(MembershipManager membershipManager) { ConsumerGroupHeartbeatResponseData.Assignment targetAssignment = new ConsumerGroupHeartbeatResponseData.Assignment() .setTopicPartitions(Collections.emptyList()); ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(targetAssignment); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); } /** @@ -2307,9 +2454,9 @@ private ConsumerGroupHeartbeatResponse createConsumerGroupHeartbeatResponseWithB .setAssignment(assignment)); } - private ConsumerGroupHeartbeatResponse createConsumerGroupHeartbeatResponseWithError() { + private ConsumerGroupHeartbeatResponse createConsumerGroupHeartbeatResponseWithError(Errors error) { return new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() - .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code()) + .setErrorCode(error.code()) .setMemberId(MEMBER_ID) .setMemberEpoch(5)); } @@ -2356,4 +2503,29 @@ private static Stream notInGroupStates() { Arguments.of(MemberState.STALE)); } + private static class SleepyRebalanceListener implements ConsumerRebalanceListener { + private long sleepMs; + private final long sleepDurationMs; + private final Time time; + SleepyRebalanceListener(long sleepDurationMs, Time time) { + this.sleepDurationMs = sleepDurationMs; + this.time = time; + } + + @Override + public void onPartitionsRevoked(Collection partitions) { + sleepMs += sleepDurationMs; + time.sleep(sleepDurationMs); + } + + @Override + public void onPartitionsAssigned(Collection partitions) { + sleepMs += sleepDurationMs; + time.sleep(sleepDurationMs); + } + + public void reset() { + sleepMs = 0; + } + } } From 52289c92be45ba3758d07376d9c64ddadbecb544 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Wed, 28 Feb 2024 05:38:02 -0800 Subject: [PATCH 003/521] MINOR: Optimize EventAccumulator (#15430) `poll(long timeout, TimeUnit unit)` is either used with `Long.MAX_VALUE` or `0`. This patch replaces it with `poll` and `take`. It removes the `awaitNanos` usage. Reviewers: Jeff Kim , Justine Olshan --- .../group/runtime/EventAccumulator.java | 37 +++++++---- .../runtime/MultiThreadedEventProcessor.java | 9 +-- .../group/runtime/EventAccumulatorTest.java | 30 ++++----- .../MultiThreadedEventProcessorTest.java | 61 +++++-------------- 4 files changed, 56 insertions(+), 81 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/EventAccumulator.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/EventAccumulator.java index f46e8b8a8b..16b61f8e99 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/EventAccumulator.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/EventAccumulator.java @@ -27,7 +27,6 @@ import java.util.Random; import java.util.Set; import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; @@ -137,31 +136,43 @@ public void add(T event) throws RejectedExecutionException { } /** - * Returns the next {{@link Event}} available. This method block indefinitely until - * one event is ready or the accumulator is closed. + * Returns the next {{@link Event}} available or null if no event is + * available. * - * @return The next event. + * @return The next event available or null. */ public T poll() { - return poll(Long.MAX_VALUE, TimeUnit.SECONDS); + lock.lock(); + try { + K key = randomKey(); + if (key == null) return null; + + Queue queue = queues.get(key); + T event = queue.poll(); + + if (queue.isEmpty()) queues.remove(key); + inflightKeys.add(key); + size--; + + return event; + } finally { + lock.unlock(); + } } /** - * Returns the next {{@link Event}} available. This method blocks for the provided - * time and returns null of not event is available. + * Returns the next {{@link Event}} available. This method blocks until an + * event is available or accumulator is closed. * - * @param timeout The timeout. - * @param unit The timeout unit. * @return The next event available or null. */ - public T poll(long timeout, TimeUnit unit) { + public T take() { lock.lock(); try { K key = randomKey(); - long nanos = unit.toNanos(timeout); - while (key == null && !closed && nanos > 0) { + while (key == null && !closed) { try { - nanos = condition.awaitNanos(nanos); + condition.await(); } catch (InterruptedException e) { // Ignore. } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java index e4adc18e95..0e3d563861 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java @@ -25,7 +25,6 @@ import java.util.List; import java.util.Objects; import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -128,7 +127,7 @@ private class EventProcessorThread extends Thread { private void handleEvents() { while (!shuttingDown) { recordPollStartTime(time.milliseconds()); - CoordinatorEvent event = accumulator.poll(); + CoordinatorEvent event = accumulator.take(); recordPollEndTime(time.milliseconds()); if (event != null) { try { @@ -148,8 +147,8 @@ private void handleEvents() { } private void drainEvents() { - CoordinatorEvent event = accumulator.poll(0, TimeUnit.MILLISECONDS); - while (event != null) { + CoordinatorEvent event; + while ((event = accumulator.poll()) != null) { try { log.debug("Draining event: {}.", event); metrics.recordEventQueueTime(time.milliseconds() - event.createdTimeMs()); @@ -159,8 +158,6 @@ private void drainEvents() { } finally { accumulator.done(event); } - - event = accumulator.poll(0, TimeUnit.MILLISECONDS); } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/EventAccumulatorTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/EventAccumulatorTest.java index 147cf08121..e077fb5e02 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/EventAccumulatorTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/EventAccumulatorTest.java @@ -78,7 +78,7 @@ public void testBasicOperations() { EventAccumulator accumulator = new EventAccumulator<>(); assertEquals(0, accumulator.size()); - assertNull(accumulator.poll(0, TimeUnit.MICROSECONDS)); + assertNull(accumulator.poll()); List events = Arrays.asList( new MockEvent(1, 0), @@ -97,14 +97,14 @@ public void testBasicOperations() { Set polledEvents = new HashSet<>(); for (int i = 0; i < events.size(); i++) { - MockEvent event = accumulator.poll(0, TimeUnit.MICROSECONDS); + MockEvent event = accumulator.poll(); assertNotNull(event); polledEvents.add(event); assertEquals(events.size() - 1 - i, accumulator.size()); accumulator.done(event); } - assertNull(accumulator.poll(0, TimeUnit.MICROSECONDS)); + assertNull(accumulator.poll()); assertEquals(new HashSet<>(events), polledEvents); assertEquals(0, accumulator.size()); @@ -126,27 +126,27 @@ public void testKeyConcurrentAndOrderingGuarantees() { MockEvent event = null; // Poll event0. - event = accumulator.poll(0, TimeUnit.MICROSECONDS); + event = accumulator.poll(); assertEquals(event0, event); // Poll returns null because key is inflight. - assertNull(accumulator.poll(0, TimeUnit.MICROSECONDS)); + assertNull(accumulator.poll()); accumulator.done(event); // Poll event1. - event = accumulator.poll(0, TimeUnit.MICROSECONDS); + event = accumulator.poll(); assertEquals(event1, event); // Poll returns null because key is inflight. - assertNull(accumulator.poll(0, TimeUnit.MICROSECONDS)); + assertNull(accumulator.poll()); accumulator.done(event); // Poll event2. - event = accumulator.poll(0, TimeUnit.MICROSECONDS); + event = accumulator.poll(); assertEquals(event2, event); // Poll returns null because key is inflight. - assertNull(accumulator.poll(0, TimeUnit.MICROSECONDS)); + assertNull(accumulator.poll()); accumulator.done(event); accumulator.close(); @@ -160,9 +160,9 @@ public void testDoneUnblockWaitingThreads() throws ExecutionException, Interrupt MockEvent event1 = new MockEvent(1, 1); MockEvent event2 = new MockEvent(1, 2); - CompletableFuture future0 = CompletableFuture.supplyAsync(accumulator::poll); - CompletableFuture future1 = CompletableFuture.supplyAsync(accumulator::poll); - CompletableFuture future2 = CompletableFuture.supplyAsync(accumulator::poll); + CompletableFuture future0 = CompletableFuture.supplyAsync(accumulator::take); + CompletableFuture future1 = CompletableFuture.supplyAsync(accumulator::take); + CompletableFuture future2 = CompletableFuture.supplyAsync(accumulator::take); List> futures = Arrays.asList(future0, future1, future2); assertFalse(future0.isDone()); @@ -215,9 +215,9 @@ public void testDoneUnblockWaitingThreads() throws ExecutionException, Interrupt public void testCloseUnblockWaitingThreads() throws ExecutionException, InterruptedException, TimeoutException { EventAccumulator accumulator = new EventAccumulator<>(); - CompletableFuture future0 = CompletableFuture.supplyAsync(accumulator::poll); - CompletableFuture future1 = CompletableFuture.supplyAsync(accumulator::poll); - CompletableFuture future2 = CompletableFuture.supplyAsync(accumulator::poll); + CompletableFuture future0 = CompletableFuture.supplyAsync(accumulator::take); + CompletableFuture future1 = CompletableFuture.supplyAsync(accumulator::take); + CompletableFuture future2 = CompletableFuture.supplyAsync(accumulator::take); assertFalse(future0.isDone()); assertFalse(future1.isDone()); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java index 2714188f65..3708141827 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java @@ -27,15 +27,12 @@ import java.util.ArrayList; import java.util.Arrays; -import java.util.LinkedList; import java.util.List; -import java.util.Queue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; @@ -53,53 +50,19 @@ @Timeout(value = 60) public class MultiThreadedEventProcessorTest { - private static class MockEventAccumulator extends EventAccumulator { + private static class DelayEventAccumulator extends EventAccumulator { private final Time time; - private final Queue events; - private final long timeToPollMs; - private final AtomicBoolean isClosed; + private final long takeDelayMs; - public MockEventAccumulator(Time time, long timeToPollMs) { + public DelayEventAccumulator(Time time, long takeDelayMs) { this.time = time; - this.events = new LinkedList<>(); - this.timeToPollMs = timeToPollMs; - this.isClosed = new AtomicBoolean(false); + this.takeDelayMs = takeDelayMs; } @Override - public CoordinatorEvent poll() { - synchronized (events) { - while (events.isEmpty() && !isClosed.get()) { - try { - events.wait(); - } catch (Exception ignored) { - - } - } - time.sleep(timeToPollMs); - return events.poll(); - } - } - - @Override - public CoordinatorEvent poll(long timeout, TimeUnit unit) { - return null; - } - - @Override - public void add(CoordinatorEvent event) throws RejectedExecutionException { - synchronized (events) { - events.add(event); - events.notifyAll(); - } - } - - @Override - public void close() { - isClosed.set(true); - synchronized (events) { - events.notifyAll(); - } + public CoordinatorEvent take() { + time.sleep(takeDelayMs); + return super.take(); } } @@ -353,7 +316,11 @@ public void testEventsAreDrainedWhenClosed() throws Exception { AtomicInteger numEventsExecuted = new AtomicInteger(0); // Special event which blocks until the latch is released. - FutureEvent blockingEvent = new FutureEvent<>(new TopicPartition("foo", 0), numEventsExecuted::incrementAndGet, true); + FutureEvent blockingEvent = new FutureEvent<>( + new TopicPartition("foo", 0), + numEventsExecuted::incrementAndGet, + true + ); List> events = Arrays.asList( new FutureEvent<>(new TopicPartition("foo", 0), numEventsExecuted::incrementAndGet), @@ -428,7 +395,7 @@ public void testMetrics() throws Exception { 1, // Use a single thread to block event in the processor. mockTime, mockRuntimeMetrics, - new MockEventAccumulator<>(mockTime, 500L) + new DelayEventAccumulator(mockTime, 500L) )) { // Enqueue the blocking event. eventProcessor.enqueue(blockingEvent); @@ -501,7 +468,7 @@ public void testRecordThreadIdleRatioTwoThreads() throws Exception { 2, Time.SYSTEM, mockRuntimeMetrics, - new MockEventAccumulator<>(Time.SYSTEM, 100L) + new DelayEventAccumulator(Time.SYSTEM, 100L) )) { List recordedRatios = new ArrayList<>(); AtomicInteger numEventsExecuted = new AtomicInteger(0); From 1bb9a851744972d6b2ce534fd75f03b18a61974d Mon Sep 17 00:00:00 2001 From: John Yu <54207775+chiacyu@users.noreply.github.com> Date: Thu, 29 Feb 2024 08:14:35 +0800 Subject: [PATCH 004/521] MINOR: Remove the space between two words (#15439) Remove the space between two words Reviewers: Luke Chen --- .../kafka/server/log/remote/storage/RemoteLogMetadata.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadata.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadata.java index 74d5c3d28a..4d8eaa95bc 100644 --- a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadata.java +++ b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadata.java @@ -32,7 +32,7 @@ public abstract class RemoteLogMetadata { private final int brokerId; /** - * Epoch time in milli seconds at which this event is generated. + * Epoch time in milliseconds at which this event is generated. */ private final long eventTimestampMs; @@ -42,7 +42,7 @@ protected RemoteLogMetadata(int brokerId, long eventTimestampMs) { } /** - * @return Epoch time in milli seconds at which this event is occurred. + * @return Epoch time in milliseconds at which this event is occurred. */ public long eventTimestampMs() { return eventTimestampMs; From 55a6d30ccbe971f4d2e99aeb3b1a773ffe5792a2 Mon Sep 17 00:00:00 2001 From: Christo Lolov Date: Thu, 29 Feb 2024 00:19:55 +0000 Subject: [PATCH 005/521] KAFKA-16154: Broker returns offset for LATEST_TIERED_TIMESTAMP (#15213) This is the first part of the implementation of KIP-1005 The purpose of this pull request is for the broker to start returning the correct offset when it receives a -5 as a timestamp in a ListOffsets API request Reviewers: Luke Chen , Kamal Chandraprakash , Satish Duggana --- .../common/requests/ListOffsetsRequest.java | 2 + .../src/main/scala/kafka/log/UnifiedLog.scala | 56 ++++++++---- .../scala/unit/kafka/log/UnifiedLogTest.scala | 86 +++++++++++++++++++ 3 files changed, 127 insertions(+), 17 deletions(-) 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 efdc7da2af..fc996453d6 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,6 +47,8 @@ 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/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index c0bb9d8cd6..f6198ce9d2 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -41,7 +41,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, EpochEntry, 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, 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} @@ -150,7 +150,9 @@ 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 var highestOffsetInRemoteStorage: Long = -1L + @volatile private[kafka] var _highestOffsetInRemoteStorage: Long = -1L + + def highestOffsetInRemoteStorage(): Long = _highestOffsetInRemoteStorage locally { def updateLocalLogStartOffset(offset: Long): Unit = { @@ -544,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 @@ -1279,7 +1281,6 @@ 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 " + @@ -1300,18 +1301,39 @@ class UnifiedLog(@volatile var logStartOffset: Long, } else if (targetTimestamp == ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP) { val curLocalLogStartOffset = localLogStartOffset() - val earliestLocalLogEpochEntry = leaderEpochCache.asJava.flatMap(cache => { - val epoch = cache.epochForOffset(curLocalLogStartOffset) - if (epoch.isPresent) cache.epochEntry(epoch.getAsInt) else Optional.empty[EpochEntry]() - }) - - val epochOpt = if (earliestLocalLogEpochEntry.isPresent && earliestLocalLogEpochEntry.get().startOffset <= curLocalLogStartOffset) - Optional.of[Integer](earliestLocalLogEpochEntry.get().epoch) - else Optional.empty[Integer]() + 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() + } - Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, curLocalLogStartOffset, epochOpt)) + Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, curLocalLogStartOffset, epochResult)) } else if (targetTimestamp == ListOffsetsRequest.LATEST_TIMESTAMP) { Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, logEndOffset, latestEpochAsOptional(leaderEpochCache))) + } 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`. @@ -1448,7 +1470,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 @@ -1582,13 +1604,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 diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 7b92a9e2df..ffa585a460 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -2127,6 +2127,92 @@ 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 */ From 96c68096a26ea5e7c2333308dfbaef47cb1eac72 Mon Sep 17 00:00:00 2001 From: Ritika Reddy <98577846+rreddy-22@users.noreply.github.com> Date: Thu, 29 Feb 2024 00:38:42 -0800 Subject: [PATCH 006/521] KAFKA-15462: Add Group Type Filter for List Group to the Admin Client (#15150) In KIP-848, we introduce the notion of Group Types based on the protocol type that the members in the consumer group use. As of now we support two types of groups: * Classic : Members use the classic consumer group protocol ( existing one ) * Consumer : Members use the consumer group protocol introduced in KIP-848. Currently List Groups allows users to list all the consumer groups available. KIP-518 introduced filtering the consumer groups by the state that they are in. We now want to allow users to filter consumer groups by type. This patch includes the changes to the admin client and related files. It also includes changes to parameterize the tests to include permutations of the old GC and the new GC with the different protocol types. Reviewers: David Jacot --- checkstyle/suppressions.xml | 1 + .../clients/admin/ConsumerGroupListing.java | 64 +-- .../kafka/clients/admin/KafkaAdminClient.java | 20 +- .../admin/ListConsumerGroupsOptions.java | 25 +- .../clients/admin/KafkaAdminClientTest.java | 124 +++++- .../kafka/admin/ConsumerGroupCommand.scala | 107 +++-- .../kafka/api/BaseConsumerTest.scala | 19 +- .../admin/ConsumerGroupCommandTest.scala | 7 +- .../apache/kafka/tools/ToolsTestUtils.java | 2 + .../group/ConsumerGroupCommandTest.java | 13 +- .../consumer/group/ListConsumerGroupTest.java | 386 ++++++++++++++++-- 11 files changed, 669 insertions(+), 99 deletions(-) diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 7486ef9a80..c65cd675a9 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -46,6 +46,7 @@ + state; + private final Optional type; /** * Create an instance with the specified parameters. @@ -37,7 +39,7 @@ public class ConsumerGroupListing { * @param isSimpleConsumerGroup If consumer group is simple or not. */ public ConsumerGroupListing(String groupId, boolean isSimpleConsumerGroup) { - this(groupId, isSimpleConsumerGroup, Optional.empty()); + this(groupId, isSimpleConsumerGroup, Optional.empty(), Optional.empty()); } /** @@ -48,9 +50,27 @@ public ConsumerGroupListing(String groupId, boolean isSimpleConsumerGroup) { * @param state The state of the consumer group */ public ConsumerGroupListing(String groupId, boolean isSimpleConsumerGroup, Optional state) { + this(groupId, isSimpleConsumerGroup, state, Optional.empty()); + } + + /** + * Create an instance with the specified parameters. + * + * @param groupId Group Id. + * @param isSimpleConsumerGroup If consumer group is simple or not. + * @param state The state of the consumer group. + * @param type The type of the consumer group. + */ + public ConsumerGroupListing( + String groupId, + boolean isSimpleConsumerGroup, + Optional state, + Optional type + ) { this.groupId = groupId; this.isSimpleConsumerGroup = isSimpleConsumerGroup; this.state = Objects.requireNonNull(state); + this.type = Objects.requireNonNull(type); } /** @@ -74,42 +94,38 @@ public Optional state() { return state; } + /** + * The type of the consumer group. + * + * @return An Optional containing the type, if available. + */ + public Optional type() { + return type; + } + @Override public String toString() { return "(" + "groupId='" + groupId + '\'' + ", isSimpleConsumerGroup=" + isSimpleConsumerGroup + ", state=" + state + + ", type=" + type + ')'; } @Override public int hashCode() { - return Objects.hash(groupId, isSimpleConsumerGroup, state); + return Objects.hash(groupId, isSimpleConsumerGroup(), state, type); } @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - ConsumerGroupListing other = (ConsumerGroupListing) obj; - if (groupId == null) { - if (other.groupId != null) - return false; - } else if (!groupId.equals(other.groupId)) - return false; - if (isSimpleConsumerGroup != other.isSimpleConsumerGroup) - return false; - if (state == null) { - if (other.state != null) - return false; - } else if (!state.equals(other.state)) - return false; - return true; + public boolean equals(Object o) { + if (this == o) return true; + if (!(o instanceof ConsumerGroupListing)) return false; + ConsumerGroupListing that = (ConsumerGroupListing) o; + return isSimpleConsumerGroup() == that.isSimpleConsumerGroup() && + Objects.equals(groupId, that.groupId) && + Objects.equals(state, that.state) && + Objects.equals(type, that.type); } - } 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 85c82e2514..d98ad8ac04 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 @@ -58,6 +58,7 @@ import org.apache.kafka.clients.consumer.internals.ConsumerProtocol; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.ConsumerGroupState; +import org.apache.kafka.common.GroupType; import org.apache.kafka.common.ElectionType; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaFuture; @@ -3382,7 +3383,14 @@ ListGroupsRequest.Builder createRequest(int timeoutMs) { .stream() .map(ConsumerGroupState::toString) .collect(Collectors.toList()); - return new ListGroupsRequest.Builder(new ListGroupsRequestData().setStatesFilter(states)); + List groupTypes = options.types() + .stream() + .map(GroupType::toString) + .collect(Collectors.toList()); + return new ListGroupsRequest.Builder(new ListGroupsRequestData() + .setStatesFilter(states) + .setTypesFilter(groupTypes) + ); } private void maybeAddConsumerGroup(ListGroupsResponseData.ListedGroup group) { @@ -3392,7 +3400,15 @@ private void maybeAddConsumerGroup(ListGroupsResponseData.ListedGroup group) { final Optional state = group.groupState().equals("") ? Optional.empty() : Optional.of(ConsumerGroupState.parse(group.groupState())); - final ConsumerGroupListing groupListing = new ConsumerGroupListing(groupId, protocolType.isEmpty(), state); + final Optional type = group.groupType().equals("") + ? Optional.empty() + : Optional.of(GroupType.parse(group.groupType())); + final ConsumerGroupListing groupListing = new ConsumerGroupListing( + groupId, + protocolType.isEmpty(), + state, + type + ); results.addListing(groupListing); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupsOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupsOptions.java index 9f1f38dd4a..c240da159f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupsOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupsOptions.java @@ -22,6 +22,7 @@ import java.util.Set; import org.apache.kafka.common.ConsumerGroupState; +import org.apache.kafka.common.GroupType; import org.apache.kafka.common.annotation.InterfaceStability; /** @@ -34,20 +35,38 @@ public class ListConsumerGroupsOptions extends AbstractOptions states = Collections.emptySet(); + private Set types = Collections.emptySet(); + /** - * If states is set, only groups in these states will be returned by listConsumerGroups() + * If states is set, only groups in these states will be returned by listConsumerGroups(). * Otherwise, all groups are returned. * This operation is supported by brokers with version 2.6.0 or later. */ public ListConsumerGroupsOptions inStates(Set states) { - this.states = (states == null) ? Collections.emptySet() : new HashSet<>(states); + this.states = (states == null || states.isEmpty()) ? Collections.emptySet() : new HashSet<>(states); return this; } /** - * Returns the list of States that are requested or empty if no states have been specified + * If types is set, only groups of these types will be returned by listConsumerGroups(). + * Otherwise, all groups are returned. + */ + public ListConsumerGroupsOptions withTypes(Set types) { + this.types = (types == null || types.isEmpty()) ? Collections.emptySet() : new HashSet<>(types); + return this; + } + + /** + * Returns the list of States that are requested or empty if no states have been specified. */ public Set states() { return states; } + + /** + * Returns the list of group types that are requested or empty if no types have been specified. + */ + public Set types() { + return types; + } } 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 b8b3d54ef4..43d391a220 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 @@ -243,6 +243,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.OptionalInt; import java.util.OptionalLong; @@ -2811,6 +2812,68 @@ public void testListConsumerGroupsWithStates() throws Exception { } } + @Test + public void testListConsumerGroupsWithTypes() throws Exception { + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + + // Test with a specific state filter but no type filter in list consumer group options. + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + + env.kafkaClient().prepareResponseFrom( + expectListGroupsRequestWithFilters(singleton(ConsumerGroupState.STABLE.toString()), Collections.emptySet()), + new ListGroupsResponse(new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(Arrays.asList( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-1") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) + .setGroupState("Stable") + .setGroupType(GroupType.CLASSIC.toString())))), + env.cluster().nodeById(0)); + + final ListConsumerGroupsOptions options = new ListConsumerGroupsOptions().inStates(singleton(ConsumerGroupState.STABLE)); + final ListConsumerGroupsResult result = env.adminClient().listConsumerGroups(options); + Collection listings = result.valid().get(); + + assertEquals(1, listings.size()); + List expected = new ArrayList<>(); + expected.add(new ConsumerGroupListing("group-1", false, Optional.of(ConsumerGroupState.STABLE), Optional.of(GroupType.CLASSIC))); + assertEquals(expected, listings); + assertEquals(0, result.errors().get().size()); + + // Test with list consumer group options. + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + + env.kafkaClient().prepareResponseFrom( + expectListGroupsRequestWithFilters(Collections.emptySet(), singleton(GroupType.CONSUMER.toString())), + new ListGroupsResponse(new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(Arrays.asList( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-1") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) + .setGroupState("Stable") + .setGroupType(GroupType.CONSUMER.toString()), + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-2") + .setGroupState("Empty") + .setGroupType(GroupType.CONSUMER.toString())))), + env.cluster().nodeById(0)); + + final ListConsumerGroupsOptions options2 = new ListConsumerGroupsOptions().withTypes(singleton(GroupType.CONSUMER)); + final ListConsumerGroupsResult result2 = env.adminClient().listConsumerGroups(options2); + Collection listings2 = result2.valid().get(); + + assertEquals(2, listings2.size()); + List expected2 = new ArrayList<>(); + expected2.add(new ConsumerGroupListing("group-2", true, Optional.of(ConsumerGroupState.EMPTY), Optional.of(GroupType.CONSUMER))); + expected2.add(new ConsumerGroupListing("group-1", false, Optional.of(ConsumerGroupState.STABLE), Optional.of(GroupType.CONSUMER))); + assertEquals(expected2, listings2); + assertEquals(0, result.errors().get().size()); + } + } + @Test public void testListConsumerGroupsWithStatesOlderBrokerVersion() throws Exception { ApiVersion listGroupV3 = new ApiVersion() @@ -2835,7 +2898,7 @@ public void testListConsumerGroupsWithStatesOlderBrokerVersion() throws Exceptio ListConsumerGroupsResult result = env.adminClient().listConsumerGroups(options); Collection listing = result.all().get(); assertEquals(1, listing.size()); - List expected = Collections.singletonList(new ConsumerGroupListing("group-1", false, Optional.empty())); + List expected = Collections.singletonList(new ConsumerGroupListing("group-1", false)); assertEquals(expected, listing); // But we cannot set a state filter with older broker @@ -2849,6 +2912,65 @@ public void testListConsumerGroupsWithStatesOlderBrokerVersion() throws Exceptio } } + @Test + public void testListConsumerGroupsWithTypesOlderBrokerVersion() throws Exception { + ApiVersion listGroupV4 = new ApiVersion() + .setApiKey(ApiKeys.LIST_GROUPS.id) + .setMinVersion((short) 0) + .setMaxVersion((short) 4); + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create(Collections.singletonList(listGroupV4))); + + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + + // Check if we can list groups with older broker if we specify states and don't specify types. + env.kafkaClient().prepareResponseFrom( + expectListGroupsRequestWithFilters(singleton(ConsumerGroupState.STABLE.toString()), Collections.emptySet()), + new ListGroupsResponse(new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(Collections.singletonList( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-1") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) + .setGroupState(ConsumerGroupState.STABLE.toString())))), + env.cluster().nodeById(0)); + + ListConsumerGroupsOptions options = new ListConsumerGroupsOptions().inStates(singleton(ConsumerGroupState.STABLE)); + ListConsumerGroupsResult result = env.adminClient().listConsumerGroups(options); + + Collection listing = result.all().get(); + assertEquals(1, listing.size()); + List expected = Collections.singletonList( + new ConsumerGroupListing("group-1", false, Optional.of(ConsumerGroupState.STABLE)) + ); + assertEquals(expected, listing); + + // Check that we cannot set a type filter with an older broker. + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + env.kafkaClient().prepareUnsupportedVersionResponse(request -> + request instanceof ListGroupsRequest && !((ListGroupsRequest) request).data().typesFilter().isEmpty() + ); + + options = new ListConsumerGroupsOptions().withTypes(singleton(GroupType.CLASSIC)); + result = env.adminClient().listConsumerGroups(options); + TestUtils.assertFutureThrows(result.all(), UnsupportedVersionException.class); + } + } + + private MockClient.RequestMatcher expectListGroupsRequestWithFilters( + Set expectedStates, + Set expectedTypes + ) { + return body -> { + if (body instanceof ListGroupsRequest) { + ListGroupsRequest request = (ListGroupsRequest) body; + return Objects.equals(new HashSet<>(request.data().statesFilter()), expectedStates) + && Objects.equals(new HashSet<>(request.data().typesFilter()), expectedTypes); + } + return false; + }; + } + @Test public void testOffsetCommitNumRetries() throws Exception { final Cluster cluster = mockCluster(3, 0); diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala index 4187274a22..160b9a70aa 100755 --- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala +++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala @@ -29,7 +29,7 @@ import org.apache.kafka.clients.admin._ import org.apache.kafka.clients.consumer.OffsetAndMetadata import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.common.utils.Utils -import org.apache.kafka.common.{KafkaException, Node, TopicPartition} +import org.apache.kafka.common.{ConsumerGroupState, GroupType, KafkaException, Node, TopicPartition} import org.apache.kafka.server.util.{CommandDefaultOptions, CommandLineUtils} import scala.jdk.CollectionConverters._ @@ -41,7 +41,6 @@ import org.apache.kafka.common.protocol.Errors import scala.collection.immutable.TreeMap import scala.reflect.ClassTag -import org.apache.kafka.common.ConsumerGroupState import org.apache.kafka.common.requests.ListOffsetsResponse object ConsumerGroupCommand extends Logging { @@ -104,6 +103,15 @@ object ConsumerGroupCommand extends Logging { parsedStates } + def consumerGroupTypesFromString(input: String): Set[GroupType] = { + val parsedTypes = input.toLowerCase.split(',').map(s => GroupType.parse(s.trim)).toSet + if (parsedTypes.contains(GroupType.UNKNOWN)) { + val validTypes = GroupType.values().filter(_ != GroupType.UNKNOWN) + throw new IllegalArgumentException(s"Invalid types list '$input'. Valid types are: ${validTypes.mkString(", ")}") + } + parsedTypes + } + val MISSING_COLUMN_VALUE = "-" private def printError(msg: String, e: Option[Throwable] = None): Unit = { @@ -135,7 +143,7 @@ object ConsumerGroupCommand extends Logging { private[admin] case class MemberAssignmentState(group: String, consumerId: String, host: String, clientId: String, groupInstanceId: String, numPartitions: Int, assignment: List[TopicPartition]) - case class GroupState(group: String, coordinator: Node, assignmentStrategy: String, state: String, numMembers: Int) + private[admin] case class GroupState(group: String, coordinator: Node, assignmentStrategy: String, state: String, numMembers: Int) private[admin] sealed trait CsvRecord private[admin] case class CsvRecordWithGroup(group: String, topic: String, partition: Int, offset: Long) extends CsvRecord @@ -189,16 +197,65 @@ object ConsumerGroupCommand extends Logging { } def listGroups(): Unit = { - if (opts.options.has(opts.stateOpt)) { - val stateValue = opts.options.valueOf(opts.stateOpt) - val states = if (stateValue == null || stateValue.isEmpty) - Set[ConsumerGroupState]() - else - consumerGroupStatesFromString(stateValue) - val listings = listConsumerGroupsWithState(states) - printGroupStates(listings.map(e => (e.groupId, e.state.get.toString))) - } else + val includeType = opts.options.has(opts.typeOpt) + val includeState = opts.options.has(opts.stateOpt) + + if (includeType || includeState) { + val types = typeValues() + val states = stateValues() + val listings = listConsumerGroupsWithFilters(types, states) + + printGroupInfo(listings, includeType, includeState) + + } else { listConsumerGroups().foreach(println(_)) + } + } + + private def stateValues(): Set[ConsumerGroupState] = { + val stateValue = opts.options.valueOf(opts.stateOpt) + if (stateValue == null || stateValue.isEmpty) + Set[ConsumerGroupState]() + else + consumerGroupStatesFromString(stateValue) + } + + private def typeValues(): Set[GroupType] = { + val typeValue = opts.options.valueOf(opts.typeOpt) + if (typeValue == null || typeValue.isEmpty) + Set[GroupType]() + else + consumerGroupTypesFromString(typeValue) + } + + private def printGroupInfo(groups: List[ConsumerGroupListing], includeType: Boolean, includeState: Boolean): Unit = { + def groupId(groupListing: ConsumerGroupListing): String = groupListing.groupId + def groupType(groupListing: ConsumerGroupListing): String = groupListing.`type`().orElse(GroupType.UNKNOWN).toString + def groupState(groupListing: ConsumerGroupListing): String = groupListing.state.orElse(ConsumerGroupState.UNKNOWN).toString + + val maxGroupLen = groups.foldLeft(15)((maxLen, groupListing) => Math.max(maxLen, groupId(groupListing).length)) + 10 + var format = s"%-${maxGroupLen}s" + var header = List("GROUP") + var extractors: List[ConsumerGroupListing => String] = List(groupId) + + if (includeType) { + header = header :+ "TYPE" + extractors = extractors :+ groupType _ + format += " %-20s" + } + + if (includeState) { + header = header :+ "STATE" + extractors = extractors :+ groupState _ + format += " %-20s" + } + + println(format.format(header: _*)) + + groups.foreach { groupListing => + val info = extractors.map(extractor => extractor(groupListing)) + println(format.format(info: _*)) + } } def listConsumerGroups(): List[String] = { @@ -207,26 +264,15 @@ object ConsumerGroupCommand extends Logging { listings.map(_.groupId).toList } - def listConsumerGroupsWithState(states: Set[ConsumerGroupState]): List[ConsumerGroupListing] = { + def listConsumerGroupsWithFilters(types: Set[GroupType], states: Set[ConsumerGroupState]): List[ConsumerGroupListing] = { val listConsumerGroupsOptions = withTimeoutMs(new ListConsumerGroupsOptions()) - listConsumerGroupsOptions.inStates(states.asJava) + listConsumerGroupsOptions + .inStates(states.asJava) + .withTypes(types.asJava) val result = adminClient.listConsumerGroups(listConsumerGroupsOptions) result.all.get.asScala.toList } - private def printGroupStates(groupsAndStates: List[(String, String)]): Unit = { - // find proper columns width - var maxGroupLen = 15 - for ((groupId, _) <- groupsAndStates) { - maxGroupLen = Math.max(maxGroupLen, groupId.length) - } - val format = s"%${-maxGroupLen}s %s" - println(format.format("GROUP", "STATE")) - for ((groupId, state) <- groupsAndStates) { - println(format.format(groupId, state)) - } - } - private def shouldPrintMemberState(group: String, state: Option[String], numRows: Option[Int]): Boolean = { // numRows contains the number of data rows, if any, compiled from the API call in the caller method. // if it's undefined or 0, there is no relevant group information to display. @@ -1024,6 +1070,9 @@ object ConsumerGroupCommand extends Logging { "When specified with '--list', it displays the state of all groups. It can also be used to list groups with specific states." + nl + "Example: --bootstrap-server localhost:9092 --list --state stable,empty" + nl + "This option may be used with '--describe', '--list' and '--bootstrap-server' options only." + private val TypeDoc = "When specified with '--list', it displays the types of all the groups. It can also be used to list groups with specific types." + nl + + "Example: --bootstrap-server localhost:9092 --list --type classic,consumer" + nl + + "This option may be used with the '--list' option only." private val DeleteOffsetsDoc = "Delete offsets of consumer group. Supports one consumer group at the time, and multiple topics." val bootstrapServerOpt: OptionSpec[String] = parser.accepts("bootstrap-server", BootstrapServerDoc) @@ -1090,6 +1139,10 @@ object ConsumerGroupCommand extends Logging { .availableIf(describeOpt, listOpt) .withOptionalArg() .ofType(classOf[String]) + val typeOpt: OptionSpec[String] = parser.accepts("type", TypeDoc) + .availableIf(listOpt) + .withOptionalArg() + .ofType(classOf[String]) options = parser.parse(args : _*) diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index bb3259baf9..2015983094 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -26,6 +26,7 @@ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{Arguments, MethodSource} +import java.util import java.util.Properties import java.util.concurrent.atomic.AtomicInteger import scala.jdk.CollectionConverters._ @@ -117,11 +118,12 @@ object BaseConsumerTest { // * KRaft with the new group coordinator enabled and the classic group protocol // * KRaft with the new group coordinator enabled and the consumer group protocol def getTestQuorumAndGroupProtocolParametersAll() : java.util.stream.Stream[Arguments] = { - java.util.stream.Stream.of( + util.Arrays.stream(Array( Arguments.of("zk", "classic"), Arguments.of("kraft", "classic"), Arguments.of("kraft+kip848", "classic"), - Arguments.of("kraft+kip848", "consumer")) + Arguments.of("kraft+kip848", "consumer") + )) } // In Scala 2.12, it is necessary to disambiguate the java.util.stream.Stream.of() method call @@ -138,10 +140,19 @@ object BaseConsumerTest { // * KRaft and the classic group protocol // * KRaft with the new group coordinator enabled and the classic group protocol def getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly() : java.util.stream.Stream[Arguments] = { - java.util.stream.Stream.of( + util.Arrays.stream(Array( Arguments.of("zk", "classic"), Arguments.of("kraft", "classic"), - Arguments.of("kraft+kip848", "classic")) + Arguments.of("kraft+kip848", "classic") + )) + } + + // For tests that only work with the consumer group protocol, we want to test the following combination: + // * KRaft with the new group coordinator enabled and the consumer group protocol + def getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly(): java.util.stream.Stream[Arguments] = { + util.Arrays.stream(Array( + Arguments.of("kraft+kip848", "consumer") + )) } val updateProducerCount = new AtomicInteger() diff --git a/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala index 18c7a0a8f8..f682df1f1d 100644 --- a/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala @@ -19,7 +19,7 @@ package kafka.admin import java.time.Duration import java.util.concurrent.{ExecutorService, Executors, TimeUnit} -import java.util.{Collections, Properties} +import java.util.{Collections, Properties, stream} import kafka.admin.ConsumerGroupCommand.{ConsumerGroupCommandOptions, ConsumerGroupService} import kafka.api.BaseConsumerTest import kafka.integration.KafkaServerTestHarness @@ -31,6 +31,7 @@ import org.apache.kafka.common.{PartitionInfo, TopicPartition} import org.apache.kafka.common.errors.WakeupException import org.apache.kafka.common.serialization.StringDeserializer import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} +import org.junit.jupiter.params.provider.Arguments import scala.jdk.CollectionConverters._ import scala.collection.mutable.ArrayBuffer @@ -122,7 +123,9 @@ class ConsumerGroupCommandTest extends KafkaServerTestHarness { } object ConsumerGroupCommandTest { - def getTestQuorumAndGroupProtocolParametersAll() = BaseConsumerTest.getTestQuorumAndGroupProtocolParametersAll() + def getTestQuorumAndGroupProtocolParametersAll(): stream.Stream[Arguments] = BaseConsumerTest.getTestQuorumAndGroupProtocolParametersAll() + def getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly(): stream.Stream[Arguments] = BaseConsumerTest.getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly() + def getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly(): stream.Stream[Arguments] = BaseConsumerTest.getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly() abstract class AbstractConsumerRunnable(broker: String, groupId: String, customPropsOpt: Option[Properties] = None, syncCommit: Boolean = false) extends Runnable { diff --git a/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java b/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java index fdc732ea29..83fa31bf5e 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java +++ b/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java @@ -45,6 +45,8 @@ public class ToolsTestUtils { /** @see TestInfoUtils#TestWithParameterizedQuorumName() */ public static final String TEST_WITH_PARAMETERIZED_QUORUM_NAME = "{displayName}.{argumentsWithNames}"; + /** @see TestInfoUtils#TestWithParameterizedQuorumAndGroupProtocolNames() */ + public static final String TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES = "{displayName}.quorum={0}.groupProtocol={1}"; private static int randomPort = 0; diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java index b78054cb4a..bde3af37a1 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java @@ -58,6 +58,7 @@ public class ConsumerGroupCommandTest extends kafka.integration.KafkaServerTestHarness { public static final String TOPIC = "foo"; public static final String GROUP = "test.group"; + public static final String PROTOCOL_GROUP = "protocol-group"; List consumerGroupService = new ArrayList<>(); List consumerGroupExecutors = new ArrayList<>(); @@ -154,8 +155,8 @@ ConsumerGroupExecutor addConsumerGroupExecutor(int numConsumers, String groupPro return addConsumerGroupExecutor(numConsumers, TOPIC, GROUP, RangeAssignor.class.getName(), remoteAssignor, Optional.empty(), false, groupProtocol); } - ConsumerGroupExecutor addConsumerGroupExecutor(int numConsumers, String topic, String group) { - return addConsumerGroupExecutor(numConsumers, topic, group, RangeAssignor.class.getName(), Optional.empty(), Optional.empty(), false, GroupProtocol.CLASSIC.name); + ConsumerGroupExecutor addConsumerGroupExecutor(int numConsumers, String group, String groupProtocol) { + return addConsumerGroupExecutor(numConsumers, TOPIC, group, RangeAssignor.class.getName(), Optional.empty(), Optional.empty(), false, groupProtocol); } ConsumerGroupExecutor addConsumerGroupExecutor(int numConsumers, String topic, String group, String groupProtocol) { @@ -342,6 +343,14 @@ public static Stream getTestQuorumAndGroupProtocolParametersAll() { return BaseConsumerTest.getTestQuorumAndGroupProtocolParametersAll(); } + public static Stream getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly() { + return BaseConsumerTest.getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly(); + } + + public static Stream getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly() { + return BaseConsumerTest.getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly(); + } + @SuppressWarnings({"deprecation"}) static Seq seq(Collection seq) { return JavaConverters.asScalaIteratorConverter(seq.iterator()).asScala().toSeq(); diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java index 894f00df5e..ba5ebd254f 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java @@ -20,83 +20,258 @@ import kafka.admin.ConsumerGroupCommand; import org.apache.kafka.clients.admin.ConsumerGroupListing; import org.apache.kafka.common.ConsumerGroupState; +import org.apache.kafka.common.GroupType; import org.apache.kafka.test.TestUtils; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; +import org.junit.jupiter.params.provider.MethodSource; import java.util.Arrays; import java.util.Collections; +import java.util.EnumSet; import java.util.HashSet; -import java.util.List; import java.util.Objects; import java.util.Optional; +import java.util.Properties; +import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; +import static org.apache.kafka.tools.ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES; import static org.apache.kafka.common.utils.Utils.mkSet; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; public class ListConsumerGroupTest extends ConsumerGroupCommandTest { - @ParameterizedTest - @ValueSource(strings = {"zk", "kraft"}) - public void testListConsumerGroups(String quorum) throws Exception { + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource("getTestQuorumAndGroupProtocolParametersAll") + public void testListConsumerGroupsWithoutFilters(String quorum, String groupProtocol) throws Exception { String simpleGroup = "simple-group"; + + createOffsetsTopic(listenerName(), new Properties()); + addSimpleGroupExecutor(simpleGroup); addConsumerGroupExecutor(1); + addConsumerGroupExecutor(1, PROTOCOL_GROUP, groupProtocol); String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--list"}; ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); - scala.collection.Set expectedGroups = set(Arrays.asList(GROUP, simpleGroup)); + + scala.collection.Set expectedGroups = set(Arrays.asList(GROUP, simpleGroup, PROTOCOL_GROUP)); final AtomicReference foundGroups = new AtomicReference<>(); + TestUtils.waitForCondition(() -> { foundGroups.set(service.listConsumerGroups().toSet()); return Objects.equals(expectedGroups, foundGroups.get()); }, "Expected --list to show groups " + expectedGroups + ", but found " + foundGroups.get() + "."); } - @ParameterizedTest - @ValueSource(strings = {"zk", "kraft"}) + @Test public void testListWithUnrecognizedNewConsumerOption() { String[] cgcArgs = new String[]{"--new-consumer", "--bootstrap-server", bootstrapServers(listenerName()), "--list"}; assertThrows(OptionException.class, () -> getConsumerGroupService(cgcArgs)); } - @ParameterizedTest - @ValueSource(strings = {"zk", "kraft"}) - public void testListConsumerGroupsWithStates() throws Exception { + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource("getTestQuorumAndGroupProtocolParametersAll") + public void testListConsumerGroupsWithStates(String quorum, String groupProtocol) throws Exception { String simpleGroup = "simple-group"; + + createOffsetsTopic(listenerName(), new Properties()); + addSimpleGroupExecutor(simpleGroup); - addConsumerGroupExecutor(1); + addConsumerGroupExecutor(1, groupProtocol); String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--list", "--state"}; ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); - scala.collection.Set expectedListing = set(Arrays.asList( - new ConsumerGroupListing(simpleGroup, true, Optional.of(ConsumerGroupState.EMPTY)), - new ConsumerGroupListing(GROUP, false, Optional.of(ConsumerGroupState.STABLE)))); + Set expectedListing = mkSet( + new ConsumerGroupListing( + simpleGroup, + true, + Optional.of(ConsumerGroupState.EMPTY), + Optional.of(GroupType.CLASSIC) + ), + new ConsumerGroupListing( + GROUP, + false, + Optional.of(ConsumerGroupState.STABLE), + Optional.of(GroupType.parse(groupProtocol)) + ) + ); - final AtomicReference foundListing = new AtomicReference<>(); - TestUtils.waitForCondition(() -> { - foundListing.set(service.listConsumerGroupsWithState(set(Arrays.asList(ConsumerGroupState.values()))).toSet()); - return Objects.equals(expectedListing, foundListing.get()); - }, "Expected to show groups " + expectedListing + ", but found " + foundListing.get()); + assertGroupListing( + service, + Collections.emptySet(), + EnumSet.allOf(ConsumerGroupState.class), + expectedListing + ); - scala.collection.Set expectedListingStable = set(Collections.singleton( - new ConsumerGroupListing(GROUP, false, Optional.of(ConsumerGroupState.STABLE)))); + expectedListing = mkSet( + new ConsumerGroupListing( + GROUP, + false, + Optional.of(ConsumerGroupState.STABLE), + Optional.of(GroupType.parse(groupProtocol)) + ) + ); - foundListing.set(null); + assertGroupListing( + service, + Collections.emptySet(), + mkSet(ConsumerGroupState.STABLE), + expectedListing + ); - TestUtils.waitForCondition(() -> { - foundListing.set(service.listConsumerGroupsWithState(set(Collections.singleton(ConsumerGroupState.STABLE))).toSet()); - return Objects.equals(expectedListingStable, foundListing.get()); - }, "Expected to show groups " + expectedListingStable + ", but found " + foundListing.get()); + assertGroupListing( + service, + Collections.emptySet(), + mkSet(ConsumerGroupState.PREPARING_REBALANCE), + Collections.emptySet() + ); } - @ParameterizedTest - @ValueSource(strings = {"zk", "kraft"}) - public void testConsumerGroupStatesFromString(String quorum) { + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly") + public void testListConsumerGroupsWithTypesClassicProtocol(String quorum, String groupProtocol) throws Exception { + String simpleGroup = "simple-group"; + + createOffsetsTopic(listenerName(), new Properties()); + + addSimpleGroupExecutor(simpleGroup); + addConsumerGroupExecutor(1); + + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--list"}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + Set expectedListing = mkSet( + new ConsumerGroupListing( + simpleGroup, + true, + Optional.of(ConsumerGroupState.EMPTY), + Optional.of(GroupType.CLASSIC) + ), + new ConsumerGroupListing( + GROUP, + false, + Optional.of(ConsumerGroupState.STABLE), + Optional.of(GroupType.CLASSIC) + ) + ); + + // No filters explicitly mentioned. Expectation is that all groups are returned. + assertGroupListing( + service, + Collections.emptySet(), + Collections.emptySet(), + expectedListing + ); + + // When group type is mentioned: + // Old Group Coordinator returns empty listings if the type is not Classic. + // New Group Coordinator returns groups according to the filter. + assertGroupListing( + service, + mkSet(GroupType.CONSUMER), + Collections.emptySet(), + Collections.emptySet() + ); + + assertGroupListing( + service, + mkSet(GroupType.CLASSIC), + Collections.emptySet(), + expectedListing + ); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource("getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly") + public void testListConsumerGroupsWithTypesConsumerProtocol(String quorum, String groupProtocol) throws Exception { + String simpleGroup = "simple-group"; + + createOffsetsTopic(listenerName(), new Properties()); + + addSimpleGroupExecutor(simpleGroup); + addConsumerGroupExecutor(1); + addConsumerGroupExecutor(1, PROTOCOL_GROUP, groupProtocol); + + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--list"}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + // No filters explicitly mentioned. Expectation is that all groups are returned. + Set expectedListing = mkSet( + new ConsumerGroupListing( + simpleGroup, + true, + Optional.of(ConsumerGroupState.EMPTY), + Optional.of(GroupType.CLASSIC) + ), + new ConsumerGroupListing( + GROUP, + false, + Optional.of(ConsumerGroupState.STABLE), + Optional.of(GroupType.CLASSIC) + ), + new ConsumerGroupListing( + PROTOCOL_GROUP, + false, + Optional.of(ConsumerGroupState.STABLE), + Optional.of(GroupType.CONSUMER) + ) + ); + + assertGroupListing( + service, + Collections.emptySet(), + Collections.emptySet(), + expectedListing + ); + + // When group type is mentioned: + // New Group Coordinator returns groups according to the filter. + expectedListing = mkSet( + new ConsumerGroupListing( + PROTOCOL_GROUP, + false, + Optional.of(ConsumerGroupState.STABLE), + Optional.of(GroupType.CONSUMER) + ) + ); + + assertGroupListing( + service, + mkSet(GroupType.CONSUMER), + Collections.emptySet(), + expectedListing + ); + + expectedListing = mkSet( + new ConsumerGroupListing( + simpleGroup, + true, + Optional.of(ConsumerGroupState.EMPTY), + Optional.of(GroupType.CLASSIC) + ), + new ConsumerGroupListing( + GROUP, + false, + Optional.of(ConsumerGroupState.STABLE), + Optional.of(GroupType.CLASSIC) + ) + ); + + assertGroupListing( + service, + mkSet(GroupType.CLASSIC), + Collections.emptySet(), + expectedListing + ); + } + + @Test + public void testConsumerGroupStatesFromString() { scala.collection.Set result = ConsumerGroupCommand.consumerGroupStatesFromString("Stable"); assertEquals(set(Collections.singleton(ConsumerGroupState.STABLE)), result); @@ -107,7 +282,7 @@ public void testConsumerGroupStatesFromString(String quorum) { assertEquals(set(Arrays.asList(ConsumerGroupState.DEAD, ConsumerGroupState.COMPLETING_REBALANCE)), result); result = ConsumerGroupCommand.consumerGroupStatesFromString("stable"); - assertEquals(set(Arrays.asList(ConsumerGroupState.STABLE)), result); + assertEquals(set(Collections.singletonList(ConsumerGroupState.STABLE)), result); result = ConsumerGroupCommand.consumerGroupStatesFromString("stable, assigning"); assertEquals(set(Arrays.asList(ConsumerGroupState.STABLE, ConsumerGroupState.ASSIGNING)), result); @@ -122,10 +297,31 @@ public void testConsumerGroupStatesFromString(String quorum) { assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupStatesFromString(" , ,")); } - @ParameterizedTest - @ValueSource(strings = {"zk", "kraft"}) - public void testListGroupCommand(String quorum) throws Exception { + @Test + public void testConsumerGroupTypesFromString() { + scala.collection.Set result = ConsumerGroupCommand.consumerGroupTypesFromString("consumer"); + assertEquals(set(Collections.singleton(GroupType.CONSUMER)), result); + + result = ConsumerGroupCommand.consumerGroupTypesFromString("consumer, classic"); + assertEquals(set(Arrays.asList(GroupType.CONSUMER, GroupType.CLASSIC)), result); + + result = ConsumerGroupCommand.consumerGroupTypesFromString("Consumer, Classic"); + assertEquals(set(Arrays.asList(GroupType.CONSUMER, GroupType.CLASSIC)), result); + + assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupTypesFromString("bad, wrong")); + + assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupTypesFromString(" bad, generic")); + + assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupTypesFromString(" , ,")); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly") + public void testListGroupCommandClassicProtocol(String quorum, String groupProtocol) throws Exception { String simpleGroup = "simple-group"; + + createOffsetsTopic(listenerName(), new Properties()); + addSimpleGroupExecutor(simpleGroup); addConsumerGroupExecutor(1); @@ -147,6 +343,24 @@ public void testListGroupCommand(String quorum) throws Exception { ) ); + validateListOutput( + Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type"), + Arrays.asList("GROUP", "TYPE"), + mkSet( + Arrays.asList(GROUP, "Classic"), + Arrays.asList(simpleGroup, "Classic") + ) + ); + + validateListOutput( + Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type", "--state"), + Arrays.asList("GROUP", "TYPE", "STATE"), + mkSet( + Arrays.asList(GROUP, "Classic", "Stable"), + Arrays.asList(simpleGroup, "Classic", "Empty") + ) + ); + validateListOutput( Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--state", "Stable"), Arrays.asList("GROUP", "STATE"), @@ -155,6 +369,7 @@ public void testListGroupCommand(String quorum) throws Exception { ) ); + // Check case-insensitivity in state filter. validateListOutput( Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--state", "stable"), Arrays.asList("GROUP", "STATE"), @@ -162,6 +377,109 @@ public void testListGroupCommand(String quorum) throws Exception { Arrays.asList(GROUP, "Stable") ) ); + + validateListOutput( + Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type", "Classic"), + Arrays.asList("GROUP", "TYPE"), + mkSet( + Arrays.asList(GROUP, "Classic"), + Arrays.asList(simpleGroup, "Classic") + ) + ); + + // Check case-insensitivity in type filter. + validateListOutput( + Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type", "classic"), + Arrays.asList("GROUP", "TYPE"), + mkSet( + Arrays.asList(GROUP, "Classic"), + Arrays.asList(simpleGroup, "Classic") + ) + ); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource("getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly") + public void testListGroupCommandConsumerProtocol(String quorum, String groupProtocol) throws Exception { + String simpleGroup = "simple-group"; + + createOffsetsTopic(listenerName(), new Properties()); + + addSimpleGroupExecutor(simpleGroup); + addConsumerGroupExecutor(1, PROTOCOL_GROUP, groupProtocol); + + validateListOutput( + Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list"), + Collections.emptyList(), + mkSet( + Collections.singletonList(PROTOCOL_GROUP), + Collections.singletonList(simpleGroup) + ) + ); + + validateListOutput( + Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--state"), + Arrays.asList("GROUP", "STATE"), + mkSet( + Arrays.asList(PROTOCOL_GROUP, "Stable"), + Arrays.asList(simpleGroup, "Empty") + ) + ); + + validateListOutput( + Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type"), + Arrays.asList("GROUP", "TYPE"), + mkSet( + Arrays.asList(PROTOCOL_GROUP, "Consumer"), + Arrays.asList(simpleGroup, "Classic") + ) + ); + + validateListOutput( + Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type", "--state"), + Arrays.asList("GROUP", "TYPE", "STATE"), + mkSet( + Arrays.asList(PROTOCOL_GROUP, "Consumer", "Stable"), + Arrays.asList(simpleGroup, "Classic", "Empty") + ) + ); + + validateListOutput( + Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type", "consumer"), + Arrays.asList("GROUP", "TYPE"), + mkSet( + Arrays.asList(PROTOCOL_GROUP, "Consumer") + ) + ); + + validateListOutput( + Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type", "consumer", "--state", "Stable"), + Arrays.asList("GROUP", "TYPE", "STATE"), + mkSet( + Arrays.asList(PROTOCOL_GROUP, "Consumer", "Stable") + ) + ); + } + + /** + * Validates the consumer group listings returned against expected values using specified filters. + * + * @param service The service to list consumer groups. + * @param typeFilterSet Filters for group types, empty for no filter. + * @param stateFilterSet Filters for group states, empty for no filter. + * @param expectedListing Expected consumer group listings. + */ + private static void assertGroupListing( + ConsumerGroupCommand.ConsumerGroupService service, + Set typeFilterSet, + Set stateFilterSet, + Set expectedListing + ) throws Exception { + final AtomicReference foundListing = new AtomicReference<>(); + TestUtils.waitForCondition(() -> { + foundListing.set(service.listConsumerGroupsWithFilters(set(typeFilterSet), set(stateFilterSet)).toSet()); + return Objects.equals(set(expectedListing), foundListing.get()); + }, () -> "Expected to show groups " + expectedListing + ", but found " + foundListing.get() + "."); } /** From f8eb4294d67b37e854aa14fb989d5e074df82ac2 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 29 Feb 2024 02:22:23 -0800 Subject: [PATCH 007/521] KAFKA-16191: Clean up of consumer client internal events (#15438) There are a few minor issues with the event sub-classes in the org.apache.kafka.clients.consumer.internals.events package that should be cleaned up: - Update the names of subclasses to remove "Application" or "Background" - Make toString() final in the base classes and clean up the implementations of toStringBase() - Fix minor whitespace inconsistencies - Make variable/method names consistent Reviewer: Bruno Cadonna --- .../internals/AsyncKafkaConsumer.java | 89 ++++++------ .../internals/CoordinatorRequestManager.java | 6 +- .../internals/HeartbeatRequestManager.java | 4 +- .../internals/MembershipManagerImpl.java | 4 +- .../internals/OffsetsRequestManager.java | 4 +- .../events/AbstractTopicMetadataEvent.java | 41 ++++++ .../events/AllTopicsMetadataEvent.java | 24 ++++ .../internals/events/ApplicationEvent.java | 35 +++-- .../events/ApplicationEventHandler.java | 6 +- .../events/ApplicationEventProcessor.java | 123 +++++++++------- ...nEvent.java => AssignmentChangeEvent.java} | 33 +---- ...cationEvent.java => AsyncCommitEvent.java} | 15 +- .../internals/events/BackgroundEvent.java | 30 ++-- .../events/BackgroundEventHandler.java | 2 +- ...ApplicationEvent.java => CommitEvent.java} | 21 +-- ...tionEvent.java => CommitOnCloseEvent.java} | 11 +- .../events/CompletableApplicationEvent.java | 45 +----- .../events/CompletableBackgroundEvent.java | 28 +--- .../internals/events/CompletableEvent.java | 1 - ...balanceListenerCallbackCompletedEvent.java | 31 +--- ...rRebalanceListenerCallbackNeededEvent.java | 30 +--- .../events/ErrorBackgroundEvent.java | 59 -------- .../consumer/internals/events/ErrorEvent.java | 38 +++++ ...t.java => FetchCommittedOffsetsEvent.java} | 31 +--- .../events/GroupMetadataUpdateEvent.java | 32 +---- ...ationEvent.java => LeaveOnCloseEvent.java} | 12 +- ...cationEvent.java => ListOffsetsEvent.java} | 33 +---- .../NewTopicsMetadataUpdateRequestEvent.java | 7 - .../events/PollApplicationEvent.java | 57 -------- .../consumer/internals/events/PollEvent.java | 36 +++++ ...ionEvent.java => ResetPositionsEvent.java} | 4 +- ...vent.java => SubscriptionChangeEvent.java} | 4 +- ...icationEvent.java => SyncCommitEvent.java} | 17 +-- .../events/TopicMetadataApplicationEvent.java | 78 ---------- .../internals/events/TopicMetadataEvent.java | 38 +++++ ...cationEvent.java => UnsubscribeEvent.java} | 5 +- ...Event.java => ValidatePositionsEvent.java} | 4 +- .../internals/AsyncKafkaConsumerTest.java | 134 +++++++++--------- .../internals/ConsumerNetworkThreadTest.java | 54 +++---- .../CoordinatorRequestManagerTest.java | 6 +- .../HeartbeatRequestManagerTest.java | 22 +-- .../internals/MembershipManagerImplTest.java | 8 +- .../internals/OffsetsRequestManagerTest.java | 6 +- .../events/ApplicationEventProcessorTest.java | 4 +- 44 files changed, 519 insertions(+), 753 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AllTopicsMetadataEvent.java rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/{AssignmentChangeApplicationEvent.java => AssignmentChangeEvent.java} (56%) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/{AsyncCommitApplicationEvent.java => AsyncCommitEvent.java} (73%) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/{CommitApplicationEvent.java => CommitEvent.java} (71%) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/{CommitOnCloseApplicationEvent.java => CommitOnCloseEvent.java} (76%) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorEvent.java rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/{FetchCommittedOffsetsApplicationEvent.java => FetchCommittedOffsetsEvent.java} (60%) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/{LeaveOnCloseApplicationEvent.java => LeaveOnCloseEvent.java} (76%) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/{ListOffsetsApplicationEvent.java => ListOffsetsEvent.java} (69%) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollApplicationEvent.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollEvent.java rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/{ResetPositionsApplicationEvent.java => ResetPositionsEvent.java} (89%) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/{SubscriptionChangeApplicationEvent.java => SubscriptionChangeEvent.java} (90%) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/{SyncCommitApplicationEvent.java => SyncCommitEvent.java} (73%) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataEvent.java rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/{UnsubscribeApplicationEvent.java => UnsubscribeEvent.java} (91%) rename clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/{ValidatePositionsApplicationEvent.java => ValidatePositionsEvent.java} (89%) 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 d4b461b014..d810c5f053 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 @@ -36,32 +36,33 @@ import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.clients.consumer.internals.events.AllTopicsMetadataEvent; 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.ApplicationEventProcessor; -import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.AsyncCommitApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeEvent; +import org.apache.kafka.clients.consumer.internals.events.AsyncCommitEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; -import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.CommitOnCloseApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CommitEvent; +import org.apache.kafka.clients.consumer.internals.events.CommitOnCloseEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; -import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; +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.FetchCommittedOffsetsApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsEvent; import org.apache.kafka.clients.consumer.internals.events.GroupMetadataUpdateEvent; -import org.apache.kafka.clients.consumer.internals.events.LeaveOnCloseApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; +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.PollApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ResetPositionsApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.SubscriptionChangeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.SyncCommitApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.TopicMetadataApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.UnsubscribeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.PollEvent; +import org.apache.kafka.clients.consumer.internals.events.ResetPositionsEvent; +import org.apache.kafka.clients.consumer.internals.events.SubscriptionChangeEvent; +import org.apache.kafka.clients.consumer.internals.events.SyncCommitEvent; +import org.apache.kafka.clients.consumer.internals.events.TopicMetadataEvent; +import org.apache.kafka.clients.consumer.internals.events.UnsubscribeEvent; +import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsEvent; import org.apache.kafka.clients.consumer.internals.metrics.KafkaConsumerMetrics; import org.apache.kafka.clients.consumer.internals.metrics.RebalanceCallbackMetricsManager; import org.apache.kafka.common.Cluster; @@ -179,7 +180,7 @@ public BackgroundEventProcessor(final LogContext logContext, /** * Process the events—if any—that were produced by the {@link ConsumerNetworkThread network thread}. - * It is possible that {@link org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent an error} + * It is possible that {@link ErrorEvent an error} * could occur when processing the events. In such cases, the processor will take a reference to the first * error, continue to process the remaining events, and then throw the first error that occurred. */ @@ -209,7 +210,7 @@ public boolean process() { public void process(final BackgroundEvent event) { switch (event.type()) { case ERROR: - process((ErrorBackgroundEvent) event); + process((ErrorEvent) event); break; case GROUP_METADATA_UPDATE: @@ -226,7 +227,7 @@ public void process(final BackgroundEvent event) { } } - private void process(final ErrorBackgroundEvent event) { + private void process(final ErrorEvent event) { throw event.error(); } @@ -703,7 +704,7 @@ public ConsumerRecords poll(final Duration timeout) { do { // Make sure to let the background thread know that we are still polling. - applicationEventHandler.add(new PollApplicationEvent(timer.currentTimeMs())); + applicationEventHandler.add(new PollEvent(timer.currentTimeMs())); // We must not allow wake-ups between polling for fetches and returning the records. // If the polled fetches are not empty the consumed position has already been updated in the polling @@ -768,7 +769,7 @@ public void commitAsync(OffsetCommitCallback callback) { public void commitAsync(Map offsets, OffsetCommitCallback callback) { acquireAndEnsureOpen(); try { - AsyncCommitApplicationEvent asyncCommitEvent = new AsyncCommitApplicationEvent(offsets); + AsyncCommitEvent asyncCommitEvent = new AsyncCommitEvent(offsets); CompletableFuture future = commit(asyncCommitEvent); future.whenComplete((r, t) -> { @@ -790,7 +791,7 @@ public void commitAsync(Map offsets, OffsetCo } } - private CompletableFuture commit(final CommitApplicationEvent commitEvent) { + private CompletableFuture commit(final CommitEvent commitEvent) { maybeInvokeCommitCallbacks(); maybeThrowFencedInstanceException(); maybeThrowInvalidGroupIdException(); @@ -936,7 +937,7 @@ public Map committed(final Set partitionsFor(String topic, Duration timeout) { throw new TimeoutException(); } - final TopicMetadataApplicationEvent topicMetadataApplicationEvent = - new TopicMetadataApplicationEvent(topic, timeout.toMillis()); - wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future()); + final TopicMetadataEvent topicMetadataEvent = new TopicMetadataEvent(topic, timeout.toMillis()); + wakeupTrigger.setActiveTask(topicMetadataEvent.future()); try { Map> topicMetadata = - applicationEventHandler.addAndGet(topicMetadataApplicationEvent, time.timer(timeout)); + applicationEventHandler.addAndGet(topicMetadataEvent, time.timer(timeout)); return topicMetadata.getOrDefault(topic, Collections.emptyList()); } finally { @@ -1017,11 +1017,10 @@ public Map> listTopics(Duration timeout) { throw new TimeoutException(); } - final TopicMetadataApplicationEvent topicMetadataApplicationEvent = - new TopicMetadataApplicationEvent(timeout.toMillis()); - wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future()); + final AllTopicsMetadataEvent topicMetadataEvent = new AllTopicsMetadataEvent(timeout.toMillis()); + wakeupTrigger.setActiveTask(topicMetadataEvent.future()); try { - return applicationEventHandler.addAndGet(topicMetadataApplicationEvent, time.timer(timeout)); + return applicationEventHandler.addAndGet(topicMetadataEvent, time.timer(timeout)); } finally { wakeupTrigger.clearTask(); } @@ -1089,7 +1088,7 @@ public Map offsetsForTimes(Map beginningOrEndOffset(Collection timestampToSearch = partitions .stream() .collect(Collectors.toMap(Function.identity(), tp -> timestamp)); - ListOffsetsApplicationEvent listOffsetsEvent = new ListOffsetsApplicationEvent( + ListOffsetsEvent listOffsetsEvent = new ListOffsetsEvent( timestampToSearch, false); Map offsetAndTimestampMap = applicationEventHandler.addAndGet( @@ -1267,11 +1266,11 @@ void prepareShutdown(final Timer timer, final AtomicReference firstEx if (!groupMetadata.isPresent()) return; maybeAutoCommitSync(autoCommitEnabled, timer, firstException); - applicationEventHandler.add(new CommitOnCloseApplicationEvent()); + applicationEventHandler.add(new CommitOnCloseEvent()); completeQuietly( () -> { maybeRevokePartitions(); - applicationEventHandler.addAndGet(new LeaveOnCloseApplicationEvent(), timer); + applicationEventHandler.addAndGet(new LeaveOnCloseEvent(), timer); }, "Failed to send leaveGroup heartbeat with a timeout(ms)=" + timer.timeoutMs(), firstException); } @@ -1349,7 +1348,7 @@ public void commitSync(Map offsets, Duration long commitStart = time.nanoseconds(); try { Timer requestTimer = time.timer(timeout.toMillis()); - SyncCommitApplicationEvent syncCommitEvent = new SyncCommitApplicationEvent(offsets, timeout.toMillis()); + SyncCommitEvent syncCommitEvent = new SyncCommitEvent(offsets, timeout.toMillis()); CompletableFuture commitFuture = commit(syncCommitEvent); wakeupTrigger.setActiveTask(commitFuture); ConsumerUtils.getResult(commitFuture, requestTimer); @@ -1429,7 +1428,7 @@ public void assign(Collection partitions) { // be no following rebalance. // // See the ApplicationEventProcessor.process() method that handles this event for more detail. - applicationEventHandler.add(new AssignmentChangeApplicationEvent(subscriptions.allConsumed(), time.milliseconds())); + applicationEventHandler.add(new AssignmentChangeEvent(subscriptions.allConsumed(), time.milliseconds())); log.info("Assigned to partition(s): {}", join(partitions, ", ")); if (subscriptions.assignFromUser(new HashSet<>(partitions))) @@ -1463,13 +1462,13 @@ public void unsubscribe() { try { fetchBuffer.retainAll(Collections.emptySet()); if (groupMetadata.isPresent()) { - UnsubscribeApplicationEvent unsubscribeApplicationEvent = new UnsubscribeApplicationEvent(); - applicationEventHandler.add(unsubscribeApplicationEvent); + UnsubscribeEvent unsubscribeEvent = new UnsubscribeEvent(); + applicationEventHandler.add(unsubscribeEvent); log.info("Unsubscribing all topics or patterns and assigned partitions"); Timer timer = time.timer(Long.MAX_VALUE); try { - processBackgroundEvents(backgroundEventProcessor, unsubscribeApplicationEvent.future(), timer); + processBackgroundEvents(backgroundEventProcessor, unsubscribeEvent.future(), timer); log.info("Unsubscribed all topics or patterns and assigned partitions"); } catch (TimeoutException e) { log.error("Failed while waiting for the unsubscribe event to complete"); @@ -1567,7 +1566,7 @@ private boolean updateFetchPositions(final Timer timer) { // Validate positions using the partition leader end offsets, to detect if any partition // has been truncated due to a leader change. This will trigger an OffsetForLeaderEpoch // request, retrieve the partition end offsets, and validate the current position against it. - applicationEventHandler.addAndGet(new ValidatePositionsApplicationEvent(), timer); + applicationEventHandler.addAndGet(new ValidatePositionsEvent(), timer); cachedSubscriptionHasAllFetchPositions = subscriptions.hasAllFetchPositions(); if (cachedSubscriptionHasAllFetchPositions) return true; @@ -1590,7 +1589,7 @@ private boolean updateFetchPositions(final Timer timer) { // which are awaiting reset. This will trigger a ListOffset request, retrieve the // partition offsets according to the strategy (ex. earliest, latest), and update the // positions. - applicationEventHandler.addAndGet(new ResetPositionsApplicationEvent(), timer); + applicationEventHandler.addAndGet(new ResetPositionsEvent(), timer); return true; } catch (TimeoutException e) { return false; @@ -1620,8 +1619,8 @@ private boolean initWithCommittedOffsetsIfNeeded(Timer timer) { log.debug("Refreshing committed offsets for partitions {}", initializingPartitions); try { - final FetchCommittedOffsetsApplicationEvent event = - new FetchCommittedOffsetsApplicationEvent( + final FetchCommittedOffsetsEvent event = + new FetchCommittedOffsetsEvent( initializingPartitions, timer.remainingMs()); final Map offsets = applicationEventHandler.addAndGet(event, timer); @@ -1770,7 +1769,7 @@ private void subscribeInternal(Collection topics, Optional topics, Optional * * As an example, take {@link #unsubscribe()}. To start unsubscribing, the application thread enqueues an - * {@link UnsubscribeApplicationEvent} on the application event queue. That event will eventually trigger the + * {@link UnsubscribeEvent} on the application event queue. That event will eventually trigger the * rebalancing logic in the background thread. Critically, as part of this rebalancing work, the * {@link ConsumerRebalanceListener#onPartitionsRevoked(Collection)} callback needs to be invoked. However, * this callback must be executed on the application thread. To achieve this, the background thread enqueues a diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java index d6a72812a5..a6cc28fb0f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java @@ -17,7 +17,7 @@ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; -import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; import org.apache.kafka.common.errors.GroupAuthorizationException; @@ -175,12 +175,12 @@ private void onFailedResponse(final long currentTimeMs, final Throwable exceptio if (exception == Errors.GROUP_AUTHORIZATION_FAILED.exception()) { log.debug("FindCoordinator request failed due to authorization error {}", exception.getMessage()); KafkaException groupAuthorizationException = GroupAuthorizationException.forGroupId(this.groupId); - backgroundEventHandler.add(new ErrorBackgroundEvent(groupAuthorizationException)); + backgroundEventHandler.add(new ErrorEvent(groupAuthorizationException)); return; } log.warn("FindCoordinator request failed due to fatal exception", exception); - backgroundEventHandler.add(new ErrorBackgroundEvent(exception)); + backgroundEventHandler.add(new ErrorEvent(exception)); } /** 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 d551dbe250..826774a6a6 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 @@ -21,7 +21,7 @@ import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; -import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.clients.consumer.internals.events.GroupMetadataUpdateEvent; import org.apache.kafka.clients.consumer.internals.metrics.HeartbeatMetricsManager; import org.apache.kafka.common.Uuid; @@ -446,7 +446,7 @@ private void logInfo(final String message, } private void handleFatalFailure(Throwable error) { - backgroundEventHandler.add(new ErrorBackgroundEvent(error)); + backgroundEventHandler.add(new ErrorEvent(error)); membershipManager.transitionToFatal(); } 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 6f3947eea4..e74c7e30a2 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 @@ -26,7 +26,7 @@ import org.apache.kafka.clients.consumer.internals.events.CompletableBackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; -import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.clients.consumer.internals.metrics.RebalanceMetricsManager; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicIdPartition; @@ -255,7 +255,7 @@ public class MembershipManagerImpl implements MembershipManager { /** * Serves as the conduit by which we can report events to the application thread. This is needed as we send * {@link ConsumerRebalanceListenerCallbackNeededEvent callbacks} and, if needed, - * {@link ErrorBackgroundEvent errors} to the application thread. + * {@link ErrorEvent errors} to the application thread. */ private final BackgroundEventHandler backgroundEventHandler; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java index 34f4b30c44..c5156e9e0b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java @@ -25,7 +25,7 @@ import org.apache.kafka.clients.consumer.internals.OffsetFetcherUtils.ListOffsetData; import org.apache.kafka.clients.consumer.internals.OffsetFetcherUtils.ListOffsetResult; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; -import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.common.ClusterResource; import org.apache.kafka.common.ClusterResourceListener; import org.apache.kafka.common.IsolationLevel; @@ -199,7 +199,7 @@ public CompletableFuture resetPositionsIfNeeded() { try { offsetResetTimestamps = offsetFetcherUtils.getOffsetResetTimestamp(); } catch (Exception e) { - backgroundEventHandler.add(new ErrorBackgroundEvent(e)); + backgroundEventHandler.add(new ErrorEvent(e)); return CompletableFuture.completedFuture(null); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java new file mode 100644 index 0000000000..31c21817d8 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java @@ -0,0 +1,41 @@ +/* + * 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.events; + +import org.apache.kafka.common.PartitionInfo; + +import java.util.List; +import java.util.Map; + +public abstract class AbstractTopicMetadataEvent extends CompletableApplicationEvent>> { + + private final long timeoutMs; + + protected AbstractTopicMetadataEvent(final Type type, final long timeoutMs) { + super(type); + this.timeoutMs = timeoutMs; + } + + public long timeoutMs() { + return timeoutMs; + } + + @Override + public String toStringBase() { + return super.toStringBase() + ", timeoutMs=" + timeoutMs; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AllTopicsMetadataEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AllTopicsMetadataEvent.java new file mode 100644 index 0000000000..154703aaee --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AllTopicsMetadataEvent.java @@ -0,0 +1,24 @@ +/* + * 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.events; + +public class AllTopicsMetadataEvent extends AbstractTopicMetadataEvent { + + public AllTopicsMetadataEvent(final long timeoutMs) { + super(Type.ALL_TOPICS_METADATA, timeoutMs); + } +} 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 ac7ccc56c5..2897117da8 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 @@ -16,52 +16,65 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer; +import org.apache.kafka.common.Uuid; + import java.util.Objects; /** - * This is the abstract definition of the events created by the KafkaConsumer API + * This is the abstract definition of the events created by the {@link AsyncKafkaConsumer} on the user's + * application thread. */ public abstract class ApplicationEvent { 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, SUBSCRIPTION_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 } private final Type type; + /** + * This identifies a particular event. It is used to disambiguate events via {@link #hashCode()} and + * {@link #equals(Object)} and can be used in log messages when debugging. + */ + private final Uuid id; + protected ApplicationEvent(Type type) { this.type = Objects.requireNonNull(type); + this.id = Uuid.randomUuid(); } public Type type() { return type; } + public Uuid id() { + return id; + } + @Override - public boolean equals(Object o) { + public final boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; - ApplicationEvent that = (ApplicationEvent) o; - - return type == that.type; + return type == that.type && id.equals(that.id); } @Override - public int hashCode() { - return type.hashCode(); + public final int hashCode() { + return Objects.hash(type, id); } protected String toStringBase() { - return "type=" + type; + return "type=" + type + ", id=" + id; } @Override - public String toString() { - return "ApplicationEvent{" + + public final String toString() { + return getClass().getSimpleName() + "{" + toStringBase() + '}'; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java index 7535edf597..eac1cc3d62 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; +import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.common.internals.IdempotentCloser; @@ -99,10 +100,9 @@ public long maximumTimeToWait() { * *

* - * See {@link CompletableApplicationEvent#get(Timer)} and {@link Future#get(long, TimeUnit)} for more details. + * See {@link ConsumerUtils#getResult(Future, Timer)} and {@link Future#get(long, TimeUnit)} for more details. * * @param event A {@link CompletableApplicationEvent} created by the polling thread - * @param timer Timer for which to wait for the event to complete * @return Value that is the result of the event * @param Type of return value of the event */ @@ -110,7 +110,7 @@ public T addAndGet(final CompletableApplicationEvent event, final Timer t Objects.requireNonNull(event, "CompletableApplicationEvent provided to addAndGet must be non-null"); Objects.requireNonNull(timer, "Timer provided to addAndGet must be non-null"); add(event); - return event.get(timer); + return ConsumerUtils.getResult(event.future(), timer); } @Override 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 9e48b4de6d..c86aa8815f 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 @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.internals.CachedSupplier; import org.apache.kafka.clients.consumer.internals.CommitRequestManager; @@ -34,6 +35,7 @@ import java.util.Objects; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; +import java.util.function.BiConsumer; import java.util.function.Supplier; /** @@ -65,23 +67,24 @@ public boolean process() { return process((event, error) -> error.ifPresent(e -> log.warn("Error processing event {}", e.getMessage(), e))); } + @SuppressWarnings({"CyclomaticComplexity"}) @Override public void process(ApplicationEvent event) { switch (event.type()) { case COMMIT_ASYNC: - process((AsyncCommitApplicationEvent) event); + process((AsyncCommitEvent) event); return; case COMMIT_SYNC: - process((SyncCommitApplicationEvent) event); + process((SyncCommitEvent) event); return; case POLL: - process((PollApplicationEvent) event); + process((PollEvent) event); return; case FETCH_COMMITTED_OFFSETS: - process((FetchCommittedOffsetsApplicationEvent) event); + process((FetchCommittedOffsetsEvent) event); return; case NEW_TOPICS_METADATA_UPDATE: @@ -89,31 +92,35 @@ public void process(ApplicationEvent event) { return; case ASSIGNMENT_CHANGE: - process((AssignmentChangeApplicationEvent) event); + process((AssignmentChangeEvent) event); return; case TOPIC_METADATA: - process((TopicMetadataApplicationEvent) event); + process((TopicMetadataEvent) event); + return; + + case ALL_TOPICS_METADATA: + process((AllTopicsMetadataEvent) event); return; case LIST_OFFSETS: - process((ListOffsetsApplicationEvent) event); + process((ListOffsetsEvent) event); return; case RESET_POSITIONS: - process((ResetPositionsApplicationEvent) event); + process((ResetPositionsEvent) event); return; case VALIDATE_POSITIONS: - process((ValidatePositionsApplicationEvent) event); + process((ValidatePositionsEvent) event); return; case SUBSCRIPTION_CHANGE: - process((SubscriptionChangeApplicationEvent) event); + process((SubscriptionChangeEvent) event); return; case UNSUBSCRIBE: - process((UnsubscribeApplicationEvent) event); + process((UnsubscribeEvent) event); return; case CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED: @@ -121,11 +128,11 @@ public void process(ApplicationEvent event) { return; case COMMIT_ON_CLOSE: - process((CommitOnCloseApplicationEvent) event); + process((CommitOnCloseEvent) event); return; case LEAVE_ON_CLOSE: - process((LeaveOnCloseApplicationEvent) event); + process((LeaveOnCloseEvent) event); return; default: @@ -133,7 +140,7 @@ public void process(ApplicationEvent event) { } } - private void process(final PollApplicationEvent event) { + private void process(final PollEvent event) { if (!requestManagers.commitRequestManager.isPresent()) { return; } @@ -142,20 +149,28 @@ private void process(final PollApplicationEvent event) { requestManagers.heartbeatRequestManager.ifPresent(hrm -> hrm.resetPollTimer(event.pollTimeMs())); } - private void process(final AsyncCommitApplicationEvent event) { + private void process(final AsyncCommitEvent event) { + if (!requestManagers.commitRequestManager.isPresent()) { + return; + } + CommitRequestManager manager = requestManagers.commitRequestManager.get(); - CompletableFuture commitResult = manager.commitAsync(event.offsets()); - event.chain(commitResult); + CompletableFuture future = manager.commitAsync(event.offsets()); + future.whenComplete(complete(event.future())); } - private void process(final SyncCommitApplicationEvent event) { + private void process(final SyncCommitEvent event) { + if (!requestManagers.commitRequestManager.isPresent()) { + return; + } + CommitRequestManager manager = requestManagers.commitRequestManager.get(); long expirationTimeoutMs = getExpirationTimeForTimeout(event.retryTimeoutMs()); - CompletableFuture commitResult = manager.commitSync(event.offsets(), expirationTimeoutMs); - event.chain(commitResult); + CompletableFuture future = manager.commitSync(event.offsets(), expirationTimeoutMs); + future.whenComplete(complete(event.future())); } - private void process(final FetchCommittedOffsetsApplicationEvent event) { + private void process(final FetchCommittedOffsetsEvent event) { if (!requestManagers.commitRequestManager.isPresent()) { event.future().completeExceptionally(new KafkaException("Unable to fetch committed " + "offset because the CommittedRequestManager is not available. Check if group.id was set correctly")); @@ -163,19 +178,19 @@ private void process(final FetchCommittedOffsetsApplicationEvent event) { } CommitRequestManager manager = requestManagers.commitRequestManager.get(); long expirationTimeMs = getExpirationTimeForTimeout(event.timeout()); - event.chain(manager.fetchOffsets(event.partitions(), expirationTimeMs)); + CompletableFuture> future = manager.fetchOffsets(event.partitions(), expirationTimeMs); + future.whenComplete(complete(event.future())); } private void process(final NewTopicsMetadataUpdateRequestEvent ignored) { metadata.requestUpdateForNewTopics(); } - /** * Commit all consumed if auto-commit is enabled. Note this will trigger an async commit, * that will not be retried if the commit request fails. */ - private void process(final AssignmentChangeApplicationEvent event) { + private void process(final AssignmentChangeEvent event) { if (!requestManagers.commitRequestManager.isPresent()) { return; } @@ -184,11 +199,11 @@ private void process(final AssignmentChangeApplicationEvent event) { manager.maybeAutoCommitAsync(); } - private void process(final ListOffsetsApplicationEvent event) { + private void process(final ListOffsetsEvent event) { final CompletableFuture> future = requestManagers.offsetsRequestManager.fetchOffsets(event.timestampsToSearch(), event.requireTimestamps()); - event.chain(future); + future.whenComplete(complete(event.future())); } /** @@ -196,7 +211,7 @@ private void process(final ListOffsetsApplicationEvent event) { * consumer join the group if it is not part of it yet, or send the updated subscription if * it is already a member. */ - private void process(final SubscriptionChangeApplicationEvent ignored) { + private void process(final SubscriptionChangeEvent ignored) { if (!requestManagers.heartbeatRequestManager.isPresent()) { log.warn("Group membership manager not present when processing a subscribe event"); return; @@ -213,38 +228,39 @@ private void process(final SubscriptionChangeApplicationEvent ignored) { * execution for releasing the assignment completes, and the request to leave * the group is sent out. */ - private void process(final UnsubscribeApplicationEvent event) { + private void process(final UnsubscribeEvent event) { if (!requestManagers.heartbeatRequestManager.isPresent()) { KafkaException error = new KafkaException("Group membership manager not present when processing an unsubscribe event"); event.future().completeExceptionally(error); return; } MembershipManager membershipManager = requestManagers.heartbeatRequestManager.get().membershipManager(); - CompletableFuture result = membershipManager.leaveGroup(); - event.chain(result); + CompletableFuture future = membershipManager.leaveGroup(); + future.whenComplete(complete(event.future())); } - private void process(final ResetPositionsApplicationEvent event) { - CompletableFuture result = requestManagers.offsetsRequestManager.resetPositionsIfNeeded(); - event.chain(result); + private void process(final ResetPositionsEvent event) { + CompletableFuture future = requestManagers.offsetsRequestManager.resetPositionsIfNeeded(); + future.whenComplete(complete(event.future())); } - private void process(final ValidatePositionsApplicationEvent event) { - CompletableFuture result = requestManagers.offsetsRequestManager.validatePositionsIfNeeded(); - event.chain(result); + private void process(final ValidatePositionsEvent event) { + CompletableFuture future = requestManagers.offsetsRequestManager.validatePositionsIfNeeded(); + future.whenComplete(complete(event.future())); } - private void process(final TopicMetadataApplicationEvent event) { - final CompletableFuture>> future; - - long expirationTimeMs = getExpirationTimeForTimeout(event.getTimeoutMs()); - if (event.isAllTopics()) { - future = requestManagers.topicMetadataRequestManager.requestAllTopicsMetadata(expirationTimeMs); - } else { - future = requestManagers.topicMetadataRequestManager.requestTopicMetadata(event.topic(), expirationTimeMs); - } + private void process(final TopicMetadataEvent event) { + final long expirationTimeMs = getExpirationTimeForTimeout(event.timeoutMs()); + final CompletableFuture>> future = + requestManagers.topicMetadataRequestManager.requestTopicMetadata(event.topic(), expirationTimeMs); + future.whenComplete(complete(event.future())); + } - event.chain(future); + private void process(final AllTopicsMetadataEvent event) { + final long expirationTimeMs = getExpirationTimeForTimeout(event.timeoutMs()); + final CompletableFuture>> future = + requestManagers.topicMetadataRequestManager.requestAllTopicsMetadata(expirationTimeMs); + future.whenComplete(complete(event.future())); } private void process(final ConsumerRebalanceListenerCallbackCompletedEvent event) { @@ -259,14 +275,14 @@ private void process(final ConsumerRebalanceListenerCallbackCompletedEvent event manager.consumerRebalanceListenerCallbackCompleted(event); } - private void process(final CommitOnCloseApplicationEvent event) { + private void process(final CommitOnCloseEvent event) { if (!requestManagers.commitRequestManager.isPresent()) return; log.debug("Signal CommitRequestManager closing"); requestManagers.commitRequestManager.get().signalClose(); } - private void process(final LeaveOnCloseApplicationEvent event) { + private void process(final LeaveOnCloseEvent event) { if (!requestManagers.heartbeatRequestManager.isPresent()) { event.future().complete(null); return; @@ -277,7 +293,7 @@ private void process(final LeaveOnCloseApplicationEvent event) { log.debug("Leaving group before closing"); CompletableFuture future = membershipManager.leaveGroup(); // The future will be completed on heartbeat sent - event.chain(future); + future.whenComplete(complete(event.future())); } /** @@ -293,6 +309,15 @@ long getExpirationTimeForTimeout(final long timeoutMs) { return expiration; } + private BiConsumer complete(final CompletableFuture b) { + return (value, exception) -> { + if (exception != null) + b.completeExceptionally(exception); + else + b.complete(value); + }; + } + /** * Creates a {@link Supplier} for deferred creation during invocation by * {@link ConsumerNetworkThread}. diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeEvent.java similarity index 56% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeEvent.java index ccf7199f26..c9efa2e9df 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeEvent.java @@ -22,13 +22,12 @@ import java.util.Collections; import java.util.Map; -public class AssignmentChangeApplicationEvent extends ApplicationEvent { +public class AssignmentChangeEvent extends ApplicationEvent { private final Map offsets; private final long currentTimeMs; - public AssignmentChangeApplicationEvent(final Map offsets, - final long currentTimeMs) { + public AssignmentChangeEvent(final Map offsets, final long currentTimeMs) { super(Type.ASSIGNMENT_CHANGE); this.offsets = Collections.unmodifiableMap(offsets); this.currentTimeMs = currentTimeMs; @@ -43,31 +42,7 @@ public long currentTimeMs() { } @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - AssignmentChangeApplicationEvent that = (AssignmentChangeApplicationEvent) o; - - if (currentTimeMs != that.currentTimeMs) return false; - return offsets.equals(that.offsets); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + offsets.hashCode(); - result = 31 * result + (int) (currentTimeMs ^ (currentTimeMs >>> 32)); - return result; - } - - @Override - public String toString() { - return "AssignmentChangeApplicationEvent{" + - toStringBase() + - ", offsets=" + offsets + - ", currentTimeMs=" + currentTimeMs + - '}'; + protected String toStringBase() { + return super.toStringBase() + ", offsets=" + offsets + ", currentTimeMs=" + currentTimeMs; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java similarity index 73% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java index 7a939ce3cf..2f03fdfb1e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java @@ -18,22 +18,15 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; + import java.util.Map; /** * Event to commit offsets without waiting for a response, so the request won't be retried. */ -public class AsyncCommitApplicationEvent extends CommitApplicationEvent { - - public AsyncCommitApplicationEvent(final Map offsets) { - super(offsets, Type.COMMIT_ASYNC); - } +public class AsyncCommitEvent extends CommitEvent { - @Override - public String toString() { - return "AsyncCommitApplicationEvent{" + - toStringBase() + - ", offsets=" + offsets() + - '}'; + public AsyncCommitEvent(final Map offsets) { + super(Type.COMMIT_ASYNC, offsets); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java index e5d522201e..9bc3fbebc3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; +import org.apache.kafka.common.Uuid; import java.util.Objects; @@ -31,36 +32,45 @@ public enum Type { private final Type type; - public BackgroundEvent(Type type) { + /** + * This identifies a particular event. It is used to disambiguate events via {@link #hashCode()} and + * {@link #equals(Object)} and can be used in log messages when debugging. + */ + private final Uuid id; + + protected BackgroundEvent(Type type) { this.type = Objects.requireNonNull(type); + this.id = Uuid.randomUuid(); } public Type type() { return type; } + public Uuid id() { + return id; + } + @Override - public boolean equals(Object o) { + public final boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; - BackgroundEvent that = (BackgroundEvent) o; - - return type == that.type; + return type == that.type && id.equals(that.id); } @Override - public int hashCode() { - return type.hashCode(); + public final int hashCode() { + return Objects.hash(type, id); } protected String toStringBase() { - return "type=" + type; + return "type=" + type + ", id=" + id; } @Override - public String toString() { - return "BackgroundEvent{" + + public final String toString() { + return getClass().getSimpleName() + "{" + toStringBase() + '}'; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java index 103493d253..48421484f1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java @@ -26,7 +26,7 @@ /** * An event handler that receives {@link BackgroundEvent background events} from the * {@link ConsumerNetworkThread network thread} which are then made available to the application thread - * via the {@link BackgroundEventProcessor}. + * via an {@link EventProcessor}. */ public class BackgroundEventHandler { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java similarity index 71% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java index 69d969d7b0..253d27e257 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java @@ -22,14 +22,14 @@ import java.util.Collections; import java.util.Map; -public abstract class CommitApplicationEvent extends CompletableApplicationEvent { +public abstract class CommitEvent extends CompletableApplicationEvent { /** * Offsets to commit per partition. */ private final Map offsets; - public CommitApplicationEvent(final Map offsets, Type type) { + protected CommitEvent(final Type type, final Map offsets) { super(type); this.offsets = Collections.unmodifiableMap(offsets); @@ -45,20 +45,7 @@ public Map offsets() { } @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - CommitApplicationEvent that = (CommitApplicationEvent) o; - - return offsets.equals(that.offsets); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + offsets.hashCode(); - return result; + protected String toStringBase() { + return super.toStringBase() + ", offsets=" + offsets; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseEvent.java similarity index 76% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseEvent.java index 4cc07e945f..7d2e29fced 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseEvent.java @@ -16,16 +16,9 @@ */ package org.apache.kafka.clients.consumer.internals.events; -public class CommitOnCloseApplicationEvent extends ApplicationEvent { +public class CommitOnCloseEvent extends ApplicationEvent { - public CommitOnCloseApplicationEvent() { + public CommitOnCloseEvent() { super(Type.COMMIT_ON_CLOSE); } - - @Override - public String toString() { - return "CommitOnCloseApplicationEvent{" + - toStringBase() + - '}'; - } } 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 365c620e0c..a62c3aaa4c 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 @@ -16,9 +16,6 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.clients.consumer.internals.ConsumerUtils; -import org.apache.kafka.common.utils.Timer; - import java.util.concurrent.CompletableFuture; /** @@ -31,56 +28,18 @@ public abstract class CompletableApplicationEvent extends ApplicationEvent im private final CompletableFuture future; - protected CompletableApplicationEvent(Type type) { + protected CompletableApplicationEvent(final Type type) { super(type); this.future = new CompletableFuture<>(); } + @Override public CompletableFuture future() { return future; } - public T get(Timer timer) { - return ConsumerUtils.getResult(future, timer); - } - - public void chain(final CompletableFuture providedFuture) { - providedFuture.whenComplete((value, exception) -> { - if (exception != null) { - this.future.completeExceptionally(exception); - } else { - this.future.complete(value); - } - }); - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - CompletableApplicationEvent that = (CompletableApplicationEvent) o; - - return future.equals(that.future); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + future.hashCode(); - return result; - } - @Override protected String toStringBase() { return super.toStringBase() + ", future=" + future; } - - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - '}'; - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java index 640ee6103a..1a58515a5c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java @@ -28,42 +28,18 @@ public abstract class CompletableBackgroundEvent extends BackgroundEvent impl private final CompletableFuture future; - protected CompletableBackgroundEvent(Type type) { + protected CompletableBackgroundEvent(final Type type) { super(type); this.future = new CompletableFuture<>(); } + @Override public CompletableFuture future() { return future; } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - CompletableBackgroundEvent that = (CompletableBackgroundEvent) o; - - return future.equals(that.future); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + future.hashCode(); - return result; - } - @Override protected String toStringBase() { return super.toStringBase() + ", future=" + future; } - - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - '}'; - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java index 8fdcc20fa8..97559d8cb9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java @@ -21,5 +21,4 @@ public interface CompletableEvent { CompletableFuture future(); - } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java index b260c6154e..a10e98df1d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java @@ -34,9 +34,9 @@ public class ConsumerRebalanceListenerCallbackCompletedEvent extends Application private final CompletableFuture future; private final Optional error; - public ConsumerRebalanceListenerCallbackCompletedEvent(ConsumerRebalanceListenerMethodName methodName, - CompletableFuture future, - Optional error) { + public ConsumerRebalanceListenerCallbackCompletedEvent(final ConsumerRebalanceListenerMethodName methodName, + final CompletableFuture future, + final Optional error) { super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED); this.methodName = Objects.requireNonNull(methodName); this.future = Objects.requireNonNull(future); @@ -55,24 +55,6 @@ public Optional error() { return error; } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - ConsumerRebalanceListenerCallbackCompletedEvent that = (ConsumerRebalanceListenerCallbackCompletedEvent) o; - - return methodName == that.methodName && - future.equals(that.future) && - error.equals(that.error); - } - - @Override - public int hashCode() { - return Objects.hash(methodName, future, error); - } - @Override protected String toStringBase() { return super.toStringBase() + @@ -80,11 +62,4 @@ protected String toStringBase() { ", future=" + future + ", error=" + error; } - - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - '}'; - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java index 7b17c034ab..6ce833580c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java @@ -37,8 +37,8 @@ public class ConsumerRebalanceListenerCallbackNeededEvent extends CompletableBac private final ConsumerRebalanceListenerMethodName methodName; private final SortedSet partitions; - public ConsumerRebalanceListenerCallbackNeededEvent(ConsumerRebalanceListenerMethodName methodName, - SortedSet partitions) { + public ConsumerRebalanceListenerCallbackNeededEvent(final ConsumerRebalanceListenerMethodName methodName, + final SortedSet partitions) { super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED); this.methodName = Objects.requireNonNull(methodName); this.partitions = Collections.unmodifiableSortedSet(partitions); @@ -52,36 +52,10 @@ public SortedSet partitions() { return partitions; } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - ConsumerRebalanceListenerCallbackNeededEvent that = (ConsumerRebalanceListenerCallbackNeededEvent) o; - - return methodName == that.methodName && partitions.equals(that.partitions); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + methodName.hashCode(); - result = 31 * result + partitions.hashCode(); - return result; - } - @Override protected String toStringBase() { return super.toStringBase() + ", methodName=" + methodName + ", partitions=" + partitions; } - - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - '}'; - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java deleted file mode 100644 index 2945f22986..0000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java +++ /dev/null @@ -1,59 +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 org.apache.kafka.clients.consumer.internals.events; - -import org.apache.kafka.common.KafkaException; - -public class ErrorBackgroundEvent extends BackgroundEvent { - - private final RuntimeException error; - - public ErrorBackgroundEvent(Throwable t) { - super(Type.ERROR); - this.error = t instanceof RuntimeException ? (RuntimeException) t : new KafkaException(t); - } - - public RuntimeException error() { - return error; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - ErrorBackgroundEvent that = (ErrorBackgroundEvent) o; - - return error.equals(that.error); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + error.hashCode(); - return result; - } - - @Override - public String toString() { - return "ErrorBackgroundEvent{" + - toStringBase() + - ", error=" + error + - '}'; - } -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorEvent.java new file mode 100644 index 0000000000..5e6d822382 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorEvent.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals.events; + +import org.apache.kafka.common.KafkaException; + +public class ErrorEvent extends BackgroundEvent { + + private final RuntimeException error; + + public ErrorEvent(Throwable t) { + super(Type.ERROR); + this.error = t instanceof RuntimeException ? (RuntimeException) t : new KafkaException(t); + } + + public RuntimeException error() { + return error; + } + + @Override + public String toStringBase() { + return super.toStringBase() + ", error=" + error; + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsEvent.java similarity index 60% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsEvent.java index 34b2d97705..7cf56b990b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsEvent.java @@ -23,7 +23,7 @@ import java.util.Map; import java.util.Set; -public class FetchCommittedOffsetsApplicationEvent extends CompletableApplicationEvent> { +public class FetchCommittedOffsetsEvent extends CompletableApplicationEvent> { /** * Partitions to retrieve committed offsets for. @@ -35,8 +35,7 @@ public class FetchCommittedOffsetsApplicationEvent extends CompletableApplicatio */ private final long timeoutMs; - public FetchCommittedOffsetsApplicationEvent(final Set partitions, - final long timeoutMs) { + public FetchCommittedOffsetsEvent(final Set partitions, final long timeoutMs) { super(Type.FETCH_COMMITTED_OFFSETS); this.partitions = Collections.unmodifiableSet(partitions); this.timeoutMs = timeoutMs; @@ -51,29 +50,7 @@ public long timeout() { } @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - FetchCommittedOffsetsApplicationEvent that = (FetchCommittedOffsetsApplicationEvent) o; - - return partitions.equals(that.partitions); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + partitions.hashCode(); - return result; - } - - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - ", partitions=" + partitions + - ", timeout=" + timeoutMs + "ms" + - '}'; + public String toStringBase() { + return super.toStringBase() + ", partitions=" + partitions + ", partitions=" + partitions; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java index 120e671724..001f549818 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java @@ -19,8 +19,6 @@ import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; -import java.util.Objects; - /** * This event is sent by the {@link ConsumerNetworkThread consumer's network thread} to the application thread * so that when the user calls the {@link Consumer#groupMetadata()} API, the information is up-to-date. The @@ -29,11 +27,10 @@ */ public class GroupMetadataUpdateEvent extends BackgroundEvent { - final private int memberEpoch; - final private String memberId; + private final int memberEpoch; + private final String memberId; - public GroupMetadataUpdateEvent(final int memberEpoch, - final String memberId) { + public GroupMetadataUpdateEvent(final int memberEpoch, final String memberId) { super(Type.GROUP_METADATA_UPDATE); this.memberEpoch = memberEpoch; this.memberId = memberId; @@ -47,33 +44,10 @@ public String memberId() { return memberId; } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - GroupMetadataUpdateEvent that = (GroupMetadataUpdateEvent) o; - return memberEpoch == that.memberEpoch && - Objects.equals(memberId, that.memberId); - } - - @Override - public int hashCode() { - return Objects.hash(super.hashCode(), memberEpoch, memberId); - } - @Override public String toStringBase() { return super.toStringBase() + ", memberEpoch=" + memberEpoch + ", memberId='" + memberId + '\''; } - - @Override - public String toString() { - return "GroupMetadataUpdateEvent{" + - toStringBase() + - '}'; - } - } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseEvent.java similarity index 76% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseEvent.java index ee0b6ffa61..5ee19a7cc0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseEvent.java @@ -16,15 +16,9 @@ */ package org.apache.kafka.clients.consumer.internals.events; -public class LeaveOnCloseApplicationEvent extends CompletableApplicationEvent { - public LeaveOnCloseApplicationEvent() { - super(Type.LEAVE_ON_CLOSE); - } +public class LeaveOnCloseEvent extends CompletableApplicationEvent { - @Override - public String toString() { - return "LeaveOnCloseApplicationEvent{" + - toStringBase() + - '}'; + public LeaveOnCloseEvent() { + super(Type.LEAVE_ON_CLOSE); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java similarity index 69% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java index 2466d06272..fd3b321173 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java @@ -31,12 +31,12 @@ * {@link OffsetAndTimestamp} found (offset of the first message whose timestamp is greater than * or equals to the target timestamp) */ -public class ListOffsetsApplicationEvent extends CompletableApplicationEvent> { +public class ListOffsetsEvent extends CompletableApplicationEvent> { private final Map timestampsToSearch; private final boolean requireTimestamps; - public ListOffsetsApplicationEvent(Map timestampToSearch, boolean requireTimestamps) { + public ListOffsetsEvent(final Map timestampToSearch, final boolean requireTimestamps) { super(Type.LIST_OFFSETS); this.timestampsToSearch = Collections.unmodifiableMap(timestampToSearch); this.requireTimestamps = requireTimestamps; @@ -64,31 +64,10 @@ public boolean requireTimestamps() { } @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - ListOffsetsApplicationEvent that = (ListOffsetsApplicationEvent) o; - - if (requireTimestamps != that.requireTimestamps) return false; - return timestampsToSearch.equals(that.timestampsToSearch); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + timestampsToSearch.hashCode(); - result = 31 * result + (requireTimestamps ? 1 : 0); - return result; - } - - @Override - public String toString() { - return getClass().getSimpleName() + " {" + - toStringBase() + - ", timestampsToSearch=" + timestampsToSearch + ", " + - "requireTimestamps=" + requireTimestamps + '}'; + public String toStringBase() { + return super.toStringBase() + + ", timestampsToSearch=" + timestampsToSearch + + ", requireTimestamps=" + requireTimestamps; } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NewTopicsMetadataUpdateRequestEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NewTopicsMetadataUpdateRequestEvent.java index c06a3a717d..b06bd456f5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NewTopicsMetadataUpdateRequestEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NewTopicsMetadataUpdateRequestEvent.java @@ -21,11 +21,4 @@ public class NewTopicsMetadataUpdateRequestEvent extends ApplicationEvent { public NewTopicsMetadataUpdateRequestEvent() { super(Type.NEW_TOPICS_METADATA_UPDATE); } - - @Override - public String toString() { - return "NewTopicsMetadataUpdateRequestEvent{" + - toStringBase() + - '}'; - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollApplicationEvent.java deleted file mode 100644 index b958f0ec41..0000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollApplicationEvent.java +++ /dev/null @@ -1,57 +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 org.apache.kafka.clients.consumer.internals.events; - -public class PollApplicationEvent extends ApplicationEvent { - - private final long pollTimeMs; - - public PollApplicationEvent(final long pollTimeMs) { - super(Type.POLL); - this.pollTimeMs = pollTimeMs; - } - - public long pollTimeMs() { - return pollTimeMs; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - PollApplicationEvent that = (PollApplicationEvent) o; - - return pollTimeMs == that.pollTimeMs; - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + (int) (pollTimeMs ^ (pollTimeMs >>> 32)); - return result; - } - - @Override - public String toString() { - return "PollApplicationEvent{" + - toStringBase() + - ", pollTimeMs=" + pollTimeMs + - '}'; - } -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollEvent.java new file mode 100644 index 0000000000..96614c06e9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollEvent.java @@ -0,0 +1,36 @@ +/* + * 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.events; + +public class PollEvent extends ApplicationEvent { + + private final long pollTimeMs; + + public PollEvent(final long pollTimeMs) { + super(Type.POLL); + this.pollTimeMs = pollTimeMs; + } + + public long pollTimeMs() { + return pollTimeMs; + } + + @Override + public String toStringBase() { + return super.toStringBase() + ", pollTimeMs=" + pollTimeMs; + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsEvent.java similarity index 89% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsEvent.java index 5d9b07f9de..06f6ebbb68 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsEvent.java @@ -22,9 +22,9 @@ * asynchronous event that generates ListOffsets requests, and completes by updating in-memory * positions when responses are received. */ -public class ResetPositionsApplicationEvent extends CompletableApplicationEvent { +public class ResetPositionsEvent extends CompletableApplicationEvent { - public ResetPositionsApplicationEvent() { + public ResetPositionsEvent() { super(Type.RESET_POSITIONS); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SubscriptionChangeApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SubscriptionChangeEvent.java similarity index 90% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SubscriptionChangeApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SubscriptionChangeEvent.java index 73fd15fb14..ad5fd34c06 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SubscriptionChangeApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SubscriptionChangeEvent.java @@ -22,9 +22,9 @@ * calls the subscribe API. This will make the consumer join a consumer group if not part of it * yet, or just send the updated subscription to the broker if it's already a member of the group. */ -public class SubscriptionChangeApplicationEvent extends ApplicationEvent { +public class SubscriptionChangeEvent extends ApplicationEvent { - public SubscriptionChangeApplicationEvent() { + public SubscriptionChangeEvent() { super(Type.SUBSCRIPTION_CHANGE); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitEvent.java similarity index 73% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitEvent.java index 43dfee6ab1..7e00e0da59 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitEvent.java @@ -18,22 +18,23 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; + import java.util.Map; /** * Event to commit offsets waiting for a response and retrying on expected retriable errors until * the timer expires. */ -public class SyncCommitApplicationEvent extends CommitApplicationEvent { +public class SyncCommitEvent extends CommitEvent { /** * Time to wait for a response, retrying on retriable errors. */ private final long retryTimeoutMs; - public SyncCommitApplicationEvent(final Map offsets, - final long retryTimeoutMs) { - super(offsets, Type.COMMIT_SYNC); + public SyncCommitEvent(final Map offsets, + final long retryTimeoutMs) { + super(Type.COMMIT_SYNC, offsets); this.retryTimeoutMs = retryTimeoutMs; } @@ -42,11 +43,7 @@ public Long retryTimeoutMs() { } @Override - public String toString() { - return "SyncCommitApplicationEvent{" + - toStringBase() + - ", offsets=" + offsets() + - ", retryTimeout=" + retryTimeoutMs + "ms" + - '}'; + public String toStringBase() { + return super.toStringBase() + ", offsets=" + offsets() + ", retryTimeoutMs=" + retryTimeoutMs; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java deleted file mode 100644 index dd6f842cc2..0000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java +++ /dev/null @@ -1,78 +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 org.apache.kafka.clients.consumer.internals.events; - -import org.apache.kafka.common.PartitionInfo; - -import java.util.List; -import java.util.Map; -import java.util.Objects; - -public class TopicMetadataApplicationEvent extends CompletableApplicationEvent>> { - private final String topic; - private final boolean allTopics; - private final long timeoutMs; - - public TopicMetadataApplicationEvent(final long timeoutMs) { - super(Type.TOPIC_METADATA); - this.topic = null; - this.allTopics = true; - this.timeoutMs = timeoutMs; - } - - public TopicMetadataApplicationEvent(final String topic, final long timeoutMs) { - super(Type.TOPIC_METADATA); - this.topic = topic; - this.allTopics = false; - this.timeoutMs = timeoutMs; - } - - public String topic() { - return topic; - } - - public boolean isAllTopics() { - return allTopics; - } - - public long getTimeoutMs() { - return timeoutMs; - } - @Override - public String toString() { - return getClass().getSimpleName() + " {" + toStringBase() + - ", topic=" + topic + - ", allTopics=" + allTopics + - ", timeoutMs=" + timeoutMs + "}"; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (!(o instanceof TopicMetadataApplicationEvent)) return false; - if (!super.equals(o)) return false; - - TopicMetadataApplicationEvent that = (TopicMetadataApplicationEvent) o; - - return topic.equals(that.topic) && (allTopics == that.allTopics) && (timeoutMs == that.timeoutMs); - } - - @Override - public int hashCode() { - return Objects.hash(super.hashCode(), topic, allTopics, timeoutMs); - } -} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataEvent.java new file mode 100644 index 0000000000..ebbb2a6c46 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataEvent.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals.events; + +import java.util.Objects; + +public class TopicMetadataEvent extends AbstractTopicMetadataEvent { + + private final String topic; + + public TopicMetadataEvent(final String topic, final long timeoutMs) { + super(Type.TOPIC_METADATA, timeoutMs); + this.topic = Objects.requireNonNull(topic); + } + + public String topic() { + return topic; + } + + @Override + public String toStringBase() { + return super.toStringBase() + ", topic=" + topic; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java similarity index 91% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java index a1ccb896fd..07af36e5fe 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java @@ -24,8 +24,9 @@ * complete and the heartbeat to leave the group is sent out (minimal effort to send the * leave group heartbeat, without waiting for any response or considering timeouts). */ -public class UnsubscribeApplicationEvent extends CompletableApplicationEvent { - public UnsubscribeApplicationEvent() { +public class UnsubscribeEvent extends CompletableApplicationEvent { + + public UnsubscribeEvent() { super(Type.UNSUBSCRIBE); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsEvent.java similarity index 89% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsEvent.java index 3b093e0b68..efa358b4c7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsEvent.java @@ -22,9 +22,9 @@ * detected. This is an asynchronous event that generates OffsetForLeaderEpoch requests, and * completes by validating in-memory positions against the offsets received in the responses. */ -public class ValidatePositionsApplicationEvent extends CompletableApplicationEvent { +public class ValidatePositionsEvent extends CompletableApplicationEvent { - public ValidatePositionsApplicationEvent() { + public ValidatePositionsEvent() { super(Type.VALIDATE_POSITIONS); } } \ No newline at end of file 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 976677dec8..2d6899612f 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 @@ -31,26 +31,26 @@ import org.apache.kafka.clients.consumer.RetriableCommitFailedException; import org.apache.kafka.clients.consumer.RoundRobinAssignor; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; -import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.AsyncCommitApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeEvent; +import org.apache.kafka.clients.consumer.internals.events.AsyncCommitEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; -import org.apache.kafka.clients.consumer.internals.events.CommitOnCloseApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CommitOnCloseEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableBackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; -import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; +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.FetchCommittedOffsetsApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsEvent; import org.apache.kafka.clients.consumer.internals.events.GroupMetadataUpdateEvent; -import org.apache.kafka.clients.consumer.internals.events.LeaveOnCloseApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; +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.PollApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ResetPositionsApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.SubscriptionChangeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.SyncCommitApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.UnsubscribeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.PollEvent; +import org.apache.kafka.clients.consumer.internals.events.ResetPositionsEvent; +import org.apache.kafka.clients.consumer.internals.events.SubscriptionChangeEvent; +import org.apache.kafka.clients.consumer.internals.events.SyncCommitEvent; +import org.apache.kafka.clients.consumer.internals.events.UnsubscribeEvent; +import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsEvent; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Metric; @@ -256,9 +256,9 @@ public void testCommitAsyncWithNullCallback() { consumer.commitAsync(offsets, null); - final ArgumentCaptor commitEventCaptor = ArgumentCaptor.forClass(AsyncCommitApplicationEvent.class); + final ArgumentCaptor commitEventCaptor = ArgumentCaptor.forClass(AsyncCommitEvent.class); verify(applicationEventHandler).add(commitEventCaptor.capture()); - final AsyncCommitApplicationEvent commitEvent = commitEventCaptor.getValue(); + final AsyncCommitEvent commitEvent = commitEventCaptor.getValue(); assertEquals(offsets, commitEvent.offsets()); assertDoesNotThrow(() -> commitEvent.future().complete(null)); assertDoesNotThrow(() -> consumer.commitAsync(offsets, null)); @@ -310,9 +310,9 @@ public void testCommitAsyncWithFencedException() { assertDoesNotThrow(() -> consumer.commitAsync(offsets, callback)); - final ArgumentCaptor commitEventCaptor = ArgumentCaptor.forClass(AsyncCommitApplicationEvent.class); + final ArgumentCaptor commitEventCaptor = ArgumentCaptor.forClass(AsyncCommitEvent.class); verify(applicationEventHandler).add(commitEventCaptor.capture()); - final AsyncCommitApplicationEvent commitEvent = commitEventCaptor.getValue(); + final AsyncCommitEvent commitEvent = commitEventCaptor.getValue(); commitEvent.future().completeExceptionally(Errors.FENCED_INSTANCE_ID.exception()); assertThrows(Errors.FENCED_INSTANCE_ID.exception().getClass(), () -> consumer.commitAsync()); @@ -325,7 +325,7 @@ public void testCommitted() { completeFetchedCommittedOffsetApplicationEventSuccessfully(topicPartitionOffsets); assertEquals(topicPartitionOffsets, consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), any()); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class), any()); final Metric metric = consumer.metrics() .get(consumer.metricsRegistry().metricName("committed-time-ns-total", "consumer-metrics")); assertTrue((double) metric.metricValue() > 0); @@ -347,7 +347,7 @@ public void testCommittedLeaderEpochUpdate() { verify(metadata).updateLastSeenEpochIfNewer(t0, 2); verify(metadata).updateLastSeenEpochIfNewer(t2, 3); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), any()); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class), any()); } @Test @@ -355,9 +355,9 @@ public void testCommittedExceptionThrown() { consumer = newConsumer(); Map offsets = mockTopicPartitionOffset(); when(applicationEventHandler.addAndGet( - any(FetchCommittedOffsetsApplicationEvent.class), any())).thenAnswer(invocation -> { + any(FetchCommittedOffsetsEvent.class), any())).thenAnswer(invocation -> { CompletableApplicationEvent event = invocation.getArgument(0); - assertInstanceOf(FetchCommittedOffsetsApplicationEvent.class, event); + assertInstanceOf(FetchCommittedOffsetsEvent.class, event); throw new KafkaException("Test exception"); }); @@ -530,7 +530,7 @@ public void testCommitSyncLeaderEpochUpdate() { verify(metadata).updateLastSeenEpochIfNewer(t0, 2); verify(metadata).updateLastSeenEpochIfNewer(t1, 1); - verify(applicationEventHandler).add(ArgumentMatchers.isA(SyncCommitApplicationEvent.class)); + verify(applicationEventHandler).add(ArgumentMatchers.isA(SyncCommitEvent.class)); } @Test @@ -564,7 +564,7 @@ public void testCommitAsyncLeaderEpochUpdate() { verify(metadata).updateLastSeenEpochIfNewer(t0, 2); verify(metadata).updateLastSeenEpochIfNewer(t1, 1); - verify(applicationEventHandler).add(ArgumentMatchers.isA(AsyncCommitApplicationEvent.class)); + verify(applicationEventHandler).add(ArgumentMatchers.isA(AsyncCommitEvent.class)); } @Test @@ -598,8 +598,8 @@ public void testVerifyApplicationEventOnShutdown() { consumer = newConsumer(); doReturn(null).when(applicationEventHandler).addAndGet(any(), any()); consumer.close(); - verify(applicationEventHandler).addAndGet(any(LeaveOnCloseApplicationEvent.class), any()); - verify(applicationEventHandler).add(any(CommitOnCloseApplicationEvent.class)); + verify(applicationEventHandler).addAndGet(any(LeaveOnCloseEvent.class), any()); + verify(applicationEventHandler).add(any(CommitOnCloseEvent.class)); } @Test @@ -641,7 +641,7 @@ public void testFailedPartitionRevocationOnClose() { subscriptions.assignFromSubscribed(singleton(tp)); doThrow(new KafkaException()).when(listener).onPartitionsRevoked(eq(singleton(tp))); assertThrows(KafkaException.class, () -> consumer.close(Duration.ZERO)); - verify(applicationEventHandler, never()).addAndGet(any(LeaveOnCloseApplicationEvent.class), any()); + verify(applicationEventHandler, never()).addAndGet(any(LeaveOnCloseEvent.class), any()); verify(listener).onPartitionsRevoked(eq(singleton(tp))); assertEquals(emptySet(), subscriptions.assignedPartitions()); } @@ -677,7 +677,7 @@ public void testAutoCommitSyncEnabled() { subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); subscriptions.seek(new TopicPartition("topic", 0), 100); consumer.maybeAutoCommitSync(true, time.timer(100), null); - verify(applicationEventHandler).add(any(SyncCommitApplicationEvent.class)); + verify(applicationEventHandler).add(any(SyncCommitEvent.class)); } @Test @@ -695,7 +695,7 @@ public void testAutoCommitSyncDisabled() { subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); subscriptions.seek(new TopicPartition("topic", 0), 100); consumer.maybeAutoCommitSync(false, time.timer(100), null); - verify(applicationEventHandler, never()).add(any(SyncCommitApplicationEvent.class)); + verify(applicationEventHandler, never()).add(any(SyncCommitEvent.class)); } private void assertMockCommitCallbackInvoked(final Executable task, @@ -729,7 +729,7 @@ public void testAssign() { consumer.assign(singleton(tp)); assertTrue(consumer.subscription().isEmpty()); assertTrue(consumer.assignment().contains(tp)); - verify(applicationEventHandler).add(any(AssignmentChangeApplicationEvent.class)); + verify(applicationEventHandler).add(any(AssignmentChangeEvent.class)); verify(applicationEventHandler).add(any(NewTopicsMetadataUpdateRequestEvent.class)); } @@ -781,7 +781,7 @@ public void testBeginningOffsets() { Map expectedOffsets = expectedOffsetsAndTimestamp.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset())); assertEquals(expectedOffsets, result); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsEvent.class), ArgumentMatchers.isA(Timer.class)); } @@ -792,13 +792,13 @@ public void testBeginningOffsetsThrowsKafkaExceptionForUnderlyingExecutionFailur Throwable eventProcessingFailure = new KafkaException("Unexpected failure " + "processing List Offsets event"); doThrow(eventProcessingFailure).when(applicationEventHandler).addAndGet( - any(ListOffsetsApplicationEvent.class), + any(ListOffsetsEvent.class), any()); Throwable consumerError = assertThrows(KafkaException.class, () -> consumer.beginningOffsets(partitions, Duration.ofMillis(1))); assertEquals(eventProcessingFailure, consumerError); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsEvent.class), ArgumentMatchers.isA(Timer.class)); } @@ -810,7 +810,7 @@ public void testBeginningOffsetsTimeoutOnEventProcessingTimeout() { () -> consumer.beginningOffsets( Collections.singletonList(new TopicPartition("t1", 0)), Duration.ofMillis(1))); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsEvent.class), ArgumentMatchers.isA(Timer.class)); } @@ -850,7 +850,7 @@ public void testOffsetsForTimes() { Map result = assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ofMillis(1))); assertEquals(expectedResult, result); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsEvent.class), ArgumentMatchers.isA(Timer.class)); } @@ -869,7 +869,7 @@ public void testOffsetsForTimesWithZeroTimeout() { assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ZERO)); assertEquals(expectedResult, result); - verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsEvent.class), ArgumentMatchers.isA(Timer.class)); } @@ -880,12 +880,12 @@ public void testWakeupCommitted() { doAnswer(invocation -> { CompletableApplicationEvent event = invocation.getArgument(0); Timer timer = invocation.getArgument(1); - assertInstanceOf(FetchCommittedOffsetsApplicationEvent.class, event); + assertInstanceOf(FetchCommittedOffsetsEvent.class, event); assertTrue(event.future().isCompletedExceptionally()); return ConsumerUtils.getResult(event.future(), timer); }) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + .addAndGet(any(FetchCommittedOffsetsEvent.class), any(Timer.class)); consumer.wakeup(); assertThrows(WakeupException.class, () -> consumer.committed(offsets.keySet())); @@ -1011,7 +1011,7 @@ public void testSubscribeGeneratesEvent() { consumer.subscribe(singletonList(topic)); assertEquals(singleton(topic), consumer.subscription()); assertTrue(consumer.assignment().isEmpty()); - verify(applicationEventHandler).add(ArgumentMatchers.isA(SubscriptionChangeApplicationEvent.class)); + verify(applicationEventHandler).add(ArgumentMatchers.isA(SubscriptionChangeEvent.class)); } @Test @@ -1023,7 +1023,7 @@ public void testUnsubscribeGeneratesUnsubscribeEvent() { assertTrue(consumer.subscription().isEmpty()); assertTrue(consumer.assignment().isEmpty()); - verify(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeApplicationEvent.class)); + verify(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeEvent.class)); } @Test @@ -1034,7 +1034,7 @@ public void testSubscribeToEmptyListActsAsUnsubscribe() { consumer.subscribe(Collections.emptyList()); assertTrue(consumer.subscription().isEmpty()); assertTrue(consumer.assignment().isEmpty()); - verify(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeApplicationEvent.class)); + verify(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeEvent.class)); } @Test @@ -1356,8 +1356,8 @@ public void testBackgroundError() { consumer = newConsumer(config); final KafkaException expectedException = new KafkaException("Nobody expects the Spanish Inquisition"); - final ErrorBackgroundEvent errorBackgroundEvent = new ErrorBackgroundEvent(expectedException); - backgroundEventQueue.add(errorBackgroundEvent); + final ErrorEvent errorEvent = new ErrorEvent(expectedException); + backgroundEventQueue.add(errorEvent); consumer.assign(singletonList(new TopicPartition("topic", 0))); final KafkaException exception = assertThrows(KafkaException.class, () -> consumer.poll(Duration.ZERO)); @@ -1371,11 +1371,11 @@ public void testMultipleBackgroundErrors() { consumer = newConsumer(config); final KafkaException expectedException1 = new KafkaException("Nobody expects the Spanish Inquisition"); - final ErrorBackgroundEvent errorBackgroundEvent1 = new ErrorBackgroundEvent(expectedException1); - backgroundEventQueue.add(errorBackgroundEvent1); + final ErrorEvent errorEvent1 = new ErrorEvent(expectedException1); + backgroundEventQueue.add(errorEvent1); final KafkaException expectedException2 = new KafkaException("Spam, Spam, Spam"); - final ErrorBackgroundEvent errorBackgroundEvent2 = new ErrorBackgroundEvent(expectedException2); - backgroundEventQueue.add(errorBackgroundEvent2); + final ErrorEvent errorEvent2 = new ErrorEvent(expectedException2); + backgroundEventQueue.add(errorEvent2); consumer.assign(singletonList(new TopicPartition("topic", 0))); final KafkaException exception = assertThrows(KafkaException.class, () -> consumer.poll(Duration.ZERO)); @@ -1472,7 +1472,7 @@ public void testEnsurePollEventSentOnConsumerPoll() { consumer.subscribe(singletonList("topic1")); consumer.poll(Duration.ofMillis(100)); - verify(applicationEventHandler).add(any(PollApplicationEvent.class)); + verify(applicationEventHandler).add(any(PollEvent.class)); } private void testInvalidGroupId(final String groupId) { @@ -1510,20 +1510,20 @@ private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean co consumer.poll(Duration.ZERO); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ValidatePositionsEvent.class), ArgumentMatchers.isA(Timer.class)); if (committedOffsetsEnabled) { // Verify there was an FetchCommittedOffsets event and no ResetPositions event verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class), ArgumentMatchers.isA(Timer.class)); verify(applicationEventHandler, never()) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsEvent.class), ArgumentMatchers.isA(Timer.class)); } else { // Verify there was not any FetchCommittedOffsets event but there should be a ResetPositions verify(applicationEventHandler, never()) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class), ArgumentMatchers.isA(Timer.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsEvent.class), ArgumentMatchers.isA(Timer.class)); } } @@ -1538,11 +1538,11 @@ private void testRefreshCommittedOffsetsSuccess(Set partitions, consumer.poll(Duration.ZERO); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ValidatePositionsEvent.class), ArgumentMatchers.isA(Timer.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class), ArgumentMatchers.isA(Timer.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsEvent.class), ArgumentMatchers.isA(Timer.class)); } @Test @@ -1697,54 +1697,54 @@ private HashMap mockTimestampToSearch() { private void completeCommitAsyncApplicationEventExceptionally(Exception ex) { doAnswer(invocation -> { - AsyncCommitApplicationEvent event = invocation.getArgument(0); + AsyncCommitEvent event = invocation.getArgument(0); event.future().completeExceptionally(ex); return null; - }).when(applicationEventHandler).add(ArgumentMatchers.isA(AsyncCommitApplicationEvent.class)); + }).when(applicationEventHandler).add(ArgumentMatchers.isA(AsyncCommitEvent.class)); } private void completeCommitSyncApplicationEventExceptionally(Exception ex) { doAnswer(invocation -> { - SyncCommitApplicationEvent event = invocation.getArgument(0); + SyncCommitEvent event = invocation.getArgument(0); event.future().completeExceptionally(ex); return null; - }).when(applicationEventHandler).add(ArgumentMatchers.isA(SyncCommitApplicationEvent.class)); + }).when(applicationEventHandler).add(ArgumentMatchers.isA(SyncCommitEvent.class)); } private void completeCommitAsyncApplicationEventSuccessfully() { doAnswer(invocation -> { - AsyncCommitApplicationEvent event = invocation.getArgument(0); + AsyncCommitEvent event = invocation.getArgument(0); event.future().complete(null); return null; - }).when(applicationEventHandler).add(ArgumentMatchers.isA(AsyncCommitApplicationEvent.class)); + }).when(applicationEventHandler).add(ArgumentMatchers.isA(AsyncCommitEvent.class)); } private void completeCommitSyncApplicationEventSuccessfully() { doAnswer(invocation -> { - SyncCommitApplicationEvent event = invocation.getArgument(0); + SyncCommitEvent event = invocation.getArgument(0); event.future().complete(null); return null; - }).when(applicationEventHandler).add(ArgumentMatchers.isA(SyncCommitApplicationEvent.class)); + }).when(applicationEventHandler).add(ArgumentMatchers.isA(SyncCommitEvent.class)); } private void completeFetchedCommittedOffsetApplicationEventSuccessfully(final Map committedOffsets) { doReturn(committedOffsets) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + .addAndGet(any(FetchCommittedOffsetsEvent.class), any(Timer.class)); } private void completeFetchedCommittedOffsetApplicationEventExceptionally(Exception ex) { doThrow(ex) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + .addAndGet(any(FetchCommittedOffsetsEvent.class), any(Timer.class)); } private void completeUnsubscribeApplicationEventSuccessfully() { doAnswer(invocation -> { - UnsubscribeApplicationEvent event = invocation.getArgument(0); + UnsubscribeEvent event = invocation.getArgument(0); event.future().complete(null); return null; - }).when(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeApplicationEvent.class)); + }).when(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeEvent.class)); } private void forceCommitCallbackInvocation() { 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 a491df417d..cbd56d8b5e 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 @@ -20,16 +20,16 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; -import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.AsyncCommitApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeEvent; +import org.apache.kafka.clients.consumer.internals.events.AsyncCommitEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; +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.PollApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ResetPositionsApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.SyncCommitApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.TopicMetadataApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.PollEvent; +import org.apache.kafka.clients.consumer.internals.events.ResetPositionsEvent; +import org.apache.kafka.clients.consumer.internals.events.SyncCommitEvent; +import org.apache.kafka.clients.consumer.internals.events.TopicMetadataEvent; +import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsEvent; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.message.FindCoordinatorRequestData; @@ -137,7 +137,7 @@ public void testStartupAndTearDown() throws InterruptedException { @Test public void testApplicationEvent() { - ApplicationEvent e = new PollApplicationEvent(100); + ApplicationEvent e = new PollEvent(100); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor, times(1)).process(e); @@ -153,36 +153,36 @@ public void testMetadataUpdateEvent() { @Test public void testAsyncCommitEvent() { - ApplicationEvent e = new AsyncCommitApplicationEvent(new HashMap<>()); + ApplicationEvent e = new AsyncCommitEvent(new HashMap<>()); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); - verify(applicationEventProcessor).process(any(AsyncCommitApplicationEvent.class)); + verify(applicationEventProcessor).process(any(AsyncCommitEvent.class)); } @Test public void testSyncCommitEvent() { - ApplicationEvent e = new SyncCommitApplicationEvent(new HashMap<>(), 100L); + ApplicationEvent e = new SyncCommitEvent(new HashMap<>(), 100L); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); - verify(applicationEventProcessor).process(any(SyncCommitApplicationEvent.class)); + verify(applicationEventProcessor).process(any(SyncCommitEvent.class)); } @Test public void testListOffsetsEventIsProcessed() { Map timestamps = Collections.singletonMap(new TopicPartition("topic1", 1), 5L); - ApplicationEvent e = new ListOffsetsApplicationEvent(timestamps, true); + ApplicationEvent e = new ListOffsetsEvent(timestamps, true); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); - verify(applicationEventProcessor).process(any(ListOffsetsApplicationEvent.class)); + verify(applicationEventProcessor).process(any(ListOffsetsEvent.class)); assertTrue(applicationEventsQueue.isEmpty()); } @Test public void testResetPositionsEventIsProcessed() { - ResetPositionsApplicationEvent e = new ResetPositionsApplicationEvent(); + ResetPositionsEvent e = new ResetPositionsEvent(); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); - verify(applicationEventProcessor).process(any(ResetPositionsApplicationEvent.class)); + verify(applicationEventProcessor).process(any(ResetPositionsEvent.class)); assertTrue(applicationEventsQueue.isEmpty()); } @@ -190,19 +190,19 @@ public void testResetPositionsEventIsProcessed() { public void testResetPositionsProcessFailureIsIgnored() { doThrow(new NullPointerException()).when(offsetsRequestManager).resetPositionsIfNeeded(); - ResetPositionsApplicationEvent event = new ResetPositionsApplicationEvent(); + ResetPositionsEvent event = new ResetPositionsEvent(); applicationEventsQueue.add(event); assertDoesNotThrow(() -> consumerNetworkThread.runOnce()); - verify(applicationEventProcessor).process(any(ResetPositionsApplicationEvent.class)); + verify(applicationEventProcessor).process(any(ResetPositionsEvent.class)); } @Test public void testValidatePositionsEventIsProcessed() { - ValidatePositionsApplicationEvent e = new ValidatePositionsApplicationEvent(); + ValidatePositionsEvent e = new ValidatePositionsEvent(); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); - verify(applicationEventProcessor).process(any(ValidatePositionsApplicationEvent.class)); + verify(applicationEventProcessor).process(any(ValidatePositionsEvent.class)); assertTrue(applicationEventsQueue.isEmpty()); } @@ -211,11 +211,11 @@ public void testAssignmentChangeEvent() { HashMap offset = mockTopicPartitionOffset(); final long currentTimeMs = time.milliseconds(); - ApplicationEvent e = new AssignmentChangeApplicationEvent(offset, currentTimeMs); + ApplicationEvent e = new AssignmentChangeEvent(offset, currentTimeMs); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); - verify(applicationEventProcessor).process(any(AssignmentChangeApplicationEvent.class)); + verify(applicationEventProcessor).process(any(AssignmentChangeEvent.class)); verify(networkClient, times(1)).poll(anyLong(), anyLong()); verify(commitRequestManager, times(1)).updateAutoCommitTimer(currentTimeMs); // Assignment change should generate an async commit (not retried). @@ -224,9 +224,9 @@ public void testAssignmentChangeEvent() { @Test void testFetchTopicMetadata() { - applicationEventsQueue.add(new TopicMetadataApplicationEvent("topic", Long.MAX_VALUE)); + applicationEventsQueue.add(new TopicMetadataEvent("topic", Long.MAX_VALUE)); consumerNetworkThread.runOnce(); - verify(applicationEventProcessor).process(any(TopicMetadataApplicationEvent.class)); + verify(applicationEventProcessor).process(any(TopicMetadataEvent.class)); } @Test @@ -283,8 +283,8 @@ void testEnsureEventsAreCompleted() { coordinatorRequestManager.markCoordinatorUnknown("test", time.milliseconds()); client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "group-id", node)); prepareOffsetCommitRequest(new HashMap<>(), Errors.NONE, false); - CompletableApplicationEvent event1 = spy(new AsyncCommitApplicationEvent(Collections.emptyMap())); - ApplicationEvent event2 = new AsyncCommitApplicationEvent(Collections.emptyMap()); + CompletableApplicationEvent event1 = spy(new AsyncCommitEvent(Collections.emptyMap())); + ApplicationEvent event2 = new AsyncCommitEvent(Collections.emptyMap()); CompletableFuture future = new CompletableFuture<>(); when(event1.future()).thenReturn(future); applicationEventsQueue.add(event1); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java index d7ad1b5573..afee3fff39 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java @@ -18,7 +18,7 @@ import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; -import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.common.Node; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.TimeoutException; @@ -114,10 +114,10 @@ public void testPropagateAndBackoffAfterFatalError() { expectFindCoordinatorRequest(coordinatorManager, Errors.GROUP_AUTHORIZATION_FAILED); verify(backgroundEventHandler).add(argThat(backgroundEvent -> { - if (!(backgroundEvent instanceof ErrorBackgroundEvent)) + if (!(backgroundEvent instanceof ErrorEvent)) return false; - RuntimeException exception = ((ErrorBackgroundEvent) backgroundEvent).error(); + RuntimeException exception = ((ErrorEvent) backgroundEvent).error(); if (!(exception instanceof GroupAuthorizationException)) return false; 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 4d4492bcb4..90cfb90bb1 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 @@ -328,12 +328,8 @@ public void testValidateConsumerGroupHeartbeatRequest(final short version) { @Test public void testConsumerGroupMetadataFirstUpdate() { final GroupMetadataUpdateEvent groupMetadataUpdateEvent = makeFirstGroupMetadataUpdate(memberId, memberEpoch); - - final GroupMetadataUpdateEvent expectedGroupMetadataUpdateEvent = new GroupMetadataUpdateEvent( - memberEpoch, - memberId - ); - assertEquals(expectedGroupMetadataUpdateEvent, groupMetadataUpdateEvent); + assertEquals(memberEpoch, groupMetadataUpdateEvent.memberEpoch()); + assertEquals(memberId, groupMetadataUpdateEvent.memberId()); } @Test @@ -370,11 +366,8 @@ public void testConsumerGroupMetadataUpdateWithMemberIdNullButMemberEpochUpdated final BackgroundEvent eventWithUpdatedMemberEpoch = backgroundEventQueue.poll(); assertEquals(BackgroundEvent.Type.GROUP_METADATA_UPDATE, eventWithUpdatedMemberEpoch.type()); final GroupMetadataUpdateEvent groupMetadataUpdateEvent = (GroupMetadataUpdateEvent) eventWithUpdatedMemberEpoch; - final GroupMetadataUpdateEvent expectedGroupMetadataUpdateEvent = new GroupMetadataUpdateEvent( - updatedMemberEpoch, - memberId - ); - assertEquals(expectedGroupMetadataUpdateEvent, groupMetadataUpdateEvent); + assertEquals(updatedMemberEpoch, groupMetadataUpdateEvent.memberEpoch()); + assertEquals(memberId, groupMetadataUpdateEvent.memberId()); } @Test @@ -398,11 +391,8 @@ public void testConsumerGroupMetadataUpdateWithMemberIdUpdatedAndMemberEpochSame final BackgroundEvent eventWithUpdatedMemberEpoch = backgroundEventQueue.poll(); assertEquals(BackgroundEvent.Type.GROUP_METADATA_UPDATE, eventWithUpdatedMemberEpoch.type()); final GroupMetadataUpdateEvent groupMetadataUpdateEvent = (GroupMetadataUpdateEvent) eventWithUpdatedMemberEpoch; - final GroupMetadataUpdateEvent expectedGroupMetadataUpdateEvent = new GroupMetadataUpdateEvent( - memberEpoch, - updatedMemberId - ); - assertEquals(expectedGroupMetadataUpdateEvent, groupMetadataUpdateEvent); + assertEquals(memberEpoch, groupMetadataUpdateEvent.memberEpoch()); + assertEquals(updatedMemberId, groupMetadataUpdateEvent.memberId()); } private GroupMetadataUpdateEvent makeFirstGroupMetadataUpdate(final String memberId, final int memberEpoch) { 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 8a0fcf8575..49c5819be7 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 @@ -1890,10 +1890,10 @@ private Map> topicIdPartitionsMap(Uuid topicId, int... } private ConsumerRebalanceListenerCallbackCompletedEvent performCallback(MembershipManagerImpl membershipManager, - ConsumerRebalanceListenerInvoker invoker, - ConsumerRebalanceListenerMethodName expectedMethodName, - SortedSet expectedPartitions, - boolean complete) { + ConsumerRebalanceListenerInvoker invoker, + ConsumerRebalanceListenerMethodName expectedMethodName, + SortedSet expectedPartitions, + boolean complete) { // We expect only our enqueued event in the background queue. assertEquals(1, backgroundEventQueue.size()); assertNotNull(backgroundEventQueue.peek()); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java index 5ca034d636..59ed7d440e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java @@ -24,7 +24,7 @@ import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; -import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.ClusterResource; import org.apache.kafka.common.IsolationLevel; @@ -553,8 +553,8 @@ public void testResetOffsetsAuthorizationFailure() { assertNotNull(event); // Check that the event itself is of the expected type - assertInstanceOf(ErrorBackgroundEvent.class, event); - ErrorBackgroundEvent errorEvent = (ErrorBackgroundEvent) event; + assertInstanceOf(ErrorEvent.class, event); + ErrorEvent errorEvent = (ErrorEvent) event; assertNotNull(errorEvent.error()); // Check that the error held in the event is of the expected type 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 8ea8cb7a72..f3e2557ae9 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 @@ -92,7 +92,7 @@ public void setup() { public void testPrepClosingCommitEvents() { List results = mockCommitResults(); doReturn(new NetworkClientDelegate.PollResult(100, results)).when(commitRequestManager).pollOnClose(); - processor.process(new CommitOnCloseApplicationEvent()); + processor.process(new CommitOnCloseEvent()); verify(commitRequestManager).signalClose(); } @@ -107,7 +107,7 @@ public void testExpirationCalculation() { @Test public void testPrepClosingLeaveGroupEvent() { - LeaveOnCloseApplicationEvent event = new LeaveOnCloseApplicationEvent(); + LeaveOnCloseEvent event = new LeaveOnCloseEvent(); when(heartbeatRequestManager.membershipManager()).thenReturn(membershipManager); when(membershipManager.leaveGroup()).thenReturn(CompletableFuture.completedFuture(null)); processor.process(event); From d066b94c8103cca166d7ea01a4b5bf5f65a3b838 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Thu, 29 Feb 2024 07:01:21 -0800 Subject: [PATCH 008/521] MINOR: Fix UpdatedImage and HighWatermarkUpdated events' logs (#15432) I have noticed the following log when a __consumer_offsets partition immigrate from a broker. It appends because the event is queued up after the event that unloads the state machine. This patch fixes it and fixes another similar one. ``` [2024-02-06 17:14:51,359] ERROR [GroupCoordinator id=1] Execution of UpdateImage(tp=__consumer_offsets-28, offset=13251) failed due to This is not the correct coordinator.. (org.apache.kafka.coordinator.group.runtime.CoordinatorRuntime) org.apache.kafka.common.errors.NotCoordinatorException: This is not the correct coordinator. ``` Reviewers: Justine Olshan --- .../group/runtime/CoordinatorRuntime.java | 144 +++++++++--------- 1 file changed, 76 insertions(+), 68 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java index b0be84c7a9..ccb4caf04f 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java @@ -1192,11 +1192,28 @@ public void onHighWatermarkUpdated( ) { log.debug("High watermark of {} incremented to {}.", tp, offset); scheduleInternalOperation("HighWatermarkUpdated(tp=" + tp + ", offset=" + offset + ")", tp, () -> { - withActiveContextOrThrow(tp, context -> { - context.coordinator.updateLastCommittedOffset(offset); - context.deferredEventQueue.completeUpTo(offset); - coordinatorMetrics.onUpdateLastCommittedOffset(tp, offset); - }); + CoordinatorContext context = coordinators.get(tp); + if (context != null) { + context.lock.lock(); + try { + if (context.state == CoordinatorState.ACTIVE) { + // The updated high watermark can be applied to the coordinator only if the coordinator + // exists and is in the active state. + log.debug("Updating high watermark of {} to {}.", tp, offset); + context.coordinator.updateLastCommittedOffset(offset); + context.deferredEventQueue.completeUpTo(offset); + coordinatorMetrics.onUpdateLastCommittedOffset(tp, offset); + } else { + log.debug("Ignored high watermark updated for {} to {} because the coordinator is not active.", + tp, offset); + } + } finally { + context.lock.unlock(); + } + } else { + log.debug("Ignored high watermark updated for {} to {} because the coordinator does not exist.", + tp, offset); + } }); } } @@ -1350,14 +1367,11 @@ private void enqueue(CoordinatorEvent event) { } /** - * Creates the context if it does not exist. - * - * @param tp The topic partition. - * - * Visible for testing. + * @return The coordinator context or a new context if it does not exist. + * Package private for testing. */ - void maybeCreateContext(TopicPartition tp) { - coordinators.computeIfAbsent(tp, CoordinatorContext::new); + CoordinatorContext maybeCreateContext(TopicPartition tp) { + return coordinators.computeIfAbsent(tp, CoordinatorContext::new); } /** @@ -1376,29 +1390,6 @@ CoordinatorContext contextOrThrow(TopicPartition tp) throws NotCoordinatorExcept } } - /** - * Calls the provided function with the context; throws an exception otherwise. - * This method ensures that the context lock is acquired before calling the - * function and releases afterwards. - * - * @param tp The topic partition. - * @param func The function that will receive the context. - * @throws NotCoordinatorException - */ - private void withContextOrThrow( - TopicPartition tp, - Consumer func - ) throws NotCoordinatorException { - CoordinatorContext context = contextOrThrow(tp); - - try { - context.lock.lock(); - func.accept(context); - } finally { - context.lock.unlock(); - } - } - /** * Calls the provided function with the context iff the context is active; throws * an exception otherwise. This method ensures that the context lock is acquired @@ -1609,7 +1600,11 @@ public void scheduleLoadOperation( maybeCreateContext(tp); scheduleInternalOperation("Load(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> { - withContextOrThrow(tp, context -> { + // The context is re-created if it does not exist. + CoordinatorContext context = maybeCreateContext(tp); + + context.lock.lock(); + try { if (context.epoch < partitionEpoch) { context.epoch = partitionEpoch; @@ -1617,16 +1612,13 @@ public void scheduleLoadOperation( case FAILED: case INITIAL: context.transitionTo(CoordinatorState.LOADING); - loader.load( - tp, - context.coordinator - ).whenComplete((summary, exception) -> { + loader.load(tp, context.coordinator).whenComplete((summary, exception) -> { scheduleInternalOperation("CompleteLoad(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> { - withContextOrThrow(tp, ctx -> { + CoordinatorContext ctx = coordinators.get(tp); + if (ctx != null) { if (ctx.state != CoordinatorState.LOADING) { - log.info("Ignoring load completion from {} because context is in {} state.", - ctx.tp, ctx.state - ); + log.info("Ignored load completion from {} because context is in {} state.", + ctx.tp, ctx.state); return; } try { @@ -1635,18 +1627,19 @@ public void scheduleLoadOperation( if (summary != null) { runtimeMetrics.recordPartitionLoadSensor(summary.startTimeMs(), summary.endTimeMs()); log.info("Finished loading of metadata from {} with epoch {} in {}ms where {}ms " + - "was spent in the scheduler. Loaded {} records which total to {} bytes.", + "was spent in the scheduler. Loaded {} records which total to {} bytes.", tp, partitionEpoch, summary.endTimeMs() - summary.startTimeMs(), - summary.schedulerQueueTimeMs(), summary.numRecords(), summary.numBytes() - ); + summary.schedulerQueueTimeMs(), summary.numRecords(), summary.numBytes()); } } catch (Throwable ex) { log.error("Failed to load metadata from {} with epoch {} due to {}.", - tp, partitionEpoch, ex.toString() - ); + tp, partitionEpoch, ex.toString()); ctx.transitionTo(CoordinatorState.FAILED); } - }); + } else { + log.debug("Failed to complete the loading of metadata for {} in epoch {} since the coordinator does not exist.", + tp, partitionEpoch); + } }); }); break; @@ -1663,11 +1656,12 @@ public void scheduleLoadOperation( log.error("Cannot load coordinator {} in state {}.", tp, context.state); } } else { - log.info("Ignoring loading metadata from {} since current epoch {} is larger than or equals to {}.", - context.tp, context.epoch, partitionEpoch - ); + log.info("Ignored loading metadata from {} since current epoch {} is larger than or equals to {}.", + context.tp, context.epoch, partitionEpoch); } - }); + } finally { + context.lock.unlock(); + } }); } @@ -1689,8 +1683,8 @@ public void scheduleUnloadOperation( scheduleInternalOperation("UnloadCoordinator(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> { CoordinatorContext context = coordinators.get(tp); if (context != null) { + context.lock.lock(); try { - context.lock.lock(); if (!partitionEpoch.isPresent() || context.epoch < partitionEpoch.getAsInt()) { log.info("Started unloading metadata for {} with epoch {}.", tp, partitionEpoch); context.transitionTo(CoordinatorState.CLOSED); @@ -1698,16 +1692,14 @@ public void scheduleUnloadOperation( log.info("Finished unloading metadata for {} with epoch {}.", tp, partitionEpoch); } else { log.info("Ignored unloading metadata for {} in epoch {} since current epoch is {}.", - tp, partitionEpoch, context.epoch - ); + tp, partitionEpoch, context.epoch); } } finally { context.lock.unlock(); } } else { log.info("Ignored unloading metadata for {} in epoch {} since metadata was never loaded.", - tp, partitionEpoch - ); + tp, partitionEpoch); } }); } @@ -1731,15 +1723,26 @@ public void onNewMetadataImage( // Push an event for each coordinator. coordinators.keySet().forEach(tp -> { scheduleInternalOperation("UpdateImage(tp=" + tp + ", offset=" + newImage.offset() + ")", tp, () -> { - withContextOrThrow(tp, context -> { - if (context.state == CoordinatorState.ACTIVE) { - log.debug("Applying new metadata image with offset {} to {}.", newImage.offset(), tp); - context.coordinator.onNewMetadataImage(newImage, delta); - } else { - log.debug("Ignoring new metadata image with offset {} for {} because the coordinator is not active.", - newImage.offset(), tp); + CoordinatorContext context = coordinators.get(tp); + if (context != null) { + context.lock.lock(); + try { + if (context.state == CoordinatorState.ACTIVE) { + // The new image can be applied to the coordinator only if the coordinator + // exists and is in the active state. + log.debug("Applying new metadata image with offset {} to {}.", newImage.offset(), tp); + context.coordinator.onNewMetadataImage(newImage, delta); + } else { + log.debug("Ignored new metadata image with offset {} for {} because the coordinator is not active.", + newImage.offset(), tp); + } + } finally { + context.lock.unlock(); } - }); + } else { + log.debug("Ignored new metadata image with offset {} for {} because the coordinator does not exist.", + newImage.offset(), tp); + } }); }); } @@ -1764,7 +1767,12 @@ public void close() throws Exception { Utils.closeQuietly(processor, "event processor"); // Unload all the coordinators. coordinators.forEach((tp, context) -> { - context.transitionTo(CoordinatorState.CLOSED); + context.lock.lock(); + try { + context.transitionTo(CoordinatorState.CLOSED); + } finally { + context.lock.unlock(); + } }); coordinators.clear(); Utils.closeQuietly(runtimeMetrics, "runtime metrics"); From c8843f06841d7f3c94b640ec9dbf69ec4682ec11 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Fri, 1 Mar 2024 11:14:13 +0100 Subject: [PATCH 009/521] KAFKA-16167: Disable wakeups during autocommit on close (#15445) When the consumer is closed, we perform a sychronous autocommit. We don't want to be woken up here, because we are already executing a close operation under a deadline. This is in line with the behavior of the old consumer. This fixes PlaintextConsumerTest.testAutoCommitOnCloseAfterWakeup which is flaky on trunk - because we return immediately from the synchronous commit with a WakeupException, which causes us to not wait for the commit to finish and thereby sometimes miss the committed offset when a new consumer is created. Reviewers: Matthias J. Sax , Bruno Cadonna --- .../internals/AsyncKafkaConsumer.java | 8 ++-- .../consumer/internals/WakeupTrigger.java | 11 ++++++ .../internals/AsyncKafkaConsumerTest.java | 31 ++++++++++++++- .../consumer/internals/WakeupTriggerTest.java | 39 +++++++++++++++++++ 4 files changed, 84 insertions(+), 5 deletions(-) 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 d810c5f053..e706898b70 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 @@ -1228,6 +1228,9 @@ private void close(Duration timeout, boolean swallowException) { log.trace("Closing the Kafka consumer"); AtomicReference firstException = new AtomicReference<>(); + // We are already closing with a timeout, don't allow wake-ups from here on. + wakeupTrigger.disableWakeups(); + final Timer closeTimer = time.timer(timeout); clientTelemetryReporter.ifPresent(reporter -> reporter.initiateClose(timeout.toMillis())); closeTimer.update(); @@ -1265,7 +1268,7 @@ private void close(Duration timeout, boolean swallowException) { void prepareShutdown(final Timer timer, final AtomicReference firstException) { if (!groupMetadata.isPresent()) return; - maybeAutoCommitSync(autoCommitEnabled, timer, firstException); + maybeAutoCommitSync(autoCommitEnabled, timer); applicationEventHandler.add(new CommitOnCloseEvent()); completeQuietly( () -> { @@ -1277,8 +1280,7 @@ void prepareShutdown(final Timer timer, final AtomicReference firstEx // Visible for testing void maybeAutoCommitSync(final boolean shouldAutoCommit, - final Timer timer, - final AtomicReference firstException) { + final Timer timer) { if (!shouldAutoCommit) return; Map allConsumed = subscriptions.allConsumed(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/WakeupTrigger.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/WakeupTrigger.java index 5a030f6307..209d5e41be 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/WakeupTrigger.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/WakeupTrigger.java @@ -73,6 +73,8 @@ public CompletableFuture setActiveTask(final CompletableFuture current } else if (task instanceof WakeupFuture) { currentTask.completeExceptionally(new WakeupException()); return null; + } else if (task instanceof DisabledWakeups) { + return task; } // last active state is still active throw new KafkaException("Last active task is still active"); @@ -88,6 +90,8 @@ public void setFetchAction(final FetchBuffer fetchBuffer) { } else if (task instanceof WakeupFuture) { throwWakeupException.set(true); return null; + } else if (task instanceof DisabledWakeups) { + return task; } // last active state is still active throw new IllegalStateException("Last active task is still active"); @@ -97,6 +101,10 @@ public void setFetchAction(final FetchBuffer fetchBuffer) { } } + public void disableWakeups() { + pendingTask.set(new DisabledWakeups()); + } + public void clearTask() { pendingTask.getAndUpdate(task -> { if (task == null) { @@ -131,6 +139,9 @@ Wakeupable getPendingTask() { interface Wakeupable { } + // Set to block wakeups from happening and pending actions to be registered. + static class DisabledWakeups implements Wakeupable { } + static class ActiveFuture implements Wakeupable { private final CompletableFuture future; 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 2d6899612f..35e742fbd0 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 @@ -30,6 +30,7 @@ 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; import org.apache.kafka.clients.consumer.internals.events.AsyncCommitEvent; @@ -120,6 +121,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -676,7 +678,7 @@ public void testAutoCommitSyncEnabled() { consumer.subscribe(singleton("topic"), mock(ConsumerRebalanceListener.class)); subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); subscriptions.seek(new TopicPartition("topic", 0), 100); - consumer.maybeAutoCommitSync(true, time.timer(100), null); + consumer.maybeAutoCommitSync(true, time.timer(100)); verify(applicationEventHandler).add(any(SyncCommitEvent.class)); } @@ -694,7 +696,7 @@ public void testAutoCommitSyncDisabled() { consumer.subscribe(singleton("topic"), mock(ConsumerRebalanceListener.class)); subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); subscriptions.seek(new TopicPartition("topic", 0), 100); - consumer.maybeAutoCommitSync(false, time.timer(100), null); + consumer.maybeAutoCommitSync(false, time.timer(100)); verify(applicationEventHandler, never()).add(any(SyncCommitEvent.class)); } @@ -892,6 +894,31 @@ public void testWakeupCommitted() { assertNull(consumer.wakeupTrigger().getPendingTask()); } + @Test + public void testNoWakeupInCloseCommit() { + TopicPartition tp = new TopicPartition("topic1", 0); + consumer = newConsumer(); + consumer.assign(Collections.singleton(tp)); + doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); + consumer.seek(tp, 10); + consumer.wakeup(); + + AtomicReference capturedEvent = new AtomicReference<>(); + doAnswer(invocation -> { + ApplicationEvent event = invocation.getArgument(0); + if (event instanceof SyncCommitEvent) { + capturedEvent.set((SyncCommitEvent) event); + } + return null; + }).when(applicationEventHandler).add(any()); + + consumer.close(Duration.ZERO); + + // A commit was triggered and not completed exceptionally by the wakeup + assertNotNull(capturedEvent.get()); + assertFalse(capturedEvent.get().future().isCompletedExceptionally()); + } + @Test public void testInterceptorAutoCommitOnClose() { Properties props = requiredConsumerPropertiesAndGroupId("test-id"); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/WakeupTriggerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/WakeupTriggerTest.java index 235ec78168..3e1518814e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/WakeupTriggerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/WakeupTriggerTest.java @@ -28,12 +28,14 @@ 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.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; @MockitoSettings(strictness = Strictness.STRICT_STUBS) @@ -133,6 +135,43 @@ public void testManualTriggerWhenWakeupCalledAndFetchActionSet() { } } + @Test + public void testDisableWakeupWithoutPendingTask() { + wakeupTrigger.disableWakeups(); + wakeupTrigger.wakeup(); + assertDoesNotThrow(() -> wakeupTrigger.maybeTriggerWakeup()); + } + + @Test + public void testDisableWakeupWithPendingTask() { + final CompletableFuture future = new CompletableFuture<>(); + wakeupTrigger.disableWakeups(); + wakeupTrigger.setActiveTask(future); + wakeupTrigger.wakeup(); + assertFalse(future.isCompletedExceptionally()); + assertDoesNotThrow(() -> wakeupTrigger.maybeTriggerWakeup()); + } + + @Test + public void testDisableWakeupWithFetchAction() { + try (final FetchBuffer fetchBuffer = mock(FetchBuffer.class)) { + wakeupTrigger.disableWakeups(); + wakeupTrigger.setFetchAction(fetchBuffer); + wakeupTrigger.wakeup(); + verify(fetchBuffer, never()).wakeup(); + } + } + + @Test + public void testDisableWakeupPreservedByClearTask() { + final CompletableFuture future = new CompletableFuture<>(); + wakeupTrigger.disableWakeups(); + wakeupTrigger.setActiveTask(future); + wakeupTrigger.clearTask(); + wakeupTrigger.wakeup(); + assertDoesNotThrow(() -> wakeupTrigger.maybeTriggerWakeup()); + } + private void assertWakeupExceptionIsThrown(final CompletableFuture future) { assertTrue(future.isCompletedExceptionally()); try { From 8e1516f88b88b2c815ccbca074af5a5c2d14b5c9 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Fri, 1 Mar 2024 16:42:08 +0100 Subject: [PATCH 010/521] KAFKA-16261: updateSubscription fails if already empty subscription (#15440) The internal SubscriptionState object keeps track of whether the assignment is user-assigned, or auto-assigned. If there are no assigned partitions, the assignment resets to NONE. If you call SubscriptionState.assignFromSubscribed in this state it fails. This change makes sure to check SubscriptionState.hasAutoAssignedPartitions() so that assignFromSubscribed is going to be permitted. Also, a minor refactoring to make clearing the subscription a bit easier to follow in MembershipManagerImpl. Testing via new unit test. Reviewers: Bruno Cadonna , Andrew Schofield --- .../internals/MembershipManagerImpl.java | 37 ++++++------------- .../internals/MembershipManagerImplTest.java | 20 ++++++++++ 2 files changed, 32 insertions(+), 25 deletions(-) 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 e74c7e30a2..35322fb51b 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 @@ -550,7 +550,7 @@ public void transitionToFenced() { log.error("onPartitionsLost callback invocation failed while releasing assignment" + " after member got fenced. Member will rejoin the group anyways.", error); } - updateSubscription(new TreeSet<>(TOPIC_ID_PARTITION_COMPARATOR), true); + clearSubscription(); if (state == MemberState.FENCED) { transitionToJoining(); } else { @@ -583,7 +583,7 @@ public void transitionToFatal() { log.error("onPartitionsLost callback invocation failed while releasing assignment" + "after member failed with fatal error.", error); } - updateSubscription(new TreeSet<>(TOPIC_ID_PARTITION_COMPARATOR), true); + clearSubscription(); }); } @@ -597,16 +597,14 @@ public void onSubscriptionUpdated() { } /** - * Update a new assignment by setting the assigned partitions in the member subscription. - * - * @param assignedPartitions Topic partitions to take as the new subscription assignment - * @param clearAssignments True if the pending assignments and metadata cache should be cleared + * Clear the assigned partitions in the member subscription, pending assignments and metadata cache. */ - private void updateSubscription(SortedSet assignedPartitions, - boolean clearAssignments) { - Collection assignedTopicPartitions = toTopicPartitionSet(assignedPartitions); - subscriptions.assignFromSubscribed(assignedTopicPartitions); - updateAssignmentLocally(assignedPartitions, clearAssignments); + private void clearSubscription() { + if (subscriptions.hasAutoAssignedPartitions()) { + subscriptions.assignFromSubscribed(Collections.emptySet()); + } + updateCurrentAssignment(Collections.emptySet()); + clearPendingAssignmentsAndLocalNamesCache(); } /** @@ -621,18 +619,7 @@ private void updateSubscriptionAwaitingCallback(SortedSet assi SortedSet addedPartitions) { Collection assignedTopicPartitions = toTopicPartitionSet(assignedPartitions); subscriptions.assignFromSubscribedAwaitingCallback(assignedTopicPartitions, addedPartitions); - updateAssignmentLocally(assignedPartitions, false); - } - - /** - * Make assignment effective on the group manager. - */ - private void updateAssignmentLocally(SortedSet assignedPartitions, - boolean clearAssignments) { updateCurrentAssignment(assignedPartitions); - if (clearAssignments) { - clearPendingAssignmentsAndLocalNamesCache(); - } } /** @@ -660,7 +647,7 @@ public void transitionToJoining() { public CompletableFuture leaveGroup() { if (isNotInGroup()) { if (state == MemberState.FENCED) { - updateSubscription(new TreeSet<>(TOPIC_ID_PARTITION_COMPARATOR), true); + clearSubscription(); transitionTo(MemberState.UNSUBSCRIBED); } return CompletableFuture.completedFuture(null); @@ -679,7 +666,7 @@ public CompletableFuture leaveGroup() { CompletableFuture callbackResult = invokeOnPartitionsRevokedOrLostToReleaseAssignment(); callbackResult.whenComplete((result, error) -> { // Clear the subscription, no matter if the callback execution failed or succeeded. - updateSubscription(new TreeSet<>(TOPIC_ID_PARTITION_COMPARATOR), true); + clearSubscription(); // 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 @@ -880,7 +867,7 @@ private void transitionToStale() { log.error("onPartitionsLost callback invocation failed while releasing assignment" + " after member left group due to expired poll timer.", error); } - updateSubscription(new TreeSet<>(TOPIC_ID_PARTITION_COMPARATOR), true); + clearSubscription(); 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); 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 49c5819be7..bc7be88c3b 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 @@ -771,6 +771,21 @@ public void testLeaveGroupWhenMemberOwnsAssignment() { testLeaveGroupReleasesAssignmentAndResetsEpochToSendLeaveGroup(membershipManager); } + @Test + public void testFencedWhenAssignmentEmpty() { + MembershipManager membershipManager = createMemberInStableState(); + + // Clear the assignment + when(subscriptionState.assignedPartitions()).thenReturn(Collections.emptySet()); + when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(false); + + membershipManager.transitionToFenced(); + + // Make sure to never call `assignFromSubscribed` again + verify(subscriptionState, never()).assignFromSubscribed(Collections.emptySet()); + } + + @Test public void testLeaveGroupWhenMemberAlreadyLeaving() { MembershipManager membershipManager = createMemberInStableState(); @@ -1678,6 +1693,7 @@ private void assertLeaveGroupDueToExpiredPollAndTransitionToStale(MembershipMana public void testTransitionToLeavingWhileReconcilingDueToStaleMember() { MembershipManagerImpl membershipManager = memberJoinWithAssignment(); clearInvocations(subscriptionState); + when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); assertEquals(MemberState.RECONCILING, membershipManager.state()); assertLeaveGroupDueToExpiredPollAndTransitionToStale(membershipManager); } @@ -1686,6 +1702,7 @@ public void testTransitionToLeavingWhileReconcilingDueToStaleMember() { public void testTransitionToLeavingWhileJoiningDueToStaleMember() { MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); doNothing().when(subscriptionState).assignFromSubscribed(any()); + when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); assertEquals(MemberState.JOINING, membershipManager.state()); assertLeaveGroupDueToExpiredPollAndTransitionToStale(membershipManager); } @@ -1695,6 +1712,7 @@ public void testTransitionToLeavingWhileStableDueToStaleMember() { MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(null); membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); + when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); doNothing().when(subscriptionState).assignFromSubscribed(any()); assertEquals(MemberState.STABLE, membershipManager.state()); assertLeaveGroupDueToExpiredPollAndTransitionToStale(membershipManager); @@ -1705,6 +1723,7 @@ public void testTransitionToLeavingWhileAcknowledgingDueToStaleMember() { MembershipManagerImpl membershipManager = mockJoinAndReceiveAssignment(true); doNothing().when(subscriptionState).assignFromSubscribed(any()); clearInvocations(subscriptionState); + when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); assertEquals(MemberState.ACKNOWLEDGING, membershipManager.state()); assertLeaveGroupDueToExpiredPollAndTransitionToStale(membershipManager); } @@ -1775,6 +1794,7 @@ public void testStaleMemberWaitsForCallbackToRejoinWhenTimerReset() { private MembershipManagerImpl mockStaleMember() { MembershipManagerImpl membershipManager = createMemberInStableState(); doNothing().when(subscriptionState).assignFromSubscribed(any()); + when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); membershipManager.transitionToSendingLeaveGroup(true); membershipManager.onHeartbeatRequestSent(); return membershipManager; From 52a3fa07446f9c108399d47dbfb1685989a5d6eb Mon Sep 17 00:00:00 2001 From: Jamie Date: Sat, 2 Mar 2024 08:13:56 +1300 Subject: [PATCH 011/521] KAFKA-15878: KIP-768 - Extend support for opaque (i.e. non-JWT) tokens in SASL/OAUTHBEARER (#14818) # Overview * This change pertains to [SASL/OAUTHBEARER ](https://kafka.apache.org/documentation/#security_sasl_oauthbearer) mechanism of Kafka authentication. * Kafka clients can use [SASL/OAUTHBEARER ](https://kafka.apache.org/documentation/#security_sasl_oauthbearer) mechanism by overriding the [custom call back handlers](https://kafka.apache.org/documentation/#security_sasl_oauthbearer_prod) . * [KIP-768](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=186877575) available from v3.1 further extends the mechanism with a production grade implementation. * Kafka's [SASL/OAUTHBEARER ](https://kafka.apache.org/documentation/#security_sasl_oauthbearer) mechanism currently **rejects the non-JWT (i.e. opaque) tokens**. This is because of a more restrictive set of characters than what [RFC-6750](https://datatracker.ietf.org/doc/html/rfc6750#section-2.1) recommends. * This JIRA can be considered an extension of [KIP-768](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=186877575) to support the opaque tokens as well apart from the JWT tokens. # Solution * Have updated the regex in the the offending class to be compliant with the [RFC-6750](https://datatracker.ietf.org/doc/html/rfc6750#section-2.1) * Have provided a supporting test case that includes the possible character set defined in [RFC-6750](https://datatracker.ietf.org/doc/html/rfc6750#section-2.1) --------- Co-authored-by: Anuj Sharma Co-authored-by: Jamie Holmes Co-authored-by: Christopher Webb <31657038+cwebbtw@users.noreply.github.com> Reviewers: Manikumar Reddy , Kirk True --- .../internals/OAuthBearerClientInitialResponse.java | 2 +- .../OAuthBearerClientInitialResponseTest.java | 12 ++++++++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerClientInitialResponse.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerClientInitialResponse.java index 73bfcd15c1..3b340131cf 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerClientInitialResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerClientInitialResponse.java @@ -34,7 +34,7 @@ public class OAuthBearerClientInitialResponse { private static final String VALUE = "[\\x21-\\x7E \t\r\n]+"; private static final String KVPAIRS = String.format("(%s=%s%s)*", KEY, VALUE, SEPARATOR); - private static final Pattern AUTH_PATTERN = Pattern.compile("(?[\\w]+)[ ]+(?[-_\\.a-zA-Z0-9]+)"); + private static final Pattern AUTH_PATTERN = Pattern.compile("(?[\\w]+)[ ]+(?[-_~+/\\.a-zA-Z0-9]+([=]*))"); private static final Pattern CLIENT_INITIAL_RESPONSE_PATTERN = Pattern.compile( String.format("n,(a=(?%s))?,%s(?%s)%s", SASLNAME, SEPARATOR, KVPAIRS, SEPARATOR)); public static final String AUTH_KEY = "auth"; diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerClientInitialResponseTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerClientInitialResponseTest.java index 3b3c90bf1d..fc44297a2f 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerClientInitialResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerClientInitialResponseTest.java @@ -102,6 +102,18 @@ public void testRfc7688Example() throws Exception { assertEquals("143", response.extensions().map().get("port")); } + // RFC 6750 token format 1*( ALPHA / DIGIT /"-" / "." / "_" / "~" / "+" / "/" ) *"=" + @Test + public void testCharSupportForRfc6750Token() throws Exception { + String message = "n,a=user@example.com,\u0001host=server.example.com\u0001port=143\u0001" + + "auth=Bearer vF-9.df_t4qm~Tc2Nvb3RlckBhbHR+hdmlzdGEuY29/tCg==\u0001\u0001"; + OAuthBearerClientInitialResponse response = new OAuthBearerClientInitialResponse(message.getBytes(StandardCharsets.UTF_8)); + assertEquals("vF-9.df_t4qm~Tc2Nvb3RlckBhbHR+hdmlzdGEuY29/tCg==", response.tokenValue()); + assertEquals("user@example.com", response.authorizationId()); + assertEquals("server.example.com", response.extensions().map().get("host")); + assertEquals("143", response.extensions().map().get("port")); + } + @Test public void testNoExtensionsFromByteArray() throws Exception { String message = "n,a=user@example.com,\u0001" + From 21a5bbd84c3c80bcee57ceb05f47005d9122d46e Mon Sep 17 00:00:00 2001 From: Said Boudjelda Date: Sun, 3 Mar 2024 17:32:28 +0100 Subject: [PATCH 012/521] MINOR: Upgrade jqwik to version 1.8.3 (#14365) This minor pull request consist of upgrading version of jqwik library to version 1.8.0 that brings some bug fixing and some enhancements, upgrading the version now will make future upgrades easier For breaking changes: We are not using ArbitraryConfiguratorBase, so there is no overriding of configure method We are not using TypeUsage.canBeAssignedTo(TypeUsage) No breaking is related to @Provide and @ForAll usage no Exception CannotFindArbitraryException is thrown during tests running No usage of StringArbitrary.repeatChars(0.0) We are not affected by the removal of method TypeArbitrary.use(Executable) We are not affected by the removal or methods ActionChainArbitrary.addAction(action) and ActionChainArbitrary.addAction(weight, action) For more details check the release notes: https://jqwik.net/release-notes.html#180 Reviewers: Chia-Ping Tsai , Yash Mayya --- gradle/dependencies.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 9cd1515a79..5ac978f9b0 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -119,7 +119,7 @@ versions += [ jopt: "5.0.4", jose4j: "0.9.4", junit: "5.10.2", - jqwik: "1.7.4", + jqwik: "1.8.3", kafka_0100: "0.10.0.1", kafka_0101: "0.10.1.1", kafka_0102: "0.10.2.2", From 44af72fd77376cd5fb0c0a6019b0e5908928f0e1 Mon Sep 17 00:00:00 2001 From: PoAn Yang Date: Mon, 4 Mar 2024 00:36:57 +0800 Subject: [PATCH 013/521] MINOR: fix link for ListTransactionsOptions#filterOnDuration (#15459) Reviewers: Chia-Ping Tsai --- clients/src/main/java/org/apache/kafka/clients/admin/Admin.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 ff0e60e766..e9f0a58e41 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 @@ -1634,7 +1634,7 @@ default ListTransactionsResult listTransactions() { * should typically attempt to reduce the size of the result set using * {@link ListTransactionsOptions#filterProducerIds(Collection)} or * {@link ListTransactionsOptions#filterStates(Collection)} or - * {@link ListTransactionsOptions#durationFilter(Long)} + * {@link ListTransactionsOptions#filterOnDuration(long)}. * * @param options Options to control the method behavior (including filters) * @return The result From 907e945c0b70263d312038d448196fff1f49a98b Mon Sep 17 00:00:00 2001 From: Ayoub Omari Date: Sun, 3 Mar 2024 18:03:04 +0100 Subject: [PATCH 014/521] MINOR: fix SessionStore java doc (#15412) Reviewers: Chia-Ping Tsai --- .../streams/state/ReadOnlyKeyValueStore.java | 2 +- .../streams/state/ReadOnlySessionStore.java | 17 ++++++++--------- .../streams/state/ReadOnlyWindowStore.java | 4 ++-- 3 files changed, 11 insertions(+), 12 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java index f905a29625..4ef656c212 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java @@ -93,7 +93,7 @@ default KeyValueIterator reverseRange(K from, K to) { * and must not return null values. * Order is not guaranteed as bytes lexicographical ordering might not represent key order. * - * @return An reverse iterator of all key/value pairs in the store, from largest to smallest key bytes. + * @return A reverse iterator of all key/value pairs in the store, from largest to smallest key bytes. * @throws InvalidStateStoreException if the store is not initialized */ default KeyValueIterator reverseAll() { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java index 7fe11a6bea..5a52f00f60 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java @@ -37,7 +37,7 @@ public interface ReadOnlySessionStore { * is the upper bound of the search interval, and the method returns all sessions that overlap * with the search interval. * Thus, if a session ends before earliestSessionEndTime, or starts after latestSessionStartTime - * if won't be contained in the result: + * it won't be contained in the result: *

{@code
      * earliestSessionEndTime: ESET
      * latestSessionStartTime: LSST
@@ -48,7 +48,6 @@ public interface ReadOnlySessionStore {
      * 

* This iterator must be closed after use. * - * * @param key the key to return sessions for * @param earliestSessionEndTime the end timestamp of the earliest session to search for, where * iteration starts. @@ -72,7 +71,7 @@ default KeyValueIterator, AGG> findSessions(final K key, * is the upper bound of the search interval, and the method returns all sessions that overlap * with the search interval. * Thus, if a session ends before earliestSessionEndTime, or starts after latestSessionStartTime - * if won't be contained in the result: + * it won't be contained in the result: *

{@code
      * earliestSessionEndTime: ESET
      * latestSessionStartTime: LSST
@@ -106,7 +105,7 @@ default KeyValueIterator, AGG> findSessions(final K key,
      * is the upper bound of the search interval, and the method returns all sessions that overlap
      * with the search interval.
      * Thus, if a session ends before earliestSessionEndTime, or starts after latestSessionStartTime
-     * if won't be contained in the result:
+     * it won't be contained in the result:
      * 
{@code
      * earliestSessionEndTime: ESET
      * latestSessionStartTime: LSST
@@ -140,7 +139,7 @@ default KeyValueIterator, AGG> backwardFindSessions(final K key,
      * is the upper bound of the search interval, and the method returns all sessions that overlap
      * with the search interval.
      * Thus, if a session ends before earliestSessionEndTime, or starts after latestSessionStartTime
-     * if won't be contained in the result:
+     * it won't be contained in the result:
      * 
{@code
      * earliestSessionEndTime: ESET
      * latestSessionStartTime: LSST
@@ -175,7 +174,7 @@ default KeyValueIterator, AGG> backwardFindSessions(final K key,
      * is the upper bound of the search interval, and the method returns all sessions that overlap
      * with the search interval.
      * Thus, if a session ends before earliestSessionEndTime, or starts after latestSessionStartTime
-     * if won't be contained in the result:
+     * it won't be contained in the result:
      * 
{@code
      * earliestSessionEndTime: ESET
      * latestSessionStartTime: LSST
@@ -213,7 +212,7 @@ default KeyValueIterator, AGG> findSessions(final K keyFrom,
      * is the upper bound of the search interval, and the method returns all sessions that overlap
      * with the search interval.
      * Thus, if a session ends before earliestSessionEndTime, or starts after latestSessionStartTime
-     * if won't be contained in the result:
+     * it won't be contained in the result:
      * 
{@code
      * earliestSessionEndTime: ESET
      * latestSessionStartTime: LSST
@@ -251,7 +250,7 @@ default KeyValueIterator, AGG> findSessions(final K keyFrom,
      * is the upper bound of the search interval, and the method returns all sessions that overlap
      * with the search interval.
      * Thus, if a session ends before earliestSessionEndTime, or starts after latestSessionStartTime
-     * if won't be contained in the result:
+     * it won't be contained in the result:
      * 
{@code
      * earliestSessionEndTime: ESET
      * latestSessionStartTime: LSST
@@ -289,7 +288,7 @@ default KeyValueIterator, AGG> backwardFindSessions(final K keyFrom,
      * is the upper bound of the search interval, and the method returns all sessions that overlap
      * with the search interval.
      * Thus, if a session ends before earliestSessionEndTime, or starts after latestSessionStartTime
-     * if won't be contained in the result:
+     * it won't be contained in the result:
      * 
{@code
      * earliestSessionEndTime: ESET
      * latestSessionStartTime: LSST
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java
index 3df170d5ab..30ca72a64c 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java
@@ -174,7 +174,7 @@ default KeyValueIterator, V> backwardFetch(K keyFrom, K keyTo, Insta
      * Gets all the key-value pairs in the existing windows in backward order
      * with respect to time (from end to beginning of time).
      *
-     * @return an backward iterator over windowed key-value pairs {@code , value>}, from the end to beginning of time.
+     * @return a backward iterator over windowed key-value pairs {@code , value>}, from the end to beginning of time.
      * @throws InvalidStateStoreException if the store is not initialized
      */
     default KeyValueIterator, V> backwardAll() {
@@ -199,7 +199,7 @@ default KeyValueIterator, V> backwardAll() {
      *
      * @param timeFrom the beginning of the time slot from which to search (inclusive), where iteration ends.
      * @param timeTo   the end of the time slot from which to search (inclusive), where iteration starts.
-     * @return an backward iterator over windowed key-value pairs {@code , value>}, from end to beginning of time.
+     * @return a backward iterator over windowed key-value pairs {@code , value>}, from end to beginning of time.
      * @throws InvalidStateStoreException if the store is not initialized
      * @throws NullPointerException       if {@code null} is used for any key
      * @throws IllegalArgumentException   if duration is negative or can't be represented as {@code long milliseconds}

From 2c0cab39aedab3a8635510acfac2551aaeb62ffb Mon Sep 17 00:00:00 2001
From: Ismael Juma 
Date: Sun, 3 Mar 2024 23:31:57 -0800
Subject: [PATCH 015/521] MINOR: Remove unnecessary easymock/powermock
 dependencies (#15460)

These projects don't actually use easymock/powermock.

Reviewers: Chia-Ping Tsai 
---
 build.gradle | 7 +------
 1 file changed, 1 insertion(+), 6 deletions(-)

diff --git a/build.gradle b/build.gradle
index b362c2f673..f2ba22e86b 100644
--- a/build.gradle
+++ b/build.gradle
@@ -2183,8 +2183,6 @@ project(':streams') {
     testImplementation libs.junitJupiter
     testImplementation libs.junitVintageEngine
     testImplementation libs.easymock
-    testImplementation libs.powermockJunit4
-    testImplementation libs.powermockEasymock
     testImplementation libs.bcpkix
     testImplementation libs.hamcrest
     testImplementation libs.mockitoCore
@@ -2332,7 +2330,6 @@ project(':streams:streams-scala') {
     testImplementation project(':streams:test-utils')
 
     testImplementation libs.junitJupiter
-    testImplementation libs.easymock
     testImplementation libs.mockitoCore
     testImplementation libs.mockitoJunitJupiter // supports MockitoExtension
     testImplementation libs.hamcrest
@@ -2869,7 +2866,6 @@ project(':connect:transforms') {
 
     implementation libs.slf4jApi
 
-    testImplementation libs.easymock
     testImplementation libs.junitJupiter
 
     testRuntimeOnly libs.slf4jlog4j
@@ -2909,8 +2905,7 @@ project(':connect:json') {
     api libs.jacksonAfterburner
 
     implementation libs.slf4jApi
-
-    testImplementation libs.easymock
+    
     testImplementation libs.junitJupiter
 
     testRuntimeOnly libs.slf4jlog4j

From 7dbdc15c668dfb5a4a91c79f339c22fb7178c368 Mon Sep 17 00:00:00 2001
From: Ayoub Omari 
Date: Mon, 4 Mar 2024 10:19:59 +0100
Subject: [PATCH 016/521] KAFKA-15625: Do not flush global state store at each
 commit (#15361)

Global state stores are currently flushed at each commit, which may impact performance, especially for EOS (commit each 200ms).
The goal of this improvement is to flush global state stores only when the delta between the current offset and the last checkpointed offset exceeds a threshold.
This is the same logic we apply on local state store, with a threshold of 10000 records.
The implementation only flushes if the time interval elapsed and the threshold of 10000 records is exceeded.

Reviewers: Jeff Kim , Bruno Cadonna 
---
 .../internals/GlobalStateMaintainer.java      |   2 +
 .../internals/GlobalStateUpdateTask.java      |  20 +++-
 .../internals/GlobalStreamThread.java         |  25 +---
 .../internals/GlobalStateTaskTest.java        | 111 ++++++++++++++++--
 .../internals/StateConsumerTest.java          |  31 ++---
 .../kafka/test/GlobalStateManagerStub.java    |  10 +-
 .../kafka/streams/TopologyTestDriver.java     |   4 +-
 7 files changed, 147 insertions(+), 56 deletions(-)

diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateMaintainer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateMaintainer.java
index 9a8aab6eb3..06afb6fde4 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateMaintainer.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateMaintainer.java
@@ -34,4 +34,6 @@ interface GlobalStateMaintainer {
     void close(final boolean wipeStateStore) throws IOException;
 
     void update(ConsumerRecord record);
+
+    void maybeCheckpoint();
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java
index 523228542a..da7ebba209 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java
@@ -19,6 +19,7 @@
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.common.TopicPartition;
 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.streams.errors.DeserializationExceptionHandler;
 import org.apache.kafka.streams.errors.StreamsException;
@@ -45,18 +46,26 @@ public class GlobalStateUpdateTask implements GlobalStateMaintainer {
     private final Map deserializers = new HashMap<>();
     private final GlobalStateManager stateMgr;
     private final DeserializationExceptionHandler deserializationExceptionHandler;
+    private final Time time;
+    private final long flushInterval;
+    private long lastFlush;
 
     public GlobalStateUpdateTask(final LogContext logContext,
                                  final ProcessorTopology topology,
                                  final InternalProcessorContext processorContext,
                                  final GlobalStateManager stateMgr,
-                                 final DeserializationExceptionHandler deserializationExceptionHandler) {
+                                 final DeserializationExceptionHandler deserializationExceptionHandler,
+                                 final Time time,
+                                 final long flushInterval
+                                 ) {
         this.logContext = logContext;
         this.log = logContext.logger(getClass());
         this.topology = topology;
         this.stateMgr = stateMgr;
         this.processorContext = processorContext;
         this.deserializationExceptionHandler = deserializationExceptionHandler;
+        this.time = time;
+        this.flushInterval = flushInterval;
     }
 
     /**
@@ -86,6 +95,7 @@ public Map initialize() {
         }
         initTopology();
         processorContext.initialize();
+        lastFlush = time.milliseconds();
         return stateMgr.changelogOffsets();
     }
 
@@ -150,5 +160,13 @@ private void initTopology() {
         }
     }
 
+    @Override
+    public void maybeCheckpoint() {
+        final long now = time.milliseconds();
+        if (now - flushInterval >= lastFlush && StateManagerUtil.checkpointNeeded(false, stateMgr.changelogOffsets(), offsets)) {
+            flushState();
+            lastFlush = now;
+        }
+    }
 
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java
index 82a0cc5113..1ed517b15d 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java
@@ -228,25 +228,17 @@ public GlobalStreamThread(final ProcessorTopology topology,
     static class StateConsumer {
         private final Consumer globalConsumer;
         private final GlobalStateMaintainer stateMaintainer;
-        private final Time time;
         private final Duration pollTime;
-        private final long flushInterval;
         private final Logger log;
 
-        private long lastFlush;
-
         StateConsumer(final LogContext logContext,
                       final Consumer globalConsumer,
                       final GlobalStateMaintainer stateMaintainer,
-                      final Time time,
-                      final Duration pollTime,
-                      final long flushInterval) {
+                      final Duration pollTime) {
             this.log = logContext.logger(getClass());
             this.globalConsumer = globalConsumer;
             this.stateMaintainer = stateMaintainer;
-            this.time = time;
             this.pollTime = pollTime;
-            this.flushInterval = flushInterval;
         }
 
         /**
@@ -259,7 +251,6 @@ void initialize() {
             for (final Map.Entry entry : partitionOffsets.entrySet()) {
                 globalConsumer.seek(entry.getKey(), entry.getValue());
             }
-            lastFlush = time.milliseconds();
         }
 
         void pollAndUpdate() {
@@ -267,11 +258,7 @@ void pollAndUpdate() {
             for (final ConsumerRecord record : received) {
                 stateMaintainer.update(record);
             }
-            final long now = time.milliseconds();
-            if (now - flushInterval >= lastFlush) {
-                stateMaintainer.flushState();
-                lastFlush = now;
-            }
+            stateMaintainer.maybeCheckpoint();
         }
 
         public void close(final boolean wipeStateStore) throws IOException {
@@ -418,11 +405,11 @@ private StateConsumer initialize() {
                     topology,
                     globalProcessorContext,
                     stateMgr,
-                    config.defaultDeserializationExceptionHandler()
+                    config.defaultDeserializationExceptionHandler(),
+                    time,
+                    config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG)
                 ),
-                time,
-                Duration.ofMillis(config.getLong(StreamsConfig.POLL_MS_CONFIG)),
-                config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG)
+                Duration.ofMillis(config.getLong(StreamsConfig.POLL_MS_CONFIG))
             );
 
             try {
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java
index 31be9dc2a4..af5dc68103 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java
@@ -25,6 +25,7 @@
 import org.apache.kafka.common.serialization.LongSerializer;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.MockTime;
 import org.apache.kafka.common.utils.Utils;
 import org.apache.kafka.streams.errors.LogAndContinueExceptionHandler;
 import org.apache.kafka.streams.errors.LogAndFailExceptionHandler;
@@ -46,8 +47,6 @@
 
 import static java.util.Arrays.asList;
 import static org.apache.kafka.streams.processor.internals.testutil.ConsumerRecordUtil.record;
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -71,8 +70,12 @@ public class GlobalStateTaskTest {
     private final MockProcessorNode processorTwo = new MockProcessorNode<>();
 
     private final Map offsets = new HashMap<>();
-    private File testDirectory = TestUtils.tempDirectory("global-store");
+    private final File testDirectory = TestUtils.tempDirectory("global-store");
     private final NoOpProcessorContext context = new NoOpProcessorContext();
+    private final MockTime time = new MockTime();
+    private final long flushInterval = 1000L;
+    private final long currentOffsetT1 = 50;
+    private final long currentOffsetT2 = 100;
 
     private ProcessorTopology topology;
     private GlobalStateManagerStub stateMgr;
@@ -101,7 +104,9 @@ public void before() {
             topology,
             context,
             stateMgr,
-            new LogAndFailExceptionHandler()
+            new LogAndFailExceptionHandler(),
+            time,
+            flushInterval
         );
     }
 
@@ -188,7 +193,9 @@ public void shouldNotThrowStreamsExceptionWhenKeyDeserializationFailsWithSkipHan
             topology,
             context,
             stateMgr,
-            new LogAndContinueExceptionHandler()
+            new LogAndContinueExceptionHandler(),
+            time,
+            flushInterval
         );
         final byte[] key = new LongSerializer().serialize(topic2, 1L);
         final byte[] recordValue = new IntegerSerializer().serialize(topic2, 10);
@@ -203,7 +210,9 @@ public void shouldNotThrowStreamsExceptionWhenValueDeserializationFails() {
             topology,
             context,
             stateMgr,
-            new LogAndContinueExceptionHandler()
+            new LogAndContinueExceptionHandler(),
+            time,
+            flushInterval
         );
         final byte[] key = new IntegerSerializer().serialize(topic2, 1);
         final byte[] recordValue = new LongSerializer().serialize(topic2, 10L);
@@ -217,10 +226,13 @@ public void shouldFlushStateManagerWithOffsets() {
         final Map expectedOffsets = new HashMap<>();
         expectedOffsets.put(t1, 52L);
         expectedOffsets.put(t2, 100L);
+
         globalStateTask.initialize();
-        globalStateTask.update(record(topic1, 1, 51, "foo".getBytes(), "foo".getBytes()));
+        globalStateTask.update(record(topic1, 1, currentOffsetT1 + 1, "foo".getBytes(), "foo".getBytes()));
         globalStateTask.flushState();
+
         assertEquals(expectedOffsets, stateMgr.changelogOffsets());
+        assertTrue(stateMgr.flushed);
     }
 
     @Test
@@ -228,12 +240,93 @@ public void shouldCheckpointOffsetsWhenStateIsFlushed() {
         final Map expectedOffsets = new HashMap<>();
         expectedOffsets.put(t1, 102L);
         expectedOffsets.put(t2, 100L);
+
         globalStateTask.initialize();
-        globalStateTask.update(record(topic1, 1, 101, "foo".getBytes(), "foo".getBytes()));
+        globalStateTask.update(record(topic1, 1, currentOffsetT1 + 51L, "foo".getBytes(), "foo".getBytes()));
         globalStateTask.flushState();
-        assertThat(stateMgr.changelogOffsets(), equalTo(expectedOffsets));
+
+        assertEquals(expectedOffsets, stateMgr.changelogOffsets());
+        assertTrue(stateMgr.checkpointWritten);
+    }
+
+    @Test
+    public void shouldNotCheckpointIfNotReceivedEnoughRecords() {
+        globalStateTask.initialize();
+        globalStateTask.update(record(topic1, 1, currentOffsetT1 + 9000L, "foo".getBytes(), "foo".getBytes()));
+        time.sleep(flushInterval); // flush interval elapsed
+        globalStateTask.maybeCheckpoint();
+
+        assertEquals(offsets, stateMgr.changelogOffsets());
+        assertFalse(stateMgr.flushed);
+        assertFalse(stateMgr.checkpointWritten);
+    }
+
+    @Test
+    public void shouldNotCheckpointWhenFlushIntervalHasNotLapsed() {
+        globalStateTask.initialize();
+
+        // offset delta exceeded
+        globalStateTask.update(record(topic1, 1, currentOffsetT1 + 10000L, "foo".getBytes(), "foo".getBytes()));
+
+        time.sleep(flushInterval / 2);
+        globalStateTask.maybeCheckpoint();
+
+        assertEquals(offsets, stateMgr.changelogOffsets());
+        assertFalse(stateMgr.flushed);
+        assertFalse(stateMgr.checkpointWritten);
+    }
+
+    @Test
+    public void shouldCheckpointIfReceivedEnoughRecordsAndFlushIntervalHasElapsed() {
+        final Map expectedOffsets = new HashMap<>();
+        expectedOffsets.put(t1, 10051L); // topic1 advanced with 10001 records
+        expectedOffsets.put(t2, 100L);
+
+        globalStateTask.initialize();
+
+        time.sleep(flushInterval); // flush interval elapsed
+
+        // 10000 records received since last flush => do not flush
+        globalStateTask.update(record(topic1, 1, currentOffsetT1 + 9999L, "foo".getBytes(), "foo".getBytes()));
+        globalStateTask.maybeCheckpoint();
+
+        assertEquals(offsets, stateMgr.changelogOffsets());
+        assertFalse(stateMgr.flushed);
+        assertFalse(stateMgr.checkpointWritten);
+
+        // 1 more record received => triggers the flush
+        globalStateTask.update(record(topic1, 1, currentOffsetT1 + 10000L, "foo".getBytes(), "foo".getBytes()));
+        globalStateTask.maybeCheckpoint();
+
+        assertEquals(expectedOffsets, stateMgr.changelogOffsets());
+        assertTrue(stateMgr.flushed);
+        assertTrue(stateMgr.checkpointWritten);
     }
 
+    @Test
+    public void shouldCheckpointIfReceivedEnoughRecordsFromMultipleTopicsAndFlushIntervalElapsed() {
+        final byte[] integerBytes = new IntegerSerializer().serialize(topic2, 1);
+
+        final Map expectedOffsets = new HashMap<>();
+        expectedOffsets.put(t1, 9050L); // topic1 advanced with 9000 records
+        expectedOffsets.put(t2, 1101L); // topic2 advanced with 1001 records
+
+        globalStateTask.initialize();
+
+        time.sleep(flushInterval);
+
+        // received 9000 records in topic1
+        globalStateTask.update(record(topic1, 1, currentOffsetT1 + 8999L, "foo".getBytes(), "foo".getBytes()));
+        // received 1001 records in topic2
+        globalStateTask.update(record(topic2, 1, currentOffsetT2 + 1000L, integerBytes, integerBytes));
+        globalStateTask.maybeCheckpoint();
+
+        assertEquals(expectedOffsets, stateMgr.changelogOffsets());
+        assertTrue(stateMgr.flushed);
+        assertTrue(stateMgr.checkpointWritten);
+    }
+
+
     @Test
     public void shouldWipeGlobalStateDirectory() throws Exception {
         assertTrue(stateMgr.baseDir().exists());
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateConsumerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateConsumerTest.java
index 1f98eb456d..5e57939483 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateConsumerTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateConsumerTest.java
@@ -21,7 +21,6 @@
 import org.apache.kafka.clients.consumer.OffsetResetStrategy;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.utils.LogContext;
-import org.apache.kafka.common.utils.MockTime;
 import org.apache.kafka.common.utils.Utils;
 import org.junit.Before;
 import org.junit.Test;
@@ -32,16 +31,13 @@
 import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 
 public class StateConsumerTest {
 
-    private static final long FLUSH_INTERVAL = 1000L;
     private final TopicPartition topicOne = new TopicPartition("topic-one", 1);
     private final TopicPartition topicTwo = new TopicPartition("topic-two", 1);
-    private final MockTime time = new MockTime();
     private final MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST);
     private final Map partitionOffsets = new HashMap<>();
     private final LogContext logContext = new LogContext("test ");
@@ -53,7 +49,7 @@ public void setUp() {
         partitionOffsets.put(topicOne, 20L);
         partitionOffsets.put(topicTwo, 30L);
         stateMaintainer = new TaskStub(partitionOffsets);
-        stateConsumer = new GlobalStreamThread.StateConsumer(logContext, consumer, stateMaintainer, time, Duration.ofMillis(10L), FLUSH_INTERVAL);
+        stateConsumer = new GlobalStreamThread.StateConsumer(logContext, consumer, stateMaintainer, Duration.ofMillis(10L));
     }
 
     @Test
@@ -76,6 +72,7 @@ public void shouldUpdateStateWithReceivedRecordsForPartition() {
         consumer.addRecord(new ConsumerRecord<>("topic-one", 1, 21L, new byte[0], new byte[0]));
         stateConsumer.pollAndUpdate();
         assertEquals(2, stateMaintainer.updatedPartitions.get(topicOne).intValue());
+        assertTrue(stateMaintainer.flushed);
     }
 
     @Test
@@ -87,27 +84,9 @@ public void shouldUpdateStateWithReceivedRecordsForAllTopicPartition() {
         stateConsumer.pollAndUpdate();
         assertEquals(1, stateMaintainer.updatedPartitions.get(topicOne).intValue());
         assertEquals(2, stateMaintainer.updatedPartitions.get(topicTwo).intValue());
-    }
-
-    @Test
-    public void shouldFlushStoreWhenFlushIntervalHasLapsed() {
-        stateConsumer.initialize();
-        consumer.addRecord(new ConsumerRecord<>("topic-one", 1, 20L, new byte[0], new byte[0]));
-        time.sleep(FLUSH_INTERVAL);
-
-        stateConsumer.pollAndUpdate();
         assertTrue(stateMaintainer.flushed);
     }
 
-    @Test
-    public void shouldNotFlushOffsetsWhenFlushIntervalHasNotLapsed() {
-        stateConsumer.initialize();
-        consumer.addRecord(new ConsumerRecord<>("topic-one", 1, 20L, new byte[0], new byte[0]));
-        time.sleep(FLUSH_INTERVAL / 2);
-        stateConsumer.pollAndUpdate();
-        assertFalse(stateMaintainer.flushed);
-    }
-
     @Test
     public void shouldCloseConsumer() throws IOException {
         stateConsumer.close(false);
@@ -161,6 +140,10 @@ public void update(final ConsumerRecord record) {
             updatedPartitions.put(tp, updatedPartitions.get(tp) + 1);
         }
 
+        @Override
+        public void maybeCheckpoint() {
+            flushState();
+        }
     }
 
-}
\ No newline at end of file
+}
diff --git a/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java b/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java
index d34b3c8029..3031649944 100644
--- a/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java
+++ b/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java
@@ -35,6 +35,8 @@ public class GlobalStateManagerStub implements GlobalStateManager {
     private final File baseDirectory;
     public boolean initialized;
     public boolean closed;
+    public boolean flushed;
+    public boolean checkpointWritten;
 
     public GlobalStateManagerStub(final Set storeNames,
                                   final Map offsets,
@@ -64,7 +66,9 @@ public void registerStore(final StateStore store,
                               final CommitCallback checkpoint) {}
 
     @Override
-    public void flush() {}
+    public void flush() {
+        flushed = true;
+    }
 
     @Override
     public void close() {
@@ -77,7 +81,9 @@ public void updateChangelogOffsets(final Map writtenOffset
     }
 
     @Override
-    public void checkpoint() {}
+    public void checkpoint() {
+        checkpointWritten = true;
+    }
 
     @Override
     public StateStore getStore(final String name) {
diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java
index 20abaa5407..5767ed9d20 100644
--- a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java
+++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java
@@ -459,7 +459,9 @@ private void setupGlobalTask(final Time mockWallClockTime,
                 globalTopology,
                 globalProcessorContext,
                 globalStateManager,
-                new LogAndContinueExceptionHandler()
+                new LogAndContinueExceptionHandler(),
+                mockWallClockTime,
+                streamsConfig.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG)
             );
             globalStateTask.initialize();
             globalProcessorContext.setRecordContext(null);

From 21a888c4ca0189924a82e4e4ee4909b214c4c9bc Mon Sep 17 00:00:00 2001
From: "Gyeongwon, Do" 
Date: Mon, 4 Mar 2024 20:36:46 +0900
Subject: [PATCH 017/521] MINOR: Updating comments to match the code (#15388)

This comment was added by #12862

The method with the comment was originally named updateLastSend, but its name was later changed to onSendAttempt.
This method doesn't increment numAttempts.

It seems that the numAttempts is only modified after a Request succeeds or fails.

Reviewers: Chia-Ping Tsai 
---
 .../apache/kafka/clients/consumer/internals/RequestState.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestState.java
index 321be5e8fe..a888e7831a 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestState.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestState.java
@@ -102,7 +102,7 @@ public boolean requestInFlight() {
     }
 
     public void onSendAttempt(final long currentTimeMs) {
-        // Here we update the timer everytime we try to send a request. Also increment number of attempts.
+        // Here we update the timer everytime we try to send a request.
         this.lastSentMs = currentTimeMs;
     }
 

From aa0443eb607c2d1d3004312f55f7583102127cb8 Mon Sep 17 00:00:00 2001
From: Bruno Cadonna 
Date: Mon, 4 Mar 2024 12:42:24 +0100
Subject: [PATCH 018/521] KAFKA-16285: Make group metadata available when a new
 assignment is set (#15426)

Currently, in the async Kafka consumer updates to the group metadata
that are received by the heartbeat are propagated to the application thread
in form of an event. Group metadata is updated when a new assignment is
received. The new assignment is directly set in the subscription without
sending an update event from the background thread to the application thread.
That means that there might be a delay between the application thread being
aware of the update to the assignment and the application thread being
aware of the update to the group metadata. This delay can cause stale
group metadata returned by the application thread that then causes
issues when data of the new assignment is committed. A concrete
example is
producer.sendOffsetsToTransaction(offsetsToCommit, groupMetadata)
The offsets to commit might already stem from the new assignment
but the group metadata might relate to the previous assignment.

Reviewers: Kirk True , Andrew Schofield , Lucas Brutschy 
---
 .../internals/AsyncKafkaConsumer.java         | 118 ++++----
 .../internals/HeartbeatRequestManager.java    |  18 --
 .../internals/MembershipManagerImpl.java      |   6 +
 .../consumer/internals/RequestManagers.java   |   4 +-
 .../internals/events/BackgroundEvent.java     |   2 +-
 .../events/GroupMetadataUpdateEvent.java      |  53 ----
 .../internals/AsyncKafkaConsumerTest.java     | 271 +++++-------------
 .../HeartbeatRequestManagerTest.java          |  87 ------
 .../internals/RequestManagersTest.java        |  70 +++++
 .../java/org/apache/kafka/test/TestUtils.java |   9 +
 10 files changed, 227 insertions(+), 411 deletions(-)
 delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java
 create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java

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 e706898b70..dafd3293c7 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
@@ -52,7 +52,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.GroupMetadataUpdateEvent;
 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;
@@ -213,10 +212,6 @@ public void process(final BackgroundEvent event) {
                     process((ErrorEvent) event);
                     break;
 
-                case GROUP_METADATA_UPDATE:
-                    process((GroupMetadataUpdateEvent) event);
-                    break;
-
                 case CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED:
                     process((ConsumerRebalanceListenerCallbackNeededEvent) event);
                     break;
@@ -231,18 +226,6 @@ private void process(final ErrorEvent event) {
             throw event.error();
         }
 
-        private void process(final GroupMetadataUpdateEvent event) {
-            if (AsyncKafkaConsumer.this.groupMetadata.isPresent()) {
-                final ConsumerGroupMetadata currentGroupMetadata = AsyncKafkaConsumer.this.groupMetadata.get();
-                AsyncKafkaConsumer.this.groupMetadata = Optional.of(new ConsumerGroupMetadata(
-                    currentGroupMetadata.groupId(),
-                    event.memberEpoch(),
-                    event.memberId(),
-                    currentGroupMetadata.groupInstanceId()
-                ));
-            }
-        }
-
         private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
             ApplicationEvent invokedEvent = invokeRebalanceCallbacks(
                 rebalanceListenerInvoker,
@@ -256,7 +239,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
 
     private final ApplicationEventHandler applicationEventHandler;
     private final Time time;
-    private Optional groupMetadata = Optional.empty();
+    private final AtomicReference> groupMetadata = new AtomicReference<>(Optional.empty());
     private final KafkaConsumerMetrics kafkaConsumerMetrics;
     private Logger log;
     private final String clientId;
@@ -370,6 +353,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
                     fetchMetricsManager,
                     clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null));
             this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors);
+            this.groupMetadata.set(initializeGroupMetadata(config, groupRebalanceConfig));
             final Supplier requestManagersSupplier = RequestManagers.supplier(time,
                     logContext,
                     backgroundEventHandler,
@@ -383,7 +367,9 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
                     networkClientDelegateSupplier,
                     clientTelemetryReporter,
                     metrics,
-                    offsetCommitCallbackInvoker);
+                    offsetCommitCallbackInvoker,
+                    this::updateGroupMetadata
+            );
             final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext,
                     metadata,
                     applicationEventQueue,
@@ -413,8 +399,6 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
                     config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId))
             );
 
-            this.groupMetadata = initializeGroupMetadata(config, groupRebalanceConfig);
-
             // The FetchCollector is only used on the application thread.
             this.fetchCollector = fetchCollectorFactory.build(logContext,
                     metadata,
@@ -426,7 +410,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
 
             this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, CONSUMER_METRIC_GROUP_PREFIX);
 
-            if (groupMetadata.isPresent() &&
+            if (groupMetadata.get().isPresent() &&
                 GroupProtocol.of(config.getString(ConsumerConfig.GROUP_PROTOCOL_CONFIG)) == GroupProtocol.CONSUMER) {
                 config.ignore(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG); // Used by background thread
             }
@@ -478,7 +462,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
                 rebalanceListenerInvoker
         );
         this.metrics = metrics;
-        this.groupMetadata = initializeGroupMetadata(groupId, Optional.empty());
+        this.groupMetadata.set(initializeGroupMetadata(groupId, Optional.empty()));
         this.metadata = metadata;
         this.retryBackoffMs = retryBackoffMs;
         this.defaultApiTimeoutMs = defaultApiTimeoutMs;
@@ -532,7 +516,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
             GroupRebalanceConfig.ProtocolType.CONSUMER
         );
 
-        this.groupMetadata = initializeGroupMetadata(config, groupRebalanceConfig);
+        this.groupMetadata.set(initializeGroupMetadata(config, groupRebalanceConfig));
 
         BlockingQueue applicationEventQueue = new LinkedBlockingQueue<>();
         BlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>();
@@ -568,7 +552,8 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
             networkClientDelegateSupplier,
             clientTelemetryReporter,
             metrics,
-            offsetCommitCallbackInvoker
+            offsetCommitCallbackInvoker,
+            this::updateGroupMetadata
         );
         Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(
                 logContext,
@@ -651,17 +636,35 @@ private Optional initializeGroupMetadata(final String gro
                 throw new InvalidGroupIdException("The configured " + ConsumerConfig.GROUP_ID_CONFIG
                     + " should not be an empty string or whitespace.");
             } else {
-                return Optional.of(new ConsumerGroupMetadata(
-                    groupId,
-                    JoinGroupRequest.UNKNOWN_GENERATION_ID,
-                    JoinGroupRequest.UNKNOWN_MEMBER_ID,
-                    groupInstanceId
-                ));
+                return Optional.of(initializeConsumerGroupMetadata(groupId, groupInstanceId));
             }
         }
         return Optional.empty();
     }
 
+    private ConsumerGroupMetadata initializeConsumerGroupMetadata(final String groupId,
+                                                                  final Optional groupInstanceId) {
+        return new ConsumerGroupMetadata(
+            groupId,
+            JoinGroupRequest.UNKNOWN_GENERATION_ID,
+            JoinGroupRequest.UNKNOWN_MEMBER_ID,
+            groupInstanceId
+        );
+    }
+
+    private void updateGroupMetadata(final Optional memberEpoch, final Optional memberId) {
+        groupMetadata.updateAndGet(
+            oldGroupMetadataOptional -> oldGroupMetadataOptional.map(
+                oldGroupMetadata -> new ConsumerGroupMetadata(
+                    oldGroupMetadata.groupId(),
+                    memberEpoch.orElse(oldGroupMetadata.generationId()),
+                    memberId.orElse(oldGroupMetadata.memberId()),
+                    oldGroupMetadata.groupInstanceId()
+                )
+            )
+        );
+    }
+
     /**
      * poll implementation using {@link ApplicationEventHandler}.
      *  1. Poll for background events. If there's a fetch response event, process the record and return it. If it is
@@ -713,18 +716,14 @@ public ConsumerRecords poll(final Duration timeout) {
                 wakeupTrigger.maybeTriggerWakeup();
 
                 updateAssignmentMetadataIfNeeded(timer);
-                if (isGenerationKnownOrPartitionsUserAssigned()) {
-                    final Fetch fetch = pollForFetches(timer);
-                    if (!fetch.isEmpty()) {
-                        if (fetch.records().isEmpty()) {
-                            log.trace("Returning empty records from `poll()` "
-                                + "since the consumer's position has advanced for at least one topic partition");
-                        }
-
-                        return interceptors.onConsume(new ConsumerRecords<>(fetch.records()));
+                final Fetch fetch = pollForFetches(timer);
+                if (!fetch.isEmpty()) {
+                    if (fetch.records().isEmpty()) {
+                        log.trace("Returning empty records from `poll()` "
+                            + "since the consumer's position has advanced for at least one topic partition");
                     }
-                } else {
-                    timer.update();
+
+                    return interceptors.onConsume(new ConsumerRecords<>(fetch.records()));
                 }
                 // We will wait for retryBackoffMs
             } while (timer.notExpired());
@@ -736,13 +735,6 @@ public ConsumerRecords poll(final Duration timeout) {
         }
     }
 
-    private boolean isGenerationKnownOrPartitionsUserAssigned() {
-        if (subscriptions.hasAutoAssignedPartitions()) {
-            return groupMetadata.filter(g -> g.generationId() != JoinGroupRequest.UNKNOWN_GENERATION_ID).isPresent();
-        }
-        return true;
-    }
-
     /**
      * Commit offsets returned on the last {@link #poll(Duration) poll()} for all the subscribed list of topics and
      * partitions.
@@ -960,7 +952,7 @@ public Map committed(final Set firstException) {
-        if (!groupMetadata.isPresent())
+        if (!groupMetadata.get().isPresent())
             return;
         maybeAutoCommitSync(autoCommitEnabled, timer);
         applicationEventHandler.add(new CommitOnCloseEvent());
@@ -1463,7 +1455,7 @@ public void unsubscribe() {
         acquireAndEnsureOpen();
         try {
             fetchBuffer.retainAll(Collections.emptySet());
-            if (groupMetadata.isPresent()) {
+            if (groupMetadata.get().isPresent()) {
                 UnsubscribeEvent unsubscribeEvent = new UnsubscribeEvent();
                 applicationEventHandler.add(unsubscribeEvent);
                 log.info("Unsubscribing all topics or patterns and assigned partitions");
@@ -1475,7 +1467,7 @@ public void unsubscribe() {
                 } catch (TimeoutException e) {
                     log.error("Failed while waiting for the unsubscribe event to complete");
                 }
-                groupMetadata = initializeGroupMetadata(groupMetadata.get().groupId(), Optional.empty());
+                resetGroupMetadata();
             }
             subscriptions.unsubscribe();
         } finally {
@@ -1483,6 +1475,16 @@ public void unsubscribe() {
         }
     }
 
+    private void resetGroupMetadata() {
+        groupMetadata.updateAndGet(
+            oldGroupMetadataOptional -> oldGroupMetadataOptional
+                .map(oldGroupMetadata -> initializeConsumerGroupMetadata(
+                    oldGroupMetadata.groupId(),
+                    oldGroupMetadata.groupInstanceId()
+                ))
+        );
+    }
+
     @Override
     @Deprecated
     public ConsumerRecords poll(final long timeoutMs) {
@@ -1604,7 +1606,7 @@ private boolean updateFetchPositions(final Timer timer) {
      * according to config {@link CommonClientConfigs#GROUP_ID_CONFIG}
      */
     private boolean isCommittedOffsetsManagementEnabled() {
-        return groupMetadata.isPresent();
+        return groupMetadata.get().isPresent();
     }
 
     /**
@@ -1905,12 +1907,12 @@ public KafkaConsumerMetrics kafkaConsumerMetrics() {
     private void maybeThrowFencedInstanceException() {
         if (offsetCommitCallbackInvoker.hasFencedException()) {
             String groupInstanceId = "unknown";
-            if (!groupMetadata.isPresent()) {
+            if (!groupMetadata.get().isPresent()) {
                 log.error("No group metadata found although a group ID was provided. This is a bug!");
-            } else if (!groupMetadata.get().groupInstanceId().isPresent()) {
+            } else if (!groupMetadata.get().get().groupInstanceId().isPresent()) {
                 log.error("No group instance ID found although the consumer is fenced. This is a bug!");
             } else {
-                groupInstanceId = groupMetadata.get().groupInstanceId().get();
+                groupInstanceId = groupMetadata.get().get().groupInstanceId().get();
             }
             throw new FencedInstanceIdException("Get fenced exception for group.instance.id " + groupInstanceId);
         }
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 826774a6a6..d2a7205d5f 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
@@ -22,7 +22,6 @@
 import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor;
 import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler;
 import org.apache.kafka.clients.consumer.internals.events.ErrorEvent;
-import org.apache.kafka.clients.consumer.internals.events.GroupMetadataUpdateEvent;
 import org.apache.kafka.clients.consumer.internals.metrics.HeartbeatMetricsManager;
 import org.apache.kafka.common.Uuid;
 import org.apache.kafka.common.errors.GroupAuthorizationException;
@@ -110,7 +109,6 @@ public class HeartbeatRequestManager implements RequestManager {
      * sending heartbeat until the next poll.
      */
     private final Timer pollTimer;
-    private GroupMetadataUpdateEvent previousGroupMetadataUpdateEvent = null;
 
     /**
      * Holding the heartbeat sensor to measure heartbeat timing and response latency
@@ -328,27 +326,11 @@ private void onResponse(final ConsumerGroupHeartbeatResponse response, long curr
             heartbeatRequestState.onSuccessfulAttempt(currentTimeMs);
             heartbeatRequestState.resetTimer();
             membershipManager.onHeartbeatSuccess(response.data());
-            maybeSendGroupMetadataUpdateEvent();
             return;
         }
         onErrorResponse(response, currentTimeMs);
     }
 
-    private void maybeSendGroupMetadataUpdateEvent() {
-        if (previousGroupMetadataUpdateEvent == null ||
-            !previousGroupMetadataUpdateEvent.memberId().equals(membershipManager.memberId()) ||
-            previousGroupMetadataUpdateEvent.memberEpoch() != membershipManager.memberEpoch()) {
-
-            final GroupMetadataUpdateEvent currentGroupMetadataUpdateEvent = new GroupMetadataUpdateEvent(
-                membershipManager.memberEpoch(),
-                previousGroupMetadataUpdateEvent != null && membershipManager.memberId() == null ?
-                    previousGroupMetadataUpdateEvent.memberId() : membershipManager.memberId()
-            );
-            this.backgroundEventHandler.add(currentGroupMetadataUpdateEvent);
-            previousGroupMetadataUpdateEvent = currentGroupMetadataUpdateEvent;
-        }
-    }
-
     private void onErrorResponse(final ConsumerGroupHeartbeatResponse response,
                                  final long currentTimeMs) {
         Errors error = Errors.forCode(response.data().errorCode());
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 35322fb51b..c2bdd3f860 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
@@ -57,6 +57,7 @@
 import java.util.concurrent.CompletableFuture;
 import java.util.stream.Collectors;
 
+import static java.util.Collections.unmodifiableList;
 import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED;
 import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.ON_PARTITIONS_LOST;
 import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED;
@@ -1491,4 +1492,9 @@ public PollResult poll(final long currentTimeMs) {
         }
         return PollResult.EMPTY;
     }
+
+    // visible for testing
+    List stateListeners() {
+        return unmodifiableList(stateUpdatesListeners);
+    }
 }
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 0b4c043d4a..75d87432db 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
@@ -123,7 +123,8 @@ public static Supplier supplier(final Time time,
                                                      final Supplier networkClientDelegateSupplier,
                                                      final Optional clientTelemetryReporter,
                                                      final Metrics metrics,
-                                                     final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker
+                                                     final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker,
+                                                     final MemberStateListener applicationThreadMemberStateListener
                                                      ) {
         return new CachedSupplier() {
             @Override
@@ -192,6 +193,7 @@ protected RequestManagers create() {
                             time,
                             metrics);
                     membershipManager.registerStateListener(commit);
+                    membershipManager.registerStateListener(applicationThreadMemberStateListener);
                     heartbeatRequestManager = new HeartbeatRequestManager(
                             logContext,
                             time,
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java
index 9bc3fbebc3..4241482bca 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java
@@ -27,7 +27,7 @@
 public abstract class BackgroundEvent {
 
     public enum Type {
-        ERROR, CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED, GROUP_METADATA_UPDATE
+        ERROR, CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED
     }
 
     private final Type type;
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java
deleted file mode 100644
index 001f549818..0000000000
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java
+++ /dev/null
@@ -1,53 +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 org.apache.kafka.clients.consumer.internals.events;
-
-import org.apache.kafka.clients.consumer.Consumer;
-import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread;
-
-/**
- * This event is sent by the {@link ConsumerNetworkThread consumer's network thread} to the application thread
- * so that when the user calls the {@link Consumer#groupMetadata()} API, the information is up-to-date. The
- * information for the current state of the group member is managed on the consumer network thread and thus
- * requires this interplay between threads.
- */
-public class GroupMetadataUpdateEvent extends BackgroundEvent {
-
-    private final int memberEpoch;
-    private final String memberId;
-
-    public GroupMetadataUpdateEvent(final int memberEpoch, final String memberId) {
-        super(Type.GROUP_METADATA_UPDATE);
-        this.memberEpoch = memberEpoch;
-        this.memberId = memberId;
-    }
-
-    public int memberEpoch() {
-        return memberEpoch;
-    }
-
-    public String memberId() {
-        return memberId;
-    }
-
-    @Override
-    public String toStringBase() {
-        return super.toStringBase() +
-            ", memberEpoch=" + memberEpoch +
-            ", memberId='" + memberId + '\'';
-    }
-}
\ No newline at end of file
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 35e742fbd0..fe83a41789 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
@@ -42,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.GroupMetadataUpdateEvent;
 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;
@@ -52,7 +51,6 @@
 import org.apache.kafka.clients.consumer.internals.events.SyncCommitEvent;
 import org.apache.kafka.clients.consumer.internals.events.UnsubscribeEvent;
 import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsEvent;
-import org.apache.kafka.common.Cluster;
 import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.Metric;
 import org.apache.kafka.common.Node;
@@ -80,6 +78,7 @@
 import org.junit.jupiter.params.provider.MethodSource;
 import org.mockito.ArgumentCaptor;
 import org.mockito.ArgumentMatchers;
+import org.mockito.MockedStatic;
 import org.mockito.Mockito;
 
 import java.time.Duration;
@@ -102,7 +101,6 @@
 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.Collectors;
 import java.util.stream.Stream;
 
@@ -117,10 +115,12 @@
 import static org.apache.kafka.clients.consumer.internals.MembershipManagerImpl.TOPIC_PARTITION_COMPARATOR;
 import static org.apache.kafka.common.utils.Utils.mkEntry;
 import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.apache.kafka.test.TestUtils.requiredConsumerConfig;
 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.assertInstanceOf;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertSame;
@@ -133,6 +133,7 @@
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.mockStatic;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
@@ -164,19 +165,19 @@ public void resetAll() {
     }
 
     private AsyncKafkaConsumer newConsumer() {
-        final Properties props = requiredConsumerProperties();
+        final Properties props = requiredConsumerConfig();
         props.put(ConsumerConfig.GROUP_ID_CONFIG, "group-id");
         return newConsumer(props);
     }
 
     private AsyncKafkaConsumer newConsumerWithoutGroupId() {
-        final Properties props = requiredConsumerProperties();
+        final Properties props = requiredConsumerConfig();
         return newConsumer(props);
     }
 
     @SuppressWarnings("UnusedReturnValue")
     private AsyncKafkaConsumer newConsumerWithEmptyGroupId() {
-        final Properties props = requiredConsumerPropertiesAndGroupId("");
+        final Properties props = requiredConsumerConfigAndGroupId("");
         return newConsumer(props);
     }
 
@@ -921,7 +922,7 @@ public void testNoWakeupInCloseCommit() {
 
     @Test
     public void testInterceptorAutoCommitOnClose() {
-        Properties props = requiredConsumerPropertiesAndGroupId("test-id");
+        Properties props = requiredConsumerConfigAndGroupId("test-id");
         props.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, MockConsumerInterceptor.class.getName());
         props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true");
 
@@ -937,7 +938,7 @@ public void testInterceptorAutoCommitOnClose() {
 
     @Test
     public void testInterceptorCommitSync() {
-        Properties props = requiredConsumerPropertiesAndGroupId("test-id");
+        Properties props = requiredConsumerConfigAndGroupId("test-id");
         props.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, MockConsumerInterceptor.class.getName());
         props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
 
@@ -952,7 +953,7 @@ public void testInterceptorCommitSync() {
 
     @Test
     public void testNoInterceptorCommitSyncFailed() {
-        Properties props = requiredConsumerPropertiesAndGroupId("test-id");
+        Properties props = requiredConsumerConfigAndGroupId("test-id");
         props.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, MockConsumerInterceptor.class.getName());
         props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
 
@@ -968,7 +969,7 @@ public void testNoInterceptorCommitSyncFailed() {
 
     @Test
     public void testInterceptorCommitAsync() {
-        Properties props = requiredConsumerPropertiesAndGroupId("test-id");
+        Properties props = requiredConsumerConfigAndGroupId("test-id");
         props.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, MockConsumerInterceptor.class.getName());
         props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
 
@@ -985,7 +986,7 @@ public void testInterceptorCommitAsync() {
 
     @Test
     public void testNoInterceptorCommitAsyncFailed() {
-        Properties props = requiredConsumerPropertiesAndGroupId("test-id");
+        Properties props = requiredConsumerConfigAndGroupId("test-id");
         props.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, MockConsumerInterceptor.class.getName());
         props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
 
@@ -1085,7 +1086,7 @@ public void testSubscriptionOnEmptyTopic() {
 
     @Test
     public void testGroupMetadataAfterCreationWithGroupIdIsNull() {
-        final Properties props = requiredConsumerProperties();
+        final Properties props = requiredConsumerConfig();
         final ConsumerConfig config = new ConsumerConfig(props);
         consumer = newConsumer(config);
 
@@ -1102,7 +1103,7 @@ public void testGroupMetadataAfterCreationWithGroupIdIsNull() {
     @Test
     public void testGroupMetadataAfterCreationWithGroupIdIsNotNull() {
         final String groupId = "consumerGroupA";
-        final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId));
+        final ConsumerConfig config = new ConsumerConfig(requiredConsumerConfigAndGroupId(groupId));
         consumer = newConsumer(config);
 
         final ConsumerGroupMetadata groupMetadata = consumer.groupMetadata();
@@ -1117,7 +1118,7 @@ public void testGroupMetadataAfterCreationWithGroupIdIsNotNull() {
     public void testGroupMetadataAfterCreationWithGroupIdIsNotNullAndGroupInstanceIdSet() {
         final String groupId = "consumerGroupA";
         final String groupInstanceId = "groupInstanceId1";
-        final Properties props = requiredConsumerPropertiesAndGroupId(groupId);
+        final Properties props = requiredConsumerConfigAndGroupId(groupId);
         props.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, groupInstanceId);
         final ConsumerConfig config = new ConsumerConfig(props);
         consumer = newConsumer(config);
@@ -1130,164 +1131,65 @@ public void testGroupMetadataAfterCreationWithGroupIdIsNotNullAndGroupInstanceId
         assertEquals(JoinGroupRequest.UNKNOWN_MEMBER_ID, groupMetadata.memberId());
     }
 
-    @Test
-    public void testGroupMetadataUpdateSingleCall() {
+    private MemberStateListener captureGroupMetadataUpdateListener(final MockedStatic requestManagers) {
+        ArgumentCaptor applicationThreadMemberStateListener = ArgumentCaptor.forClass(MemberStateListener.class);
+        requestManagers.verify(() -> RequestManagers.supplier(
+            any(),
+            any(),
+            any(),
+            any(),
+            any(),
+            any(),
+            any(),
+            any(),
+            any(),
+            any(),
+            any(),
+            any(),
+            any(),
+            any(),
+            applicationThreadMemberStateListener.capture()
+        ));
+        return applicationThreadMemberStateListener.getValue();
+    }
+
+    @Test
+    public void testGroupMetadataUpdate() {
         final String groupId = "consumerGroupA";
-        final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId));
-        consumer = newConsumer(config);
-
-        doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class));
-        completeFetchedCommittedOffsetApplicationEventSuccessfully(mkMap());
-
-        final int generation = 1;
-        final String memberId = "newMemberId";
-        final ConsumerGroupMetadata expectedGroupMetadata = new ConsumerGroupMetadata(
-            groupId,
-            generation,
-            memberId,
-            Optional.empty()
-        );
-        final GroupMetadataUpdateEvent groupMetadataUpdateEvent = new GroupMetadataUpdateEvent(
-            generation,
-            memberId
-        );
-        backgroundEventQueue.add(groupMetadataUpdateEvent);
-        consumer.assign(singletonList(new TopicPartition("topic", 0)));
-        consumer.poll(Duration.ZERO);
-
-        final ConsumerGroupMetadata actualGroupMetadata = consumer.groupMetadata();
-
-        assertEquals(expectedGroupMetadata, actualGroupMetadata);
-
-        final ConsumerGroupMetadata secondActualGroupMetadataWithoutUpdate = consumer.groupMetadata();
-
-        assertEquals(expectedGroupMetadata, secondActualGroupMetadataWithoutUpdate);
-    }
-
-    @Test
-    public void testPollNotReturningRecordsIfGenerationUnknownAndGroupManagementIsUsedWithTopics() {
-        testPollNotReturningRecordsIfGenerationUnknownAndGroupManagementIsUsed(() -> {
-            consumer.subscribe(singletonList("topic"));
-        });
-    }
-
-    @Test
-    public void testPollNotReturningRecordsIfGenerationUnknownAndGroupManagementIsUsedWithPattern() {
-        testPollNotReturningRecordsIfGenerationUnknownAndGroupManagementIsUsed(() -> {
-            when(metadata.fetch()).thenReturn(Cluster.empty());
-            consumer.subscribe(Pattern.compile("topic"));
-        });
-    }
-
-    private void testPollNotReturningRecordsIfGenerationUnknownAndGroupManagementIsUsed(final Runnable subscription) {
-        final String groupId = "consumerGroupA";
-        final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId));
-        consumer = newConsumer(config);
-        subscription.run();
-
-        consumer.poll(Duration.ZERO);
-
-        verify(fetchCollector, never()).collectFetch(any(FetchBuffer.class));
-    }
-
-    @Test
-    public void testPollReturningRecordsIfGroupIdSetAndGroupManagementIsNotUsed() {
-        final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId("consumerGroupA"));
-        testPollReturningRecordsIfGroupMetadataHasUnknownGenerationAndGroupManagementIsNotUsed(config);
-    }
-
-    @Test
-    public void testPollReturningRecordsIfGroupIdNotSetAndGroupManagementIsNotUsed() {
-        final ConsumerConfig config = new ConsumerConfig(requiredConsumerProperties());
-        testPollReturningRecordsIfGroupMetadataHasUnknownGenerationAndGroupManagementIsNotUsed(config);
-    }
-
-    private void testPollReturningRecordsIfGroupMetadataHasUnknownGenerationAndGroupManagementIsNotUsed(final ConsumerConfig config) {
-        final String topic = "topic";
-        final TopicPartition topicPartition = new TopicPartition(topic, 0);
-        consumer = newConsumer(config);
-        consumer.assign(singletonList(topicPartition));
-        final List> records = singletonList(
-            new ConsumerRecord<>(topic, 0, 2, "key1", "value1")
-        );
-        when(fetchCollector.collectFetch(any(FetchBuffer.class)))
-            .thenReturn(Fetch.forPartition(topicPartition, records, true));
-        completeFetchedCommittedOffsetApplicationEventSuccessfully(mkMap());
-
-        consumer.poll(Duration.ZERO);
-
-        verify(fetchCollector).collectFetch(any(FetchBuffer.class));
-    }
-
-    @Test
-    public void testPollReturningRecordIfGenerationKnownAndGroupManagementIsUsedWithTopics() {
-        final String topic = "topic";
-        testPollReturningRecordIfGenerationKnownAndGroupManagementIsUsed(
-            topic,
-            () -> {
-                consumer.subscribe(singletonList(topic));
-            });
-    }
-
-    @Test
-    public void testPollReturningRecordIfGenerationKnownAndGroupManagementIsUsedWithPattern() {
-        final String topic = "topic";
-        testPollReturningRecordIfGenerationKnownAndGroupManagementIsUsed(
-            topic,
-            () -> {
-                when(metadata.fetch()).thenReturn(Cluster.empty());
-                consumer.subscribe(Pattern.compile(topic));
-            });
-    }
-
-    private void testPollReturningRecordIfGenerationKnownAndGroupManagementIsUsed(final String topic,
-                                                                                  final Runnable subscription) {
-        final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId("consumerGroupA"));
-        final int generation = 1;
-        final String memberId = "newMemberId";
-        final GroupMetadataUpdateEvent groupMetadataUpdateEvent = new GroupMetadataUpdateEvent(
-            generation,
-            memberId
-        );
-        backgroundEventQueue.add(groupMetadataUpdateEvent);
-        final TopicPartition topicPartition = new TopicPartition(topic, 0);
-        final List> records = singletonList(
-            new ConsumerRecord<>(topic, 0, 2, "key1", "value1")
-        );
-        when(fetchCollector.collectFetch(any(FetchBuffer.class)))
-            .thenReturn(Fetch.forPartition(topicPartition, records, true));
-        consumer = newConsumer(config);
-        subscription.run();
-
-        consumer.poll(Duration.ZERO);
-
-        verify(fetchCollector).collectFetch(any(FetchBuffer.class));
+        final ConsumerConfig config = new ConsumerConfig(requiredConsumerConfigAndGroupId(groupId));
+        try (final MockedStatic requestManagers = mockStatic(RequestManagers.class)) {
+            consumer = newConsumer(config);
+            final ConsumerGroupMetadata oldGroupMetadata = consumer.groupMetadata();
+            final MemberStateListener groupMetadataUpdateListener = captureGroupMetadataUpdateListener(requestManagers);
+            final int expectedMemberEpoch = 42;
+            final String expectedMemberId = "memberId";
+            groupMetadataUpdateListener.onMemberEpochUpdated(
+                Optional.of(expectedMemberEpoch),
+                Optional.of(expectedMemberId)
+            );
+            final ConsumerGroupMetadata newGroupMetadata = consumer.groupMetadata();
+            assertEquals(oldGroupMetadata.groupId(), newGroupMetadata.groupId());
+            assertEquals(expectedMemberId, newGroupMetadata.memberId());
+            assertEquals(expectedMemberEpoch, newGroupMetadata.generationId());
+            assertEquals(oldGroupMetadata.groupInstanceId(), newGroupMetadata.groupInstanceId());
+        }
     }
 
     @Test
     public void testGroupMetadataIsResetAfterUnsubscribe() {
         final String groupId = "consumerGroupA";
-        final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId));
-        consumer = newConsumer(config);
-        consumer.subscribe(singletonList("topic"));
-        final int generation = 1;
-        final String memberId = "newMemberId";
-        final ConsumerGroupMetadata groupMetadataAfterSubscription = new ConsumerGroupMetadata(
-            groupId,
-            generation,
-            memberId,
-            Optional.empty()
-        );
-        final GroupMetadataUpdateEvent groupMetadataUpdateEvent = new GroupMetadataUpdateEvent(
-            generation,
-            memberId
-        );
-        backgroundEventQueue.add(groupMetadataUpdateEvent);
-        when(fetchCollector.collectFetch(any(FetchBuffer.class))).thenReturn(Fetch.empty());
-        consumer.poll(Duration.ZERO);
-
-        assertEquals(groupMetadataAfterSubscription, consumer.groupMetadata());
-
+        final ConsumerConfig config = new ConsumerConfig(requiredConsumerConfigAndGroupId(groupId));
+        try (final MockedStatic requestManagers = mockStatic(RequestManagers.class)) {
+            consumer = newConsumer(config);
+            final MemberStateListener groupMetadataUpdateListener = captureGroupMetadataUpdateListener(requestManagers);
+            consumer.subscribe(singletonList("topic"));
+            final int memberEpoch = 42;
+            final String memberId = "memberId";
+            groupMetadataUpdateListener.onMemberEpochUpdated(Optional.of(memberEpoch), Optional.of(memberId));
+            final ConsumerGroupMetadata groupMetadata = consumer.groupMetadata();
+            assertNotEquals(JoinGroupRequest.UNKNOWN_GENERATION_ID, groupMetadata.generationId());
+            assertNotEquals(JoinGroupRequest.UNKNOWN_MEMBER_ID, groupMetadata.memberId());
+        }
         completeUnsubscribeApplicationEventSuccessfully();
 
         consumer.unsubscribe();
@@ -1298,7 +1200,6 @@ public void testGroupMetadataIsResetAfterUnsubscribe() {
             JoinGroupRequest.UNKNOWN_MEMBER_ID,
             Optional.empty()
         );
-
         assertEquals(groupMetadataAfterUnsubscription, consumer.groupMetadata());
     }
 
@@ -1379,7 +1280,7 @@ private static Stream listenerCallbacksInvokeSource() {
     @Test
     public void testBackgroundError() {
         final String groupId = "consumerGroupA";
-        final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId));
+        final ConsumerConfig config = new ConsumerConfig(requiredConsumerConfigAndGroupId(groupId));
         consumer = newConsumer(config);
 
         final KafkaException expectedException = new KafkaException("Nobody expects the Spanish Inquisition");
@@ -1394,7 +1295,7 @@ public void testBackgroundError() {
     @Test
     public void testMultipleBackgroundErrors() {
         final String groupId = "consumerGroupA";
-        final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId));
+        final ConsumerConfig config = new ConsumerConfig(requiredConsumerConfigAndGroupId(groupId));
         consumer = newConsumer(config);
 
         final KafkaException expectedException1 = new KafkaException("Nobody expects the Spanish Inquisition");
@@ -1412,7 +1313,7 @@ public void testMultipleBackgroundErrors() {
 
     @Test
     public void testGroupRemoteAssignorUnusedIfGroupIdUndefined() {
-        final Properties props = requiredConsumerProperties();
+        final Properties props = requiredConsumerConfig();
         props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor");
         final ConsumerConfig config = new ConsumerConfig(props);
         consumer = newConsumer(config);
@@ -1422,7 +1323,7 @@ public void testGroupRemoteAssignorUnusedIfGroupIdUndefined() {
 
     @Test
     public void testGroupRemoteAssignorUnusedInGenericProtocol() {
-        final Properties props = requiredConsumerProperties();
+        final Properties props = requiredConsumerConfig();
         props.put(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroupA");
         props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name().toLowerCase(Locale.ROOT));
         props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor");
@@ -1434,7 +1335,7 @@ public void testGroupRemoteAssignorUnusedInGenericProtocol() {
 
     @Test
     public void testGroupRemoteAssignorUsedInConsumerProtocol() {
-        final Properties props = requiredConsumerProperties();
+        final Properties props = requiredConsumerConfig();
         props.put(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroupA");
         props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CONSUMER.name().toLowerCase(Locale.ROOT));
         props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor");
@@ -1446,7 +1347,7 @@ public void testGroupRemoteAssignorUsedInConsumerProtocol() {
 
     @Test
     public void testGroupIdNull() {
-        final Properties props = requiredConsumerProperties();
+        final Properties props = requiredConsumerConfig();
         props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 10000);
         props.put(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED, true);
         final ConsumerConfig config = new ConsumerConfig(props);
@@ -1458,7 +1359,7 @@ public void testGroupIdNull() {
 
     @Test
     public void testGroupIdNotNullAndValid() {
-        final Properties props = requiredConsumerPropertiesAndGroupId("consumerGroupA");
+        final Properties props = requiredConsumerConfigAndGroupId("consumerGroupA");
         props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 10000);
         props.put(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED, true);
         final ConsumerConfig config = new ConsumerConfig(props);
@@ -1492,7 +1393,6 @@ public void testEnsurePollEventSentOnConsumerPoll() {
         final TopicPartition tp = new TopicPartition("topic", 0);
         final List> records = singletonList(
                 new ConsumerRecord<>("topic", 0, 2, "key1", "value1"));
-        backgroundEventQueue.add(new GroupMetadataUpdateEvent(1, "memberId"));
         doAnswer(invocation -> Fetch.forPartition(tp, records, true))
                 .when(fetchCollector)
                 .collectFetch(Mockito.any(FetchBuffer.class));
@@ -1503,7 +1403,7 @@ public void testEnsurePollEventSentOnConsumerPoll() {
     }
 
     private void testInvalidGroupId(final String groupId) {
-        final Properties props = requiredConsumerPropertiesAndGroupId(groupId);
+        final Properties props = requiredConsumerConfigAndGroupId(groupId);
         final ConsumerConfig config = new ConsumerConfig(props);
 
         final Exception exception = assertThrows(
@@ -1514,20 +1414,12 @@ private void testInvalidGroupId(final String groupId) {
         assertEquals("Failed to construct kafka consumer", exception.getMessage());
     }
 
-    private Properties requiredConsumerPropertiesAndGroupId(final String groupId) {
-        final Properties props = requiredConsumerProperties();
+    private Properties requiredConsumerConfigAndGroupId(final String groupId) {
+        final Properties props = requiredConsumerConfig();
         props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId);
         return props;
     }
 
-    private Properties requiredConsumerProperties() {
-        final Properties props = new Properties();
-        props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
-        props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
-        props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9091");
-        return props;
-    }
-
     private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean committedOffsetsEnabled) {
         completeFetchedCommittedOffsetApplicationEventExceptionally(new TimeoutException());
         doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class));
@@ -1588,13 +1480,6 @@ public void testLongPollWaitIsLimited() {
             new ConsumerRecord<>(topicName, partition, 3, "key2", "value2")
         );
 
-        final int generation = 1;
-        final String memberId = "newMemberId";
-        final GroupMetadataUpdateEvent groupMetadataUpdateEvent = new GroupMetadataUpdateEvent(
-            generation,
-            memberId
-        );
-        backgroundEventQueue.add(groupMetadataUpdateEvent);
         // On the first iteration, return no data; on the second, return two records
         doAnswer(invocation -> {
             // Mock the subscription being assigned as the first fetch is collected
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 90cfb90bb1..8e05e505be 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
@@ -21,7 +21,6 @@
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent;
 import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler;
-import org.apache.kafka.clients.consumer.internals.events.GroupMetadataUpdateEvent;
 import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.Node;
 import org.apache.kafka.common.Uuid;
@@ -325,91 +324,6 @@ public void testValidateConsumerGroupHeartbeatRequest(final short version) {
         assertEquals(DEFAULT_REMOTE_ASSIGNOR, heartbeatRequest.data().serverAssignor());
     }
 
-    @Test
-    public void testConsumerGroupMetadataFirstUpdate() {
-        final GroupMetadataUpdateEvent groupMetadataUpdateEvent = makeFirstGroupMetadataUpdate(memberId, memberEpoch);
-        assertEquals(memberEpoch, groupMetadataUpdateEvent.memberEpoch());
-        assertEquals(memberId, groupMetadataUpdateEvent.memberId());
-    }
-
-    @Test
-    public void testConsumerGroupMetadataUpdateWithSameUpdate() {
-        makeFirstGroupMetadataUpdate(memberId, memberEpoch);
-
-        time.sleep(2000);
-        NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds());
-
-        assertEquals(1, result.unsentRequests.size());
-        NetworkClientDelegate.UnsentRequest request = result.unsentRequests.get(0);
-        ClientResponse responseWithSameUpdate = createHeartbeatResponse(request, Errors.NONE);
-        request.handler().onComplete(responseWithSameUpdate);
-        assertEquals(0, backgroundEventQueue.size());
-    }
-
-    @Test
-    public void testConsumerGroupMetadataUpdateWithMemberIdNullButMemberEpochUpdated() {
-        makeFirstGroupMetadataUpdate(memberId, memberEpoch);
-
-        time.sleep(2000);
-        NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds());
-
-        assertEquals(1, result.unsentRequests.size());
-        NetworkClientDelegate.UnsentRequest request = result.unsentRequests.get(0);
-        final int updatedMemberEpoch = 2;
-        ClientResponse responseWithMemberEpochUpdate = createHeartbeatResponseWithMemberIdNull(
-            request,
-            Errors.NONE,
-            updatedMemberEpoch
-        );
-        request.handler().onComplete(responseWithMemberEpochUpdate);
-        assertEquals(1, backgroundEventQueue.size());
-        final BackgroundEvent eventWithUpdatedMemberEpoch = backgroundEventQueue.poll();
-        assertEquals(BackgroundEvent.Type.GROUP_METADATA_UPDATE, eventWithUpdatedMemberEpoch.type());
-        final GroupMetadataUpdateEvent groupMetadataUpdateEvent = (GroupMetadataUpdateEvent) eventWithUpdatedMemberEpoch;
-        assertEquals(updatedMemberEpoch, groupMetadataUpdateEvent.memberEpoch());
-        assertEquals(memberId, groupMetadataUpdateEvent.memberId());
-    }
-
-    @Test
-    public void testConsumerGroupMetadataUpdateWithMemberIdUpdatedAndMemberEpochSame() {
-        makeFirstGroupMetadataUpdate(memberId, memberEpoch);
-
-        time.sleep(2000);
-        NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds());
-
-        assertEquals(1, result.unsentRequests.size());
-        NetworkClientDelegate.UnsentRequest request = result.unsentRequests.get(0);
-        final String updatedMemberId = "updatedMemberId";
-        ClientResponse responseWithMemberIdUpdate = createHeartbeatResponse(
-            request,
-            Errors.NONE,
-            updatedMemberId,
-            memberEpoch
-        );
-        request.handler().onComplete(responseWithMemberIdUpdate);
-        assertEquals(1, backgroundEventQueue.size());
-        final BackgroundEvent eventWithUpdatedMemberEpoch = backgroundEventQueue.poll();
-        assertEquals(BackgroundEvent.Type.GROUP_METADATA_UPDATE, eventWithUpdatedMemberEpoch.type());
-        final GroupMetadataUpdateEvent groupMetadataUpdateEvent = (GroupMetadataUpdateEvent) eventWithUpdatedMemberEpoch;
-        assertEquals(memberEpoch, groupMetadataUpdateEvent.memberEpoch());
-        assertEquals(updatedMemberId, groupMetadataUpdateEvent.memberId());
-    }
-
-    private GroupMetadataUpdateEvent makeFirstGroupMetadataUpdate(final String memberId, final int memberEpoch) {
-        resetWithZeroHeartbeatInterval(Optional.empty());
-        mockStableMember();
-        when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999)));
-        NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds());
-        assertEquals(1, result.unsentRequests.size());
-        NetworkClientDelegate.UnsentRequest request = result.unsentRequests.get(0);
-        ClientResponse firstResponse = createHeartbeatResponse(request, Errors.NONE, memberId, memberEpoch);
-        request.handler().onComplete(firstResponse);
-        assertEquals(1, backgroundEventQueue.size());
-        final BackgroundEvent event = backgroundEventQueue.poll();
-        assertEquals(BackgroundEvent.Type.GROUP_METADATA_UPDATE, event.type());
-        return (GroupMetadataUpdateEvent) event;
-    }
-
     @ParameterizedTest
     @MethodSource("errorProvider")
     public void testHeartbeatResponseOnErrorHandling(final Errors error, final boolean isFatal) {
@@ -430,7 +344,6 @@ public void testHeartbeatResponseOnErrorHandling(final Errors error, final boole
 
         switch (error) {
             case NONE:
-                verify(backgroundEventHandler).add(any(GroupMetadataUpdateEvent.class));
                 verify(membershipManager, times(2)).onHeartbeatSuccess(mockResponse.data());
                 assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, heartbeatRequestState.nextHeartbeatMs(time.milliseconds()));
                 break;
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java
new file mode 100644
index 0000000000..640c7e98e4
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java
@@ -0,0 +1,70 @@
+/*
+ * 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.clients.ApiVersions;
+import org.apache.kafka.clients.GroupRebalanceConfig;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.MockTime;
+import org.junit.jupiter.api.Test;
+
+import java.util.Optional;
+import java.util.Properties;
+
+import static org.apache.kafka.test.TestUtils.requiredConsumerConfig;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.mock;
+
+public class RequestManagersTest {
+
+    @Test
+    public void testMemberStateListenerRegistered() {
+
+        final MemberStateListener listener = (memberEpoch, memberId) -> { };
+
+        final Properties properties = requiredConsumerConfig();
+        properties.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroup");
+        final ConsumerConfig config = new ConsumerConfig(properties);
+        final GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig(
+            config,
+            GroupRebalanceConfig.ProtocolType.CONSUMER
+        );
+        final RequestManagers requestManagers = RequestManagers.supplier(
+            new MockTime(),
+            new LogContext(),
+            mock(BackgroundEventHandler.class),
+            mock(ConsumerMetadata.class),
+            mock(SubscriptionState.class),
+            mock(FetchBuffer.class),
+            config,
+            groupRebalanceConfig,
+            mock(ApiVersions.class),
+            mock(FetchMetricsManager.class),
+            () -> mock(NetworkClientDelegate.class),
+            Optional.empty(),
+            new Metrics(),
+            mock(OffsetCommitCallbackInvoker.class),
+            listener
+        ).get();
+        requestManagers.membershipManager.ifPresent(
+            membershipManager -> assertTrue(((MembershipManagerImpl) membershipManager).stateListeners().contains(listener))
+        );
+    }
+}
\ No newline at end of file
diff --git a/clients/src/test/java/org/apache/kafka/test/TestUtils.java b/clients/src/test/java/org/apache/kafka/test/TestUtils.java
index db86558f7d..3a7dcfc930 100644
--- a/clients/src/test/java/org/apache/kafka/test/TestUtils.java
+++ b/clients/src/test/java/org/apache/kafka/test/TestUtils.java
@@ -37,6 +37,7 @@
 import org.apache.kafka.common.requests.ByteBufferChannel;
 import org.apache.kafka.common.requests.MetadataResponse.PartitionMetadata;
 import org.apache.kafka.common.requests.RequestHeader;
+import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.utils.Exit;
 import org.apache.kafka.common.utils.Utils;
 import org.slf4j.Logger;
@@ -279,6 +280,14 @@ public static Properties producerConfig(final String bootstrapServers, final Cla
         return producerConfig(bootstrapServers, keySerializer, valueSerializer, new Properties());
     }
 
+    public static Properties requiredConsumerConfig() {
+        final Properties consumerConfig = new Properties();
+        consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9091");
+        consumerConfig.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
+        consumerConfig.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
+        return consumerConfig;
+    }
+
     public static Properties consumerConfig(final String bootstrapServers,
                                             final String groupId,
                                             final Class keyDeserializer,

From c254b22a4877e70617b2710b95ef44b8cc55ce97 Mon Sep 17 00:00:00 2001
From: PoAn Yang 
Date: Mon, 4 Mar 2024 19:50:56 +0800
Subject: [PATCH 019/521] MINOR: simplify ensure topic exists condition
 (#15458)

Reviewers: Chia-Ping Tsai 
---
 .../main/java/org/apache/kafka/tools/TopicCommand.java | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)

diff --git a/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java b/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java
index 043ae521f5..66650cb9db 100644
--- a/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java
+++ b/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java
@@ -208,9 +208,9 @@ private static Integer getReplicationFactor(TopicPartitionInfo tpi, PartitionRea
      *                           If set to true, the command will throw an exception if the topic with the
      *                           requested name does not exist.
      */
-    private static void ensureTopicExists(List foundTopics, String requestedTopic, Boolean requireTopicExists) {
+    private static void ensureTopicExists(List foundTopics, Optional requestedTopic, Boolean requireTopicExists) {
         // If no topic name was mentioned, do not need to throw exception.
-        if (!(requestedTopic.isEmpty() || !Optional.ofNullable(requestedTopic).isPresent()) && requireTopicExists && foundTopics.isEmpty()) {
+        if (requestedTopic.isPresent() && !requestedTopic.get().isEmpty() && requireTopicExists && foundTopics.isEmpty()) {
             // If given topic doesn't exist then throw exception
             throw new IllegalArgumentException(String.format("Topic '%s' does not exist as expected", requestedTopic));
         }
@@ -490,7 +490,7 @@ public void listTopics(TopicCommandOptions opts) throws ExecutionException, Inte
         public void alterTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
             CommandTopicPartition topic = new CommandTopicPartition(opts);
             List topics = getTopics(opts.topic(), opts.excludeInternalTopics());
-            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+            ensureTopicExists(topics, opts.topic(), !opts.ifExists());
 
             if (!topics.isEmpty()) {
                 Map> topicsInfo = adminClient.describeTopics(topics).topicNameValues();
@@ -556,7 +556,7 @@ public void describeTopic(TopicCommandOptions opts) throws ExecutionException, I
             if (useTopicId) {
                 ensureTopicIdExists(topicIds, inputTopicId.get(), !opts.ifExists());
             } else {
-                ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+                ensureTopicExists(topics, opts.topic(), !opts.ifExists());
             }
             List topicDescriptions = new ArrayList<>();
 
@@ -632,7 +632,7 @@ numPartitions, getReplicationFactor(firstPartition, reassignment),
 
         public void deleteTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
             List topics = getTopics(opts.topic(), opts.excludeInternalTopics());
-            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+            ensureTopicExists(topics, opts.topic(), !opts.ifExists());
             adminClient.deleteTopics(Collections.unmodifiableList(topics),
                 new DeleteTopicsOptions().retryOnQuotaViolation(false)
             ).all().get();

From 4f92a3f0afda96c04059be81cf7867a0bbc7c276 Mon Sep 17 00:00:00 2001
From: Ayoub Omari 
Date: Tue, 5 Mar 2024 00:56:40 +0100
Subject: [PATCH 020/521] KAFKA-14747: record discarded FK join subscription
 responses (#15395)

A foreign-key-join might drop a "subscription response" message, if the value-hash changed.
This PR adds support to record such event via the existing "dropped records" sensor.

Reviewers: Matthias J. Sax 
---
 .../ResponseJoinProcessorSupplier.java        | 13 +++++
 .../SubscriptionJoinProcessorSupplier.java    |  6 +-
 ...ForeignTableJoinProcessorSupplierTest.java |  2 +-
 .../ResponseJoinProcessorSupplierTest.java    | 56 +++++++++++++++++--
 4 files changed, 69 insertions(+), 8 deletions(-)

diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ResponseJoinProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ResponseJoinProcessorSupplier.java
index 600b28078b..cbb66f98fa 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ResponseJoinProcessorSupplier.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ResponseJoinProcessorSupplier.java
@@ -18,6 +18,7 @@
 package org.apache.kafka.streams.kstream.internals.foreignkeyjoin;
 
 import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.metrics.Sensor;
 import org.apache.kafka.common.serialization.Serializer;
 import org.apache.kafka.streams.kstream.ValueJoiner;
 import org.apache.kafka.streams.kstream.internals.KTableValueGetter;
@@ -27,6 +28,8 @@
 import org.apache.kafka.streams.processor.api.ProcessorContext;
 import org.apache.kafka.streams.processor.api.ProcessorSupplier;
 import org.apache.kafka.streams.processor.api.Record;
+import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
 import org.apache.kafka.streams.state.ValueAndTimestamp;
 import org.apache.kafka.streams.state.internals.Murmur3;
 import org.slf4j.Logger;
@@ -71,6 +74,8 @@ public Processor, K, VR> get() {
             private Serializer runtimeValueSerializer = constructionTimeValueSerializer;
 
             private KTableValueGetter valueGetter;
+            private Sensor droppedRecordsSensor;
+
 
             @SuppressWarnings("unchecked")
             @Override
@@ -82,6 +87,13 @@ public void init(final ProcessorContext context) {
                 if (runtimeValueSerializer == null) {
                     runtimeValueSerializer = (Serializer) context.valueSerde().serializer();
                 }
+
+                final InternalProcessorContext internalProcessorContext = (InternalProcessorContext) context;
+                droppedRecordsSensor = TaskMetrics.droppedRecordsSensor(
+                        Thread.currentThread().getName(),
+                        internalProcessorContext.taskId().toString(),
+                        internalProcessorContext.metrics()
+                );
             }
 
             @Override
@@ -112,6 +124,7 @@ public void process(final Record> record) {
                     context().forward(record.withValue(result));
                 } else {
                     LOG.trace("Dropping FK-join response due to hash mismatch. Expected {}. Actual {}", messageHash, currentHash);
+                    droppedRecordsSensor.record();
                 }
             }
         };
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinProcessorSupplier.java
index a8677ce295..388b669e98 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinProcessorSupplier.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinProcessorSupplier.java
@@ -107,7 +107,11 @@ public void process(final Record, Change(value.getHash(), valueToSend, value.getPrimaryPartition()))
+                                .withValue(new SubscriptionResponseWrapper<>(
+                                        value.getHash(),
+                                        valueToSend,
+                                        value.getPrimaryPartition()
+                                ))
                                 .withTimestamp(resultTimestamp)
                         );
                         break;
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignTableJoinProcessorSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignTableJoinProcessorSupplierTest.java
index f4f35e6ff0..c292dd2e34 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignTableJoinProcessorSupplierTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignTableJoinProcessorSupplierTest.java
@@ -379,4 +379,4 @@ SubscriptionResponseWrapper> processor(final KTableValueGetterSupplier(valueGetterSupplier);
         return supplier.get();
     }
-}
\ No newline at end of file
+}
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ResponseJoinProcessorSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ResponseJoinProcessorSupplierTest.java
index 4c26efe236..b32c51a3ba 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ResponseJoinProcessorSupplierTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ResponseJoinProcessorSupplierTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.kafka.streams.kstream.internals.foreignkeyjoin;
 
+import org.apache.kafka.common.MetricName;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.kafka.streams.kstream.ValueJoiner;
 import org.apache.kafka.streams.kstream.internals.KTableValueGetter;
@@ -25,17 +26,23 @@
 import org.apache.kafka.streams.processor.api.Processor;
 import org.apache.kafka.streams.processor.api.ProcessorContext;
 import org.apache.kafka.streams.processor.api.Record;
+import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
 import org.apache.kafka.streams.state.ValueAndTimestamp;
 import org.apache.kafka.streams.state.internals.Murmur3;
+import org.apache.kafka.test.MockInternalNewProcessorContext;
 import org.junit.Test;
 
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.collection.IsEmptyCollection.empty;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
 
 public class ResponseJoinProcessorSupplierTest {
     private static final StringSerializer STRING_SERIALIZER = new StringSerializer();
@@ -88,7 +95,7 @@ public void shouldNotForwardWhenHashDoesNotMatch() {
                 leftJoin
             );
         final Processor, String, String> processor = processorSupplier.get();
-        final org.apache.kafka.streams.processor.api.MockProcessorContext context = new org.apache.kafka.streams.processor.api.MockProcessorContext<>();
+        final MockInternalNewProcessorContext context = new MockInternalNewProcessorContext<>();
         processor.init(context);
         context.setRecordMetadata("topic", 0, 0);
 
@@ -97,6 +104,10 @@ public void shouldNotForwardWhenHashDoesNotMatch() {
         processor.process(new Record<>("lhs1", new SubscriptionResponseWrapper<>(oldHash, "rhsValue", 0), 0));
         final List> forwarded = context.forwarded();
         assertThat(forwarded, empty());
+
+        // test dropped-records sensors
+        assertEquals(1.0, getDroppedRecordsTotalMetric(context));
+        assertNotEquals(0.0, getDroppedRecordsRateMetric(context));
     }
 
     @Test
@@ -113,7 +124,7 @@ public void shouldIgnoreUpdateWhenLeftHasBecomeNull() {
                 leftJoin
             );
         final Processor, String, String> processor = processorSupplier.get();
-        final MockProcessorContext context = new MockProcessorContext<>();
+        final MockInternalNewProcessorContext context = new MockInternalNewProcessorContext<>();
         processor.init(context);
         context.setRecordMetadata("topic", 0, 0);
 
@@ -122,6 +133,10 @@ public void shouldIgnoreUpdateWhenLeftHasBecomeNull() {
         processor.process(new Record<>("lhs1", new SubscriptionResponseWrapper<>(hash, "rhsValue", 0), 0));
         final List> forwarded = context.forwarded();
         assertThat(forwarded, empty());
+
+        // test dropped-records sensors
+        assertEquals(1.0, getDroppedRecordsTotalMetric(context));
+        assertNotEquals(0.0, getDroppedRecordsRateMetric(context));
     }
 
     @Test
@@ -138,7 +153,7 @@ public void shouldForwardWhenHashMatches() {
                 leftJoin
             );
         final Processor, String, String> processor = processorSupplier.get();
-        final MockProcessorContext context = new MockProcessorContext<>();
+        final MockInternalNewProcessorContext context = new MockInternalNewProcessorContext<>();
         processor.init(context);
         context.setRecordMetadata("topic", 0, 0);
 
@@ -164,7 +179,7 @@ public void shouldEmitTombstoneForInnerJoinWhenRightIsNull() {
                 leftJoin
             );
         final Processor, String, String> processor = processorSupplier.get();
-        final MockProcessorContext context = new MockProcessorContext<>();
+        final MockInternalNewProcessorContext context = new MockInternalNewProcessorContext<>();
         processor.init(context);
         context.setRecordMetadata("topic", 0, 0);
 
@@ -190,7 +205,7 @@ public void shouldEmitResultForLeftJoinWhenRightIsNull() {
                 leftJoin
             );
         final Processor, String, String> processor = processorSupplier.get();
-        final MockProcessorContext context = new MockProcessorContext<>();
+        final MockInternalNewProcessorContext context = new MockInternalNewProcessorContext<>();
         processor.init(context);
         context.setRecordMetadata("topic", 0, 0);
 
@@ -216,7 +231,7 @@ public void shouldEmitTombstoneForLeftJoinWhenRightIsNullAndLeftIsNull() {
                 leftJoin
             );
         final Processor, String, String> processor = processorSupplier.get();
-        final MockProcessorContext context = new MockProcessorContext<>();
+        final MockInternalNewProcessorContext context = new MockInternalNewProcessorContext<>();
         processor.init(context);
         context.setRecordMetadata("topic", 0, 0);
 
@@ -227,4 +242,33 @@ public void shouldEmitTombstoneForLeftJoinWhenRightIsNullAndLeftIsNull() {
         assertThat(forwarded.size(), is(1));
         assertThat(forwarded.get(0).record(), is(new Record<>("lhs1", null, 0)));
     }
+
+    private Object getDroppedRecordsTotalMetric(final InternalProcessorContext context) {
+        final MetricName dropTotalMetric = new MetricName(
+            "dropped-records-total",
+            "stream-task-metrics",
+            "The total number of dropped records",
+            mkMap(
+                mkEntry("thread-id", Thread.currentThread().getName()),
+                mkEntry("task-id", "0_0")
+            )
+        );
+
+        return context.metrics().metrics().get(dropTotalMetric).metricValue();
+    }
+
+    private Object getDroppedRecordsRateMetric(final InternalProcessorContext context) {
+        final MetricName dropRateMetric = new MetricName(
+            "dropped-records-rate",
+            "stream-task-metrics",
+            "The average number of dropped records per second",
+            mkMap(
+                mkEntry("thread-id", Thread.currentThread().getName()),
+                mkEntry("task-id", "0_0")
+            )
+        );
+
+        return context.metrics().metrics().get(dropRateMetric).metricValue();
+    }
+
 }

From 99e511c706b7da08b559a3ff6a2c207cacd47b86 Mon Sep 17 00:00:00 2001
From: Greg Harris 
Date: Mon, 4 Mar 2024 15:59:47 -0800
Subject: [PATCH 021/521] KAFKA-16288, KAFKA-16289: Fix Values convertToDecimal
 exception and parseString corruption (#15399)

* KAFKA-16288: Prevent ClassCastExceptions for strings in Values.convertToDecimal
* KAFKA-16289: Values inferred schemas for map and arrays should ignore element order

Signed-off-by: Greg Harris 
Reviewers: Chris Egerton 
---
 .../org/apache/kafka/connect/data/Values.java |  20 +++-
 .../apache/kafka/connect/data/ValuesTest.java | 110 +++++++++++++++---
 2 files changed, 108 insertions(+), 22 deletions(-)

diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/Values.java b/connect/api/src/main/java/org/apache/kafka/connect/data/Values.java
index cbaeeae66f..7b78c64af0 100644
--- a/connect/api/src/main/java/org/apache/kafka/connect/data/Values.java
+++ b/connect/api/src/main/java/org/apache/kafka/connect/data/Values.java
@@ -424,7 +424,7 @@ protected static Object convertTo(Schema toSchema, Schema fromSchema, Object val
                         return BigDecimal.valueOf(converted);
                     }
                     if (value instanceof String) {
-                        return new BigDecimal(value.toString()).doubleValue();
+                        return new BigDecimal(value.toString());
                     }
                 }
                 if (value instanceof ByteBuffer) {
@@ -802,11 +802,12 @@ protected static SchemaAndValue parse(Parser parser, boolean embedded) throws No
         try {
             if (parser.canConsume(ARRAY_BEGIN_DELIMITER)) {
                 List result = new ArrayList<>();
+                boolean compatible = true;
                 Schema elementSchema = null;
                 while (parser.hasNext()) {
                     if (parser.canConsume(ARRAY_END_DELIMITER)) {
                         Schema listSchema;
-                        if (elementSchema != null) {
+                        if (elementSchema != null && compatible) {
                             listSchema = SchemaBuilder.array(elementSchema).schema();
                             result = alignListEntriesWithSchema(listSchema, result);
                         } else {
@@ -821,6 +822,9 @@ protected static SchemaAndValue parse(Parser parser, boolean embedded) throws No
                     }
                     SchemaAndValue element = parse(parser, true);
                     elementSchema = commonSchemaFor(elementSchema, element);
+                    if (elementSchema == null && element != null && element.schema() != null) {
+                        compatible = false;
+                    }
                     result.add(element != null ? element.value() : null);
 
                     int currentPosition = parser.mark();
@@ -840,15 +844,17 @@ protected static SchemaAndValue parse(Parser parser, boolean embedded) throws No
 
             if (parser.canConsume(MAP_BEGIN_DELIMITER)) {
                 Map result = new LinkedHashMap<>();
+                boolean keyCompatible = true;
                 Schema keySchema = null;
+                boolean valueCompatible = true;
                 Schema valueSchema = null;
                 while (parser.hasNext()) {
                     if (parser.canConsume(MAP_END_DELIMITER)) {
                         Schema mapSchema;
-                        if (keySchema != null && valueSchema != null) {
+                        if (keySchema != null && valueSchema != null && keyCompatible && valueCompatible) {
                             mapSchema = SchemaBuilder.map(keySchema, valueSchema).build();
                             result = alignMapKeysAndValuesWithSchema(mapSchema, result);
-                        } else if (keySchema != null) {
+                        } else if (keySchema != null && keyCompatible) {
                             mapSchema = SchemaBuilder.mapWithNullValues(keySchema);
                             result = alignMapKeysWithSchema(mapSchema, result);
                         } else {
@@ -876,7 +882,13 @@ protected static SchemaAndValue parse(Parser parser, boolean embedded) throws No
 
                     parser.canConsume(COMMA_DELIMITER);
                     keySchema = commonSchemaFor(keySchema, key);
+                    if (keySchema == null && key.schema() != null) {
+                        keyCompatible = false;
+                    }
                     valueSchema = commonSchemaFor(valueSchema, value);
+                    if (valueSchema == null && value != null && value.schema() != null) {
+                        valueCompatible = false;
+                    }
                 }
                 // Missing either a comma or an end delimiter
                 if (COMMA_DELIMITER.equals(parser.previous())) {
diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java
index 3700a6ee4e..abb6ea4221 100644
--- a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java
+++ b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java
@@ -25,6 +25,7 @@
 
 import java.math.BigDecimal;
 import java.math.BigInteger;
+import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
@@ -363,33 +364,73 @@ public void shouldConvertStringOfListWithOnlyNumericElementTypesIntoListOfLarges
     }
 
     /**
-     * The parsed array has byte values and one int value, so we should return list with single unified type of integers.
+     * We parse into different element types, but cannot infer a common element schema.
+     * This behavior should be independent of the order that the elements appear in the string
      */
     @Test
-    public void shouldConvertStringOfListWithMixedElementTypesIntoListWithDifferentElementTypes() {
-        String str = "[1, 2, \"three\"]";
-        List list = Values.convertToList(Schema.STRING_SCHEMA, str);
-        assertEquals(3, list.size());
-        assertEquals(1, ((Number) list.get(0)).intValue());
-        assertEquals(2, ((Number) list.get(1)).intValue());
-        assertEquals("three", list.get(2));
+    public void shouldParseStringListWithMultipleElementTypes() {
+        assertParseStringArrayWithNoSchema(
+                Arrays.asList((byte) 1, (byte) 2, (short) 300, "four"),
+                "[1, 2, 300, \"four\"]");
+        assertParseStringArrayWithNoSchema(
+                Arrays.asList((byte) 2, (short) 300, "four", (byte) 1),
+                "[2, 300, \"four\", 1]");
+        assertParseStringArrayWithNoSchema(
+                Arrays.asList((short) 300, "four", (byte) 1, (byte) 2),
+                "[300, \"four\", 1, 2]");
+        assertParseStringArrayWithNoSchema(
+                Arrays.asList("four", (byte) 1, (byte) 2, (short) 300),
+                "[\"four\", 1, 2, 300]");
+    }
+
+    private void assertParseStringArrayWithNoSchema(List expected, String str) {
+        SchemaAndValue result = Values.parseString(str);
+        assertEquals(Type.ARRAY, result.schema().type());
+        assertNull(result.schema().valueSchema());
+        List list = (List) result.value();
+        assertEquals(expected, list);
     }
 
     /**
-     * We parse into different element types, but cannot infer a common element schema.
+     * Maps with an inconsistent key type don't find a common type for the keys or the values
+     * This behavior should be independent of the order that the pairs appear in the string
+     */
+    @Test
+    public void shouldParseStringMapWithMultipleKeyTypes() {
+        Map expected = new HashMap<>();
+        expected.put((byte) 1, (byte) 1);
+        expected.put((byte) 2, (byte) 1);
+        expected.put((short) 300, (short) 300);
+        expected.put("four", (byte) 1);
+        assertParseStringMapWithNoSchema(expected, "{1:1, 2:1, 300:300, \"four\":1}");
+        assertParseStringMapWithNoSchema(expected, "{2:1, 300:300, \"four\":1, 1:1}");
+        assertParseStringMapWithNoSchema(expected, "{300:300, \"four\":1, 1:1, 2:1}");
+        assertParseStringMapWithNoSchema(expected, "{\"four\":1, 1:1, 2:1, 300:300}");
+    }
+
+    /**
+     * Maps with a consistent key type may still not have a common type for the values
+     * This behavior should be independent of the order that the pairs appear in the string
      */
     @Test
-    public void shouldParseStringListWithMultipleElementTypesAndReturnListWithNoSchema() {
-        String str = "[1, 2, 3, \"four\"]";
+    public void shouldParseStringMapWithMultipleValueTypes() {
+        Map expected = new HashMap<>();
+        expected.put((short) 1, (byte) 1);
+        expected.put((short) 2, (byte) 1);
+        expected.put((short) 300, (short) 300);
+        expected.put((short) 4, "four");
+        assertParseStringMapWithNoSchema(expected, "{1:1, 2:1, 300:300, 4:\"four\"}");
+        assertParseStringMapWithNoSchema(expected, "{2:1, 300:300, 4:\"four\", 1:1}");
+        assertParseStringMapWithNoSchema(expected, "{300:300, 4:\"four\", 1:1, 2:1}");
+        assertParseStringMapWithNoSchema(expected, "{4:\"four\", 1:1, 2:1, 300:300}");
+    }
+
+    private void assertParseStringMapWithNoSchema(Map expected, String str) {
         SchemaAndValue result = Values.parseString(str);
-        assertEquals(Type.ARRAY, result.schema().type());
+        assertEquals(Type.MAP, result.schema().type());
         assertNull(result.schema().valueSchema());
-        List list = (List) result.value();
-        assertEquals(4, list.size());
-        assertEquals(1, ((Number) list.get(0)).intValue());
-        assertEquals(2, ((Number) list.get(1)).intValue());
-        assertEquals(3, ((Number) list.get(2)).intValue());
-        assertEquals("four", list.get(3));
+        Map list = (Map) result.value();
+        assertEquals(expected, list);
     }
 
     /**
@@ -744,6 +785,39 @@ public void shouldConvertTimestampValues() {
         assertEquals(current, ts4);
     }
 
+    @Test
+    public void shouldConvertDecimalValues() {
+        // Various forms of the same number should all be parsed to the same BigDecimal
+        Number number = 1.0f;
+        String string = number.toString();
+        BigDecimal value = new BigDecimal(string);
+        byte[] bytes = Decimal.fromLogical(Decimal.schema(1), value);
+        ByteBuffer buffer = ByteBuffer.wrap(bytes);
+
+        assertEquals(value, Values.convertToDecimal(null, number, 1));
+        assertEquals(value, Values.convertToDecimal(null, string, 1));
+        assertEquals(value, Values.convertToDecimal(null, value, 1));
+        assertEquals(value, Values.convertToDecimal(null, bytes, 1));
+        assertEquals(value, Values.convertToDecimal(null, buffer, 1));
+    }
+
+    /**
+     * Test parsing distinct number-like types (strings containing numbers, and logical Decimals) in the same list
+     * The parser does not convert Numbers to Decimals, or Strings containing numbers to Numbers automatically.
+     */
+    @Test
+    public void shouldNotConvertArrayValuesToDecimal() {
+        List decimals = Arrays.asList("\"1.0\"", BigDecimal.valueOf(Long.MAX_VALUE).add(BigDecimal.ONE),
+                BigDecimal.valueOf(Long.MIN_VALUE).subtract(BigDecimal.ONE), (byte) 1, (byte) 1);
+        List expected = new ArrayList<>(decimals); // most values are directly reproduced with the same type
+        expected.set(0, "1.0"); // The quotes are parsed away, but the value remains a string
+        SchemaAndValue schemaAndValue = Values.parseString(decimals.toString());
+        Schema schema = schemaAndValue.schema();
+        assertEquals(Type.ARRAY, schema.type());
+        assertNull(schema.valueSchema());
+        assertEquals(expected, schemaAndValue.value());
+    }
+
     @Test
     public void canConsume() {
     }

From 47792770a28a0b4a6c321a1822b522f4e81068d6 Mon Sep 17 00:00:00 2001
From: Lucas Brutschy 
Date: Tue, 5 Mar 2024 09:44:51 +0100
Subject: [PATCH 022/521] KAFKA-16169: FencedException in commitAsync not
 propagated without callback (#15437)

The javadocs for commitAsync() (w/o callback) say:

@throws org.apache.kafka.common.errors.FencedInstanceIdException
if this consumer instance gets fenced by broker.
If no callback is passed into commitAsync(), no offset commit callback invocation is submitted. However, we only check for a FencedInstanceIdException when we execute a callback. When the consumer gets fenced by another consumer with the same group.instance.id, and we do not use a callback, we miss the exception.

This change modifies the behavior to propagate the FencedInstanceIdException even if no callback is used. The code is kept very similar to the original consumer.

We also change the order - first try to throw the fenced exception, then execute callbacks. That is the order in the original consumer so it's safer to keep it this way.

For testing, we add a unit test that verifies that the FencedInstanceIdException is thrown in that case.

Reviewers: Philip Nee , Matthias J. Sax 
---
 .../internals/AsyncKafkaConsumer.java         | 15 ++++-
 .../internals/CommitRequestManager.java       |  2 +-
 .../OffsetCommitCallbackInvoker.java          | 10 ---
 .../internals/AsyncKafkaConsumerTest.java     | 67 +++++++++++++++++++
 .../internals/CommitRequestManagerTest.java   |  2 -
 5 files changed, 80 insertions(+), 16 deletions(-)

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 dafd3293c7..5354503c01 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
@@ -111,6 +111,7 @@
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
@@ -271,6 +272,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
     private boolean cachedSubscriptionHasAllFetchPositions;
     private final WakeupTrigger wakeupTrigger = new WakeupTrigger();
     private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker;
+    private final AtomicBoolean asyncCommitFenced;
 
     // currentThread holds the threadId of the current thread accessing the AsyncKafkaConsumer
     // and is used to prevent multithreaded access
@@ -353,6 +355,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
                     fetchMetricsManager,
                     clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null));
             this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors);
+            this.asyncCommitFenced = new AtomicBoolean(false);
             this.groupMetadata.set(initializeGroupMetadata(config, groupRebalanceConfig));
             final Supplier requestManagersSupplier = RequestManagers.supplier(time,
                     logContext,
@@ -473,6 +476,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
         this.clientTelemetryReporter = Optional.empty();
         this.autoCommitEnabled = autoCommitEnabled;
         this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors);
+        this.asyncCommitFenced = new AtomicBoolean(false);
     }
 
     AsyncKafkaConsumer(LogContext logContext,
@@ -538,6 +542,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
             client
         );
         this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors);
+        this.asyncCommitFenced = new AtomicBoolean(false);
         Supplier requestManagersSupplier = RequestManagers.supplier(
             time,
             logContext,
@@ -769,6 +774,10 @@ public void commitAsync(Map offsets, OffsetCo
                     offsetCommitCallbackInvoker.enqueueInterceptorInvocation(offsets);
                 }
 
+                if (t instanceof FencedInstanceIdException) {
+                    asyncCommitFenced.set(true);
+                }
+
                 if (callback == null) {
                     if (t != null) {
                         log.error("Offset commit with offsets {} failed", offsets, t);
@@ -784,8 +793,8 @@ public void commitAsync(Map offsets, OffsetCo
     }
 
     private CompletableFuture commit(final CommitEvent commitEvent) {
-        maybeInvokeCommitCallbacks();
         maybeThrowFencedInstanceException();
+        maybeInvokeCommitCallbacks();
         maybeThrowInvalidGroupIdException();
 
         Map offsets = commitEvent.offsets();
@@ -1649,8 +1658,8 @@ private void updateLastSeenEpochIfNewer(TopicPartition topicPartition, OffsetAnd
 
     @Override
     public boolean updateAssignmentMetadataIfNeeded(Timer timer) {
-        maybeInvokeCommitCallbacks();
         maybeThrowFencedInstanceException();
+        maybeInvokeCommitCallbacks();
         backgroundEventProcessor.process();
 
         // Keeping this updateAssignmentMetadataIfNeeded wrapping up the updateFetchPositions as
@@ -1905,7 +1914,7 @@ public KafkaConsumerMetrics kafkaConsumerMetrics() {
     }
 
     private void maybeThrowFencedInstanceException() {
-        if (offsetCommitCallbackInvoker.hasFencedException()) {
+        if (asyncCommitFenced.get()) {
             String groupInstanceId = "unknown";
             if (!groupMetadata.get().isPresent()) {
                 log.error("No group metadata found although a group ID was provided. This is a bug!");
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 9206783d56..f7acbde60b 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
@@ -438,7 +438,7 @@ private Throwable commitSyncExceptionForError(Throwable error) {
 
     private Throwable commitAsyncExceptionForError(Throwable error) {
         if (error instanceof RetriableException) {
-            return new RetriableCommitFailedException(error.getMessage());
+            return new RetriableCommitFailedException(error);
         }
         return error;
     }
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java
index db7770cbda..47a5df6d1d 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java
@@ -19,7 +19,6 @@
 import org.apache.kafka.clients.consumer.OffsetAndMetadata;
 import org.apache.kafka.clients.consumer.OffsetCommitCallback;
 import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.errors.FencedInstanceIdException;
 
 import java.util.Map;
 import java.util.concurrent.BlockingQueue;
@@ -33,7 +32,6 @@
  */
 public class OffsetCommitCallbackInvoker {
     private final ConsumerInterceptors interceptors;
-    private boolean hasFencedException = false;
 
     OffsetCommitCallbackInvoker(ConsumerInterceptors interceptors) {
         this.interceptors = interceptors;
@@ -62,19 +60,11 @@ public void executeCallbacks() {
         while (!callbackQueue.isEmpty()) {
             OffsetCommitCallbackTask task = callbackQueue.poll();
             if (task != null) {
-
-                if (task.exception instanceof FencedInstanceIdException)
-                    hasFencedException = true;
-
                 task.callback.onComplete(task.offsets, task.exception);
             }
         }
     }
 
-    public boolean hasFencedException() {
-        return hasFencedException;
-    }
-
     private static class OffsetCommitCallbackTask {
         public final Map offsets;
         public final Exception exception;
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 fe83a41789..5777aa245a 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
@@ -55,6 +55,7 @@
 import org.apache.kafka.common.Metric;
 import org.apache.kafka.common.Node;
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.FencedInstanceIdException;
 import org.apache.kafka.common.errors.GroupAuthorizationException;
 import org.apache.kafka.common.errors.InvalidGroupIdException;
 import org.apache.kafka.common.errors.RetriableException;
@@ -570,6 +571,72 @@ public void testCommitAsyncLeaderEpochUpdate() {
         verify(applicationEventHandler).add(ArgumentMatchers.isA(AsyncCommitEvent.class));
     }
 
+    @Test
+    public void testCommitAsyncTriggersFencedExceptionFromCommitAsync() {
+        final String groupId = "consumerGroupA";
+        final String groupInstanceId = "groupInstanceId1";
+        final Properties props = requiredConsumerConfigAndGroupId(groupId);
+        props.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, groupInstanceId);
+        final ConsumerConfig config = new ConsumerConfig(props);
+        consumer = newConsumer(config);
+        completeCommitAsyncApplicationEventExceptionally(Errors.FENCED_INSTANCE_ID.exception());
+        doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class));
+        completeFetchedCommittedOffsetApplicationEventSuccessfully(mkMap());
+        doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any());
+        final TopicPartition tp = new TopicPartition("foo", 0);
+        consumer.assign(Collections.singleton(tp));
+        consumer.seek(tp, 20);
+
+        assertDoesNotThrow(() -> consumer.commitAsync());
+
+        Exception e = assertThrows(FencedInstanceIdException.class, () -> consumer.commitAsync());
+        assertEquals("Get fenced exception for group.instance.id groupInstanceId1", e.getMessage());
+    }
+
+    @Test
+    public void testCommitSyncTriggersFencedExceptionFromCommitAsync() {
+        final String groupId = "consumerGroupA";
+        final String groupInstanceId = "groupInstanceId1";
+        final Properties props = requiredConsumerConfigAndGroupId(groupId);
+        props.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, groupInstanceId);
+        final ConsumerConfig config = new ConsumerConfig(props);
+        consumer = newConsumer(config);
+        completeCommitAsyncApplicationEventExceptionally(Errors.FENCED_INSTANCE_ID.exception());
+        doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class));
+        completeFetchedCommittedOffsetApplicationEventSuccessfully(mkMap());
+        doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any());
+        final TopicPartition tp = new TopicPartition("foo", 0);
+        consumer.assign(Collections.singleton(tp));
+        consumer.seek(tp, 20);
+
+        assertDoesNotThrow(() -> consumer.commitAsync());
+
+        Exception e =  assertThrows(FencedInstanceIdException.class, () -> consumer.commitSync());
+        assertEquals("Get fenced exception for group.instance.id groupInstanceId1", e.getMessage());
+    }
+
+    @Test
+    public void testPollTriggersFencedExceptionFromCommitAsync() {
+        final String groupId = "consumerGroupA";
+        final String groupInstanceId = "groupInstanceId1";
+        final Properties props = requiredConsumerConfigAndGroupId(groupId);
+        props.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, groupInstanceId);
+        final ConsumerConfig config = new ConsumerConfig(props);
+        consumer = newConsumer(config);
+        completeCommitAsyncApplicationEventExceptionally(Errors.FENCED_INSTANCE_ID.exception());
+        doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class));
+        completeFetchedCommittedOffsetApplicationEventSuccessfully(mkMap());
+        doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any());
+        final TopicPartition tp = new TopicPartition("foo", 0);
+        consumer.assign(Collections.singleton(tp));
+        consumer.seek(tp, 20);
+
+        assertDoesNotThrow(() -> consumer.commitAsync());
+
+        Exception e = assertThrows(FencedInstanceIdException.class, () -> consumer.poll(Duration.ZERO));
+        assertEquals("Get fenced exception for group.instance.id groupInstanceId1", e.getMessage());
+    }
+
     @Test
     public void testEnsurePollExecutedCommitAsyncCallbacks() {
         consumer = newConsumer();
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 c27494d69a..7e18924e7b 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
@@ -75,7 +75,6 @@
 import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
 import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_GROUP_ID;
 import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_GROUP_INSTANCE_ID;
-import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX;
 import static org.apache.kafka.test.TestUtils.assertFutureThrows;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
@@ -96,7 +95,6 @@ public class CommitRequestManagerTest {
 
     private long retryBackoffMs = 100;
     private long retryBackoffMaxMs = 1000;
-    private String consumerMetricGroupPrefix = CONSUMER_METRIC_GROUP_PREFIX;
     private static final String CONSUMER_COORDINATOR_METRICS = "consumer-coordinator-metrics";
     private Node mockedNode = new Node(1, "host1", 9092);
     private SubscriptionState subscriptionState;

From eea369af947dcff567f849183ba2217ac5e9a2ba Mon Sep 17 00:00:00 2001
From: Nikolay 
Date: Tue, 5 Mar 2024 13:11:56 +0300
Subject: [PATCH 023/521] KAFKA-14588 Log cleaner configuration move to
 CleanerConfig (#15387)

In order to move ConfigCommand to tools we must move all it's dependencies which includes KafkaConfig and other core classes to java. This PR moves log cleaner configuration to CleanerConfig class of storage module.

Reviewers: Chia-Ping Tsai 
---
 build.gradle                                  |  2 +
 checkstyle/import-control-core.xml            |  1 +
 checkstyle/import-control.xml                 |  2 +
 .../util/clusters/EmbeddedKafkaCluster.java   |  3 +-
 .../src/main/scala/kafka/log/LogCleaner.scala | 12 +--
 .../main/scala/kafka/server/KafkaConfig.scala | 76 ++++++-------------
 .../kafka/testkit/KafkaClusterTestKit.java    |  3 +-
 .../api/PlaintextAdminIntegrationTest.scala   | 12 +--
 .../DynamicBrokerReconfigurationTest.scala    | 32 ++++----
 ...gCleanerParameterizedIntegrationTest.scala | 12 +--
 .../scala/unit/kafka/log/LogCleanerTest.scala |  8 +-
 .../kafka/server/ControllerApisTest.scala     | 13 ++--
 .../server/DynamicBrokerConfigTest.scala      | 18 ++---
 .../unit/kafka/server/KafkaConfigTest.scala   | 18 ++---
 .../scala/unit/kafka/utils/TestUtils.scala    |  2 +-
 .../config/ServerTopicConfigSynonyms.java     |  2 +-
 .../apache/kafka/server/config/Defaults.java  |  7 --
 .../storage/internals/log/CleanerConfig.java  | 42 ++++++++++
 .../utils/EmbeddedKafkaCluster.java           |  3 +-
 19 files changed, 141 insertions(+), 127 deletions(-)

diff --git a/build.gradle b/build.gradle
index f2ba22e86b..7dc8f50342 100644
--- a/build.gradle
+++ b/build.gradle
@@ -2177,6 +2177,7 @@ project(':streams') {
     testImplementation project(':core')
     testImplementation project(':tools')
     testImplementation project(':core').sourceSets.test.output
+    testImplementation project(':storage')
     testImplementation project(':server-common')
     testImplementation project(':server-common').sourceSets.test.output
     testImplementation libs.log4j
@@ -2975,6 +2976,7 @@ project(':connect:runtime') {
     testImplementation project(':metadata')
     testImplementation project(':core').sourceSets.test.output
     testImplementation project(':server-common')
+    testImplementation project(':storage')
     testImplementation project(':connect:test-plugins')
 
     testImplementation libs.easymock
diff --git a/checkstyle/import-control-core.xml b/checkstyle/import-control-core.xml
index 3f9a21fffc..782d2fe461 100644
--- a/checkstyle/import-control-core.xml
+++ b/checkstyle/import-control-core.xml
@@ -56,6 +56,7 @@
     
     
     
+    
   
 
   
diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index caf1fe5ebe..a52b3d94e3 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -410,6 +410,7 @@
       
       
       
+      
     
 
     
@@ -601,6 +602,7 @@
         
         
         
+        
       
     
 
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 e3694cae29..c15aa27ae5 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
@@ -55,6 +55,7 @@
 import org.apache.kafka.common.utils.Utils;
 import org.apache.kafka.connect.errors.ConnectException;
 import org.apache.kafka.metadata.BrokerState;
+import org.apache.kafka.storage.internals.log.CleanerConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -160,7 +161,7 @@ private void doStart() {
         putIfAbsent(brokerConfig, KafkaConfig.OffsetsTopicReplicationFactorProp(), (short) brokers.length);
         putIfAbsent(brokerConfig, KafkaConfig.AutoCreateTopicsEnableProp(), false);
         // reduce the size of the log cleaner map to reduce test memory usage
-        putIfAbsent(brokerConfig, KafkaConfig.LogCleanerDedupeBufferSizeProp(), 2 * 1024 * 1024L);
+        putIfAbsent(brokerConfig, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, 2 * 1024 * 1024L);
 
         Object listenerConfig = brokerConfig.get(KafkaConfig.InterBrokerListenerNameProp());
         if (listenerConfig == null)
diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala
index 8098ea237e..b653f40b28 100644
--- a/core/src/main/scala/kafka/log/LogCleaner.scala
+++ b/core/src/main/scala/kafka/log/LogCleaner.scala
@@ -499,13 +499,13 @@ class LogCleaner(initialConfig: CleanerConfig,
 
 object LogCleaner {
   val ReconfigurableConfigs: Set[String] = Set(
-    KafkaConfig.LogCleanerThreadsProp,
-    KafkaConfig.LogCleanerDedupeBufferSizeProp,
-    KafkaConfig.LogCleanerDedupeBufferLoadFactorProp,
-    KafkaConfig.LogCleanerIoBufferSizeProp,
+    CleanerConfig.LOG_CLEANER_THREADS_PROP,
+    CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP,
+    CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP,
+    CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP,
     KafkaConfig.MessageMaxBytesProp,
-    KafkaConfig.LogCleanerIoMaxBytesPerSecondProp,
-    KafkaConfig.LogCleanerBackoffMsProp
+    CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP,
+    CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP
   )
 
   def cleanerConfig(config: KafkaConfig): CleanerConfig = {
diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala
index c6f51a000e..eaddb047da 100755
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -49,7 +49,7 @@ import org.apache.kafka.server.config.{Defaults, ServerTopicConfigSynonyms}
 import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
 import org.apache.kafka.server.record.BrokerCompressionType
 import org.apache.kafka.server.util.Csv
-import org.apache.kafka.storage.internals.log.{LogConfig, ProducerStateManagerConfig}
+import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, ProducerStateManagerConfig}
 import org.apache.kafka.storage.internals.log.LogConfig.MessageFormatVersion
 import org.apache.zookeeper.client.ZKClientConfig
 
@@ -211,17 +211,6 @@ object KafkaConfig {
   val LogRetentionBytesProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.RETENTION_BYTES_CONFIG)
   val LogCleanupIntervalMsProp = LogConfigPrefix + "retention.check.interval.ms"
   val LogCleanupPolicyProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.CLEANUP_POLICY_CONFIG)
-  val LogCleanerThreadsProp = LogConfigPrefix + "cleaner.threads"
-  val LogCleanerIoMaxBytesPerSecondProp = LogConfigPrefix + "cleaner.io.max.bytes.per.second"
-  val LogCleanerDedupeBufferSizeProp = LogConfigPrefix + "cleaner.dedupe.buffer.size"
-  val LogCleanerIoBufferSizeProp = LogConfigPrefix + "cleaner.io.buffer.size"
-  val LogCleanerDedupeBufferLoadFactorProp = LogConfigPrefix + "cleaner.io.buffer.load.factor"
-  val LogCleanerBackoffMsProp = LogConfigPrefix + "cleaner.backoff.ms"
-  val LogCleanerMinCleanRatioProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG)
-  val LogCleanerEnableProp = LogConfigPrefix + "cleaner.enable"
-  val LogCleanerDeleteRetentionMsProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.DELETE_RETENTION_MS_CONFIG)
-  val LogCleanerMinCompactionLagMsProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG)
-  val LogCleanerMaxCompactionLagMsProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG)
   val LogIndexSizeMaxBytesProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG)
   val LogIndexIntervalBytesProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG)
   val LogFlushIntervalMessagesProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG)
@@ -652,25 +641,6 @@ object KafkaConfig {
   val LogRetentionBytesDoc = "The maximum size of the log before deleting it"
   val LogCleanupIntervalMsDoc = "The frequency in milliseconds that the log cleaner checks whether any log is eligible for deletion"
   val LogCleanupPolicyDoc = "The default cleanup policy for segments beyond the retention window. A comma separated list of valid policies. Valid policies are: \"delete\" and \"compact\""
-  val LogCleanerThreadsDoc = "The number of background threads to use for log cleaning"
-  val LogCleanerIoMaxBytesPerSecondDoc = "The log cleaner will be throttled so that the sum of its read and write i/o will be less than this value on average"
-  val LogCleanerDedupeBufferSizeDoc = "The total memory used for log deduplication across all cleaner threads"
-  val LogCleanerIoBufferSizeDoc = "The total memory used for log cleaner I/O buffers across all cleaner threads"
-  val LogCleanerDedupeBufferLoadFactorDoc = "Log cleaner dedupe buffer load factor. The percentage full the dedupe buffer can become. A higher value " +
-  "will allow more log to be cleaned at once but will lead to more hash collisions"
-  val LogCleanerBackoffMsDoc = "The amount of time to sleep when there are no logs to clean"
-  val LogCleanerMinCleanRatioDoc = "The minimum ratio of dirty log to total log for a log to eligible for cleaning. " +
-    "If the " + LogCleanerMaxCompactionLagMsProp + " or the " + LogCleanerMinCompactionLagMsProp +
-    " configurations are also specified, then the log compactor considers the log eligible for compaction " +
-    "as soon as either: (i) the dirty ratio threshold has been met and the log has had dirty (uncompacted) " +
-    "records for at least the " + LogCleanerMinCompactionLagMsProp + " duration, or (ii) if the log has had " +
-    "dirty (uncompacted) records for at most the " + LogCleanerMaxCompactionLagMsProp + " period."
-  val LogCleanerEnableDoc = "Enable the log cleaner process to run on the server. Should be enabled if using any topics with a cleanup.policy=compact including the internal offsets topic. If disabled those topics will not be compacted and continually grow in size."
-  val LogCleanerDeleteRetentionMsDoc = "The amount of time to retain tombstone message markers for log compacted topics. This setting also gives a bound " +
-    "on the time in which a consumer must complete a read if they begin from offset 0 to ensure that they get a valid snapshot of the final stage (otherwise  " +
-    "tombstones messages may be collected before a consumer completes their scan)."
-  val LogCleanerMinCompactionLagMsDoc = "The minimum time a message will remain uncompacted in the log. Only applicable for logs that are being compacted."
-  val LogCleanerMaxCompactionLagMsDoc = "The maximum time a message will remain ineligible for compaction in the log. Only applicable for logs that are being compacted."
   val LogIndexSizeMaxBytesDoc = "The maximum size in bytes of the offset index"
   val LogIndexIntervalBytesDoc = "The interval with which we add an entry to the offset index."
   val LogFlushIntervalMessagesDoc = "The number of messages accumulated on a log partition before messages are flushed to disk."
@@ -1075,17 +1045,17 @@ object KafkaConfig {
       .define(LogRetentionBytesProp, LONG, LogConfig.DEFAULT_RETENTION_BYTES, HIGH, LogRetentionBytesDoc)
       .define(LogCleanupIntervalMsProp, LONG, Defaults.LOG_CLEANUP_INTERVAL_MS, atLeast(1), MEDIUM, LogCleanupIntervalMsDoc)
       .define(LogCleanupPolicyProp, LIST, LogConfig.DEFAULT_CLEANUP_POLICY, ValidList.in(TopicConfig.CLEANUP_POLICY_COMPACT, TopicConfig.CLEANUP_POLICY_DELETE), MEDIUM, LogCleanupPolicyDoc)
-      .define(LogCleanerThreadsProp, INT, Defaults.LOG_CLEANER_THREADS, atLeast(0), MEDIUM, LogCleanerThreadsDoc)
-      .define(LogCleanerIoMaxBytesPerSecondProp, DOUBLE, Defaults.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND, MEDIUM, LogCleanerIoMaxBytesPerSecondDoc)
-      .define(LogCleanerDedupeBufferSizeProp, LONG, Defaults.LOG_CLEANER_DEDUPE_BUFFER_SIZE, MEDIUM, LogCleanerDedupeBufferSizeDoc)
-      .define(LogCleanerIoBufferSizeProp, INT, Defaults.LOG_CLEANER_IO_BUFFER_SIZE, atLeast(0), MEDIUM, LogCleanerIoBufferSizeDoc)
-      .define(LogCleanerDedupeBufferLoadFactorProp, DOUBLE, Defaults.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR, MEDIUM, LogCleanerDedupeBufferLoadFactorDoc)
-      .define(LogCleanerBackoffMsProp, LONG, Defaults.LOG_CLEANER_BACKOFF_MS, atLeast(0), MEDIUM, LogCleanerBackoffMsDoc)
-      .define(LogCleanerMinCleanRatioProp, DOUBLE, LogConfig.DEFAULT_MIN_CLEANABLE_DIRTY_RATIO, between(0, 1), MEDIUM, LogCleanerMinCleanRatioDoc)
-      .define(LogCleanerEnableProp, BOOLEAN, Defaults.LOG_CLEANER_ENABLE, MEDIUM, LogCleanerEnableDoc)
-      .define(LogCleanerDeleteRetentionMsProp, LONG, LogConfig.DEFAULT_DELETE_RETENTION_MS, atLeast(0), MEDIUM, LogCleanerDeleteRetentionMsDoc)
-      .define(LogCleanerMinCompactionLagMsProp, LONG, LogConfig.DEFAULT_MIN_COMPACTION_LAG_MS, atLeast(0), MEDIUM, LogCleanerMinCompactionLagMsDoc)
-      .define(LogCleanerMaxCompactionLagMsProp, LONG, LogConfig.DEFAULT_MAX_COMPACTION_LAG_MS, atLeast(1), MEDIUM, LogCleanerMaxCompactionLagMsDoc)
+      .define(CleanerConfig.LOG_CLEANER_THREADS_PROP, INT, CleanerConfig.LOG_CLEANER_THREADS, atLeast(0), MEDIUM, CleanerConfig.LOG_CLEANER_THREADS_DOC)
+      .define(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, DOUBLE, CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND, MEDIUM, CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_DOC)
+      .define(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, LONG, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE, MEDIUM, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_DOC)
+      .define(CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP, INT, CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE, atLeast(0), MEDIUM, CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_DOC)
+      .define(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP, DOUBLE, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR, MEDIUM, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_DOC)
+      .define(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP, LONG, CleanerConfig.LOG_CLEANER_BACKOFF_MS, atLeast(0), MEDIUM, CleanerConfig.LOG_CLEANER_BACKOFF_MS_DOC)
+      .define(CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_PROP, DOUBLE, LogConfig.DEFAULT_MIN_CLEANABLE_DIRTY_RATIO, between(0, 1), MEDIUM, CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_DOC)
+      .define(CleanerConfig.LOG_CLEANER_ENABLE_PROP, BOOLEAN, CleanerConfig.LOG_CLEANER_ENABLE, MEDIUM, CleanerConfig.LOG_CLEANER_ENABLE_DOC)
+      .define(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP, LONG, LogConfig.DEFAULT_DELETE_RETENTION_MS, atLeast(0), MEDIUM, CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_DOC)
+      .define(CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP, LONG, LogConfig.DEFAULT_MIN_COMPACTION_LAG_MS, atLeast(0), MEDIUM, CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_DOC)
+      .define(CleanerConfig.LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP, LONG, LogConfig.DEFAULT_MAX_COMPACTION_LAG_MS, atLeast(1), MEDIUM, CleanerConfig.LOG_CLEANER_MAX_COMPACTION_LAG_MS_DOC)
       .define(LogIndexSizeMaxBytesProp, INT, LogConfig.DEFAULT_SEGMENT_INDEX_BYTES, atLeast(4), MEDIUM, LogIndexSizeMaxBytesDoc)
       .define(LogIndexIntervalBytesProp, INT, LogConfig.DEFAULT_INDEX_INTERVAL_BYTES, atLeast(0), MEDIUM, LogIndexIntervalBytesDoc)
       .define(LogFlushIntervalMessagesProp, LONG, LogConfig.DEFAULT_FLUSH_MESSAGES_INTERVAL, atLeast(1), HIGH, LogFlushIntervalMessagesDoc)
@@ -1652,7 +1622,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
   val logDirs = CoreUtils.parseCsvList(Option(getString(KafkaConfig.LogDirsProp)).getOrElse(getString(KafkaConfig.LogDirProp)))
   def logSegmentBytes = getInt(KafkaConfig.LogSegmentBytesProp)
   def logFlushIntervalMessages = getLong(KafkaConfig.LogFlushIntervalMessagesProp)
-  val logCleanerThreads = getInt(KafkaConfig.LogCleanerThreadsProp)
+  val logCleanerThreads = getInt(CleanerConfig.LOG_CLEANER_THREADS_PROP)
   def numRecoveryThreadsPerDataDir = getInt(KafkaConfig.NumRecoveryThreadsPerDataDirProp)
   val logFlushSchedulerIntervalMs = getLong(KafkaConfig.LogFlushSchedulerIntervalMsProp)
   val logFlushOffsetCheckpointIntervalMs = getInt(KafkaConfig.LogFlushOffsetCheckpointIntervalMsProp).toLong
@@ -1662,16 +1632,16 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
   val offsetsRetentionMinutes = getInt(KafkaConfig.OffsetsRetentionMinutesProp)
   val offsetsRetentionCheckIntervalMs = getLong(KafkaConfig.OffsetsRetentionCheckIntervalMsProp)
   def logRetentionBytes = getLong(KafkaConfig.LogRetentionBytesProp)
-  val logCleanerDedupeBufferSize = getLong(KafkaConfig.LogCleanerDedupeBufferSizeProp)
-  val logCleanerDedupeBufferLoadFactor = getDouble(KafkaConfig.LogCleanerDedupeBufferLoadFactorProp)
-  val logCleanerIoBufferSize = getInt(KafkaConfig.LogCleanerIoBufferSizeProp)
-  val logCleanerIoMaxBytesPerSecond = getDouble(KafkaConfig.LogCleanerIoMaxBytesPerSecondProp)
-  def logCleanerDeleteRetentionMs = getLong(KafkaConfig.LogCleanerDeleteRetentionMsProp)
-  def logCleanerMinCompactionLagMs = getLong(KafkaConfig.LogCleanerMinCompactionLagMsProp)
-  def logCleanerMaxCompactionLagMs = getLong(KafkaConfig.LogCleanerMaxCompactionLagMsProp)
-  val logCleanerBackoffMs = getLong(KafkaConfig.LogCleanerBackoffMsProp)
-  def logCleanerMinCleanRatio = getDouble(KafkaConfig.LogCleanerMinCleanRatioProp)
-  val logCleanerEnable = getBoolean(KafkaConfig.LogCleanerEnableProp)
+  val logCleanerDedupeBufferSize = getLong(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP)
+  val logCleanerDedupeBufferLoadFactor = getDouble(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP)
+  val logCleanerIoBufferSize = getInt(CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP)
+  val logCleanerIoMaxBytesPerSecond = getDouble(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP)
+  def logCleanerDeleteRetentionMs = getLong(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP)
+  def logCleanerMinCompactionLagMs = getLong(CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP)
+  def logCleanerMaxCompactionLagMs = getLong(CleanerConfig.LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP)
+  val logCleanerBackoffMs = getLong(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP)
+  def logCleanerMinCleanRatio = getDouble(CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_PROP)
+  val logCleanerEnable = getBoolean(CleanerConfig.LOG_CLEANER_ENABLE_PROP)
   def logIndexSizeMaxBytes = getInt(KafkaConfig.LogIndexSizeMaxBytesProp)
   def logIndexIntervalBytes = getInt(KafkaConfig.LogIndexIntervalBytesProp)
   def logDeleteDelayMs = getLong(KafkaConfig.LogDeleteDelayMsProp)
diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java
index fa11fa9b18..4c6d1f9ef4 100644
--- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java
+++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java
@@ -41,6 +41,7 @@
 import org.apache.kafka.server.common.ApiMessageAndVersion;
 import org.apache.kafka.server.fault.FaultHandler;
 import org.apache.kafka.server.fault.MockFaultHandler;
+import org.apache.kafka.storage.internals.log.CleanerConfig;
 import org.apache.kafka.test.TestUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -192,7 +193,7 @@ private KafkaConfig createNodeConfig(TestKitNode node) {
             props.put(RaftConfig.QUORUM_VOTERS_CONFIG, uninitializedQuorumVotersString);
 
             // reduce log cleaner offset map memory usage
-            props.put(KafkaConfig$.MODULE$.LogCleanerDedupeBufferSizeProp(), "2097152");
+            props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "2097152");
 
             // Add associated broker node property overrides
             if (brokerNode != null) {
diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
index 54be333762..e084454f5f 100644
--- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
@@ -45,7 +45,7 @@ import org.apache.kafka.common.utils.{Time, Utils}
 import org.apache.kafka.common.{ConsumerGroupState, ElectionType, TopicCollection, TopicPartition, TopicPartitionInfo, TopicPartitionReplica, Uuid}
 import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT
 import org.apache.kafka.server.config.Defaults
-import org.apache.kafka.storage.internals.log.LogConfig
+import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig}
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, TestInfo}
 import org.junit.jupiter.params.ParameterizedTest
@@ -447,7 +447,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
       configs.get(brokerResource2).entries.size)
     assertEquals(brokers(2).config.brokerId.toString, configs.get(brokerResource2).get(KafkaConfig.BrokerIdProp).value)
     assertEquals(brokers(2).config.logCleanerThreads.toString,
-      configs.get(brokerResource2).get(KafkaConfig.LogCleanerThreadsProp).value)
+      configs.get(brokerResource2).get(CleanerConfig.LOG_CLEANER_THREADS_PROP).value)
 
     checkValidAlterConfigs(client, this, topicResource1, topicResource2)
   }
@@ -2532,7 +2532,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
       .all().get(15, TimeUnit.SECONDS)
 
     val newLogCleanerDeleteRetention = new Properties
-    newLogCleanerDeleteRetention.put(KafkaConfig.LogCleanerDeleteRetentionMsProp, "34")
+    newLogCleanerDeleteRetention.put(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP, "34")
     TestUtils.incrementalAlterConfigs(brokers, client, newLogCleanerDeleteRetention, perBrokerConfig = true)
       .all().get(15, TimeUnit.SECONDS)
 
@@ -2543,14 +2543,14 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
         controllerServer.config.nodeId.toString)
       controllerServer.controller.incrementalAlterConfigs(ANONYMOUS_CONTEXT,
         Collections.singletonMap(controllerNodeResource,
-          Collections.singletonMap(KafkaConfig.LogCleanerDeleteRetentionMsProp,
+          Collections.singletonMap(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP,
             new SimpleImmutableEntry(AlterConfigOp.OpType.SET, "34"))), false).get()
       ensureConsistentKRaftMetadata()
     }
 
     waitUntilTrue(() => brokers.forall(_.config.originals.getOrDefault(
-      KafkaConfig.LogCleanerDeleteRetentionMsProp, "").toString.equals("34")),
-      s"Timed out waiting for change to ${KafkaConfig.LogCleanerDeleteRetentionMsProp}",
+      CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP, "").toString.equals("34")),
+      s"Timed out waiting for change to ${CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP}",
       waitTimeMs = 60000L)
 
     waitUntilTrue(() => brokers.forall(_.config.originals.getOrDefault(
diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
index 4be668aa32..c984eae027 100644
--- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
+++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
@@ -254,7 +254,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
     expectedProps.setProperty(KafkaConfig.LogRetentionTimeMillisProp, "1680000000")
     expectedProps.setProperty(KafkaConfig.LogRetentionTimeHoursProp, "168")
     expectedProps.setProperty(KafkaConfig.LogRollTimeHoursProp, "168")
-    expectedProps.setProperty(KafkaConfig.LogCleanerThreadsProp, "1")
+    expectedProps.setProperty(CleanerConfig.LOG_CLEANER_THREADS_PROP, "1")
     val logRetentionMs = configEntry(configDesc, KafkaConfig.LogRetentionTimeMillisProp)
     verifyConfig(KafkaConfig.LogRetentionTimeMillisProp, logRetentionMs,
       isSensitive = false, isReadOnly = false, expectedProps)
@@ -264,8 +264,8 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
     val logRollHours = configEntry(configDesc, KafkaConfig.LogRollTimeHoursProp)
     verifyConfig(KafkaConfig.LogRollTimeHoursProp, logRollHours,
       isSensitive = false, isReadOnly = true, expectedProps)
-    val logCleanerThreads = configEntry(configDesc, KafkaConfig.LogCleanerThreadsProp)
-    verifyConfig(KafkaConfig.LogCleanerThreadsProp, logCleanerThreads,
+    val logCleanerThreads = configEntry(configDesc, CleanerConfig.LOG_CLEANER_THREADS_PROP)
+    verifyConfig(CleanerConfig.LOG_CLEANER_THREADS_PROP, logCleanerThreads,
       isSensitive = false, isReadOnly = false, expectedProps)
 
     def synonymsList(configEntry: ConfigEntry): List[(String, ConfigSource)] =
@@ -278,7 +278,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
       (KafkaConfig.LogRetentionTimeHoursProp, ConfigSource.DEFAULT_CONFIG)),
       synonymsList(logRetentionHours))
     assertEquals(List((KafkaConfig.LogRollTimeHoursProp, ConfigSource.DEFAULT_CONFIG)), synonymsList(logRollHours))
-    assertEquals(List((KafkaConfig.LogCleanerThreadsProp, ConfigSource.DEFAULT_CONFIG)), synonymsList(logCleanerThreads))
+    assertEquals(List((CleanerConfig.LOG_CLEANER_THREADS_PROP, ConfigSource.DEFAULT_CONFIG)), synonymsList(logCleanerThreads))
   }
 
   @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
@@ -536,19 +536,19 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
     verifyThreads("kafka-log-cleaner-thread-", countPerBroker = 1)
 
     val props = new Properties
-    props.put(KafkaConfig.LogCleanerThreadsProp, "2")
-    props.put(KafkaConfig.LogCleanerDedupeBufferSizeProp, "20000000")
-    props.put(KafkaConfig.LogCleanerDedupeBufferLoadFactorProp, "0.8")
-    props.put(KafkaConfig.LogCleanerIoBufferSizeProp, "300000")
+    props.put(CleanerConfig.LOG_CLEANER_THREADS_PROP, "2")
+    props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "20000000")
+    props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP, "0.8")
+    props.put(CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP, "300000")
     props.put(KafkaConfig.MessageMaxBytesProp, "40000")
-    props.put(KafkaConfig.LogCleanerIoMaxBytesPerSecondProp, "50000000")
-    props.put(KafkaConfig.LogCleanerBackoffMsProp, "6000")
+    props.put(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, "50000000")
+    props.put(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP, "6000")
 
     // Verify cleaner config was updated. Wait for one of the configs to be updated and verify
     // that all other others were updated at the same time since they are reconfigured together
     var newCleanerConfig: CleanerConfig = null
     TestUtils.waitUntilTrue(() => {
-      reconfigureServers(props, perBrokerConfig = false, (KafkaConfig.LogCleanerThreadsProp, "2"))
+      reconfigureServers(props, perBrokerConfig = false, (CleanerConfig.LOG_CLEANER_THREADS_PROP, "2"))
       newCleanerConfig = servers.head.logManager.cleaner.currentConfig
       newCleanerConfig.numThreads == 2
     }, "Log cleaner not reconfigured", 60000)
@@ -566,8 +566,8 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
     def cleanerThreads = Thread.getAllStackTraces.keySet.asScala.filter(_.getName.startsWith("kafka-log-cleaner-thread-"))
     cleanerThreads.take(2).foreach(_.interrupt())
     TestUtils.waitUntilTrue(() => cleanerThreads.size == (2 * numServers) - 2, "Threads did not exit")
-    props.put(KafkaConfig.LogCleanerBackoffMsProp, "8000")
-    reconfigureServers(props, perBrokerConfig = false, (KafkaConfig.LogCleanerBackoffMsProp, "8000"))
+    props.put(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP, "8000")
+    reconfigureServers(props, perBrokerConfig = false, (CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP, "8000"))
     verifyThreads("kafka-log-cleaner-thread-", countPerBroker = 2)
 
     // Verify that produce/consume worked throughout this test without any retries in producer
@@ -635,10 +635,10 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
     props.put(KafkaConfig.LogRetentionTimeMillisProp, TimeUnit.DAYS.toMillis(1).toString)
     props.put(KafkaConfig.MessageMaxBytesProp, "100000")
     props.put(KafkaConfig.LogIndexIntervalBytesProp, "10000")
-    props.put(KafkaConfig.LogCleanerDeleteRetentionMsProp, TimeUnit.DAYS.toMillis(1).toString)
-    props.put(KafkaConfig.LogCleanerMinCompactionLagMsProp, "60000")
+    props.put(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP, TimeUnit.DAYS.toMillis(1).toString)
+    props.put(CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP, "60000")
     props.put(KafkaConfig.LogDeleteDelayMsProp, "60000")
-    props.put(KafkaConfig.LogCleanerMinCleanRatioProp, "0.3")
+    props.put(CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_PROP, "0.3")
     props.put(KafkaConfig.LogCleanupPolicyProp, "delete")
     props.put(KafkaConfig.UncleanLeaderElectionEnableProp, "false")
     props.put(KafkaConfig.MinInSyncReplicasProp, "2")
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala
index 8729045db7..49e518ac2a 100755
--- a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala
@@ -251,13 +251,13 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
 
     def kafkaConfigWithCleanerConfig(cleanerConfig: CleanerConfig): KafkaConfig = {
       val props = TestUtils.createBrokerConfig(0, "localhost:2181")
-      props.put(KafkaConfig.LogCleanerThreadsProp, cleanerConfig.numThreads.toString)
-      props.put(KafkaConfig.LogCleanerDedupeBufferSizeProp, cleanerConfig.dedupeBufferSize.toString)
-      props.put(KafkaConfig.LogCleanerDedupeBufferLoadFactorProp, cleanerConfig.dedupeBufferLoadFactor.toString)
-      props.put(KafkaConfig.LogCleanerIoBufferSizeProp, cleanerConfig.ioBufferSize.toString)
+      props.put(CleanerConfig.LOG_CLEANER_THREADS_PROP, cleanerConfig.numThreads.toString)
+      props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, cleanerConfig.dedupeBufferSize.toString)
+      props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP, cleanerConfig.dedupeBufferLoadFactor.toString)
+      props.put(CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP, cleanerConfig.ioBufferSize.toString)
       props.put(KafkaConfig.MessageMaxBytesProp, cleanerConfig.maxMessageSize.toString)
-      props.put(KafkaConfig.LogCleanerBackoffMsProp, cleanerConfig.backoffMs.toString)
-      props.put(KafkaConfig.LogCleanerIoMaxBytesPerSecondProp, cleanerConfig.maxIoBytesPerSecond.toString)
+      props.put(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP, cleanerConfig.backoffMs.toString)
+      props.put(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, cleanerConfig.maxIoBytesPerSecond.toString)
       KafkaConfig.fromProps(props)
     }
 
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
index 161493457e..b3a7259219 100644
--- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
@@ -1959,7 +1959,7 @@ class LogCleanerTest extends Logging {
   @Test
   def testReconfigureLogCleanerIoMaxBytesPerSecond(): Unit = {
     val oldKafkaProps = TestUtils.createBrokerConfig(1, "localhost:2181")
-    oldKafkaProps.setProperty(KafkaConfig.LogCleanerIoMaxBytesPerSecondProp, "10000000")
+    oldKafkaProps.setProperty(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, "10000000")
 
     val logCleaner = new LogCleaner(LogCleaner.cleanerConfig(new KafkaConfig(oldKafkaProps)),
       logDirs = Array(TestUtils.tempDir()),
@@ -1973,14 +1973,14 @@ class LogCleanerTest extends Logging {
     }
 
     try {
-      assertEquals(10000000, logCleaner.throttler.desiredRatePerSec, s"Throttler.desiredRatePerSec should be initialized from initial `${KafkaConfig.LogCleanerIoMaxBytesPerSecondProp}` config.")
+      assertEquals(10000000, logCleaner.throttler.desiredRatePerSec, s"Throttler.desiredRatePerSec should be initialized from initial `${CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP}` config.")
 
       val newKafkaProps = TestUtils.createBrokerConfig(1, "localhost:2181")
-      newKafkaProps.setProperty(KafkaConfig.LogCleanerIoMaxBytesPerSecondProp, "20000000")
+      newKafkaProps.setProperty(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, "20000000")
 
       logCleaner.reconfigure(new KafkaConfig(oldKafkaProps), new KafkaConfig(newKafkaProps))
 
-      assertEquals(20000000, logCleaner.throttler.desiredRatePerSec, s"Throttler.desiredRatePerSec should be updated with new `${KafkaConfig.LogCleanerIoMaxBytesPerSecondProp}` config.")
+      assertEquals(20000000, logCleaner.throttler.desiredRatePerSec, s"Throttler.desiredRatePerSec should be updated with new `${CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP}` config.")
     } finally {
       logCleaner.shutdown()
     }
diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala
index 00777e9677..135de6e514 100644
--- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala
@@ -55,6 +55,7 @@ import org.apache.kafka.image.publisher.ControllerRegistrationsPublisher
 import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult, Authorizer}
 import org.apache.kafka.server.common.{ApiMessageAndVersion, Features, MetadataVersion, ProducerIdsBlock}
 import org.apache.kafka.server.util.FutureUtils
+import org.apache.kafka.storage.internals.log.CleanerConfig
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{AfterEach, Test}
 import org.junit.jupiter.params.ParameterizedTest
@@ -318,19 +319,19 @@ class ControllerApisTest {
           setResourceName("1").
           setResourceType(ConfigResource.Type.BROKER.id()).
           setConfigs(new OldAlterableConfigCollection(util.Arrays.asList(new OldAlterableConfig().
-            setName(KafkaConfig.LogCleanerBackoffMsProp).
+            setName(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP).
             setValue("100000")).iterator())),
         new OldAlterConfigsResource().
           setResourceName("2").
           setResourceType(ConfigResource.Type.BROKER.id()).
           setConfigs(new OldAlterableConfigCollection(util.Arrays.asList(new OldAlterableConfig().
-            setName(KafkaConfig.LogCleanerBackoffMsProp).
+            setName(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP).
             setValue("100000")).iterator())),
         new OldAlterConfigsResource().
           setResourceName("2").
           setResourceType(ConfigResource.Type.BROKER.id()).
           setConfigs(new OldAlterableConfigCollection(util.Arrays.asList(new OldAlterableConfig().
-            setName(KafkaConfig.LogCleanerBackoffMsProp).
+            setName(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP).
             setValue("100000")).iterator())),
         new OldAlterConfigsResource().
           setResourceName("baz").
@@ -472,7 +473,7 @@ class ControllerApisTest {
           setResourceName("1").
           setResourceType(ConfigResource.Type.BROKER.id()).
           setConfigs(new AlterableConfigCollection(util.Arrays.asList(new AlterableConfig().
-            setName(KafkaConfig.LogCleanerBackoffMsProp).
+            setName(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP).
             setValue("100000").
             setConfigOperation(AlterConfigOp.OpType.SET.id())).iterator())),
         new AlterConfigsResource().
@@ -536,14 +537,14 @@ class ControllerApisTest {
           setResourceName("3").
           setResourceType(ConfigResource.Type.BROKER.id()).
           setConfigs(new AlterableConfigCollection(util.Arrays.asList(new AlterableConfig().
-            setName(KafkaConfig.LogCleanerBackoffMsProp).
+            setName(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP).
             setValue("100000").
             setConfigOperation(AlterConfigOp.OpType.SET.id())).iterator())),
         new AlterConfigsResource().
           setResourceName("3").
           setResourceType(ConfigResource.Type.BROKER.id()).
           setConfigs(new AlterableConfigCollection(util.Arrays.asList(new AlterableConfig().
-            setName(KafkaConfig.LogCleanerBackoffMsProp).
+            setName(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP).
             setValue("100000").
             setConfigOperation(AlterConfigOp.OpType.SET.id())).iterator())),
         new AlterConfigsResource().
diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala
index be065f5b8c..4e1c8eed27 100755
--- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala
@@ -38,7 +38,7 @@ import org.apache.kafka.server.config.Defaults
 import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
 import org.apache.kafka.server.metrics.KafkaYammerMetrics
 import org.apache.kafka.server.util.KafkaScheduler
-import org.apache.kafka.storage.internals.log.{LogConfig, ProducerStateManagerConfig}
+import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, ProducerStateManagerConfig}
 import org.apache.kafka.test.MockMetricsReporter
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.Test
@@ -215,7 +215,7 @@ class DynamicBrokerConfigTest {
     verifyConfigUpdateWithInvalidConfig(config, origProps, validProps, nonDynamicProps)
 
     // Test update of configs with invalid type
-    val invalidProps = Map(KafkaConfig.LogCleanerThreadsProp -> "invalid")
+    val invalidProps = Map(CleanerConfig.LOG_CLEANER_THREADS_PROP -> "invalid")
     verifyConfigUpdateWithInvalidConfig(config, origProps, validProps, invalidProps)
 
     val excludedTopicConfig = Map(KafkaConfig.LogMessageFormatVersionProp -> "0.10.2")
@@ -225,21 +225,21 @@ class DynamicBrokerConfigTest {
   @Test
   def testConfigUpdateWithReconfigurableValidationFailure(): Unit = {
     val origProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
-    origProps.put(KafkaConfig.LogCleanerDedupeBufferSizeProp, "100000000")
+    origProps.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "100000000")
     val config = KafkaConfig(origProps)
     config.dynamicConfig.initialize(None, None)
 
     val validProps = Map.empty[String, String]
-    val invalidProps = Map(KafkaConfig.LogCleanerThreadsProp -> "20")
+    val invalidProps = Map(CleanerConfig.LOG_CLEANER_THREADS_PROP -> "20")
 
     def validateLogCleanerConfig(configs: util.Map[String, _]): Unit = {
-      val cleanerThreads = configs.get(KafkaConfig.LogCleanerThreadsProp).toString.toInt
+      val cleanerThreads = configs.get(CleanerConfig.LOG_CLEANER_THREADS_PROP).toString.toInt
       if (cleanerThreads <=0 || cleanerThreads >= 5)
         throw new ConfigException(s"Invalid cleaner threads $cleanerThreads")
     }
     val reconfigurable = new Reconfigurable {
       override def configure(configs: util.Map[String, _]): Unit = {}
-      override def reconfigurableConfigs(): util.Set[String] = Set(KafkaConfig.LogCleanerThreadsProp).asJava
+      override def reconfigurableConfigs(): util.Set[String] = Set(CleanerConfig.LOG_CLEANER_THREADS_PROP).asJava
       override def validateReconfiguration(configs: util.Map[String, _]): Unit = validateLogCleanerConfig(configs)
       override def reconfigure(configs: util.Map[String, _]): Unit = {}
     }
@@ -248,7 +248,7 @@ class DynamicBrokerConfigTest {
     config.dynamicConfig.removeReconfigurable(reconfigurable)
 
     val brokerReconfigurable = new BrokerReconfigurable {
-      override def reconfigurableConfigs: collection.Set[String] = Set(KafkaConfig.LogCleanerThreadsProp)
+      override def reconfigurableConfigs: collection.Set[String] = Set(CleanerConfig.LOG_CLEANER_THREADS_PROP)
       override def validateReconfiguration(newConfig: KafkaConfig): Unit = validateLogCleanerConfig(newConfig.originals)
       override def reconfigure(oldConfig: KafkaConfig, newConfig: KafkaConfig): Unit = {}
     }
@@ -260,8 +260,8 @@ class DynamicBrokerConfigTest {
   def testReconfigurableValidation(): Unit = {
     val origProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
     val config = KafkaConfig(origProps)
-    val invalidReconfigurableProps = Set(KafkaConfig.LogCleanerThreadsProp, KafkaConfig.BrokerIdProp, "some.prop")
-    val validReconfigurableProps = Set(KafkaConfig.LogCleanerThreadsProp, KafkaConfig.LogCleanerDedupeBufferSizeProp, "some.prop")
+    val invalidReconfigurableProps = Set(CleanerConfig.LOG_CLEANER_THREADS_PROP, KafkaConfig.BrokerIdProp, "some.prop")
+    val validReconfigurableProps = Set(CleanerConfig.LOG_CLEANER_THREADS_PROP, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "some.prop")
 
     def createReconfigurable(configs: Set[String]) = new Reconfigurable {
       override def configure(configs: util.Map[String, _]): Unit = {}
diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
index e9426245b1..a5d4d961fe 100755
--- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
@@ -40,7 +40,7 @@ import org.apache.kafka.server.common.MetadataVersion
 import org.apache.kafka.server.common.MetadataVersion.{IBP_0_8_2, IBP_3_0_IV1}
 import org.apache.kafka.server.config.ServerTopicConfigSynonyms
 import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
-import org.apache.kafka.storage.internals.log.LogConfig
+import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig}
 import org.junit.jupiter.api.function.Executable
 
 import scala.annotation.nowarn
@@ -845,14 +845,14 @@ class KafkaConfigTest {
         case KafkaConfig.LogRetentionBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
         case KafkaConfig.LogCleanupIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
         case KafkaConfig.LogCleanupPolicyProp => assertPropertyInvalid(baseProperties, name, "unknown_policy", "0")
-        case KafkaConfig.LogCleanerIoMaxBytesPerSecondProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
-        case KafkaConfig.LogCleanerDedupeBufferSizeProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "1024")
-        case KafkaConfig.LogCleanerDedupeBufferLoadFactorProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
-        case KafkaConfig.LogCleanerEnableProp => assertPropertyInvalid(baseProperties, name, "not_a_boolean")
-        case KafkaConfig.LogCleanerDeleteRetentionMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
-        case KafkaConfig.LogCleanerMinCompactionLagMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
-        case KafkaConfig.LogCleanerMaxCompactionLagMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
-        case KafkaConfig.LogCleanerMinCleanRatioProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
+        case CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number")
+        case CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", "1024")
+        case CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number")
+        case CleanerConfig.LOG_CLEANER_ENABLE_PROP => assertPropertyInvalid(baseProperties, name, "not_a_boolean")
+        case CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number")
+        case CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number")
+        case CleanerConfig.LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number")
+        case CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number")
         case KafkaConfig.LogIndexSizeMaxBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "3")
         case KafkaConfig.LogFlushIntervalMessagesProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
         case KafkaConfig.LogFlushSchedulerIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index 7b1ce80cbe..17ee3139a5 100755
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -373,7 +373,7 @@ object TestUtils extends Logging {
     props.put(KafkaConfig.DeleteTopicEnableProp, enableDeleteTopic.toString)
     props.put(KafkaConfig.LogDeleteDelayMsProp, "1000")
     props.put(KafkaConfig.ControlledShutdownRetryBackoffMsProp, "100")
-    props.put(KafkaConfig.LogCleanerDedupeBufferSizeProp, "2097152")
+    props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "2097152")
     props.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
     if (!props.containsKey(KafkaConfig.OffsetsTopicPartitionsProp))
       props.put(KafkaConfig.OffsetsTopicPartitionsProp, "5")
diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java
index 320de2db6b..02cdc1cc87 100644
--- a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java
+++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java
@@ -30,7 +30,7 @@
 
 public final class ServerTopicConfigSynonyms {
     private static final String LOG_PREFIX = "log.";
-    private static final String LOG_CLEANER_PREFIX = LOG_PREFIX + "cleaner.";
+    public static final String LOG_CLEANER_PREFIX = LOG_PREFIX + "cleaner.";
 
     /**
      * Maps topic configurations to their equivalent broker configurations.
diff --git a/server/src/main/java/org/apache/kafka/server/config/Defaults.java b/server/src/main/java/org/apache/kafka/server/config/Defaults.java
index e55f641790..0c7ca123b8 100644
--- a/server/src/main/java/org/apache/kafka/server/config/Defaults.java
+++ b/server/src/main/java/org/apache/kafka/server/config/Defaults.java
@@ -106,13 +106,6 @@ public class Defaults {
     public static final int NUM_PARTITIONS = 1;
     public static final String LOG_DIR = "/tmp/kafka-logs";
     public static final long LOG_CLEANUP_INTERVAL_MS = 5 * 60 * 1000L;
-    public static final int LOG_CLEANER_THREADS = 1;
-    public static final double LOG_CLEANER_IO_MAX_BYTES_PER_SECOND = Double.MAX_VALUE;
-    public static final long LOG_CLEANER_DEDUPE_BUFFER_SIZE = 128 * 1024 * 1024L;
-    public static final int LOG_CLEANER_IO_BUFFER_SIZE = 512 * 1024;
-    public static final double LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR = 0.9d;
-    public static final int LOG_CLEANER_BACKOFF_MS = 15 * 1000;
-    public static final boolean LOG_CLEANER_ENABLE = true;
     public static final int LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS = 60000;
     public static final int LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS = 60000;
     public static final int NUM_RECOVERY_THREADS_PER_DATA_DIR = 1;
diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/CleanerConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/CleanerConfig.java
index 9e38d0c02e..8168197fe0 100644
--- a/storage/src/main/java/org/apache/kafka/storage/internals/log/CleanerConfig.java
+++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/CleanerConfig.java
@@ -16,11 +16,53 @@
  */
 package org.apache.kafka.storage.internals.log;
 
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.server.config.ServerTopicConfigSynonyms;
+
 /**
  * Configuration parameters for the log cleaner.
  */
 public class CleanerConfig {
     public static final String HASH_ALGORITHM = "MD5";
+    public static final int LOG_CLEANER_THREADS = 1;
+    public static final double LOG_CLEANER_IO_MAX_BYTES_PER_SECOND = Double.MAX_VALUE;
+    public static final long LOG_CLEANER_DEDUPE_BUFFER_SIZE = 128 * 1024 * 1024L;
+    public static final int LOG_CLEANER_IO_BUFFER_SIZE = 512 * 1024;
+    public static final double LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR = 0.9d;
+    public static final int LOG_CLEANER_BACKOFF_MS = 15 * 1000;
+    public static final boolean LOG_CLEANER_ENABLE = true;
+
+    public static final String LOG_CLEANER_THREADS_PROP = ServerTopicConfigSynonyms.LOG_CLEANER_PREFIX + "threads";
+    public static final String LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP = ServerTopicConfigSynonyms.LOG_CLEANER_PREFIX + "io.max.bytes.per.second";
+    public static final String LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP = ServerTopicConfigSynonyms.LOG_CLEANER_PREFIX + "dedupe.buffer.size";
+    public static final String LOG_CLEANER_IO_BUFFER_SIZE_PROP = ServerTopicConfigSynonyms.LOG_CLEANER_PREFIX + "io.buffer.size";
+    public static final String LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP = ServerTopicConfigSynonyms.LOG_CLEANER_PREFIX + "io.buffer.load.factor";
+    public static final String LOG_CLEANER_BACKOFF_MS_PROP = ServerTopicConfigSynonyms.LOG_CLEANER_PREFIX + "backoff.ms";
+    public static final String LOG_CLEANER_MIN_CLEAN_RATIO_PROP = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG);
+    public static final String LOG_CLEANER_ENABLE_PROP = ServerTopicConfigSynonyms.LOG_CLEANER_PREFIX + "enable";
+    public static final String LOG_CLEANER_DELETE_RETENTION_MS_PROP = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.DELETE_RETENTION_MS_CONFIG);
+    public static final String LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG);
+    public static final String LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG);
+
+    public static final String LOG_CLEANER_MIN_CLEAN_RATIO_DOC = "The minimum ratio of dirty log to total log for a log to eligible for cleaning. " +
+            "If the " + LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP + " or the " + LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP +
+            " configurations are also specified, then the log compactor considers the log eligible for compaction " +
+            "as soon as either: (i) the dirty ratio threshold has been met and the log has had dirty (uncompacted) " +
+            "records for at least the " + LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP + " duration, or (ii) if the log has had " +
+            "dirty (uncompacted) records for at most the " + LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP + " period.";
+    public static final String LOG_CLEANER_THREADS_DOC = "The number of background threads to use for log cleaning";
+    public static final String LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_DOC = "The log cleaner will be throttled so that the sum of its read and write i/o will be less than this value on average";
+    public static final String LOG_CLEANER_DEDUPE_BUFFER_SIZE_DOC = "The total memory used for log deduplication across all cleaner threads";
+    public static final String LOG_CLEANER_IO_BUFFER_SIZE_DOC = "The total memory used for log cleaner I/O buffers across all cleaner threads";
+    public static final String LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_DOC = "Log cleaner dedupe buffer load factor. The percentage full the dedupe buffer can become. A higher value " +
+            "will allow more log to be cleaned at once but will lead to more hash collisions";
+    public static final String LOG_CLEANER_BACKOFF_MS_DOC = "The amount of time to sleep when there are no logs to clean";
+    public static final String LOG_CLEANER_ENABLE_DOC = "Enable the log cleaner process to run on the server. Should be enabled if using any topics with a cleanup.policy=compact including the internal offsets topic. If disabled those topics will not be compacted and continually grow in size.";
+    public static final String LOG_CLEANER_DELETE_RETENTION_MS_DOC = "The amount of time to retain tombstone message markers for log compacted topics. This setting also gives a bound " +
+            "on the time in which a consumer must complete a read if they begin from offset 0 to ensure that they get a valid snapshot of the final stage (otherwise  " +
+            "tombstones messages may be collected before a consumer completes their scan).";
+    public static final String LOG_CLEANER_MIN_COMPACTION_LAG_MS_DOC = "The minimum time a message will remain uncompacted in the log. Only applicable for logs that are being compacted.";
+    public static final String LOG_CLEANER_MAX_COMPACTION_LAG_MS_DOC = "The maximum time a message will remain ineligible for compaction in the log. Only applicable for logs that are being compacted.";
 
     public final int numThreads;
     public final long dedupeBufferSize;
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java
index 3f3e9a243b..4232e1d74c 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java
@@ -24,6 +24,7 @@
 import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
 import org.apache.kafka.server.config.ConfigType;
 import org.apache.kafka.server.util.MockTime;
+import org.apache.kafka.storage.internals.log.CleanerConfig;
 import org.apache.kafka.test.TestCondition;
 import org.apache.kafka.test.TestUtils;
 import org.slf4j.Logger;
@@ -111,7 +112,7 @@ public void start() throws IOException {
         brokerConfig.put(KafkaConfig.ZkConnectProp(), zKConnectString());
         putIfAbsent(brokerConfig, KafkaConfig.ListenersProp(), "PLAINTEXT://localhost:" + DEFAULT_BROKER_PORT);
         putIfAbsent(brokerConfig, KafkaConfig.DeleteTopicEnableProp(), true);
-        putIfAbsent(brokerConfig, KafkaConfig.LogCleanerDedupeBufferSizeProp(), 2 * 1024 * 1024L);
+        putIfAbsent(brokerConfig, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, 2 * 1024 * 1024L);
         putIfAbsent(brokerConfig, KafkaConfig.GroupMinSessionTimeoutMsProp(), 0);
         putIfAbsent(brokerConfig, KafkaConfig.GroupInitialRebalanceDelayMsProp(), 0);
         putIfAbsent(brokerConfig, KafkaConfig.OffsetsTopicReplicationFactorProp(), (short) 1);

From 2f401ff4c85f6797391b8a3dd57d651f4de3d6ad Mon Sep 17 00:00:00 2001
From: Dongnuo Lyu <139248811+dongnuo123@users.noreply.github.com>
Date: Tue, 5 Mar 2024 09:50:58 -0500
Subject: [PATCH 024/521] MINOR: parameterize group-id in
 GroupMetadataManagerTestContext (#15467)

This pr parameterize some group ids in GroupMetadataManagerTestContext that are now constant strings.

Reviewers: Chia-Ping Tsai 
---
 .../group/GroupMetadataManagerTestContext.java       | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)

diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java
index d2d00c1582..ad70107506 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java
@@ -643,7 +643,7 @@ public JoinGroupResponseData joinClassicGroupAsDynamicMemberAndCompleteRebalance
 
         JoinGroupResponseData leaderJoinResponse =
             joinClassicGroupAsDynamicMemberAndCompleteJoin(new JoinGroupRequestBuilder()
-                .withGroupId("group-id")
+                .withGroupId(groupId)
                 .withMemberId(UNKNOWN_MEMBER_ID)
                 .withDefaultProtocolTypeAndProtocols()
                 .withRebalanceTimeoutMs(10000)
@@ -654,7 +654,7 @@ public JoinGroupResponseData joinClassicGroupAsDynamicMemberAndCompleteRebalance
         assertTrue(group.isInState(COMPLETING_REBALANCE));
 
         SyncResult syncResult = sendClassicGroupSync(new SyncGroupRequestBuilder()
-            .withGroupId("group-id")
+            .withGroupId(groupId)
             .withMemberId(leaderJoinResponse.memberId())
             .withGenerationId(leaderJoinResponse.generationId())
             .build());
@@ -804,7 +804,7 @@ public RebalanceResult staticMembersJoinAndRebalance(
         int rebalanceTimeoutMs,
         int sessionTimeoutMs
     ) throws Exception {
-        ClassicGroup group = createClassicGroup("group-id");
+        ClassicGroup group = createClassicGroup(groupId);
 
         JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder()
             .withGroupId(groupId)
@@ -901,7 +901,7 @@ public RebalanceResult staticMembersJoinAndRebalance(
     public PendingMemberGroupResult setupGroupWithPendingMember(ClassicGroup group) throws Exception {
         // Add the first member
         JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder()
-            .withGroupId("group-id")
+            .withGroupId(group.groupId())
             .withMemberId(UNKNOWN_MEMBER_ID)
             .withDefaultProtocolTypeAndProtocols()
             .withRebalanceTimeoutMs(10000)
@@ -914,7 +914,7 @@ public PendingMemberGroupResult setupGroupWithPendingMember(ClassicGroup group)
         List assignment = new ArrayList<>();
         assignment.add(new SyncGroupRequestData.SyncGroupRequestAssignment().setMemberId(leaderJoinResponse.memberId()));
         SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder()
-            .withGroupId("group-id")
+            .withGroupId(group.groupId())
             .withMemberId(leaderJoinResponse.memberId())
             .withGenerationId(leaderJoinResponse.generationId())
             .withAssignment(assignment)
@@ -1191,7 +1191,7 @@ public void verifyDescribeGroupsReturnsDeadGroup(String groupId) {
 
         assertEquals(
             Collections.singletonList(new DescribeGroupsResponseData.DescribedGroup()
-                .setGroupId("group-id")
+                .setGroupId(groupId)
                 .setGroupState(DEAD.toString())
             ),
             describedGroups

From 2c1943d836d828755af382b9d7996ff092854fe2 Mon Sep 17 00:00:00 2001
From: Colin Patrick McCabe 
Date: Tue, 5 Mar 2024 12:02:19 -0800
Subject: [PATCH 025/521] MINOR: remove test constructor for
 PartitionAssignment (#15435)

Remove the test constructor for PartitionAssignment and remove the TODO.
Also add KRaftClusterTest.testCreatePartitions to get more coverage for
createPartitions.

Reviewers: David Arthur , Chia-Ping Tsai 
---
 .../kafka/server/KRaftClusterTest.scala       | 35 ++++++++++++++++++-
 .../placement/PartitionAssignment.java        |  6 ----
 .../PartitionChangeBuilderTest.java           |  6 ++--
 .../PartitionReassignmentReplicasTest.java    | 22 ++++++------
 .../ReplicationControlManagerTest.java        | 18 +++++-----
 .../placement/PartitionAssignmentTest.java    | 12 ++++---
 .../placement/StripedReplicaPlacerTest.java   | 19 +++++-----
 .../placement/TopicAssignmentTest.java        |  9 ++---
 8 files changed, 80 insertions(+), 47 deletions(-)

diff --git a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
index 130d0e5642..3be1b400ab 100644
--- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
+++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
@@ -26,7 +26,7 @@ import org.apache.kafka.clients.admin._
 import org.apache.kafka.common.acl.{AclBinding, AclBindingFilter}
 import org.apache.kafka.common.config.{ConfigException, ConfigResource}
 import org.apache.kafka.common.config.ConfigResource.Type
-import org.apache.kafka.common.errors.{PolicyViolationException, UnsupportedVersionException}
+import org.apache.kafka.common.errors.{InvalidPartitionsException,PolicyViolationException, UnsupportedVersionException}
 import org.apache.kafka.common.message.DescribeClusterRequestData
 import org.apache.kafka.common.metadata.{ConfigRecord, FeatureLevelRecord}
 import org.apache.kafka.common.metrics.Metrics
@@ -792,6 +792,39 @@ class KRaftClusterTest {
     }
   }
 
+  @ParameterizedTest
+  @ValueSource(strings = Array("3.7-IV0", "3.7-IV2"))
+  def testCreatePartitions(metadataVersionString: String): Unit = {
+    val cluster = new KafkaClusterTestKit.Builder(
+      new TestKitNodes.Builder().
+        setNumBrokerNodes(4).
+        setBootstrapMetadataVersion(MetadataVersion.fromVersionString(metadataVersionString)).
+        setNumControllerNodes(3).build()).
+      build()
+    try {
+      cluster.format()
+      cluster.startup()
+      cluster.waitForReadyBrokers()
+      val admin = Admin.create(cluster.clientProperties())
+      try {
+        val createResults = admin.createTopics(Arrays.asList(
+          new NewTopic("foo", 1, 3.toShort),
+          new NewTopic("bar", 2, 3.toShort))).values()
+        createResults.get("foo").get()
+        createResults.get("bar").get()
+        val increaseResults = admin.createPartitions(Map(
+          "foo" -> NewPartitions.increaseTo(3),
+          "bar" -> NewPartitions.increaseTo(2)).asJava).values()
+        increaseResults.get("foo").get()
+        assertEquals(classOf[InvalidPartitionsException], assertThrows(
+          classOf[ExecutionException], () => increaseResults.get("bar").get()).getCause.getClass)
+      } finally {
+        admin.close()
+      }
+    } finally {
+      cluster.close()
+    }
+  }
   private def clusterImage(
     cluster: KafkaClusterTestKit,
     brokerId: Int
diff --git a/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java b/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java
index 177d5311af..a7012d1505 100644
--- a/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java
+++ b/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java
@@ -17,7 +17,6 @@
 
 package org.apache.kafka.metadata.placement;
 
-import org.apache.kafka.common.DirectoryId;
 import org.apache.kafka.common.Uuid;
 
 import java.util.ArrayList;
@@ -39,11 +38,6 @@ public class PartitionAssignment {
     private final List replicas;
     private final List directories;
 
-    // TODO remove -- just here for testing
-    public PartitionAssignment(List replicas) {
-        this(replicas, brokerId -> DirectoryId.UNASSIGNED);
-    }
-
     public PartitionAssignment(List replicas, DefaultDirProvider defaultDirProvider) {
         this.replicas = Collections.unmodifiableList(new ArrayList<>(replicas));
         Uuid[] directories = new Uuid[replicas.size()];
diff --git a/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java b/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java
index efc9bd2a24..b956a7f2ef 100644
--- a/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java
+++ b/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java
@@ -27,7 +27,6 @@
 import org.apache.kafka.metadata.PartitionRegistration;
 import org.apache.kafka.metadata.Replicas;
 import org.apache.kafka.metadata.placement.DefaultDirProvider;
-import org.apache.kafka.metadata.placement.PartitionAssignment;
 import org.apache.kafka.server.common.ApiMessageAndVersion;
 import org.apache.kafka.server.common.MetadataVersion;
 import org.junit.jupiter.api.Test;
@@ -49,6 +48,7 @@
 import static org.apache.kafka.controller.PartitionChangeBuilder.changeRecordIsNoOp;
 import static org.apache.kafka.metadata.LeaderConstants.NO_LEADER;
 import static org.apache.kafka.metadata.LeaderConstants.NO_LEADER_CHANGE;
+import static org.apache.kafka.metadata.placement.PartitionAssignmentTest.partitionAssignment;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -502,7 +502,7 @@ public void testRevertReassignment(short version) {
     @MethodSource("partitionChangeRecordVersions")
     public void testRemovingReplicaReassignment(short version) {
         PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas(
-            new PartitionAssignment(Replicas.toList(FOO.replicas)), new PartitionAssignment(Arrays.asList(1, 2)));
+            partitionAssignment(Replicas.toList(FOO.replicas)), partitionAssignment(Arrays.asList(1, 2)));
         assertEquals(Collections.singletonList(3), replicas.removing());
         assertEquals(Collections.emptyList(), replicas.adding());
         assertEquals(Arrays.asList(1, 2, 3), replicas.replicas());
@@ -527,7 +527,7 @@ public void testRemovingReplicaReassignment(short version) {
     @MethodSource("partitionChangeRecordVersions")
     public void testAddingReplicaReassignment(short version) {
         PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas(
-            new PartitionAssignment(Replicas.toList(FOO.replicas)), new PartitionAssignment(Arrays.asList(1, 2, 3, 4)));
+            partitionAssignment(Replicas.toList(FOO.replicas)), partitionAssignment(Arrays.asList(1, 2, 3, 4)));
         assertEquals(Collections.emptyList(), replicas.removing());
         assertEquals(Collections.singletonList(4), replicas.adding());
         assertEquals(Arrays.asList(1, 2, 3, 4), replicas.replicas());
diff --git a/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentReplicasTest.java b/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentReplicasTest.java
index b2bc540bda..17be98d47f 100644
--- a/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentReplicasTest.java
+++ b/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentReplicasTest.java
@@ -24,10 +24,10 @@
 import org.apache.kafka.common.Uuid;
 import org.apache.kafka.metadata.LeaderRecoveryState;
 import org.apache.kafka.metadata.PartitionRegistration;
-import org.apache.kafka.metadata.placement.PartitionAssignment;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.Timeout;
 
+import static org.apache.kafka.metadata.placement.PartitionAssignmentTest.partitionAssignment;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -38,7 +38,7 @@ public class PartitionReassignmentReplicasTest {
     @Test
     public void testNoneAddedOrRemoved() {
         PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas(
-            new PartitionAssignment(Arrays.asList(3, 2, 1)), new PartitionAssignment(Arrays.asList(3, 2, 1)));
+            partitionAssignment(Arrays.asList(3, 2, 1)), partitionAssignment(Arrays.asList(3, 2, 1)));
         assertEquals(Collections.emptyList(), replicas.removing());
         assertEquals(Collections.emptyList(), replicas.adding());
         assertEquals(Arrays.asList(3, 2, 1), replicas.replicas());
@@ -47,7 +47,7 @@ public void testNoneAddedOrRemoved() {
     @Test
     public void testAdditions() {
         PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas(
-            new PartitionAssignment(Arrays.asList(3, 2, 1)), new PartitionAssignment(Arrays.asList(3, 6, 2, 1, 5)));
+            partitionAssignment(Arrays.asList(3, 2, 1)), partitionAssignment(Arrays.asList(3, 6, 2, 1, 5)));
         assertEquals(Collections.emptyList(), replicas.removing());
         assertEquals(Arrays.asList(5, 6), replicas.adding());
         assertEquals(Arrays.asList(3, 6, 2, 1, 5), replicas.replicas());
@@ -56,7 +56,7 @@ public void testAdditions() {
     @Test
     public void testRemovals() {
         PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas(
-            new PartitionAssignment(Arrays.asList(3, 2, 1, 0)), new PartitionAssignment(Arrays.asList(3, 1)));
+            partitionAssignment(Arrays.asList(3, 2, 1, 0)), partitionAssignment(Arrays.asList(3, 1)));
         assertEquals(Arrays.asList(0, 2), replicas.removing());
         assertEquals(Collections.emptyList(), replicas.adding());
         assertEquals(Arrays.asList(3, 1, 0, 2), replicas.replicas());
@@ -65,7 +65,7 @@ public void testRemovals() {
     @Test
     public void testAdditionsAndRemovals() {
         PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas(
-            new PartitionAssignment(Arrays.asList(3, 2, 1, 0)), new PartitionAssignment(Arrays.asList(7, 3, 1, 9)));
+            partitionAssignment(Arrays.asList(3, 2, 1, 0)), partitionAssignment(Arrays.asList(7, 3, 1, 9)));
         assertEquals(Arrays.asList(0, 2), replicas.removing());
         assertEquals(Arrays.asList(7, 9), replicas.adding());
         assertEquals(Arrays.asList(7, 3, 1, 9, 0, 2), replicas.replicas());
@@ -74,7 +74,7 @@ public void testAdditionsAndRemovals() {
     @Test
     public void testRearrangement() {
         PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas(
-            new PartitionAssignment(Arrays.asList(3, 2, 1, 0)), new PartitionAssignment(Arrays.asList(0, 1, 3, 2)));
+            partitionAssignment(Arrays.asList(3, 2, 1, 0)), partitionAssignment(Arrays.asList(0, 1, 3, 2)));
         assertEquals(Collections.emptyList(), replicas.removing());
         assertEquals(Collections.emptyList(), replicas.adding());
         assertEquals(Arrays.asList(0, 1, 3, 2), replicas.replicas());
@@ -83,7 +83,7 @@ public void testRearrangement() {
     @Test
     public void testDoesNotCompleteReassignment() {
         PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas(
-            new PartitionAssignment(Arrays.asList(0, 1, 2)), new PartitionAssignment(Arrays.asList(3, 4, 5)));
+            partitionAssignment(Arrays.asList(0, 1, 2)), partitionAssignment(Arrays.asList(3, 4, 5)));
         assertTrue(replicas.isReassignmentInProgress());
         Optional reassignmentOptional =
             replicas.maybeCompleteReassignment(Arrays.asList(0, 1, 2, 3, 4));
@@ -107,7 +107,7 @@ public void testDoesNotCompleteReassignmentIfNoneOngoing() {
     @Test
     public void testDoesCompleteReassignmentAllNewReplicas() {
         PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas(
-            new PartitionAssignment(Arrays.asList(0, 1, 2)), new PartitionAssignment(Arrays.asList(3, 4, 5)));
+            partitionAssignment(Arrays.asList(0, 1, 2)), partitionAssignment(Arrays.asList(3, 4, 5)));
         assertTrue(replicas.isReassignmentInProgress());
         Optional reassignmentOptional =
             replicas.maybeCompleteReassignment(Arrays.asList(0, 1, 2, 3, 4, 5));
@@ -120,7 +120,7 @@ public void testDoesCompleteReassignmentAllNewReplicas() {
     @Test
     public void testDoesCompleteReassignmentSomeNewReplicas() {
         PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas(
-            new PartitionAssignment(Arrays.asList(0, 1, 2)), new PartitionAssignment(Arrays.asList(0, 1, 3)));
+            partitionAssignment(Arrays.asList(0, 1, 2)), partitionAssignment(Arrays.asList(0, 1, 3)));
         assertTrue(replicas.isReassignmentInProgress());
         Optional reassignmentOptional =
             replicas.maybeCompleteReassignment(Arrays.asList(0, 1, 2, 3));
@@ -199,7 +199,7 @@ public void testIsReassignmentInProgress() {
     @Test
     public void testDoesNotCompleteReassignmentIfIsrDoesNotHaveAllTargetReplicas() {
         PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas(
-            new PartitionAssignment(Arrays.asList(0, 1, 2)), new PartitionAssignment(Arrays.asList(0, 1, 3)));
+            partitionAssignment(Arrays.asList(0, 1, 2)), partitionAssignment(Arrays.asList(0, 1, 3)));
         assertTrue(replicas.isReassignmentInProgress());
         Optional reassignmentOptional =
             replicas.maybeCompleteReassignment(Arrays.asList(3));
@@ -209,7 +209,7 @@ public void testDoesNotCompleteReassignmentIfIsrDoesNotHaveAllTargetReplicas() {
     @Test
     public void testOriginalReplicas() {
         PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas(
-            new PartitionAssignment(Arrays.asList(0, 1, 2)), new PartitionAssignment(Arrays.asList(0, 1, 3)));
+            partitionAssignment(Arrays.asList(0, 1, 2)), partitionAssignment(Arrays.asList(0, 1, 3)));
         assertEquals(Arrays.asList(0, 1, 2), replicas.originalReplicas());
     }
 }
diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
index 3d54720be9..e9885fa8e9 100644
--- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
+++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
@@ -87,7 +87,6 @@
 import org.apache.kafka.metadata.PartitionRegistration;
 import org.apache.kafka.metadata.RecordTestUtils;
 import org.apache.kafka.metadata.Replicas;
-import org.apache.kafka.metadata.placement.PartitionAssignment;
 import org.apache.kafka.metadata.placement.StripedReplicaPlacer;
 import org.apache.kafka.metadata.placement.UsableBroker;
 import org.apache.kafka.server.common.ApiMessageAndVersion;
@@ -148,6 +147,7 @@
 import static org.apache.kafka.controller.ControllerRequestContextUtil.anonymousContextFor;
 import static org.apache.kafka.controller.ControllerRequestContextUtil.anonymousContextWithMutationQuotaExceededFor;
 import static org.apache.kafka.metadata.LeaderConstants.NO_LEADER;
+import static org.apache.kafka.metadata.placement.PartitionAssignmentTest.partitionAssignment;
 import static org.junit.jupiter.api.Assertions.assertArrayEquals;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
@@ -1584,13 +1584,13 @@ public void testCreatePartitionsISRInvariants() throws Exception {
     public void testValidateGoodManualPartitionAssignments() throws Exception {
         ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder().build();
         ctx.registerBrokers(1, 2, 3);
-        ctx.replicationControl.validateManualPartitionAssignment(new PartitionAssignment(asList(1)),
+        ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(asList(1)),
             OptionalInt.of(1));
-        ctx.replicationControl.validateManualPartitionAssignment(new PartitionAssignment(asList(1)),
+        ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(asList(1)),
             OptionalInt.empty());
-        ctx.replicationControl.validateManualPartitionAssignment(new PartitionAssignment(asList(1, 2, 3)),
+        ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(asList(1, 2, 3)),
             OptionalInt.of(3));
-        ctx.replicationControl.validateManualPartitionAssignment(new PartitionAssignment(asList(1, 2, 3)),
+        ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(asList(1, 2, 3)),
             OptionalInt.empty());
     }
 
@@ -1600,20 +1600,20 @@ public void testValidateBadManualPartitionAssignments() throws Exception {
         ctx.registerBrokers(1, 2);
         assertEquals("The manual partition assignment includes an empty replica list.",
             assertThrows(InvalidReplicaAssignmentException.class, () ->
-                ctx.replicationControl.validateManualPartitionAssignment(new PartitionAssignment(asList()),
+                ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(asList()),
                     OptionalInt.empty())).getMessage());
         assertEquals("The manual partition assignment includes broker 3, but no such " +
             "broker is registered.", assertThrows(InvalidReplicaAssignmentException.class, () ->
-                ctx.replicationControl.validateManualPartitionAssignment(new PartitionAssignment(asList(1, 2, 3)),
+                ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(asList(1, 2, 3)),
                     OptionalInt.empty())).getMessage());
         assertEquals("The manual partition assignment includes the broker 2 more than " +
             "once.", assertThrows(InvalidReplicaAssignmentException.class, () ->
-                ctx.replicationControl.validateManualPartitionAssignment(new PartitionAssignment(asList(1, 2, 2)),
+                ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(asList(1, 2, 2)),
                     OptionalInt.empty())).getMessage());
         assertEquals("The manual partition assignment includes a partition with 2 " +
             "replica(s), but this is not consistent with previous partitions, which have " +
                 "3 replica(s).", assertThrows(InvalidReplicaAssignmentException.class, () ->
-                    ctx.replicationControl.validateManualPartitionAssignment(new PartitionAssignment(asList(1, 2)),
+                    ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(asList(1, 2)),
                         OptionalInt.of(3))).getMessage());
     }
 
diff --git a/metadata/src/test/java/org/apache/kafka/metadata/placement/PartitionAssignmentTest.java b/metadata/src/test/java/org/apache/kafka/metadata/placement/PartitionAssignmentTest.java
index 06cf5ae50d..6dca18b4dd 100644
--- a/metadata/src/test/java/org/apache/kafka/metadata/placement/PartitionAssignmentTest.java
+++ b/metadata/src/test/java/org/apache/kafka/metadata/placement/PartitionAssignmentTest.java
@@ -20,6 +20,7 @@
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotEquals;
 
+import org.apache.kafka.common.DirectoryId;
 import org.apache.kafka.common.Uuid;
 import org.junit.jupiter.api.Test;
 
@@ -27,20 +28,23 @@
 import java.util.List;
 
 public class PartitionAssignmentTest {
+    public static PartitionAssignment partitionAssignment(List replicas) {
+        return new PartitionAssignment(replicas, __ -> DirectoryId.MIGRATING);
+    }
 
     @Test
     public void testPartitionAssignmentReplicas() {
         List replicas = Arrays.asList(0, 1, 2);
-        assertEquals(replicas, new PartitionAssignment(replicas).replicas());
+        assertEquals(replicas, partitionAssignment(replicas).replicas());
     }
 
     @Test
     public void testConsistentEqualsAndHashCode() {
         List partitionAssignments = Arrays.asList(
-            new PartitionAssignment(
+            partitionAssignment(
                 Arrays.asList(0, 1, 2)
             ),
-            new PartitionAssignment(
+            partitionAssignment(
                 Arrays.asList(1, 2, 0)
             )
         );
@@ -49,7 +53,7 @@ public void testConsistentEqualsAndHashCode() {
             for (int j = 0; j < partitionAssignments.size(); j++) {
                 if (i == j) {
                     assertEquals(partitionAssignments.get(i), partitionAssignments.get(j));
-                    assertEquals(partitionAssignments.get(i), new PartitionAssignment(partitionAssignments.get(i).replicas()));
+                    assertEquals(partitionAssignments.get(i), partitionAssignment(partitionAssignments.get(i).replicas()));
                     assertEquals(partitionAssignments.get(i).hashCode(), partitionAssignments.get(j).hashCode());
                 } else {
                     assertNotEquals(partitionAssignments.get(i), partitionAssignments.get(j));
diff --git a/metadata/src/test/java/org/apache/kafka/metadata/placement/StripedReplicaPlacerTest.java b/metadata/src/test/java/org/apache/kafka/metadata/placement/StripedReplicaPlacerTest.java
index 8b02416d2b..924fcdb755 100644
--- a/metadata/src/test/java/org/apache/kafka/metadata/placement/StripedReplicaPlacerTest.java
+++ b/metadata/src/test/java/org/apache/kafka/metadata/placement/StripedReplicaPlacerTest.java
@@ -34,6 +34,7 @@
 import java.util.Map;
 import java.util.Optional;
 
+import static org.apache.kafka.metadata.placement.PartitionAssignmentTest.partitionAssignment;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 
@@ -105,7 +106,7 @@ public Iterator usableBrokers() {
 
             @Override
             public Uuid defaultDir(int brokerId) {
-                return DirectoryId.UNASSIGNED;
+                return DirectoryId.MIGRATING;
             }
         });
     }
@@ -118,9 +119,9 @@ public Uuid defaultDir(int brokerId) {
     public void testMultiPartitionTopicPlacementOnSingleUnfencedBroker() {
         MockRandom random = new MockRandom();
         StripedReplicaPlacer placer = new StripedReplicaPlacer(random);
-        assertEquals(new TopicAssignment(Arrays.asList(new PartitionAssignment(Arrays.asList(0)),
-                new PartitionAssignment(Arrays.asList(0)),
-                new PartitionAssignment(Arrays.asList(0)))),
+        assertEquals(new TopicAssignment(Arrays.asList(partitionAssignment(Arrays.asList(0)),
+                partitionAssignment(Arrays.asList(0)),
+                partitionAssignment(Arrays.asList(0)))),
                 place(placer, 0, 3, (short) 1, Arrays.asList(
                         new UsableBroker(0, Optional.empty(), false),
                         new UsableBroker(1, Optional.empty(), true))));
@@ -224,11 +225,11 @@ public void testNonPositiveReplicationFactor() {
     public void testSuccessfulPlacement() {
         MockRandom random = new MockRandom();
         StripedReplicaPlacer placer = new StripedReplicaPlacer(random);
-        assertEquals(new TopicAssignment(Arrays.asList(new PartitionAssignment(Arrays.asList(2, 3, 0)),
-                new PartitionAssignment(Arrays.asList(3, 0, 1)),
-                new PartitionAssignment(Arrays.asList(0, 1, 2)),
-                new PartitionAssignment(Arrays.asList(1, 2, 3)),
-                new PartitionAssignment(Arrays.asList(1, 0, 2)))),
+        assertEquals(new TopicAssignment(Arrays.asList(partitionAssignment(Arrays.asList(2, 3, 0)),
+                partitionAssignment(Arrays.asList(3, 0, 1)),
+                partitionAssignment(Arrays.asList(0, 1, 2)),
+                partitionAssignment(Arrays.asList(1, 2, 3)),
+                partitionAssignment(Arrays.asList(1, 0, 2)))),
             place(placer, 0, 5, (short) 3, Arrays.asList(
                 new UsableBroker(0, Optional.empty(), false),
                 new UsableBroker(3, Optional.empty(), false),
diff --git a/metadata/src/test/java/org/apache/kafka/metadata/placement/TopicAssignmentTest.java b/metadata/src/test/java/org/apache/kafka/metadata/placement/TopicAssignmentTest.java
index 7b5a24c3b8..26f8841d83 100644
--- a/metadata/src/test/java/org/apache/kafka/metadata/placement/TopicAssignmentTest.java
+++ b/metadata/src/test/java/org/apache/kafka/metadata/placement/TopicAssignmentTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.kafka.metadata.placement;
 
+import static org.apache.kafka.metadata.placement.PartitionAssignmentTest.partitionAssignment;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotEquals;
 
@@ -33,8 +34,8 @@ public void testTopicAssignmentReplicas() {
         List replicasP0 = Arrays.asList(0, 1, 2);
         List replicasP1 = Arrays.asList(1, 2, 0);
         List partitionAssignments = Arrays.asList(
-            new PartitionAssignment(replicasP0),
-            new PartitionAssignment(replicasP1)
+            partitionAssignment(replicasP0),
+            partitionAssignment(replicasP1)
         );
         assertEquals(partitionAssignments, new TopicAssignment(partitionAssignments).assignments());
     }
@@ -44,14 +45,14 @@ public void testConsistentEqualsAndHashCode() {
         List topicAssignments = Arrays.asList(
             new TopicAssignment(
                 Arrays.asList(
-                    new PartitionAssignment(
+                    partitionAssignment(
                         Arrays.asList(0, 1, 2)
                     )
                 )
             ),
             new TopicAssignment(
                 Arrays.asList(
-                    new PartitionAssignment(
+                    partitionAssignment(
                         Arrays.asList(1, 2, 0)
                     )
                 )

From 1ca939128521459ef921d3ec71dda2a507ba1f15 Mon Sep 17 00:00:00 2001
From: "Gyeongwon, Do" 
Date: Wed, 6 Mar 2024 06:16:35 +0900
Subject: [PATCH 026/521] MINOR: Remove controlPlaneRequestProcessor in
 BrokerServer (#15245)

It seems likely that BrokerServer was built upon the KafkaServer codebase.(#10113)
KafkaServer, using Zookeeper, separates controlPlane and dataPlane to implement KIP-291.
In KRaft, the roles of DataPlane and ControlPlane in KafkaServer seem to be divided into BrokerServer and ControllerServer.

It appears that the initial implementation of BrokerServer initialized and used the controlPlaneRequestProcessor, but it seems to have been removed, except for the code used in the shutdown method, through subsequent modifications.(#10931)

Reviewers: Chia-Ping Tsai 
---
 core/src/main/scala/kafka/server/BrokerServer.scala | 3 ---
 1 file changed, 3 deletions(-)

diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala
index dc72ceb35c..8ac568a5e3 100644
--- a/core/src/main/scala/kafka/server/BrokerServer.scala
+++ b/core/src/main/scala/kafka/server/BrokerServer.scala
@@ -96,7 +96,6 @@ class BrokerServer(
   var status: ProcessStatus = SHUTDOWN
 
   @volatile var dataPlaneRequestProcessor: KafkaApis = _
-  var controlPlaneRequestProcessor: KafkaApis = _
 
   var authorizer: Option[Authorizer] = None
   @volatile var socketServer: SocketServer = _
@@ -655,8 +654,6 @@ class BrokerServer(
         CoreUtils.swallow(dataPlaneRequestHandlerPool.shutdown(), this)
       if (dataPlaneRequestProcessor != null)
         CoreUtils.swallow(dataPlaneRequestProcessor.close(), this)
-      if (controlPlaneRequestProcessor != null)
-        CoreUtils.swallow(controlPlaneRequestProcessor.close(), this)
       CoreUtils.swallow(authorizer.foreach(_.close()), this)
 
       /**

From 554fa57af85ec337a556f35cbc6d2970ff252dc4 Mon Sep 17 00:00:00 2001
From: John Yu <54207775+chiacyu@users.noreply.github.com>
Date: Wed, 6 Mar 2024 09:00:58 +0800
Subject: [PATCH 027/521] KAFKA-16209 : fetchSnapshot might return null if
 topic is created before v2.8 (#15444)

Change the function with a better way to deal with the NULL pointer exception.

Reviewers: Luke Chen 
---
 .../scala/unit/kafka/log/ProducerStateManagerTest.scala     | 6 ++++++
 .../kafka/storage/internals/log/ProducerStateManager.java   | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)

diff --git a/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala b/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala
index 03aa847ded..810e0b1e4e 100644
--- a/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala
@@ -579,6 +579,12 @@ class ProducerStateManagerTest {
     assertTrue(logDir.list().head.nonEmpty, "Snapshot file is empty")
   }
 
+  @Test
+  def testFetchSnapshotEmptySnapShot(): Unit = {
+    val offset = 1
+    assertEquals(Optional.empty(), stateManager.fetchSnapshot(offset))
+  }
+
   @Test
   def testRecoverFromSnapshotUnfinishedTransaction(): Unit = {
     val epoch = 0.toShort
diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java
index 270aa0a42f..da56ddaccc 100644
--- a/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java
+++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java
@@ -605,7 +605,7 @@ public void deleteSnapshotsBefore(long offset) throws IOException {
     }
 
     public Optional fetchSnapshot(long offset) {
-        return Optional.of(snapshots.get(offset)).map(x -> x.file());
+        return Optional.ofNullable(snapshots.get(offset)).map(x -> x.file());
     }
 
     private Optional oldestSnapshotFile() {

From e81379d3fea956dd8900b7f4b68e0c1328401871 Mon Sep 17 00:00:00 2001
From: Victor van den Hoven 
Date: Wed, 6 Mar 2024 02:06:20 +0100
Subject: [PATCH 028/521] KAFKA-15417: flip joinSpuriousLookBackTimeMs and emit
 non-joined items (#14426)

Kafka Streams support asymmetric join windows. Depending on the window configuration
we need to compute window close time etc differently.

This PR flips `joinSpuriousLookBackTimeMs`, because they were not correct, and
introduced the `windowsAfterIntervalMs`-field that is used to find if emitting records can be skipped.

Reviewers: Hao Li , Guozhang Wang , Matthias J. Sax 
---
 .../kstream/internals/KStreamKStreamJoin.java |  38 +++-
 .../KStreamKStreamIntegrationTest.java        |   2 +
 .../internals/KStreamKStreamLeftJoinTest.java | 183 +++++++++++++++++-
 .../KStreamKStreamOuterJoinTest.java          |  41 ++--
 4 files changed, 236 insertions(+), 28 deletions(-)

diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java
index 603e1e8255..124386b9bc 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java
@@ -51,7 +51,8 @@ class KStreamKStreamJoin implements ProcessorSupplier implements ProcessorSupplier record) {
                 return;
             }
 
-            boolean needOuterJoin = outer;
             // Emit all non-joined records which window has closed
             if (inputRecordTimestamp == sharedTimeTracker.streamTime) {
                 outerJoinStore.ifPresent(store -> emitNonJoinedOuterRecords(store, record));
             }
+
+            boolean needOuterJoin = outer;
             try (final WindowStoreIterator iter = otherWindowStore.fetch(record.key(), timeFrom, timeTo)) {
                 while (iter.hasNext()) {
                     needOuterJoin = false;
@@ -200,7 +202,7 @@ private void emitNonJoinedOuterRecords(
             // to reduce runtime cost, we try to avoid paying those cost
 
             // only try to emit left/outer join results if there _might_ be any result records
-            if (sharedTimeTracker.minTime >= sharedTimeTracker.streamTime - joinSpuriousLookBackTimeMs - joinGraceMs) {
+            if (sharedTimeTracker.minTime + joinAfterMs + joinGraceMs >= sharedTimeTracker.streamTime) {
                 return;
             }
             // throttle the emit frequency to a (configurable) interval;
@@ -222,6 +224,8 @@ private void emitNonJoinedOuterRecords(
                 TimestampedKeyAndJoinSide prevKey = null;
 
                 while (it.hasNext()) {
+                    boolean outerJoinLeftBreak = false;
+                    boolean outerJoinRightBreak = false;
                     final KeyValue, LeftOrRightValue> next = it.next();
                     final TimestampedKeyAndJoinSide timestampedKeyAndJoinSide = next.key;
                     final LeftOrRightValue value = next.value;
@@ -230,8 +234,19 @@ private void emitNonJoinedOuterRecords(
                     sharedTimeTracker.minTime = timestamp;
 
                     // Skip next records if window has not closed
-                    if (timestamp + joinAfterMs + joinGraceMs >= sharedTimeTracker.streamTime) {
-                        break;
+                    final long outerJoinLookBackTimeMs = getOuterJoinLookBackTimeMs(timestampedKeyAndJoinSide);
+                    if (sharedTimeTracker.minTime + outerJoinLookBackTimeMs + joinGraceMs >= sharedTimeTracker.streamTime) {
+                        if (timestampedKeyAndJoinSide.isLeftSide()) {
+                            outerJoinLeftBreak = true; // there are no more candidates to emit on left-outerJoin-side
+                        } else {
+                            outerJoinRightBreak = true; // there are no more candidates to emit on right-outerJoin-side
+                        }
+                        if (outerJoinLeftBreak && outerJoinRightBreak) {
+                            break; // there are no more candidates to emit on left-outerJoin-side and
+                                    // right-outerJoin-side
+                        } else {
+                            continue; // there are possibly candidates left on the other outerJoin-side
+                        }
                     }
 
                     final VOut nullJoinedValue;
@@ -268,6 +283,15 @@ private void emitNonJoinedOuterRecords(
             }
         }
 
+        private long getOuterJoinLookBackTimeMs(final TimestampedKeyAndJoinSide timestampedKeyAndJoinSide) {
+            // depending on the JoinSide we fill in the outerJoinLookBackTimeMs
+            if (timestampedKeyAndJoinSide.isLeftSide()) {
+                return windowsAfterMs; // On the left-JoinSide we look back in time
+            } else {
+                return windowsBeforeMs; // On the right-JoinSide we look forward in time
+            }
+        }
+
         @Override
         public void close() {
             sharedTimeTrackerSupplier.remove(context().taskId());
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKStreamIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKStreamIntegrationTest.java
index 1d9a77b5bf..10ab37cee0 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKStreamIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKStreamIntegrationTest.java
@@ -27,6 +27,7 @@
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.StreamsBuilder;
 import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.StreamsConfig.InternalConfig;
 import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
 import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
 import org.apache.kafka.streams.kstream.JoinWindows;
@@ -99,6 +100,7 @@ public void before(final TestInfo testInfo) throws IOException {
         final String safeTestName = safeUniqueTestName(testInfo);
         streamsConfig = getStreamsConfig(safeTestName);
         streamsConfig.put(StreamsConfig.STATE_DIR_CONFIG, stateDirBasePath);
+        streamsConfig.put(InternalConfig.EMIT_INTERVAL_MS_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX, 0L);
     }
 
     @AfterEach
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java
index 156b553455..fd36b241b2 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java
@@ -436,6 +436,184 @@ public void testRightNonJoinedRecordsAreNeverEmittedByTheRightProcessor() {
         }
     }
 
+    @Test
+    public void testLeftJoinedRecordsWithZeroAfterAreEmitted() {
+        final StreamsBuilder builder = new StreamsBuilder();
+
+        final int[] expectedKeys = new int[] {0, 1, 2, 3};
+
+        final KStream stream1;
+        final KStream stream2;
+        final KStream joined;
+        final MockApiProcessorSupplier supplier = new MockApiProcessorSupplier<>();
+        stream1 = builder.stream(topic1, consumed);
+        stream2 = builder.stream(topic2, consumed);
+        
+        joined = stream1.leftJoin(
+            stream2,
+            MockValueJoiner.TOSTRING_JOINER,
+            JoinWindows.ofTimeDifferenceWithNoGrace(ofMillis(100)).after(ZERO),
+            StreamJoined.with(Serdes.Integer(),
+                Serdes.String(),
+                Serdes.String())
+        );
+        joined.process(supplier);
+
+        final Collection> copartitionGroups =
+            TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
+
+        assertEquals(1, copartitionGroups.size());
+        assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
+
+        try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) {
+            final TestInputTopic inputTopic1 =
+                    driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO);
+            final TestInputTopic inputTopic2 =
+                    driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO);
+            final MockApiProcessor processor = supplier.theCapturedProcessor();
+
+            processor.init(null);
+            
+            // push four items with increasing timestamps to the primary stream; the other window is empty; 
+            // this should emit the first three left-joined items;
+            // A3 is not triggered yet
+            // w1 = {}
+            // w2 = {}
+            // --> w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
+            //     w2 = {}
+            long time = 1000L;
+            for (int i = 0; i < expectedKeys.length; i++) {
+                inputTopic1.pipeInput(expectedKeys[i], "A" + expectedKeys[i], time + i);
+            }
+            processor.checkAndClearProcessResult(
+                    new KeyValueTimestamp<>(0, "A0+null", 1000L),
+                    new KeyValueTimestamp<>(1, "A1+null", 1001L),
+                    new KeyValueTimestamp<>(2, "A2+null", 1002L)
+            );
+            
+            // push four items smaller timestamps (out of window) to the secondary stream; 
+            // this should produce four joined items
+            // w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
+            // w2 = {}
+            // --> w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
+            //     w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999) }
+            time = 1000L - 1L;
+            for (final int expectedKey : expectedKeys) {
+                inputTopic2.pipeInput(expectedKey, "a" + expectedKey, time);
+            }
+            processor.checkAndClearProcessResult(
+                    new KeyValueTimestamp<>(0, "A0+a0", 1000L),
+                    new KeyValueTimestamp<>(1, "A1+a1", 1001L),
+                    new KeyValueTimestamp<>(2, "A2+a2", 1002L),
+                    new KeyValueTimestamp<>(3, "A3+a3", 1003L)
+            );
+
+            // push four items with increased timestamps to the secondary stream; 
+            // this should produce four joined item
+            // w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
+            // w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999) }
+            // --> w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
+            //     w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999),
+            //            0:b0 (ts: 1000), 1:b1 (ts: 1000), 2:b2 (ts: 1000), 3:b3 (ts: 1000) }
+            time += 1L;
+            for (final int expectedKey : expectedKeys) {
+                inputTopic2.pipeInput(expectedKey, "b" + expectedKey, time);
+            }
+            processor.checkAndClearProcessResult(
+                    new KeyValueTimestamp<>(0, "A0+b0", 1000L),
+                    new KeyValueTimestamp<>(1, "A1+b1", 1001L),
+                    new KeyValueTimestamp<>(2, "A2+b2", 1002L),
+                    new KeyValueTimestamp<>(3, "A3+b3", 1003L)
+            );
+
+            // push four items with increased timestamps to the secondary stream; 
+            // this should produce only three joined items;
+            // c0 arrives too late to be joined with A0
+            // w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
+            // w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999),
+            //        0:b0 (ts: 1000), 1:b1 (ts: 1000), 2:b2 (ts: 1000), 3:b3 (ts: 1000) }
+            // --> w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
+            //     w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999),
+            //            0:b0 (ts: 1000), 1:b1 (ts: 1000), 2:b2 (ts: 1000), 3:b3 (ts: 1000),
+            //            0:c0 (ts: 1001), 1:c1 (ts: 1001), 2:c2 (ts: 1001), 3:c3 (ts: 1001) }
+            time += 1L;
+            for (final int expectedKey : expectedKeys) {
+                inputTopic2.pipeInput(expectedKey, "c" + expectedKey, time);
+            }
+            processor.checkAndClearProcessResult(
+                    new KeyValueTimestamp<>(1, "A1+c1", 1001L),
+                    new KeyValueTimestamp<>(2, "A2+c2", 1002L),
+                    new KeyValueTimestamp<>(3, "A3+c3", 1003L)
+            );
+
+            // push four items with increased timestamps to the secondary stream;
+            // this should produce only two joined items;
+            // d0 and d1 arrive too late to be joined with A0 and A1
+            // w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
+            // w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999),
+            //        0:b0 (ts: 1000), 1:b1 (ts: 1000), 2:b2 (ts: 1000), 3:b3 (ts: 1000),
+            //        0:c0 (ts: 1001), 1:c1 (ts: 1001), 2:c2 (ts: 1001), 3:c3 (ts: 1001) }
+            // --> w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
+            //     w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999),
+            //            0:b0 (ts: 1000), 1:b1 (ts: 1000), 2:b2 (ts: 1000), 3:b3 (ts: 1000),
+            //            0:c0 (ts: 1001), 1:c1 (ts: 1001), 2:c2 (ts: 1001), 3:c3 (ts: 1001),
+            //            0:d0 (ts: 1002), 1:d1 (ts: 1002), 2:d2 (ts: 1002), 3:d3 (ts: 1002) }
+            time += 1L;
+            for (final int expectedKey : expectedKeys) {
+                inputTopic2.pipeInput(expectedKey, "d" + expectedKey, time);
+            }
+            processor.checkAndClearProcessResult(
+                    new KeyValueTimestamp<>(2, "A2+d2", 1002L),
+                    new KeyValueTimestamp<>(3, "A3+d3", 1003L)
+            );
+
+            // push four items with increased timestamps to the secondary stream; 
+            // this should produce one joined item;
+            // only e3 can be joined with A3;
+            // e0, e1 and e2 arrive too late to be joined with A0, A1 and A2
+            // w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
+            // w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999),
+            //        0:b0 (ts: 1000), 1:b1 (ts: 1000), 2:b2 (ts: 1000), 3:b3 (ts: 1000),
+            //        0:c0 (ts: 1001), 1:c1 (ts: 1001), 2:c2 (ts: 1001), 3:c3 (ts: 1001),
+            //        0:d0 (ts: 1002), 1:d1 (ts: 1002), 2:d2 (ts: 1002), 3:d3 (ts: 1002) }
+            // --> w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
+            //     w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999),
+            //            0:b0 (ts: 1000), 1:b1 (ts: 1000), 2:b2 (ts: 1000), 3:b3 (ts: 1000),
+            //            0:c0 (ts: 1001), 1:c1 (ts: 1001), 2:c2 (ts: 1001), 3:c3 (ts: 1001),
+            //            0:d0 (ts: 1002), 1:d1 (ts: 1002), 2:d2 (ts: 1002), 3:d3 (ts: 1002),
+            //            0:e0 (ts: 1003), 1:e1 (ts: 1003), 2:e2 (ts: 1003), 3:e3 (ts: 1003) }
+            time += 1L;
+            for (final int expectedKey : expectedKeys) {
+                inputTopic2.pipeInput(expectedKey, "e" + expectedKey, time);
+            }
+            processor.checkAndClearProcessResult(
+                new KeyValueTimestamp<>(3, "A3+e3", 1003L)
+            );
+
+            // push four items with larger timestamps to the secondary stream;
+            // no (non-)joined items can be produced
+            // 
+            // w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
+            // w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999),
+            //        0:b0 (ts: 1000), 1:b1 (ts: 1000), 2:b2 (ts: 1000), 3:b3 (ts: 1000),
+            //        0:c0 (ts: 1001), 1:c1 (ts: 1001), 2:c2 (ts: 1001), 3:c3 (ts: 1001),
+            //        0:d0 (ts: 1002), 1:d1 (ts: 1002), 2:d2 (ts: 1002), 3:d3 (ts: 1002),
+            //        0:e0 (ts: 1003), 1:e1 (ts: 1003), 2:e2 (ts: 1003), 3:e3 (ts: 1003) }
+            // --> w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
+            //     w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999),
+            //            0:b0 (ts: 1000), 1:b1 (ts: 1000), 2:b2 (ts: 1000), 3:b3 (ts: 1000),
+            //            0:c0 (ts: 1001), 1:c1 (ts: 1001), 2:c2 (ts: 1001), 3:c3 (ts: 1001),
+            //            0:d0 (ts: 1002), 1:d1 (ts: 1002), 2:d2 (ts: 1002), 3:d3 (ts: 1002),
+            //            0:e0 (ts: 1003), 1:e1 (ts: 1003), 2:e2 (ts: 1003), 3:e3 (ts: 1003),
+            //            0:f0 (ts: 1100), 1:f1 (ts: 1100), 2:f2 (ts: 1100), 3:f3 (ts: 1100) }
+            time = 1000 + 100L;
+            for (final int expectedKey : expectedKeys) {
+                inputTopic2.pipeInput(expectedKey, "f" + expectedKey, time);
+            }
+            processor.checkAndClearProcessResult();
+        }
+    }
+
     @Test
     public void testLeftJoinWithInMemoryCustomSuppliers() {
         final JoinWindows joinWindows = JoinWindows.ofTimeDifferenceAndGrace(ofMillis(100L), ofMillis(0L));
@@ -609,8 +787,9 @@ public void testOrdering() {
             inputTopic1.pipeInput(1, "A1", 100L);
             processor.checkAndClearProcessResult();
 
-            // push one item to the other window that has a join; this should produce non-joined records with a closed window first, then
-            // the joined records
+            // push one item to the other window that has a join; 
+            // this should produce the joined record first;
+            // then non-joined record with a closed window
             // by the time they were produced before
             // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 100) }
             // w2 = { }
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java
index 099dc5b0c8..28a5f1488f 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java
@@ -108,11 +108,11 @@ public void testOuterJoinDuplicatesWithFixDisabledOldApi() {
             inputTopic2.pipeInput(1, "b1", 0L);
 
             processor.checkAndClearProcessResult(
-                    new KeyValueTimestamp<>(0, "A0+null", 0L),
-                    new KeyValueTimestamp<>(0, "A0-0+null", 0L),
-                    new KeyValueTimestamp<>(0, "A0+a0", 0L),
-                    new KeyValueTimestamp<>(0, "A0-0+a0", 0L),
-                    new KeyValueTimestamp<>(1, "null+b1", 0L)
+                new KeyValueTimestamp<>(0, "A0+null", 0L),
+                new KeyValueTimestamp<>(0, "A0-0+null", 0L),
+                new KeyValueTimestamp<>(0, "A0+a0", 0L),
+                new KeyValueTimestamp<>(0, "A0-0+a0", 0L),
+                new KeyValueTimestamp<>(1, "null+b1", 0L)
             );
         }
     }
@@ -438,13 +438,13 @@ public void testOrdering() {
             inputTopic1.pipeInput(1, "A1", 100L);
             processor.checkAndClearProcessResult();
 
-            // push one item to the other window that has a join; this should produce non-joined records with a closed window first, then
-            // the joined records
-            // by the time they were produced before
+            // push one item to the other window that has a join;
+            // this should produce the not-joined record first;
+            // then the joined record
             // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 100) }
             // w2 = { }
-            // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) }
-            // --> w2 = { 0:a0 (ts: 110) }
+            // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 100) }
+            // --> w2 = { 1:a1 (ts: 110) }
             inputTopic2.pipeInput(1, "a1", 110L);
             processor.checkAndClearProcessResult(
                 new KeyValueTimestamp<>(0, "A0+null", 0L),
@@ -788,7 +788,7 @@ public void shouldNotEmitLeftJoinResultForAsymmetricBeforeWindow() {
                 new KeyValueTimestamp<>(1, "A1+null", 1L)
             );
 
-            // push one item to the other stream; this should not produce any items
+            // push one item to the other stream; this should produce one right-join item
             // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) }
             // w2 = { 0:a0 (ts: 100), 1:a1 (ts: 102) }
             // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) }
@@ -841,7 +841,8 @@ public void shouldNotEmitLeftJoinResultForAsymmetricAfterWindow() {
             final MockApiProcessor processor = supplier.theCapturedProcessor();
             long time = 0L;
 
-            // push two items to the primary stream; the other window is empty; this should not produce any item
+            // push two items to the primary stream; the other window is empty; 
+            // this should produce one left-joined item
             // w1 = {}
             // w2 = {}
             // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) }
@@ -849,7 +850,9 @@ public void shouldNotEmitLeftJoinResultForAsymmetricAfterWindow() {
             for (int i = 0; i < 2; i++) {
                 inputTopic1.pipeInput(expectedKeys[i], "A" + expectedKeys[i], time + i);
             }
-            processor.checkAndClearProcessResult();
+            processor.checkAndClearProcessResult(
+                new KeyValueTimestamp<>(0, "A0+null", 0L)
+            );
 
             // push one item to the other stream; this should produce one full-join item
             // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) }
@@ -863,7 +866,8 @@ public void shouldNotEmitLeftJoinResultForAsymmetricAfterWindow() {
                 new KeyValueTimestamp<>(1, "A1+a1", 1L)
             );
 
-            // push one item to the other stream; this should produce one left-join item
+            // push one item to the other stream;
+            // this should not produce any item
             // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) }
             // w2 = { 1:a1 (ts: 1) }
             // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) }
@@ -871,9 +875,7 @@ public void shouldNotEmitLeftJoinResultForAsymmetricAfterWindow() {
             time += 100;
             inputTopic2.pipeInput(expectedKeys[2], "a" + expectedKeys[2], time);
 
-            processor.checkAndClearProcessResult(
-                new KeyValueTimestamp<>(0, "A0+null", 0L)
-            );
+            processor.checkAndClearProcessResult();
 
             // push one item to the other stream; this should not produce any item
             // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) }
@@ -884,11 +886,12 @@ public void shouldNotEmitLeftJoinResultForAsymmetricAfterWindow() {
 
             processor.checkAndClearProcessResult();
 
-            // push one item to the first stream; this should produce one full-join item
+            // push one item to the first stream;
+            // this should produce one inner-join item;
             // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) }
             // w2 = { 1:a1 (ts: 1), 2:a2 (ts: 101), 3:a3 (ts: 101) }
             // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1), 2:A2 (ts: 201) }
-            // --> w2 = { 1:a1 (ts: 1), 2:a2 (ts: 101), 3:a3 (ts: 101 }
+            // --> w2 = { 1:a1 (ts: 1), 2:a2 (ts: 101), 3:a3 (ts: 101) }
             time += 100;
             inputTopic1.pipeInput(expectedKeys[2], "A" + expectedKeys[2], time);
 

From ae047bbe56e8dc37ac18472ee631a14e1b35be82 Mon Sep 17 00:00:00 2001
From: "Cheng-Kai, Zhang" 
Date: Wed, 6 Mar 2024 16:34:46 +0800
Subject: [PATCH 029/521] KAFKA-16347: Upgrade zookeeper 3.8.3 -> 3.8.4
 (#15480)

Reviewers: Luke Chen , Chia-Ping Tsai 
---
 LICENSE-binary             | 4 ++--
 gradle/dependencies.gradle | 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/LICENSE-binary b/LICENSE-binary
index 69361f878e..1031f83686 100644
--- a/LICENSE-binary
+++ b/LICENSE-binary
@@ -270,8 +270,8 @@ scala-java8-compat_2.12-1.0.2
 scala-java8-compat_2.13-1.0.2
 snappy-java-1.1.10.5
 swagger-annotations-2.2.8
-zookeeper-3.8.3
-zookeeper-jute-3.8.3
+zookeeper-3.8.4
+zookeeper-jute-3.8.4
 
 ===============================================================================
 This product bundles various third-party components under other open source
diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle
index 5ac978f9b0..6503b51e19 100644
--- a/gradle/dependencies.gradle
+++ b/gradle/dependencies.gradle
@@ -164,7 +164,7 @@ versions += [
   snappy: "1.1.10.5",
   spotbugs: "4.8.0",
   zinc: "1.9.2",
-  zookeeper: "3.8.3",
+  zookeeper: "3.8.4",
   zstd: "1.5.5-11"
 ]
 

From f6198bc075f7e8e6af9e1fb53875e92de5057872 Mon Sep 17 00:00:00 2001
From: Nikolay 
Date: Wed, 6 Mar 2024 12:13:39 +0300
Subject: [PATCH 030/521] KAFKA-14589 [3/4] Tests of ConsoleGroupCommand
 rewritten in java  (#15365)

Is contains some of ConsoleGroupCommand tests rewritten in java.
Intention of separate PR is to reduce changes and simplify review.

Reviewers: Chia-Ping Tsai 
---
 checkstyle/import-control.xml                 |   2 +
 .../AbstractAuthorizerIntegrationTest.scala   | 130 +++++++++++++
 .../kafka/api/AbstractSaslTest.scala          |  22 +++
 .../kafka/api/AuthorizerIntegrationTest.scala | 118 +-----------
 ...aslClientsWithInvalidCredentialsTest.scala |  42 +----
 .../group/AuthorizerIntegrationTest.java      |  60 ++++++
 ...SaslClientsWithInvalidCredentialsTest.java | 177 ++++++++++++++++++
 7 files changed, 395 insertions(+), 156 deletions(-)
 create mode 100644 core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala
 create mode 100644 core/src/test/scala/integration/kafka/api/AbstractSaslTest.scala
 create mode 100644 tools/src/test/java/org/apache/kafka/tools/consumer/group/AuthorizerIntegrationTest.java
 create mode 100644 tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java

diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index a52b3d94e3..8bbf572821 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -326,6 +326,8 @@
 
       
         
+        
+        
         
         
         
diff --git a/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala
new file mode 100644
index 0000000000..b3e1ba9a64
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala
@@ -0,0 +1,130 @@
+/**
+ * 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.security.authorizer.AclAuthorizer
+import kafka.security.authorizer.AclEntry.WildcardHost
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.ConsumerConfig
+import org.apache.kafka.clients.producer.ProducerConfig
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.acl.AccessControlEntry
+import org.apache.kafka.common.acl.AclOperation.CLUSTER_ACTION
+import org.apache.kafka.common.acl.AclPermissionType.ALLOW
+import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
+import org.apache.kafka.common.network.ListenerName
+import org.apache.kafka.common.resource.PatternType.LITERAL
+import org.apache.kafka.common.resource.{Resource, ResourcePattern}
+import org.apache.kafka.common.resource.ResourceType.{CLUSTER, GROUP, TOPIC, TRANSACTIONAL_ID}
+import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrincipal}
+import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder
+import org.apache.kafka.metadata.authorizer.StandardAuthorizer
+import org.junit.jupiter.api.{BeforeEach, TestInfo}
+
+import java.util.Properties
+
+object AbstractAuthorizerIntegrationTest {
+  val BrokerPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "broker")
+  val ClientPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "client")
+
+  val BrokerListenerName = "BROKER"
+  val ClientListenerName = "CLIENT"
+  val ControllerListenerName = "CONTROLLER"
+
+  class PrincipalBuilder extends DefaultKafkaPrincipalBuilder(null, null) {
+    override def build(context: AuthenticationContext): KafkaPrincipal = {
+      context.listenerName match {
+        case BrokerListenerName | ControllerListenerName => BrokerPrincipal
+        case ClientListenerName => ClientPrincipal
+        case listenerName => throw new IllegalArgumentException(s"No principal mapped to listener $listenerName")
+      }
+    }
+  }
+}
+
+/**
+ * Abstract authorizer test to be used both in scala and java tests of authorizer.
+ */
+class AbstractAuthorizerIntegrationTest extends BaseRequestTest {
+  import AbstractAuthorizerIntegrationTest._
+
+  override def interBrokerListenerName: ListenerName = new ListenerName(BrokerListenerName)
+  override def listenerName: ListenerName = new ListenerName(ClientListenerName)
+  override def brokerCount: Int = 1
+
+  def clientPrincipal: KafkaPrincipal = ClientPrincipal
+  def brokerPrincipal: KafkaPrincipal = BrokerPrincipal
+
+  val clientPrincipalString: String = clientPrincipal.toString
+
+  val brokerId: Integer = 0
+  val topic = "topic"
+  val topicPattern = "topic.*"
+  val transactionalId = "transactional.id"
+  val producerId = 83392L
+  val part = 0
+  val correlationId = 0
+  val clientId = "client-Id"
+  val tp = new TopicPartition(topic, part)
+  val logDir = "logDir"
+  val group = "my-group"
+  val protocolType = "consumer"
+  val protocolName = "consumer-range"
+  val clusterResource = new ResourcePattern(CLUSTER, Resource.CLUSTER_NAME, LITERAL)
+  val topicResource = new ResourcePattern(TOPIC, topic, LITERAL)
+  val groupResource = new ResourcePattern(GROUP, group, LITERAL)
+  val transactionalIdResource = new ResourcePattern(TRANSACTIONAL_ID, transactionalId, LITERAL)
+
+  producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "1")
+  producerConfig.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "false")
+  producerConfig.setProperty(ProducerConfig.MAX_BLOCK_MS_CONFIG, "50000")
+  consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, group)
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    addNodeProperties(properties)
+  }
+
+  override def kraftControllerConfigs(): collection.Seq[Properties] = {
+    val controllerConfigs = super.kraftControllerConfigs()
+    controllerConfigs.foreach(addNodeProperties)
+    controllerConfigs
+  }
+
+  private def addNodeProperties(properties: Properties): Unit = {
+    if (isKRaftTest()) {
+      properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName)
+      properties.put(StandardAuthorizer.SUPER_USERS_CONFIG, BrokerPrincipal.toString)
+    } else {
+      properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[AclAuthorizer].getName)
+    }
+
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+    properties.put(KafkaConfig.TransactionsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.TransactionsTopicReplicationFactorProp, "1")
+    properties.put(KafkaConfig.TransactionsTopicMinISRProp, "1")
+    properties.put(KafkaConfig.UnstableApiVersionsEnableProp, "true")
+    properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[PrincipalBuilder].getName)
+  }
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    doSetup(testInfo, createOffsetsTopic = false)
+
+    // Allow inter-broker communication
+    addAndVerifyAcls(Set(new AccessControlEntry(brokerPrincipal.toString, WildcardHost, CLUSTER_ACTION, ALLOW)), clusterResource)
+
+    createOffsetsTopic(listenerName = interBrokerListenerName)
+  }
+}
diff --git a/core/src/test/scala/integration/kafka/api/AbstractSaslTest.scala b/core/src/test/scala/integration/kafka/api/AbstractSaslTest.scala
new file mode 100644
index 0000000000..4bef3422a2
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/api/AbstractSaslTest.scala
@@ -0,0 +1,22 @@
+/**
+ * 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
+
+/**
+ * Abstract Sasl test to be used both in scala and java tests of SASL.
+ * Separate class required to overcome issues related to usage of scala trait in java code.
+ * @see SaslClientsWithInvalidCredentialsTest
+ */
+abstract class AbstractSaslTest extends IntegrationTestHarness with SaslSetup {
+}
+
diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
index c399b46450..ded470eba8 100644
--- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
@@ -18,10 +18,8 @@ import java.util
 import java.util.concurrent.ExecutionException
 import java.util.regex.Pattern
 import java.util.{Collections, Optional, Properties}
-import kafka.admin.ConsumerGroupCommand.{ConsumerGroupCommandOptions, ConsumerGroupService}
-import kafka.security.authorizer.{AclAuthorizer, AclEntry}
+import kafka.security.authorizer.AclEntry
 import kafka.security.authorizer.AclEntry.WildcardHost
-import kafka.server.{BaseRequestTest, KafkaConfig}
 import kafka.utils.{TestInfoUtils, TestUtils}
 import kafka.utils.TestUtils.waitUntilTrue
 import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, NewTopic}
@@ -30,7 +28,6 @@ import org.apache.kafka.clients.producer._
 import org.apache.kafka.common.acl.AclOperation._
 import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY}
 import org.apache.kafka.common.acl.{AccessControlEntry, AccessControlEntryFilter, AclBindingFilter, AclOperation, AclPermissionType}
-import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
 import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig, TopicConfig}
 import org.apache.kafka.common.errors._
 import org.apache.kafka.common.internals.Topic.GROUP_METADATA_TOPIC_NAME
@@ -53,14 +50,11 @@ import org.apache.kafka.common.requests._
 import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED}
 import org.apache.kafka.common.resource.ResourceType._
 import org.apache.kafka.common.resource.{PatternType, Resource, ResourcePattern, ResourcePatternFilter, ResourceType}
-import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrincipal, SecurityProtocol}
-import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder
+import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
 import org.apache.kafka.common.utils.Utils
 import org.apache.kafka.common.{ElectionType, IsolationLevel, KafkaException, Node, TopicPartition, Uuid, requests}
-import org.apache.kafka.metadata.authorizer.StandardAuthorizer
 import org.apache.kafka.test.{TestUtils => JTestUtils}
 import org.junit.jupiter.api.Assertions._
-import org.junit.jupiter.api.{BeforeEach, TestInfo}
 import org.junit.jupiter.params.ParameterizedTest
 import org.junit.jupiter.params.provider.{CsvSource, ValueSource}
 
@@ -72,55 +66,7 @@ import scala.annotation.nowarn
 import scala.collection.mutable
 import scala.jdk.CollectionConverters._
 
-object AuthorizerIntegrationTest {
-  val BrokerPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "broker")
-  val ClientPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "client")
-
-  val BrokerListenerName = "BROKER"
-  val ClientListenerName = "CLIENT"
-  val ControllerListenerName = "CONTROLLER"
-
-  class PrincipalBuilder extends DefaultKafkaPrincipalBuilder(null, null) {
-    override def build(context: AuthenticationContext): KafkaPrincipal = {
-      context.listenerName match {
-        case BrokerListenerName | ControllerListenerName => BrokerPrincipal
-        case ClientListenerName => ClientPrincipal
-        case listenerName => throw new IllegalArgumentException(s"No principal mapped to listener $listenerName")
-      }
-    }
-  }
-}
-
-class AuthorizerIntegrationTest extends BaseRequestTest {
-  import AuthorizerIntegrationTest._
-
-  override def interBrokerListenerName: ListenerName = new ListenerName(BrokerListenerName)
-  override def listenerName: ListenerName = new ListenerName(ClientListenerName)
-  override def brokerCount: Int = 1
-
-  def clientPrincipal: KafkaPrincipal = ClientPrincipal
-  def brokerPrincipal: KafkaPrincipal = BrokerPrincipal
-
-  val clientPrincipalString: String = clientPrincipal.toString
-
-  val brokerId: Integer = 0
-  val topic = "topic"
-  val topicPattern = "topic.*"
-  val transactionalId = "transactional.id"
-  val producerId = 83392L
-  val part = 0
-  val correlationId = 0
-  val clientId = "client-Id"
-  val tp = new TopicPartition(topic, part)
-  val logDir = "logDir"
-  val group = "my-group"
-  val protocolType = "consumer"
-  val protocolName = "consumer-range"
-  val clusterResource = new ResourcePattern(CLUSTER, Resource.CLUSTER_NAME, LITERAL)
-  val topicResource = new ResourcePattern(TOPIC, topic, LITERAL)
-  val groupResource = new ResourcePattern(GROUP, group, LITERAL)
-  val transactionalIdResource = new ResourcePattern(TRANSACTIONAL_ID, transactionalId, LITERAL)
-
+class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
   val groupReadAcl = Map(groupResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)))
   val groupDescribeAcl = Map(groupResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, DESCRIBE, ALLOW)))
   val groupDeleteAcl = Map(groupResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, DELETE, ALLOW)))
@@ -143,40 +89,6 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
 
   val numRecords = 1
 
-  producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "1")
-  producerConfig.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "false")
-  producerConfig.setProperty(ProducerConfig.MAX_BLOCK_MS_CONFIG, "50000")
-  consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, group)
-
-  override def brokerPropertyOverrides(properties: Properties): Unit = {
-    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
-    addNodeProperties(properties)
-  }
-
-  override def kraftControllerConfigs(): collection.Seq[Properties] = {
-    val controllerConfigs = super.kraftControllerConfigs()
-    controllerConfigs.foreach(addNodeProperties)
-    controllerConfigs
-  }
-
-  private def addNodeProperties(properties: Properties): Unit = {
-    if (isKRaftTest()) {
-      properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName)
-      properties.put(StandardAuthorizer.SUPER_USERS_CONFIG, BrokerPrincipal.toString)
-    } else {
-      properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[AclAuthorizer].getName)
-    }
-
-    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
-    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
-    properties.put(KafkaConfig.TransactionsTopicPartitionsProp, "1")
-    properties.put(KafkaConfig.TransactionsTopicReplicationFactorProp, "1")
-    properties.put(KafkaConfig.TransactionsTopicMinISRProp, "1")
-    properties.put(KafkaConfig.UnstableApiVersionsEnableProp, "true")
-    properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[PrincipalBuilder].getName)
-  }
-
-
   val requestKeyToError = (topicNames: Map[Uuid, String], version: Short) => Map[ApiKeys, Nothing => Errors](
     ApiKeys.METADATA -> ((resp: requests.MetadataResponse) => resp.errors.asScala.find(_._1 == topic).getOrElse(("test", Errors.NONE))._2),
     ApiKeys.PRODUCE -> ((resp: requests.ProduceResponse) => {
@@ -333,16 +245,6 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
     ApiKeys.DESCRIBE_TRANSACTIONS -> transactionalIdDescribeAcl
   )
 
-  @BeforeEach
-  override def setUp(testInfo: TestInfo): Unit = {
-    doSetup(testInfo, createOffsetsTopic = false)
-
-    // Allow inter-broker communication
-    addAndVerifyAcls(Set(new AccessControlEntry(brokerPrincipal.toString, WildcardHost, CLUSTER_ACTION, ALLOW)), clusterResource)
-
-    createOffsetsTopic(listenerName = interBrokerListenerName)
-  }
-
   private def createMetadataRequest(allowAutoTopicCreation: Boolean) = {
     new requests.MetadataRequest.Builder(List(topic).asJava, allowAutoTopicCreation).build()
   }
@@ -1703,20 +1605,6 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
     createAdminClient().describeConsumerGroups(Seq(group).asJava).describedGroups().get(group).get()
   }
 
-  @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
-  @ValueSource(strings = Array("zk", "kraft"))
-  def testDescribeGroupCliWithGroupDescribe(quorum: String): Unit = {
-    createTopicWithBrokerPrincipal(topic)
-    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, DESCRIBE, ALLOW)), groupResource)
-    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, DESCRIBE, ALLOW)), topicResource)
-
-    val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group)
-    val opts = new ConsumerGroupCommandOptions(cgcArgs)
-    val consumerGroupService = new ConsumerGroupService(opts)
-    consumerGroupService.describeGroups()
-    consumerGroupService.close()
-  }
-
   @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
   @ValueSource(strings = Array("zk", "kraft"))
   def testListGroupApiWithAndWithoutListGroupAcls(quorum: String): Unit = {
diff --git a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala
index eba51c968f..11afbd89d2 100644
--- a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala
+++ b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala
@@ -23,7 +23,6 @@ import org.apache.kafka.common.{KafkaException, TopicPartition}
 import org.apache.kafka.common.errors.SaslAuthenticationException
 import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
 import org.junit.jupiter.api.Assertions._
-import kafka.admin.ConsumerGroupCommand.{ConsumerGroupCommandOptions, ConsumerGroupService}
 import kafka.server.KafkaConfig
 import kafka.utils.{JaasTestUtils, TestUtils}
 import kafka.zk.ConfigEntityChangeNotificationZNode
@@ -31,7 +30,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol
 import org.junit.jupiter.params.ParameterizedTest
 import org.junit.jupiter.params.provider.ValueSource
 
-class SaslClientsWithInvalidCredentialsTest extends IntegrationTestHarness with SaslSetup {
+class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest {
   private val kafkaClientSaslMechanism = "SCRAM-SHA-256"
   private val kafkaServerSaslMechanisms = List(kafkaClientSaslMechanism)
   override protected val securityProtocol = SecurityProtocol.SASL_PLAINTEXT
@@ -166,45 +165,6 @@ class SaslClientsWithInvalidCredentialsTest extends IntegrationTestHarness with
     }
   }
 
-  @Test
-  def testConsumerGroupServiceWithAuthenticationFailure(): Unit = {
-    val consumerGroupService: ConsumerGroupService = prepareConsumerGroupService
-
-    val consumer = createConsumer()
-    try {
-      consumer.subscribe(List(topic).asJava)
-
-      verifyAuthenticationException(consumerGroupService.listGroups())
-    } finally consumerGroupService.close()
-  }
-
-  @Test
-  def testConsumerGroupServiceWithAuthenticationSuccess(): Unit = {
-    createClientCredential()
-    val consumerGroupService: ConsumerGroupService = prepareConsumerGroupService
-
-    val consumer = createConsumer()
-    try {
-      consumer.subscribe(List(topic).asJava)
-
-      verifyWithRetry(consumer.poll(Duration.ofMillis(1000)))
-      assertEquals(1, consumerGroupService.listConsumerGroups().size)
-    }
-    finally consumerGroupService.close()
-  }
-
-  private def prepareConsumerGroupService = {
-    val propsFile = TestUtils.tempPropertiesFile(Map("security.protocol" -> "SASL_PLAINTEXT", "sasl.mechanism" -> kafkaClientSaslMechanism))
-
-    val cgcArgs = Array("--bootstrap-server", bootstrapServers(),
-                        "--describe",
-                        "--group", "test.group",
-                        "--command-config", propsFile.getAbsolutePath)
-    val opts = new ConsumerGroupCommandOptions(cgcArgs)
-    val consumerGroupService = new ConsumerGroupService(opts)
-    consumerGroupService
-  }
-
   private def createClientCredential(): Unit = {
     createScramCredentialsViaPrivilegedAdminClient(JaasTestUtils.KafkaScramUser2, JaasTestUtils.KafkaScramPassword2)
   }
diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/AuthorizerIntegrationTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/AuthorizerIntegrationTest.java
new file mode 100644
index 0000000000..3094572b04
--- /dev/null
+++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/AuthorizerIntegrationTest.java
@@ -0,0 +1,60 @@
+/*
+ * 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.tools.consumer.group;
+
+import kafka.admin.ConsumerGroupCommand;
+import kafka.api.AbstractAuthorizerIntegrationTest;
+import kafka.security.authorizer.AclEntry;
+import org.apache.kafka.common.acl.AccessControlEntry;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+import scala.collection.immutable.Map$;
+
+import java.util.Collections;
+import java.util.Properties;
+
+import static org.apache.kafka.common.acl.AclOperation.DESCRIBE;
+import static org.apache.kafka.common.acl.AclPermissionType.ALLOW;
+import static org.apache.kafka.tools.ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME;
+import static org.apache.kafka.tools.consumer.group.ConsumerGroupCommandTest.set;
+
+public class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
+    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeGroupCliWithGroupDescribe(String quorum) {
+        addAndVerifyAcls(set(Collections.singleton(new AccessControlEntry(ClientPrincipal().toString(), AclEntry.WildcardHost(), DESCRIBE, ALLOW))), groupResource());
+
+        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group()};
+        ConsumerGroupCommand.ConsumerGroupCommandOptions opts = new ConsumerGroupCommand.ConsumerGroupCommandOptions(cgcArgs);
+        ConsumerGroupCommand.ConsumerGroupService consumerGroupService = new ConsumerGroupCommand.ConsumerGroupService(opts, Map$.MODULE$.empty());
+        consumerGroupService.describeGroups();
+        consumerGroupService.close();
+    }
+
+    private void createTopicWithBrokerPrincipal(String topic) {
+        // Note the principal builder implementation maps all connections on the
+        // inter-broker listener to the broker principal.
+        createTopic(
+            topic,
+            1,
+            1,
+            new Properties(),
+            interBrokerListenerName(),
+            new Properties()
+        );
+    }
+}
diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java
new file mode 100644
index 0000000000..06b727a59a
--- /dev/null
+++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java
@@ -0,0 +1,177 @@
+/*
+ * 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.tools.consumer.group;
+
+import kafka.admin.ConsumerGroupCommand;
+import kafka.api.AbstractSaslTest;
+import kafka.api.Both$;
+import kafka.utils.JaasTestUtils;
+import kafka.zk.ConfigEntityChangeNotificationZNode;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.common.errors.SaslAuthenticationException;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.api.function.Executable;
+import scala.Option;
+import scala.Some$;
+import scala.collection.JavaConverters;
+import scala.collection.Seq;
+import scala.collection.immutable.Map$;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Collections;
+import java.util.Properties;
+
+import static org.apache.kafka.tools.consumer.group.ConsumerGroupCommandTest.seq;
+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 SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest {
+    private static final String TOPIC = "topic";
+    public static final int NUM_PARTITIONS = 1;
+    public static final int BROKER_COUNT = 1;
+    public static final String KAFKA_CLIENT_SASL_MECHANISM = "SCRAM-SHA-256";
+    private static final Seq KAFKA_SERVER_SASL_MECHANISMS = seq(Collections.singletonList(KAFKA_CLIENT_SASL_MECHANISM));
+
+    @SuppressWarnings({"deprecation"})
+    private Consumer createConsumer() {
+        return createConsumer(
+            new ByteArrayDeserializer(),
+            new ByteArrayDeserializer(),
+            new Properties(),
+            JavaConverters.asScalaSet(Collections.emptySet()).toList()
+        );
+    }
+
+    @Override
+    public SecurityProtocol securityProtocol() {
+        return SecurityProtocol.SASL_PLAINTEXT;
+    }
+
+    @Override
+    public Option serverSaslProperties() {
+        return Some$.MODULE$.apply(kafkaServerSaslProperties(KAFKA_SERVER_SASL_MECHANISMS, KAFKA_CLIENT_SASL_MECHANISM));
+    }
+
+    @Override
+    public Option clientSaslProperties() {
+        return Some$.MODULE$.apply(kafkaClientSaslProperties(KAFKA_CLIENT_SASL_MECHANISM, false));
+    }
+
+    @Override
+    public int brokerCount() {
+        return 1;
+    }
+
+    @Override
+    public void configureSecurityBeforeServersStart(TestInfo testInfo) {
+        super.configureSecurityBeforeServersStart(testInfo);
+        zkClient().makeSurePersistentPathExists(ConfigEntityChangeNotificationZNode.path());
+        // Create broker credentials before starting brokers
+        createScramCredentials(zkConnect(), JaasTestUtils.KafkaScramAdmin(), JaasTestUtils.KafkaScramAdminPassword());
+    }
+
+    @Override
+    public Admin createPrivilegedAdminClient() {
+        return createAdminClient(bootstrapServers(listenerName()), securityProtocol(), trustStoreFile(), clientSaslProperties(),
+            KAFKA_CLIENT_SASL_MECHANISM, JaasTestUtils.KafkaScramAdmin(), JaasTestUtils.KafkaScramAdminPassword());
+    }
+
+    @BeforeEach
+    @Override
+    public void setUp(TestInfo testInfo) {
+        startSasl(jaasSections(KAFKA_SERVER_SASL_MECHANISMS, Some$.MODULE$.apply(KAFKA_CLIENT_SASL_MECHANISM), Both$.MODULE$,
+            JaasTestUtils.KafkaServerContextName()));
+        super.setUp(testInfo);
+        createTopic(
+            TOPIC,
+            NUM_PARTITIONS,
+            BROKER_COUNT,
+            new Properties(),
+            listenerName(),
+            new Properties());
+    }
+
+    @AfterEach
+    @Override
+    public void tearDown() {
+        super.tearDown();
+        closeSasl();
+    }
+
+    @Test
+    public void testConsumerGroupServiceWithAuthenticationFailure() throws Exception {
+        ConsumerGroupCommand.ConsumerGroupService consumerGroupService = prepareConsumerGroupService();
+        try (Consumer consumer = createConsumer()) {
+            consumer.subscribe(Collections.singletonList(TOPIC));
+
+            verifyAuthenticationException(consumerGroupService::listGroups);
+        } finally {
+            consumerGroupService.close();
+        }
+    }
+
+    @Test
+    public void testConsumerGroupServiceWithAuthenticationSuccess() throws Exception {
+        createScramCredentialsViaPrivilegedAdminClient(JaasTestUtils.KafkaScramUser2(), JaasTestUtils.KafkaScramPassword2());
+        ConsumerGroupCommand.ConsumerGroupService consumerGroupService = prepareConsumerGroupService();
+        try (Consumer consumer = createConsumer()) {
+            consumer.subscribe(Collections.singletonList(TOPIC));
+
+            TestUtils.waitForCondition(() -> {
+                try {
+                    consumer.poll(Duration.ofMillis(1000));
+                    return true;
+                } catch (SaslAuthenticationException ignored) {
+                    return false;
+                }
+            }, "failed to poll data with authentication");
+            assertEquals(1, consumerGroupService.listConsumerGroups().size());
+        } finally {
+            consumerGroupService.close();
+        }
+    }
+
+    private ConsumerGroupCommand.ConsumerGroupService prepareConsumerGroupService() throws IOException {
+        File propsFile = TestUtils.tempFile(
+            "security.protocol=SASL_PLAINTEXT\n" +
+            "sasl.mechanism=" + KAFKA_CLIENT_SASL_MECHANISM);
+
+        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()),
+            "--describe",
+            "--group", "test.group",
+            "--command-config", propsFile.getAbsolutePath()};
+        ConsumerGroupCommand.ConsumerGroupCommandOptions opts = new ConsumerGroupCommand.ConsumerGroupCommandOptions(cgcArgs);
+        return new ConsumerGroupCommand.ConsumerGroupService(opts, Map$.MODULE$.empty());
+    }
+
+    private void verifyAuthenticationException(Executable action) {
+        long startMs = System.currentTimeMillis();
+        assertThrows(Exception.class, action);
+        long elapsedMs = System.currentTimeMillis() - startMs;
+        assertTrue(elapsedMs <= 5000, "Poll took too long, elapsed=" + elapsedMs);
+    }
+}

From 8f5bb8cfb306513d215a5258a3dbcf8d0bad9c94 Mon Sep 17 00:00:00 2001
From: "Cheng-Kai, Zhang" 
Date: Wed, 6 Mar 2024 17:19:54 +0800
Subject: [PATCH 031/521] KAFKA-16252: Fix the documentation and adjust the
 format (#15473)

Currently, there are few document files generated automatically like the task genConnectMetricsDocs
However, the unwanted log information also added into it.
And the format is not aligned with other which has Mbean located of the third column.

I modified the code logic so the format could follow other section in ops.html
Also close the log since we take everything from the std as a documentation

Reviewers: Luke Chen , Chia-Ping Tsai 
---
 build.gradle                                  |  2 +-
 .../apache/kafka/common/metrics/Metrics.java  | 32 +++++++------------
 2 files changed, 12 insertions(+), 22 deletions(-)

diff --git a/build.gradle b/build.gradle
index 7dc8f50342..cba4381aae 100644
--- a/build.gradle
+++ b/build.gradle
@@ -3048,7 +3048,7 @@ project(':connect:runtime') {
   }
 
   task genConnectMetricsDocs(type: JavaExec) {
-    classpath = sourceSets.test.runtimeClasspath
+    classpath = sourceSets.main.runtimeClasspath
     mainClass = 'org.apache.kafka.connect.runtime.ConnectMetrics'
     if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() }
     standardOutput = new File(generatedDocsDir, "connect_metrics.html").newOutputStream()
diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java b/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java
index 6447cdb5c7..b52285dac6 100644
--- a/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java
@@ -277,40 +277,30 @@ public static String toHtmlTable(String domain, Iterable all
                 }
             }
         }
-        
         StringBuilder b = new StringBuilder();
-        b.append("\n");
-    
+        b.append("
\n\n"); + b.append("\n"); + b.append("\n"); + b.append("\n"); + b.append("\n"); + b.append("\n"); for (Entry> e : beansAndAttributes.entrySet()) { - b.append("\n"); - b.append(""); - b.append("\n"); - - b.append("\n"); - b.append("\n"); - b.append("\n"); - b.append("\n"); - b.append("\n"); - for (Entry e2 : e.getValue().entrySet()) { b.append("\n"); - b.append(""); b.append(""); + b.append("\n"); b.append(""); + b.append("\n"); + b.append("\n"); b.append("\n"); } - } b.append("
Metric/Attribute nameDescriptionMbean name
"); - b.append(e.getKey()); - b.append("
Attribute nameDescription
"); b.append(e2.getKey()); - b.append(""); b.append(e2.getValue()); - b.append(""); + b.append(e.getKey()); + b.append("
"); - return b.toString(); - } public MetricConfig config() { From bc0c73e944af69966c43d51d20417b441aa0f3f5 Mon Sep 17 00:00:00 2001 From: Johnny Hsu <44309740+johnnychhsu@users.noreply.github.com> Date: Wed, 6 Mar 2024 19:39:34 +0800 Subject: [PATCH 032/521] KAFKA-16322 upgrade jline from 3.22.0 to 3.25.1 (#15464) An issue in the component "GroovyEngine.execute" of jline-groovy versions through 3.24.1 allows attackers to cause an OOM (OutofMemory) error. Please refer to https://devhub.checkmarx.com/cve-details/CVE-2023-50572 for more details Reviewers: Chia-Ping Tsai --- LICENSE-binary | 2 +- gradle/dependencies.gradle | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index 1031f83686..df80c6b57e 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -333,7 +333,7 @@ zstd-jni-1.5.5-11 see: licenses/zstd-jni-BSD-2-clause --------------------------------------- BSD 3-Clause -jline-3.22.0, see: licenses/jline-BSD-3-clause +jline-3.25.1, see: licenses/jline-BSD-3-clause jsr305-3.0.2, see: licenses/jsr305-BSD-3-clause paranamer-2.8, see: licenses/paranamer-BSD-3-clause protobuf-java-3.23.4, see: licenses/protobuf-java-BSD-3-clause diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 6503b51e19..4f31bcd25c 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -108,7 +108,7 @@ versions += [ javassist: "3.29.2-GA", jetty: "9.4.53.v20231009", jersey: "2.39.1", - jline: "3.22.0", + jline: "3.25.1", jmh: "1.37", hamcrest: "2.2", scalaLogging: "3.9.4", From 6d7b25bb25ee817c03606e666608cece5cbf621d Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Wed, 6 Mar 2024 10:27:12 -0800 Subject: [PATCH 033/521] KAFKA-15797: Fix flaky EOS_v2 upgrade test (#15449) Originally, we set commit-interval to MAX_VALUE for this test, to ensure we only commit expliclity. However, we needed to decrease it later on when adding the tx-timeout verification. We did see failing test for which commit-interval hit, resulting in failing test runs. This PR increase the commit-interval close to test-timeout to avoid commit-interval from triggering. Reviewers: Bruno Cadonna --- .../kafka/streams/integration/EosV2UpgradeIntegrationTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/EosV2UpgradeIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/EosV2UpgradeIntegrationTest.java index 011090c152..bac9ae37eb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/EosV2UpgradeIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/EosV2UpgradeIntegrationTest.java @@ -939,7 +939,7 @@ public void close() {} final Properties properties = new Properties(); properties.put(StreamsConfig.CLIENT_ID_CONFIG, appDir); properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, processingGuarantee); - final long commitInterval = Duration.ofMinutes(1L).toMillis(); + final long commitInterval = Duration.ofMinutes(5L).toMillis(); properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, commitInterval); properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.METADATA_MAX_AGE_CONFIG), Duration.ofSeconds(1L).toMillis()); properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG), "earliest"); From 62998b72642109ac455564d26c08e37eb0e9f0ea Mon Sep 17 00:00:00 2001 From: Hector Geraldino Date: Wed, 6 Mar 2024 13:31:33 -0500 Subject: [PATCH 034/521] KAFKA-14683: Migrate WorkerSinkTaskTest to Mockito (3/3) (#15316) Reviewers: Greg Harris --- checkstyle/suppressions.xml | 6 +- .../kafka/connect/runtime/WorkerSinkTask.java | 5 + .../runtime/WorkerSinkTaskMockitoTest.java | 945 +++++++++++- .../connect/runtime/WorkerSinkTaskTest.java | 1298 ----------------- 4 files changed, 899 insertions(+), 1355 deletions(-) delete mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index c65cd675a9..32916fe660 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -96,7 +96,7 @@ files="(AbstractFetch|ConsumerCoordinator|CommitRequestManager|FetchCollector|OffsetFetcherUtils|KafkaProducer|Sender|ConfigDef|KerberosLogin|AbstractRequest|AbstractResponse|Selector|SslFactory|SslTransportLayer|SaslClientAuthenticator|SaslClientCallbackHandler|SaslServerAuthenticator|AbstractCoordinator|TransactionManager|AbstractStickyAssignor|DefaultSslEngineFactory|Authorizer|RecordAccumulator|MemoryRecords|FetchSessionHandler|MockAdminClient).java"/> + files="(AbstractRequest|AbstractResponse|KerberosLogin|WorkerSinkTaskTest|WorkerSinkTaskMockitoTest|TransactionManagerTest|SenderTest|KafkaAdminClient|ConsumerCoordinatorTest|KafkaAdminClientTest|KafkaRaftClientTest).java"/> @@ -173,12 +173,12 @@ files="(DistributedHerder|KafkaBasedLog|WorkerSourceTaskWithTopicCreation|WorkerSourceTask)Test.java"/> + files="(WorkerSink|WorkerSource|ErrorHandling)Task(|WithTopicCreation|Mockito)Test.java"/> + files="(RequestResponse|WorkerSinkTask|WorkerSinkTaskMockito)Test.java"/> diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java index 78a61e755f..25d8b54d4d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java @@ -354,6 +354,11 @@ Map lastCommittedOffsets() { return Collections.unmodifiableMap(lastCommittedOffsets); } + //VisibleForTesting + Map currentOffsets() { + return Collections.unmodifiableMap(currentOffsets); + } + private void doCommitSync(Map offsets, int seqno) { log.debug("{} Committing offsets synchronously using sequence number {}: {}", this, seqno, offsets); try { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskMockitoTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskMockitoTest.java index f39dce6646..83bf4fe0c7 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskMockitoTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskMockitoTest.java @@ -16,6 +16,46 @@ */ package org.apache.kafka.connect.runtime; +import static java.util.Arrays.asList; +import static java.util.Collections.singleton; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyList; +import static org.mockito.ArgumentMatchers.anyMap; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +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; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.regex.Pattern; +import java.util.stream.Collectors; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -36,6 +76,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.RetriableException; import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; import org.apache.kafka.connect.runtime.WorkerSinkTask.SinkTaskMetricsGroup; @@ -69,41 +110,6 @@ import org.mockito.stubbing.Answer; import org.mockito.stubbing.OngoingStubbing; -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; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.function.Supplier; -import java.util.regex.Pattern; - -import static java.util.Arrays.asList; -import static java.util.Collections.singleton; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyList; -import static org.mockito.ArgumentMatchers.anyMap; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.atLeastOnce; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - @RunWith(MockitoJUnitRunner.StrictStubs.class) public class WorkerSinkTaskMockitoTest { // These are fixed to keep this code simpler. In this example we assume byte[] raw values @@ -371,9 +377,9 @@ public void testPollRedelivery() { .thenAnswer(expectConsumerPoll(0)); expectConversionAndTransformation(null, new RecordHeaders()); - doAnswer(invocation -> null) + doNothing() .doThrow(new RetriableException("retry")) - .doAnswer(invocation -> null) + .doNothing() .when(sinkTask).put(anyList()); workerTask.iteration(); @@ -447,6 +453,86 @@ public void testPollRedelivery() { assertSinkMetricValue("offset-commit-completion-total", 1.0); } + @Test + @SuppressWarnings("unchecked") + public void testPollRedeliveryWithConsumerRebalance() { + createTask(initialState); + expectTaskGetTopic(); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + Set newAssignment = new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3)); + + when(consumer.assignment()) + .thenReturn(INITIAL_ASSIGNMENT, INITIAL_ASSIGNMENT, INITIAL_ASSIGNMENT) + .thenReturn(newAssignment, newAssignment, newAssignment) + .thenReturn(Collections.singleton(TOPIC_PARTITION3), + Collections.singleton(TOPIC_PARTITION3), + Collections.singleton(TOPIC_PARTITION3)); + + INITIAL_ASSIGNMENT.forEach(tp -> when(consumer.position(tp)).thenReturn(FIRST_OFFSET)); + when(consumer.position(TOPIC_PARTITION3)).thenReturn(FIRST_OFFSET); + + when(consumer.poll(any(Duration.class))) + .thenAnswer((Answer>) invocation -> { + rebalanceListener.getValue().onPartitionsAssigned(INITIAL_ASSIGNMENT); + return ConsumerRecords.empty(); + }) + .thenAnswer(expectConsumerPoll(1)) + // Empty consumer poll (all partitions are paused) with rebalance; one new partition is assigned + .thenAnswer(invocation -> { + rebalanceListener.getValue().onPartitionsRevoked(Collections.emptySet()); + rebalanceListener.getValue().onPartitionsAssigned(Collections.singleton(TOPIC_PARTITION3)); + return ConsumerRecords.empty(); + }) + .thenAnswer(expectConsumerPoll(0)) + // Non-empty consumer poll; all initially-assigned partitions are revoked in rebalance, and new partitions are allowed to resume + .thenAnswer(invocation -> { + ConsumerRecord newRecord = new ConsumerRecord<>(TOPIC, PARTITION3, FIRST_OFFSET, RAW_KEY, RAW_VALUE); + + rebalanceListener.getValue().onPartitionsRevoked(INITIAL_ASSIGNMENT); + rebalanceListener.getValue().onPartitionsAssigned(Collections.emptyList()); + return new ConsumerRecords<>(Collections.singletonMap(TOPIC_PARTITION3, Collections.singletonList(newRecord))); + }); + expectConversionAndTransformation(null, new RecordHeaders()); + + doNothing() + // If a retriable exception is thrown, we should redeliver the same batch, pausing the consumer in the meantime + .doThrow(new RetriableException("retry")) + .doThrow(new RetriableException("retry")) + .doThrow(new RetriableException("retry")) + .doNothing() + .when(sinkTask).put(any(Collection.class)); + + workerTask.iteration(); + + // Pause + workerTask.iteration(); + verify(consumer).pause(INITIAL_ASSIGNMENT); + + workerTask.iteration(); + verify(sinkTask).open(Collections.singleton(TOPIC_PARTITION3)); + // All partitions are re-paused in order to pause any newly-assigned partitions so that redelivery efforts can continue + verify(consumer).pause(newAssignment); + + workerTask.iteration(); + + final Map offsets = INITIAL_ASSIGNMENT.stream() + .collect(Collectors.toMap(Function.identity(), tp -> new OffsetAndMetadata(FIRST_OFFSET))); + when(sinkTask.preCommit(offsets)).thenReturn(offsets); + newAssignment = Collections.singleton(TOPIC_PARTITION3); + + workerTask.iteration(); + verify(sinkTask).close(INITIAL_ASSIGNMENT); + + // All partitions are resumed, as all previously paused-for-redelivery partitions were revoked + newAssignment.forEach(tp -> { + verify(consumer).resume(Collections.singleton(tp)); + }); + } + @Test public void testErrorInRebalancePartitionLoss() { RuntimeException exception = new RuntimeException("Revocation error"); @@ -601,45 +687,796 @@ public void testPartialRevocationAndAssignment() { verify(sinkTask, times(4)).put(Collections.emptyList()); } + @Test @SuppressWarnings("unchecked") + public void testPreCommitFailureAfterPartialRevocationAndAssignment() { + createTask(initialState); + expectTaskGetTopic(); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + when(consumer.assignment()) + .thenReturn(INITIAL_ASSIGNMENT, INITIAL_ASSIGNMENT) + .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2))) + .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2))) + .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2))) + .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3))) + .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3))) + .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3))); + + INITIAL_ASSIGNMENT.forEach(tp -> when(consumer.position(tp)).thenReturn(FIRST_OFFSET)); + when(consumer.position(TOPIC_PARTITION3)).thenReturn(FIRST_OFFSET); + + // First poll; assignment is [TP1, TP2] + when(consumer.poll(any(Duration.class))) + .thenAnswer((Answer>) invocation -> { + rebalanceListener.getValue().onPartitionsAssigned(INITIAL_ASSIGNMENT); + return ConsumerRecords.empty(); + }) + // Second poll; a single record is delivered from TP1 + .thenAnswer(expectConsumerPoll(1)) + // Third poll; assignment changes to [TP2] + .thenAnswer(invocation -> { + rebalanceListener.getValue().onPartitionsRevoked(Collections.singleton(TOPIC_PARTITION)); + rebalanceListener.getValue().onPartitionsAssigned(Collections.emptySet()); + return ConsumerRecords.empty(); + }) + // Fourth poll; assignment changes to [TP2, TP3] + .thenAnswer(invocation -> { + rebalanceListener.getValue().onPartitionsRevoked(Collections.emptySet()); + rebalanceListener.getValue().onPartitionsAssigned(Collections.singleton(TOPIC_PARTITION3)); + return ConsumerRecords.empty(); + }) + // Fifth poll; an offset commit takes place + .thenAnswer(expectConsumerPoll(0)); + + expectConversionAndTransformation(null, new RecordHeaders()); + + // First iteration--first call to poll, first consumer assignment + workerTask.iteration(); + // Second iteration--second call to poll, delivery of one record + workerTask.iteration(); + // Third iteration--third call to poll, partial consumer revocation + final Map offsets = new HashMap<>(); + offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); + when(sinkTask.preCommit(offsets)).thenReturn(offsets); + doNothing().when(consumer).commitSync(offsets); + + workerTask.iteration(); + verify(sinkTask).close(Collections.singleton(TOPIC_PARTITION)); + verify(sinkTask, times(2)).put(Collections.emptyList()); + + // Fourth iteration--fourth call to poll, partial consumer assignment + workerTask.iteration(); + + verify(sinkTask).open(Collections.singleton(TOPIC_PARTITION3)); + + final Map workerCurrentOffsets = new HashMap<>(); + workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + workerCurrentOffsets.put(TOPIC_PARTITION3, new OffsetAndMetadata(FIRST_OFFSET)); + when(sinkTask.preCommit(workerCurrentOffsets)).thenThrow(new ConnectException("Failed to flush")); + + // Fifth iteration--task-requested offset commit with failure in SinkTask::preCommit + sinkTaskContext.getValue().requestCommit(); + workerTask.iteration(); + + verify(consumer).seek(TOPIC_PARTITION2, FIRST_OFFSET); + verify(consumer).seek(TOPIC_PARTITION3, FIRST_OFFSET); + } + @Test - public void testTaskCancelPreventsFinalOffsetCommit() { + public void testWakeupInCommitSyncCausesRetry() { createTask(initialState); workerTask.initialize(TASK_CONFIG); + time.sleep(30000L); workerTask.initializeAndStart(); + time.sleep(30000L); verifyInitializeTask(); expectTaskGetTopic(); expectPollInitialAssignment() - // Put one message through the task to get some offsets to commit .thenAnswer(expectConsumerPoll(1)) - // the second put will return after the task is stopped and cancelled (asynchronously) - .thenAnswer(expectConsumerPoll(1)); - + .thenAnswer(invocation -> { + rebalanceListener.getValue().onPartitionsRevoked(INITIAL_ASSIGNMENT); + rebalanceListener.getValue().onPartitionsAssigned(INITIAL_ASSIGNMENT); + return ConsumerRecords.empty(); + }); expectConversionAndTransformation(null, new RecordHeaders()); - doAnswer(invocation -> null) - .doAnswer(invocation -> null) - .doAnswer(invocation -> { + workerTask.iteration(); // poll for initial assignment + time.sleep(30000L); + + final Map offsets = new HashMap<>(); + offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); + offsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + when(sinkTask.preCommit(offsets)).thenReturn(offsets); + + // first one raises wakeup + doThrow(new WakeupException()) + // and succeed the second time + .doNothing() + .when(consumer).commitSync(eq(offsets)); + + workerTask.iteration(); // first record delivered + + workerTask.iteration(); // now rebalance with the wakeup triggered + time.sleep(30000L); + + verify(sinkTask).close(INITIAL_ASSIGNMENT); + verify(sinkTask, times(2)).open(INITIAL_ASSIGNMENT); + + INITIAL_ASSIGNMENT.forEach(tp -> { + verify(consumer).resume(Collections.singleton(tp)); + }); + + verify(statusListener).onResume(taskId); + + assertSinkMetricValue("partition-count", 2); + assertSinkMetricValue("sink-record-read-total", 1.0); + assertSinkMetricValue("sink-record-send-total", 1.0); + assertSinkMetricValue("sink-record-active-count", 0.0); + assertSinkMetricValue("sink-record-active-count-max", 1.0); + assertSinkMetricValue("sink-record-active-count-avg", 0.33333); + assertSinkMetricValue("offset-commit-seq-no", 1.0); + assertSinkMetricValue("offset-commit-completion-total", 1.0); + assertSinkMetricValue("offset-commit-skip-total", 0.0); + assertTaskMetricValue("status", "running"); + assertTaskMetricValue("running-ratio", 1.0); + assertTaskMetricValue("pause-ratio", 0.0); + assertTaskMetricValue("batch-size-max", 1.0); + assertTaskMetricValue("batch-size-avg", 1.0); + assertTaskMetricValue("offset-commit-max-time-ms", 0.0); + assertTaskMetricValue("offset-commit-avg-time-ms", 0.0); + assertTaskMetricValue("offset-commit-failure-percentage", 0.0); + assertTaskMetricValue("offset-commit-success-percentage", 1.0); + } + + @Test + @SuppressWarnings("unchecked") + public void testWakeupNotThrownDuringShutdown() { + createTask(initialState); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + expectTaskGetTopic(); + expectPollInitialAssignment() + .thenAnswer(expectConsumerPoll(1)) + .thenAnswer(invocation -> { + // stop the task during its second iteration workerTask.stop(); - workerTask.cancel(); - return null; - }) - .when(sinkTask).put(anyList()); + return new ConsumerRecords<>(Collections.emptyMap()); + }); + expectConversionAndTransformation(null, new RecordHeaders()); - // task performs normal steps in advance of committing offsets final Map offsets = new HashMap<>(); - offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 2)); + offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); offsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); when(sinkTask.preCommit(offsets)).thenReturn(offsets); + // fail the first time + doThrow(new WakeupException()) + // and succeed the second time + .doNothing() + .when(consumer).commitSync(eq(offsets)); + workerTask.execute(); - // stop wakes up the consumer + assertEquals(0, workerTask.commitFailures()); verify(consumer).wakeup(); + verify(sinkTask).close(any(Collection.class)); + } - verify(sinkTask).close(any()); + @Test + public void testRequestCommit() { + createTask(initialState); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + expectTaskGetTopic(); + expectPollInitialAssignment() + .thenAnswer(expectConsumerPoll(1)) + .thenAnswer(expectConsumerPoll(0)); + expectConversionAndTransformation(null, new RecordHeaders()); + + // Initial assignment + time.sleep(30000L); + workerTask.iteration(); + assertSinkMetricValue("partition-count", 2); + + final Map offsets = new HashMap<>(); + offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); + offsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + when(sinkTask.preCommit(offsets)).thenReturn(offsets); + + // First record delivered + workerTask.iteration(); + assertSinkMetricValue("partition-count", 2); + assertSinkMetricValue("sink-record-read-total", 1.0); + assertSinkMetricValue("sink-record-send-total", 1.0); + assertSinkMetricValue("sink-record-active-count", 1.0); + assertSinkMetricValue("sink-record-active-count-max", 1.0); + assertSinkMetricValue("sink-record-active-count-avg", 0.333333); + assertSinkMetricValue("offset-commit-seq-no", 0.0); + assertSinkMetricValue("offset-commit-completion-total", 0.0); + assertSinkMetricValue("offset-commit-skip-total", 0.0); + assertTaskMetricValue("status", "running"); + assertTaskMetricValue("running-ratio", 1.0); + assertTaskMetricValue("pause-ratio", 0.0); + assertTaskMetricValue("batch-size-max", 1.0); + assertTaskMetricValue("batch-size-avg", 0.5); + assertTaskMetricValue("offset-commit-failure-percentage", 0.0); + assertTaskMetricValue("offset-commit-success-percentage", 0.0); + + // Grab the commit time prior to requesting a commit. + // This time should advance slightly after committing. + // KAFKA-8229 + final long previousCommitValue = workerTask.getNextCommit(); + sinkTaskContext.getValue().requestCommit(); + assertTrue(sinkTaskContext.getValue().isCommitRequested()); + assertNotEquals(offsets, workerTask.lastCommittedOffsets()); + + ArgumentCaptor callback = ArgumentCaptor.forClass(OffsetCommitCallback.class); + time.sleep(10000L); + workerTask.iteration(); // triggers the commit + verify(consumer).commitAsync(eq(offsets), callback.capture()); + callback.getValue().onComplete(offsets, null); + time.sleep(10000L); + + assertFalse(sinkTaskContext.getValue().isCommitRequested()); // should have been cleared + assertEquals(offsets, workerTask.lastCommittedOffsets()); + assertEquals(0, workerTask.commitFailures()); + + // Assert the next commit time advances slightly, the amount it advances + // is the normal commit time less the two sleeps since it started each + // of those sleeps were 10 seconds. + // KAFKA-8229 + assertEquals("Should have only advanced by 40 seconds", + previousCommitValue + + (WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_DEFAULT - 10000L * 2), + workerTask.getNextCommit()); + + assertSinkMetricValue("partition-count", 2); + assertSinkMetricValue("sink-record-read-total", 1.0); + assertSinkMetricValue("sink-record-send-total", 1.0); + assertSinkMetricValue("sink-record-active-count", 0.0); + assertSinkMetricValue("sink-record-active-count-max", 1.0); + assertSinkMetricValue("sink-record-active-count-avg", 0.2); + assertSinkMetricValue("offset-commit-seq-no", 1.0); + assertSinkMetricValue("offset-commit-completion-total", 1.0); + assertSinkMetricValue("offset-commit-skip-total", 0.0); + assertTaskMetricValue("status", "running"); + assertTaskMetricValue("running-ratio", 1.0); + assertTaskMetricValue("pause-ratio", 0.0); + assertTaskMetricValue("batch-size-max", 1.0); + assertTaskMetricValue("batch-size-avg", 0.33333); + assertTaskMetricValue("offset-commit-max-time-ms", 0.0); + assertTaskMetricValue("offset-commit-avg-time-ms", 0.0); + assertTaskMetricValue("offset-commit-failure-percentage", 0.0); + assertTaskMetricValue("offset-commit-success-percentage", 1.0); + } + + @Test + public void testPreCommit() { + createTask(initialState); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + expectTaskGetTopic(); + expectPollInitialAssignment() + .thenAnswer(expectConsumerPoll(2)) + .thenAnswer(expectConsumerPoll(0)); + expectConversionAndTransformation(null, new RecordHeaders()); + + workerTask.iteration(); // iter 1 -- initial assignment + + final Map workerStartingOffsets = new HashMap<>(); + workerStartingOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET)); + workerStartingOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + + assertEquals(workerStartingOffsets, workerTask.currentOffsets()); + + final Map workerCurrentOffsets = new HashMap<>(); + workerCurrentOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 2)); + workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + + final Map taskOffsets = new HashMap<>(); + taskOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); // act like FIRST_OFFSET+2 has not yet been flushed by the task + taskOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET + 1)); // should be ignored because > current offset + taskOffsets.put(new TopicPartition(TOPIC, 3), new OffsetAndMetadata(FIRST_OFFSET)); // should be ignored because this partition is not assigned + + when(sinkTask.preCommit(workerCurrentOffsets)).thenReturn(taskOffsets); + + workerTask.iteration(); // iter 2 -- deliver 2 records + + final Map committableOffsets = new HashMap<>(); + committableOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); + committableOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + + assertEquals(workerCurrentOffsets, workerTask.currentOffsets()); + assertEquals(workerStartingOffsets, workerTask.lastCommittedOffsets()); + + sinkTaskContext.getValue().requestCommit(); + workerTask.iteration(); // iter 3 -- commit + + // Expect extra invalid topic partition to be filtered, which causes the consumer assignment to be logged + ArgumentCaptor callback = ArgumentCaptor.forClass(OffsetCommitCallback.class); + verify(consumer).commitAsync(eq(committableOffsets), callback.capture()); + callback.getValue().onComplete(committableOffsets, null); + + assertEquals(committableOffsets, workerTask.lastCommittedOffsets()); + } + + @Test + public void testPreCommitFailure() { + createTask(initialState); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + expectTaskGetTopic(); + expectPollInitialAssignment() + // Put one message through the task to get some offsets to commit + .thenAnswer(expectConsumerPoll(2)) + .thenAnswer(expectConsumerPoll(0)); + + expectConversionAndTransformation(null, new RecordHeaders()); + + workerTask.iteration(); // iter 1 -- initial assignment + + workerTask.iteration(); // iter 2 -- deliver 2 records + + // iter 3 + final Map workerCurrentOffsets = new HashMap<>(); + workerCurrentOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 2)); + workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + when(sinkTask.preCommit(workerCurrentOffsets)).thenThrow(new ConnectException("Failed to flush")); + + sinkTaskContext.getValue().requestCommit(); + workerTask.iteration(); // iter 3 -- commit + + verify(consumer).seek(TOPIC_PARTITION, FIRST_OFFSET); + verify(consumer).seek(TOPIC_PARTITION2, FIRST_OFFSET); + } + + @Test + public void testIgnoredCommit() { + createTask(initialState); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + expectTaskGetTopic(); + // iter 1 + expectPollInitialAssignment() + // iter 2 + .thenAnswer(expectConsumerPoll(1)) + .thenAnswer(expectConsumerPoll(0)); + + expectConversionAndTransformation(null, new RecordHeaders()); + + workerTask.iteration(); // iter 1 -- initial assignment + + final Map workerStartingOffsets = new HashMap<>(); + workerStartingOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET)); + workerStartingOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + + assertEquals(workerStartingOffsets, workerTask.currentOffsets()); + assertEquals(workerStartingOffsets, workerTask.lastCommittedOffsets()); + + workerTask.iteration(); // iter 2 -- deliver 2 records + + // iter 3 + final Map workerCurrentOffsets = new HashMap<>(); + workerCurrentOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); + workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + + when(sinkTask.preCommit(workerCurrentOffsets)).thenReturn(workerStartingOffsets); + + sinkTaskContext.getValue().requestCommit(); + // no actual consumer.commit() triggered + workerTask.iteration(); // iter 3 -- commit + } + + // Test that the commitTimeoutMs timestamp is correctly computed and checked in WorkerSinkTask.iteration() + // when there is a long running commit in process. See KAFKA-4942 for more information. + @Test + public void testLongRunningCommitWithoutTimeout() throws InterruptedException { + createTask(initialState); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + expectTaskGetTopic(); + expectPollInitialAssignment() + .thenAnswer(expectConsumerPoll(1)) + // no actual consumer.commit() triggered + .thenAnswer(expectConsumerPoll(0)); + expectConversionAndTransformation(null, new RecordHeaders()); + + final Map workerStartingOffsets = new HashMap<>(); + workerStartingOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET)); + workerStartingOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + + workerTask.iteration(); // iter 1 -- initial assignment + assertEquals(workerStartingOffsets, workerTask.currentOffsets()); + assertEquals(workerStartingOffsets, workerTask.lastCommittedOffsets()); + + time.sleep(WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_DEFAULT); + workerTask.iteration(); // iter 2 -- deliver 2 records + + final Map workerCurrentOffsets = new HashMap<>(); + workerCurrentOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); + workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + + // iter 3 - note that we return the current offset to indicate they should be committed + when(sinkTask.preCommit(workerCurrentOffsets)).thenReturn(workerCurrentOffsets); + + sinkTaskContext.getValue().requestCommit(); + workerTask.iteration(); // iter 3 -- commit in progress + + // Make sure the "committing" flag didn't immediately get flipped back to false due to an incorrect timeout + assertTrue("Expected worker to be in the process of committing offsets", workerTask.isCommitting()); + + // Delay the result of trying to commit offsets to Kafka via the consumer.commitAsync method. + ArgumentCaptor offsetCommitCallbackArgumentCaptor = + ArgumentCaptor.forClass(OffsetCommitCallback.class); + verify(consumer).commitAsync(eq(workerCurrentOffsets), offsetCommitCallbackArgumentCaptor.capture()); + + final OffsetCommitCallback callback = offsetCommitCallbackArgumentCaptor.getValue(); + callback.onComplete(workerCurrentOffsets, null); + + assertEquals(workerCurrentOffsets, workerTask.currentOffsets()); + assertEquals(workerCurrentOffsets, workerTask.lastCommittedOffsets()); + assertFalse(workerTask.isCommitting()); + } + + @SuppressWarnings("unchecked") + @Test + public void testSinkTasksHandleCloseErrors() { + createTask(initialState); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + expectTaskGetTopic(); + expectPollInitialAssignment() + // Put one message through the task to get some offsets to commit + .thenAnswer(expectConsumerPoll(1)) + .thenAnswer(expectConsumerPoll(1)); + + expectConversionAndTransformation(null, new RecordHeaders()); + + doNothing() + .doAnswer(invocation -> { + workerTask.stop(); + return null; + }) + .when(sinkTask).put(anyList()); + + Throwable closeException = new RuntimeException(); + when(sinkTask.preCommit(anyMap())).thenReturn(Collections.emptyMap()); + + // Throw another exception while closing the task's assignment + doThrow(closeException).when(sinkTask).close(any(Collection.class)); + + try { + workerTask.execute(); + fail("workerTask.execute should have thrown an exception"); + } catch (RuntimeException e) { + assertSame("Exception from close should propagate as-is", closeException, e); + } + + verify(consumer).wakeup(); + } + + @SuppressWarnings("unchecked") + @Test + public void testSuppressCloseErrors() { + createTask(initialState); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + expectTaskGetTopic(); + expectPollInitialAssignment() + // Put one message through the task to get some offsets to commit + .thenAnswer(expectConsumerPoll(1)) + .thenAnswer(expectConsumerPoll(1)); + + expectConversionAndTransformation(null, new RecordHeaders()); + + Throwable putException = new RuntimeException(); + Throwable closeException = new RuntimeException(); + + doNothing() + // Throw an exception on the next put to trigger shutdown behavior + // This exception is the true "cause" of the failure + .doThrow(putException) + .when(sinkTask).put(anyList()); + + when(sinkTask.preCommit(anyMap())).thenReturn(Collections.emptyMap()); + + // Throw another exception while closing the task's assignment + doThrow(closeException).when(sinkTask).close(any(Collection.class)); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + try { + workerTask.execute(); + fail("workerTask.execute should have thrown an exception"); + } catch (ConnectException e) { + assertSame("Exception from put should be the cause", putException, e.getCause()); + assertTrue("Exception from close should be suppressed", e.getSuppressed().length > 0); + assertSame(closeException, e.getSuppressed()[0]); + } + } + + @SuppressWarnings("unchecked") + @Test + public void testTaskCancelPreventsFinalOffsetCommit() { + createTask(initialState); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + expectTaskGetTopic(); + expectPollInitialAssignment() + // Put one message through the task to get some offsets to commit + .thenAnswer(expectConsumerPoll(1)) + // the second put will return after the task is stopped and cancelled (asynchronously) + .thenAnswer(expectConsumerPoll(1)); + + expectConversionAndTransformation(null, new RecordHeaders()); + + doNothing() + .doNothing() + .doAnswer(invocation -> { + workerTask.stop(); + workerTask.cancel(); + return null; + }) + .when(sinkTask).put(anyList()); + + // task performs normal steps in advance of committing offsets + final Map offsets = new HashMap<>(); + offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 2)); + offsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + when(sinkTask.preCommit(offsets)).thenReturn(offsets); + + workerTask.execute(); + + // stop wakes up the consumer + verify(consumer).wakeup(); + + verify(sinkTask).close(any()); + } + + // Verify that when commitAsync is called but the supplied callback is not called by the consumer before a + // rebalance occurs, the async callback does not reset the last committed offset from the rebalance. + // See KAFKA-5731 for more information. + @Test + public void testCommitWithOutOfOrderCallback() { + createTask(initialState); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + // iter 1 + Answer> consumerPollRebalance = invocation -> { + rebalanceListener.getValue().onPartitionsAssigned(INITIAL_ASSIGNMENT); + return ConsumerRecords.empty(); + }; + + // iter 2 + expectTaskGetTopic(); + expectConversionAndTransformation(null, new RecordHeaders()); + + final Map workerStartingOffsets = new HashMap<>(); + workerStartingOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET)); + workerStartingOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + + final Map workerCurrentOffsets = new HashMap<>(); + workerCurrentOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); + workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + + final List originalPartitions = new ArrayList<>(INITIAL_ASSIGNMENT); + final List rebalancedPartitions = asList(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3); + final Map rebalanceOffsets = new HashMap<>(); + rebalanceOffsets.put(TOPIC_PARTITION, workerCurrentOffsets.get(TOPIC_PARTITION)); + rebalanceOffsets.put(TOPIC_PARTITION2, workerCurrentOffsets.get(TOPIC_PARTITION2)); + rebalanceOffsets.put(TOPIC_PARTITION3, new OffsetAndMetadata(FIRST_OFFSET)); + + final Map postRebalanceCurrentOffsets = new HashMap<>(); + postRebalanceCurrentOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 3)); + postRebalanceCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + postRebalanceCurrentOffsets.put(TOPIC_PARTITION3, new OffsetAndMetadata(FIRST_OFFSET + 2)); + + // iter 3 - note that we return the current offset to indicate they should be committed + when(sinkTask.preCommit(workerCurrentOffsets)).thenReturn(workerCurrentOffsets); + + // We need to delay the result of trying to commit offsets to Kafka via the consumer.commitAsync + // method. We do this so that we can test that the callback is not called until after the rebalance + // changes the lastCommittedOffsets. To fake this for tests we have the commitAsync build a function + // that will call the callback with the appropriate parameters, and we'll run that function later. + final AtomicReference asyncCallbackRunner = new AtomicReference<>(); + final AtomicBoolean asyncCallbackRan = new AtomicBoolean(); + + doAnswer(invocation -> { + final Map offsets = invocation.getArgument(0); + final OffsetCommitCallback callback = invocation.getArgument(1); + asyncCallbackRunner.set(() -> { + callback.onComplete(offsets, null); + asyncCallbackRan.set(true); + }); + + return null; + }).when(consumer).commitAsync(eq(workerCurrentOffsets), any(OffsetCommitCallback.class)); + + // Expect the next poll to discover and perform the rebalance, THEN complete the previous callback handler, + // and then return one record for TP1 and one for TP3. + final AtomicBoolean rebalanced = new AtomicBoolean(); + Answer> consumerPollRebalanced = invocation -> { + // Rebalance always begins with revoking current partitions ... + rebalanceListener.getValue().onPartitionsRevoked(originalPartitions); + // Respond to the rebalance + Map offsets = new HashMap<>(); + offsets.put(TOPIC_PARTITION, rebalanceOffsets.get(TOPIC_PARTITION).offset()); + offsets.put(TOPIC_PARTITION2, rebalanceOffsets.get(TOPIC_PARTITION2).offset()); + offsets.put(TOPIC_PARTITION3, rebalanceOffsets.get(TOPIC_PARTITION3).offset()); + sinkTaskContext.getValue().offset(offsets); + rebalanceListener.getValue().onPartitionsAssigned(rebalancedPartitions); + rebalanced.set(true); + + // Run the previous async commit handler + asyncCallbackRunner.get().run(); + + // And prep the two records to return + long timestamp = RecordBatch.NO_TIMESTAMP; + TimestampType timestampType = TimestampType.NO_TIMESTAMP_TYPE; + List> records = new ArrayList<>(); + records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + 1, timestamp, timestampType, + 0, 0, RAW_KEY, RAW_VALUE, new RecordHeaders(), Optional.empty())); + records.add(new ConsumerRecord<>(TOPIC, PARTITION3, FIRST_OFFSET + recordsReturnedTp3 + 1, timestamp, timestampType, + 0, 0, RAW_KEY, RAW_VALUE, new RecordHeaders(), Optional.empty())); + recordsReturnedTp1 += 1; + recordsReturnedTp3 += 1; + return new ConsumerRecords<>(Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records)); + }; + + // onPartitionsRevoked + when(sinkTask.preCommit(workerCurrentOffsets)).thenReturn(workerCurrentOffsets); + + // onPartitionsAssigned - step 1 + final long offsetTp1 = rebalanceOffsets.get(TOPIC_PARTITION).offset(); + final long offsetTp2 = rebalanceOffsets.get(TOPIC_PARTITION2).offset(); + final long offsetTp3 = rebalanceOffsets.get(TOPIC_PARTITION3).offset(); + + // iter 4 - note that we return the current offset to indicate they should be committed + when(sinkTask.preCommit(postRebalanceCurrentOffsets)).thenReturn(postRebalanceCurrentOffsets); + + // Setup mocks + when(consumer.assignment()) + .thenReturn(INITIAL_ASSIGNMENT) + .thenReturn(INITIAL_ASSIGNMENT) + .thenReturn(INITIAL_ASSIGNMENT) + .thenReturn(INITIAL_ASSIGNMENT) + .thenReturn(INITIAL_ASSIGNMENT) + .thenReturn(new HashSet<>(rebalancedPartitions)) + .thenReturn(new HashSet<>(rebalancedPartitions)) + .thenReturn(new HashSet<>(rebalancedPartitions)) + .thenReturn(new HashSet<>(rebalancedPartitions)) + .thenReturn(new HashSet<>(rebalancedPartitions)); + + when(consumer.position(TOPIC_PARTITION)) + .thenReturn(FIRST_OFFSET) + .thenReturn(offsetTp1); + + when(consumer.position(TOPIC_PARTITION2)) + .thenReturn(FIRST_OFFSET) + .thenReturn(offsetTp2); + + when(consumer.position(TOPIC_PARTITION3)) + .thenReturn(offsetTp3); + + when(consumer.poll(any(Duration.class))) + .thenAnswer(consumerPollRebalance) + .thenAnswer(expectConsumerPoll(1)) + .thenAnswer(consumerPollRebalanced) + .thenAnswer(expectConsumerPoll(1)); + + // Run the iterations + workerTask.iteration(); // iter 1 -- initial assignment + + time.sleep(WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_DEFAULT); + workerTask.iteration(); // iter 2 -- deliver records + + sinkTaskContext.getValue().requestCommit(); + workerTask.iteration(); // iter 3 -- commit in progress + + assertSinkMetricValue("partition-count", 3); + assertSinkMetricValue("sink-record-read-total", 3.0); + assertSinkMetricValue("sink-record-send-total", 3.0); + assertSinkMetricValue("sink-record-active-count", 4.0); + assertSinkMetricValue("sink-record-active-count-max", 4.0); + assertSinkMetricValue("sink-record-active-count-avg", 0.71429); + assertSinkMetricValue("offset-commit-seq-no", 2.0); + assertSinkMetricValue("offset-commit-completion-total", 1.0); + assertSinkMetricValue("offset-commit-skip-total", 1.0); + assertTaskMetricValue("status", "running"); + assertTaskMetricValue("running-ratio", 1.0); + assertTaskMetricValue("pause-ratio", 0.0); + assertTaskMetricValue("batch-size-max", 2.0); + assertTaskMetricValue("batch-size-avg", 1.0); + assertTaskMetricValue("offset-commit-max-time-ms", 0.0); + assertTaskMetricValue("offset-commit-avg-time-ms", 0.0); + assertTaskMetricValue("offset-commit-failure-percentage", 0.0); + assertTaskMetricValue("offset-commit-success-percentage", 1.0); + + assertTrue(asyncCallbackRan.get()); + assertTrue(rebalanced.get()); + + // Check that the offsets were not reset by the out-of-order async commit callback + assertEquals(postRebalanceCurrentOffsets, workerTask.currentOffsets()); + assertEquals(rebalanceOffsets, workerTask.lastCommittedOffsets()); + + // onPartitionsRevoked + verify(sinkTask).close(new ArrayList<>(workerCurrentOffsets.keySet())); + verify(consumer).commitSync(anyMap()); + + // onPartitionsAssigned - step 2 + verify(sinkTask).open(rebalancedPartitions); + + // onPartitionsAssigned - step 3 rewind + verify(consumer).seek(TOPIC_PARTITION, offsetTp1); + verify(consumer).seek(TOPIC_PARTITION2, offsetTp2); + verify(consumer).seek(TOPIC_PARTITION3, offsetTp3); + + time.sleep(WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_DEFAULT); + sinkTaskContext.getValue().requestCommit(); + workerTask.iteration(); // iter 4 -- commit in progress + + final ArgumentCaptor callback = ArgumentCaptor.forClass(OffsetCommitCallback.class); + verify(consumer).commitAsync(eq(postRebalanceCurrentOffsets), callback.capture()); + callback.getValue().onComplete(postRebalanceCurrentOffsets, null); + + // Check that the offsets were not reset by the out-of-order async commit callback + assertEquals(postRebalanceCurrentOffsets, workerTask.currentOffsets()); + assertEquals(postRebalanceCurrentOffsets, workerTask.lastCommittedOffsets()); + + assertSinkMetricValue("partition-count", 3); + assertSinkMetricValue("sink-record-read-total", 4.0); + assertSinkMetricValue("sink-record-send-total", 4.0); + assertSinkMetricValue("sink-record-active-count", 0.0); + assertSinkMetricValue("sink-record-active-count-max", 4.0); + assertSinkMetricValue("sink-record-active-count-avg", 0.5555555); + assertSinkMetricValue("offset-commit-seq-no", 3.0); + assertSinkMetricValue("offset-commit-completion-total", 2.0); + assertSinkMetricValue("offset-commit-skip-total", 1.0); + assertTaskMetricValue("status", "running"); + assertTaskMetricValue("running-ratio", 1.0); + assertTaskMetricValue("pause-ratio", 0.0); + assertTaskMetricValue("batch-size-max", 2.0); + assertTaskMetricValue("batch-size-avg", 1.0); + assertTaskMetricValue("offset-commit-max-time-ms", 0.0); + assertTaskMetricValue("offset-commit-avg-time-ms", 0.0); + assertTaskMetricValue("offset-commit-failure-percentage", 0.0); + assertTaskMetricValue("offset-commit-success-percentage", 1.0); } @Test @@ -1081,4 +1918,4 @@ private void assertTaskMetricValue(String name, String expected) { String measured = metrics.currentMetricValueAsString(taskGroup, name); assertEquals(expected, measured); } -} +} \ No newline at end of file diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java deleted file mode 100644 index e103c30157..0000000000 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java +++ /dev/null @@ -1,1298 +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 org.apache.kafka.connect.runtime; - -import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -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.consumer.OffsetAndMetadata; -import org.apache.kafka.clients.consumer.OffsetCommitCallback; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.WakeupException; -import org.apache.kafka.common.header.Header; -import org.apache.kafka.common.header.Headers; -import org.apache.kafka.common.header.internals.RecordHeaders; -import org.apache.kafka.common.record.RecordBatch; -import org.apache.kafka.common.record.TimestampType; -import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.common.utils.Time; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaAndValue; -import org.apache.kafka.connect.errors.ConnectException; -import org.apache.kafka.connect.errors.RetriableException; -import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; -import org.apache.kafka.connect.runtime.errors.ErrorHandlingMetrics; -import org.apache.kafka.connect.runtime.errors.ErrorReporter; -import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator; -import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperatorTest; -import org.apache.kafka.connect.runtime.isolation.PluginClassLoader; -import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; -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.ClusterConfigState; -import org.apache.kafka.connect.storage.Converter; -import org.apache.kafka.connect.storage.HeaderConverter; -import org.apache.kafka.connect.storage.StatusBackingStore; -import org.apache.kafka.connect.util.ConnectorTaskId; -import org.easymock.Capture; -import org.easymock.EasyMock; -import org.easymock.IExpectationSetters; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.powermock.api.easymock.PowerMock; -import org.powermock.api.easymock.annotation.Mock; -import org.powermock.core.classloader.annotations.PowerMockIgnore; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; -import org.powermock.reflect.Whitebox; - -import java.time.Duration; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Function; -import java.util.function.Supplier; -import java.util.stream.Collectors; - -import static java.util.Arrays.asList; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -@RunWith(PowerMockRunner.class) -@PrepareForTest(WorkerSinkTask.class) -@PowerMockIgnore("javax.management.*") -public class WorkerSinkTaskTest { - // These are fixed to keep this code simpler. In this example we assume byte[] raw values - // with mix of integer/string in Connect - private static final String TOPIC = "test"; - private static final int PARTITION = 12; - private static final int PARTITION2 = 13; - private static final int PARTITION3 = 14; - private static final long FIRST_OFFSET = 45; - private static final Schema KEY_SCHEMA = Schema.INT32_SCHEMA; - private static final int KEY = 12; - private static final Schema VALUE_SCHEMA = Schema.STRING_SCHEMA; - private static final String VALUE = "VALUE"; - private static final byte[] RAW_KEY = "key".getBytes(); - private static final byte[] RAW_VALUE = "value".getBytes(); - - private static final TopicPartition TOPIC_PARTITION = new TopicPartition(TOPIC, PARTITION); - private static final TopicPartition TOPIC_PARTITION2 = new TopicPartition(TOPIC, PARTITION2); - private static final TopicPartition TOPIC_PARTITION3 = new TopicPartition(TOPIC, PARTITION3); - - private static final Set INITIAL_ASSIGNMENT = - new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2)); - - private static final Map TASK_PROPS = new HashMap<>(); - static { - TASK_PROPS.put(SinkConnector.TOPICS_CONFIG, TOPIC); - TASK_PROPS.put(TaskConfig.TASK_CLASS_CONFIG, TestSinkTask.class.getName()); - } - private static final TaskConfig TASK_CONFIG = new TaskConfig(TASK_PROPS); - - private ConnectorTaskId taskId = new ConnectorTaskId("job", 0); - private TargetState initialState = TargetState.STARTED; - private MockTime time; - private WorkerSinkTask workerTask; - @Mock - private SinkTask sinkTask; - private Capture sinkTaskContext = EasyMock.newCapture(); - private WorkerConfig workerConfig; - private MockConnectMetrics metrics; - @Mock - private PluginClassLoader pluginLoader; - @Mock - private Converter keyConverter; - @Mock - private Converter valueConverter; - @Mock - private HeaderConverter headerConverter; - @Mock - private TransformationChain, SinkRecord> transformationChain; - @Mock - private TaskStatus.Listener statusListener; - @Mock - private StatusBackingStore statusBackingStore; - @Mock - private KafkaConsumer consumer; - @Mock - private ErrorHandlingMetrics errorHandlingMetrics; - private Capture rebalanceListener = EasyMock.newCapture(); - - private long recordsReturnedTp1; - private long recordsReturnedTp3; - - @Before - public void setUp() { - time = new MockTime(); - Map workerProps = new HashMap<>(); - workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); - workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); - workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); - workerConfig = new StandaloneConfig(workerProps); - pluginLoader = PowerMock.createMock(PluginClassLoader.class); - metrics = new MockConnectMetrics(time); - recordsReturnedTp1 = 0; - recordsReturnedTp3 = 0; - } - - private void createTask(TargetState initialState) { - createTask(initialState, keyConverter, valueConverter, headerConverter); - } - - private void createTask(TargetState initialState, Converter keyConverter, Converter valueConverter, HeaderConverter headerConverter) { - createTask(initialState, keyConverter, valueConverter, headerConverter, RetryWithToleranceOperatorTest.noopOperator(), Collections::emptyList); - } - - private void createTask(TargetState initialState, Converter keyConverter, Converter valueConverter, HeaderConverter headerConverter, - RetryWithToleranceOperator> retryWithToleranceOperator, Supplier>>> errorReportersSupplier) { - workerTask = new WorkerSinkTask( - taskId, sinkTask, statusListener, initialState, workerConfig, ClusterConfigState.EMPTY, metrics, - keyConverter, valueConverter, errorHandlingMetrics, headerConverter, - transformationChain, consumer, pluginLoader, time, - retryWithToleranceOperator, null, statusBackingStore, errorReportersSupplier); - } - - @After - public void tearDown() { - if (metrics != null) metrics.stop(); - } - - @Test - public void testPollRedeliveryWithConsumerRebalance() throws Exception { - createTask(initialState); - - expectInitializeTask(); - expectTaskGetTopic(true); - expectPollInitialAssignment(); - - // If a retriable exception is thrown, we should redeliver the same batch, pausing the consumer in the meantime - expectConsumerPoll(1); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall().andThrow(new RetriableException("retry")); - // Pause - EasyMock.expect(consumer.assignment()).andReturn(INITIAL_ASSIGNMENT); - consumer.pause(INITIAL_ASSIGNMENT); - PowerMock.expectLastCall(); - - // Empty consumer poll (all partitions are paused) with rebalance; one new partition is assigned - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - () -> { - rebalanceListener.getValue().onPartitionsRevoked(Collections.emptySet()); - rebalanceListener.getValue().onPartitionsAssigned(Collections.singleton(TOPIC_PARTITION3)); - return ConsumerRecords.empty(); - }); - Set newAssignment = new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3)); - EasyMock.expect(consumer.assignment()).andReturn(newAssignment).times(3); - EasyMock.expect(consumer.position(TOPIC_PARTITION3)).andReturn(FIRST_OFFSET); - sinkTask.open(Collections.singleton(TOPIC_PARTITION3)); - EasyMock.expectLastCall(); - // All partitions are re-paused in order to pause any newly-assigned partitions so that redelivery efforts can continue - consumer.pause(newAssignment); - EasyMock.expectLastCall(); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall().andThrow(new RetriableException("retry")); - - // Next delivery attempt fails again - expectConsumerPoll(0); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall().andThrow(new RetriableException("retry")); - - // Non-empty consumer poll; all initially-assigned partitions are revoked in rebalance, and new partitions are allowed to resume - ConsumerRecord newRecord = new ConsumerRecord<>(TOPIC, PARTITION3, FIRST_OFFSET, RAW_KEY, RAW_VALUE); - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - () -> { - rebalanceListener.getValue().onPartitionsRevoked(INITIAL_ASSIGNMENT); - rebalanceListener.getValue().onPartitionsAssigned(Collections.emptyList()); - return new ConsumerRecords<>(Collections.singletonMap(TOPIC_PARTITION3, Collections.singletonList(newRecord))); - }); - newAssignment = Collections.singleton(TOPIC_PARTITION3); - EasyMock.expect(consumer.assignment()).andReturn(new HashSet<>(newAssignment)).times(3); - final Map offsets = INITIAL_ASSIGNMENT.stream() - .collect(Collectors.toMap(Function.identity(), tp -> new OffsetAndMetadata(FIRST_OFFSET))); - sinkTask.preCommit(offsets); - EasyMock.expectLastCall().andReturn(offsets); - sinkTask.close(INITIAL_ASSIGNMENT); - EasyMock.expectLastCall(); - // All partitions are resumed, as all previously paused-for-redelivery partitions were revoked - newAssignment.forEach(tp -> { - consumer.resume(Collections.singleton(tp)); - EasyMock.expectLastCall(); - }); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - workerTask.iteration(); - workerTask.iteration(); - workerTask.iteration(); - workerTask.iteration(); - workerTask.iteration(); - - PowerMock.verifyAll(); - } - - @Test - public void testPreCommitFailureAfterPartialRevocationAndAssignment() throws Exception { - createTask(initialState); - - // First poll; assignment is [TP1, TP2] - expectInitializeTask(); - expectTaskGetTopic(true); - expectPollInitialAssignment(); - - // Second poll; a single record is delivered from TP1 - expectConsumerPoll(1); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - // Third poll; assignment changes to [TP2] - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - () -> { - rebalanceListener.getValue().onPartitionsRevoked(Collections.singleton(TOPIC_PARTITION)); - rebalanceListener.getValue().onPartitionsAssigned(Collections.emptySet()); - return ConsumerRecords.empty(); - }); - EasyMock.expect(consumer.assignment()).andReturn(Collections.singleton(TOPIC_PARTITION)).times(2); - final Map offsets = new HashMap<>(); - offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); - sinkTask.preCommit(offsets); - EasyMock.expectLastCall().andReturn(offsets); - consumer.commitSync(offsets); - EasyMock.expectLastCall(); - sinkTask.close(Collections.singleton(TOPIC_PARTITION)); - EasyMock.expectLastCall(); - sinkTask.put(Collections.emptyList()); - EasyMock.expectLastCall(); - - // Fourth poll; assignment changes to [TP2, TP3] - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - () -> { - rebalanceListener.getValue().onPartitionsRevoked(Collections.emptySet()); - rebalanceListener.getValue().onPartitionsAssigned(Collections.singleton(TOPIC_PARTITION3)); - return ConsumerRecords.empty(); - }); - EasyMock.expect(consumer.assignment()).andReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3))).times(2); - EasyMock.expect(consumer.position(TOPIC_PARTITION3)).andReturn(FIRST_OFFSET); - sinkTask.open(Collections.singleton(TOPIC_PARTITION3)); - EasyMock.expectLastCall(); - sinkTask.put(Collections.emptyList()); - EasyMock.expectLastCall(); - - // Fifth poll; an offset commit takes place - EasyMock.expect(consumer.assignment()).andReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3))).times(2); - final Map workerCurrentOffsets = new HashMap<>(); - workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - workerCurrentOffsets.put(TOPIC_PARTITION3, new OffsetAndMetadata(FIRST_OFFSET)); - sinkTask.preCommit(workerCurrentOffsets); - EasyMock.expectLastCall().andThrow(new ConnectException("Failed to flush")); - - consumer.seek(TOPIC_PARTITION2, FIRST_OFFSET); - EasyMock.expectLastCall(); - consumer.seek(TOPIC_PARTITION3, FIRST_OFFSET); - EasyMock.expectLastCall(); - - expectConsumerPoll(0); - sinkTask.put(EasyMock.eq(Collections.emptyList())); - EasyMock.expectLastCall(); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - // First iteration--first call to poll, first consumer assignment - workerTask.iteration(); - // Second iteration--second call to poll, delivery of one record - workerTask.iteration(); - // Third iteration--third call to poll, partial consumer revocation - workerTask.iteration(); - // Fourth iteration--fourth call to poll, partial consumer assignment - workerTask.iteration(); - // Fifth iteration--task-requested offset commit with failure in SinkTask::preCommit - sinkTaskContext.getValue().requestCommit(); - workerTask.iteration(); - - PowerMock.verifyAll(); - } - - @Test - public void testWakeupInCommitSyncCausesRetry() throws Exception { - createTask(initialState); - - expectInitializeTask(); - expectTaskGetTopic(true); - expectPollInitialAssignment(); - - expectConsumerPoll(1); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - final Map offsets = new HashMap<>(); - offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); - offsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - sinkTask.preCommit(offsets); - EasyMock.expectLastCall().andReturn(offsets); - - // first one raises wakeup - consumer.commitSync(EasyMock.>anyObject()); - EasyMock.expectLastCall().andThrow(new WakeupException()); - - // we should retry and complete the commit - consumer.commitSync(EasyMock.>anyObject()); - EasyMock.expectLastCall(); - - sinkTask.close(INITIAL_ASSIGNMENT); - EasyMock.expectLastCall(); - - INITIAL_ASSIGNMENT.forEach(tp -> EasyMock.expect(consumer.position(tp)).andReturn(FIRST_OFFSET)); - - sinkTask.open(INITIAL_ASSIGNMENT); - EasyMock.expectLastCall(); - - EasyMock.expect(consumer.assignment()).andReturn(INITIAL_ASSIGNMENT).times(5); - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - () -> { - rebalanceListener.getValue().onPartitionsRevoked(INITIAL_ASSIGNMENT); - rebalanceListener.getValue().onPartitionsAssigned(INITIAL_ASSIGNMENT); - return ConsumerRecords.empty(); - }); - - INITIAL_ASSIGNMENT.forEach(tp -> { - consumer.resume(Collections.singleton(tp)); - EasyMock.expectLastCall(); - }); - - statusListener.onResume(taskId); - EasyMock.expectLastCall(); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - time.sleep(30000L); - workerTask.initializeAndStart(); - time.sleep(30000L); - - workerTask.iteration(); // poll for initial assignment - time.sleep(30000L); - workerTask.iteration(); // first record delivered - workerTask.iteration(); // now rebalance with the wakeup triggered - time.sleep(30000L); - - assertSinkMetricValue("partition-count", 2); - assertSinkMetricValue("sink-record-read-total", 1.0); - assertSinkMetricValue("sink-record-send-total", 1.0); - assertSinkMetricValue("sink-record-active-count", 0.0); - assertSinkMetricValue("sink-record-active-count-max", 1.0); - assertSinkMetricValue("sink-record-active-count-avg", 0.33333); - assertSinkMetricValue("offset-commit-seq-no", 1.0); - assertSinkMetricValue("offset-commit-completion-total", 1.0); - assertSinkMetricValue("offset-commit-skip-total", 0.0); - assertTaskMetricValue("status", "running"); - assertTaskMetricValue("running-ratio", 1.0); - assertTaskMetricValue("pause-ratio", 0.0); - assertTaskMetricValue("batch-size-max", 1.0); - assertTaskMetricValue("batch-size-avg", 1.0); - assertTaskMetricValue("offset-commit-max-time-ms", 0.0); - assertTaskMetricValue("offset-commit-avg-time-ms", 0.0); - assertTaskMetricValue("offset-commit-failure-percentage", 0.0); - assertTaskMetricValue("offset-commit-success-percentage", 1.0); - - PowerMock.verifyAll(); - } - - @Test - public void testWakeupNotThrownDuringShutdown() throws Exception { - createTask(initialState); - - expectInitializeTask(); - expectTaskGetTopic(true); - expectPollInitialAssignment(); - - expectConsumerPoll(1); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(() -> { - // stop the task during its second iteration - workerTask.stop(); - return new ConsumerRecords<>(Collections.emptyMap()); - }); - consumer.wakeup(); - EasyMock.expectLastCall(); - - sinkTask.put(EasyMock.eq(Collections.emptyList())); - EasyMock.expectLastCall(); - - EasyMock.expect(consumer.assignment()).andReturn(INITIAL_ASSIGNMENT).times(1); - - final Map offsets = new HashMap<>(); - offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); - offsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - sinkTask.preCommit(offsets); - EasyMock.expectLastCall().andReturn(offsets); - - sinkTask.close(EasyMock.anyObject()); - PowerMock.expectLastCall(); - - // fail the first time - consumer.commitSync(EasyMock.eq(offsets)); - EasyMock.expectLastCall().andThrow(new WakeupException()); - - // and succeed the second time - consumer.commitSync(EasyMock.eq(offsets)); - EasyMock.expectLastCall(); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - workerTask.execute(); - - assertEquals(0, workerTask.commitFailures()); - - PowerMock.verifyAll(); - } - - @Test - public void testRequestCommit() throws Exception { - createTask(initialState); - - expectInitializeTask(); - expectTaskGetTopic(true); - expectPollInitialAssignment(); - - expectConsumerPoll(1); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - final Map offsets = new HashMap<>(); - offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); - offsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - sinkTask.preCommit(offsets); - EasyMock.expectLastCall().andReturn(offsets); - - EasyMock.expect(consumer.assignment()).andReturn(INITIAL_ASSIGNMENT).times(2); - - final Capture callback = EasyMock.newCapture(); - consumer.commitAsync(EasyMock.eq(offsets), EasyMock.capture(callback)); - EasyMock.expectLastCall().andAnswer(() -> { - callback.getValue().onComplete(offsets, null); - return null; - }); - - expectConsumerPoll(0); - sinkTask.put(Collections.emptyList()); - EasyMock.expectLastCall(); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - - // Initial assignment - time.sleep(30000L); - workerTask.iteration(); - assertSinkMetricValue("partition-count", 2); - - // First record delivered - workerTask.iteration(); - assertSinkMetricValue("partition-count", 2); - assertSinkMetricValue("sink-record-read-total", 1.0); - assertSinkMetricValue("sink-record-send-total", 1.0); - assertSinkMetricValue("sink-record-active-count", 1.0); - assertSinkMetricValue("sink-record-active-count-max", 1.0); - assertSinkMetricValue("sink-record-active-count-avg", 0.333333); - assertSinkMetricValue("offset-commit-seq-no", 0.0); - assertSinkMetricValue("offset-commit-completion-total", 0.0); - assertSinkMetricValue("offset-commit-skip-total", 0.0); - assertTaskMetricValue("status", "running"); - assertTaskMetricValue("running-ratio", 1.0); - assertTaskMetricValue("pause-ratio", 0.0); - assertTaskMetricValue("batch-size-max", 1.0); - assertTaskMetricValue("batch-size-avg", 0.5); - assertTaskMetricValue("offset-commit-failure-percentage", 0.0); - assertTaskMetricValue("offset-commit-success-percentage", 0.0); - - // Grab the commit time prior to requesting a commit. - // This time should advance slightly after committing. - // KAFKA-8229 - final long previousCommitValue = workerTask.getNextCommit(); - sinkTaskContext.getValue().requestCommit(); - assertTrue(sinkTaskContext.getValue().isCommitRequested()); - assertNotEquals(offsets, Whitebox.>getInternalState(workerTask, "lastCommittedOffsets")); - time.sleep(10000L); - workerTask.iteration(); // triggers the commit - time.sleep(10000L); - assertFalse(sinkTaskContext.getValue().isCommitRequested()); // should have been cleared - assertEquals(offsets, Whitebox.>getInternalState(workerTask, "lastCommittedOffsets")); - assertEquals(0, workerTask.commitFailures()); - // Assert the next commit time advances slightly, the amount it advances - // is the normal commit time less the two sleeps since it started each - // of those sleeps were 10 seconds. - // KAFKA-8229 - assertEquals("Should have only advanced by 40 seconds", - previousCommitValue + - (WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_DEFAULT - 10000L * 2), - workerTask.getNextCommit()); - - assertSinkMetricValue("partition-count", 2); - assertSinkMetricValue("sink-record-read-total", 1.0); - assertSinkMetricValue("sink-record-send-total", 1.0); - assertSinkMetricValue("sink-record-active-count", 0.0); - assertSinkMetricValue("sink-record-active-count-max", 1.0); - assertSinkMetricValue("sink-record-active-count-avg", 0.2); - assertSinkMetricValue("offset-commit-seq-no", 1.0); - assertSinkMetricValue("offset-commit-completion-total", 1.0); - assertSinkMetricValue("offset-commit-skip-total", 0.0); - assertTaskMetricValue("status", "running"); - assertTaskMetricValue("running-ratio", 1.0); - assertTaskMetricValue("pause-ratio", 0.0); - assertTaskMetricValue("batch-size-max", 1.0); - assertTaskMetricValue("batch-size-avg", 0.33333); - assertTaskMetricValue("offset-commit-max-time-ms", 0.0); - assertTaskMetricValue("offset-commit-avg-time-ms", 0.0); - assertTaskMetricValue("offset-commit-failure-percentage", 0.0); - assertTaskMetricValue("offset-commit-success-percentage", 1.0); - - PowerMock.verifyAll(); - } - - @Test - public void testPreCommit() throws Exception { - createTask(initialState); - - expectInitializeTask(); - expectTaskGetTopic(true); - - // iter 1 - expectPollInitialAssignment(); - - // iter 2 - expectConsumerPoll(2); - expectConversionAndTransformation(2); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - final Map workerStartingOffsets = new HashMap<>(); - workerStartingOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET)); - workerStartingOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - - final Map workerCurrentOffsets = new HashMap<>(); - workerCurrentOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 2)); - workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - - final Map taskOffsets = new HashMap<>(); - taskOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); // act like FIRST_OFFSET+2 has not yet been flushed by the task - taskOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET + 1)); // should be ignored because > current offset - taskOffsets.put(new TopicPartition(TOPIC, 3), new OffsetAndMetadata(FIRST_OFFSET)); // should be ignored because this partition is not assigned - - final Map committableOffsets = new HashMap<>(); - committableOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); - committableOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - - sinkTask.preCommit(workerCurrentOffsets); - EasyMock.expectLastCall().andReturn(taskOffsets); - // Expect extra invalid topic partition to be filtered, which causes the consumer assignment to be logged - EasyMock.expect(consumer.assignment()).andReturn(INITIAL_ASSIGNMENT).times(2); - final Capture callback = EasyMock.newCapture(); - consumer.commitAsync(EasyMock.eq(committableOffsets), EasyMock.capture(callback)); - EasyMock.expectLastCall().andAnswer(() -> { - callback.getValue().onComplete(committableOffsets, null); - return null; - }); - expectConsumerPoll(0); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - workerTask.iteration(); // iter 1 -- initial assignment - - assertEquals(workerStartingOffsets, Whitebox.>getInternalState(workerTask, "currentOffsets")); - workerTask.iteration(); // iter 2 -- deliver 2 records - - assertEquals(workerCurrentOffsets, Whitebox.>getInternalState(workerTask, "currentOffsets")); - assertEquals(workerStartingOffsets, Whitebox.>getInternalState(workerTask, "lastCommittedOffsets")); - sinkTaskContext.getValue().requestCommit(); - workerTask.iteration(); // iter 3 -- commit - assertEquals(committableOffsets, Whitebox.>getInternalState(workerTask, "lastCommittedOffsets")); - - PowerMock.verifyAll(); - } - - @Test - public void testPreCommitFailure() throws Exception { - createTask(initialState); - - expectInitializeTask(); - expectTaskGetTopic(true); - EasyMock.expect(consumer.assignment()).andStubReturn(INITIAL_ASSIGNMENT); - - // iter 1 - expectPollInitialAssignment(); - - // iter 2 - expectConsumerPoll(2); - expectConversionAndTransformation(2); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - // iter 3 - final Map workerCurrentOffsets = new HashMap<>(); - workerCurrentOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 2)); - workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - sinkTask.preCommit(workerCurrentOffsets); - EasyMock.expectLastCall().andThrow(new ConnectException("Failed to flush")); - - consumer.seek(TOPIC_PARTITION, FIRST_OFFSET); - EasyMock.expectLastCall(); - consumer.seek(TOPIC_PARTITION2, FIRST_OFFSET); - EasyMock.expectLastCall(); - - expectConsumerPoll(0); - sinkTask.put(EasyMock.eq(Collections.emptyList())); - EasyMock.expectLastCall(); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - workerTask.iteration(); // iter 1 -- initial assignment - workerTask.iteration(); // iter 2 -- deliver 2 records - sinkTaskContext.getValue().requestCommit(); - workerTask.iteration(); // iter 3 -- commit - - PowerMock.verifyAll(); - } - - @Test - public void testIgnoredCommit() throws Exception { - createTask(initialState); - - expectInitializeTask(); - expectTaskGetTopic(true); - - // iter 1 - expectPollInitialAssignment(); - - // iter 2 - expectConsumerPoll(1); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - final Map workerStartingOffsets = new HashMap<>(); - workerStartingOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET)); - workerStartingOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - - final Map workerCurrentOffsets = new HashMap<>(); - workerCurrentOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); - workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - - EasyMock.expect(consumer.assignment()).andReturn(INITIAL_ASSIGNMENT).times(2); - - // iter 3 - sinkTask.preCommit(workerCurrentOffsets); - EasyMock.expectLastCall().andReturn(workerStartingOffsets); - // no actual consumer.commit() triggered - expectConsumerPoll(0); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - workerTask.iteration(); // iter 1 -- initial assignment - - assertEquals(workerStartingOffsets, Whitebox.>getInternalState(workerTask, "currentOffsets")); - assertEquals(workerStartingOffsets, Whitebox.>getInternalState(workerTask, "lastCommittedOffsets")); - - workerTask.iteration(); // iter 2 -- deliver 2 records - - sinkTaskContext.getValue().requestCommit(); - workerTask.iteration(); // iter 3 -- commit - - PowerMock.verifyAll(); - } - - // Test that the commitTimeoutMs timestamp is correctly computed and checked in WorkerSinkTask.iteration() - // when there is a long running commit in process. See KAFKA-4942 for more information. - @Test - public void testLongRunningCommitWithoutTimeout() throws Exception { - createTask(initialState); - - expectInitializeTask(); - expectTaskGetTopic(true); - - // iter 1 - expectPollInitialAssignment(); - - // iter 2 - expectConsumerPoll(1); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - final Map workerStartingOffsets = new HashMap<>(); - workerStartingOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET)); - workerStartingOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - - final Map workerCurrentOffsets = new HashMap<>(); - workerCurrentOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); - workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - - EasyMock.expect(consumer.assignment()).andReturn(INITIAL_ASSIGNMENT).times(2); - - // iter 3 - note that we return the current offset to indicate they should be committed - sinkTask.preCommit(workerCurrentOffsets); - EasyMock.expectLastCall().andReturn(workerCurrentOffsets); - - // We need to delay the result of trying to commit offsets to Kafka via the consumer.commitAsync - // method. We do this so that we can test that we do not erroneously mark a commit as timed out - // while it is still running and under time. To fake this for tests we have the commit run in a - // separate thread and wait for a latch which we control back in the main thread. - final ExecutorService executor = Executors.newSingleThreadExecutor(); - final CountDownLatch latch = new CountDownLatch(1); - - consumer.commitAsync(EasyMock.eq(workerCurrentOffsets), EasyMock.anyObject()); - EasyMock.expectLastCall().andAnswer(() -> { - // Grab the arguments passed to the consumer.commitAsync method - final Object[] args = EasyMock.getCurrentArguments(); - @SuppressWarnings("unchecked") - final Map offsets = (Map) args[0]; - final OffsetCommitCallback callback = (OffsetCommitCallback) args[1]; - - executor.execute(() -> { - try { - latch.await(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - - callback.onComplete(offsets, null); - }); - - return null; - }); - - // no actual consumer.commit() triggered - expectConsumerPoll(0); - - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - workerTask.iteration(); // iter 1 -- initial assignment - - assertEquals(workerStartingOffsets, Whitebox.>getInternalState(workerTask, "currentOffsets")); - assertEquals(workerStartingOffsets, Whitebox.>getInternalState(workerTask, "lastCommittedOffsets")); - - time.sleep(WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_DEFAULT); - workerTask.iteration(); // iter 2 -- deliver 2 records - - sinkTaskContext.getValue().requestCommit(); - workerTask.iteration(); // iter 3 -- commit in progress - - // Make sure the "committing" flag didn't immediately get flipped back to false due to an incorrect timeout - assertTrue("Expected worker to be in the process of committing offsets", workerTask.isCommitting()); - - // Let the async commit finish and wait for it to end - latch.countDown(); - executor.shutdown(); - executor.awaitTermination(30, TimeUnit.SECONDS); - - assertEquals(workerCurrentOffsets, Whitebox.>getInternalState(workerTask, "currentOffsets")); - assertEquals(workerCurrentOffsets, Whitebox.>getInternalState(workerTask, "lastCommittedOffsets")); - - PowerMock.verifyAll(); - } - - @Test - public void testSinkTasksHandleCloseErrors() throws Exception { - createTask(initialState); - expectInitializeTask(); - expectTaskGetTopic(true); - - expectPollInitialAssignment(); - - // Put one message through the task to get some offsets to commit - expectConsumerPoll(1); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - PowerMock.expectLastCall().andVoid(); - - // Stop the task during the next put - expectConsumerPoll(1); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - PowerMock.expectLastCall().andAnswer(() -> { - workerTask.stop(); - return null; - }); - - consumer.wakeup(); - PowerMock.expectLastCall(); - - // Throw another exception while closing the task's assignment - EasyMock.expect(sinkTask.preCommit(EasyMock.anyObject())) - .andStubReturn(Collections.emptyMap()); - Throwable closeException = new RuntimeException(); - sinkTask.close(EasyMock.anyObject()); - PowerMock.expectLastCall().andThrow(closeException); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - try { - workerTask.execute(); - fail("workerTask.execute should have thrown an exception"); - } catch (RuntimeException e) { - PowerMock.verifyAll(); - assertSame("Exception from close should propagate as-is", closeException, e); - } - } - - @Test - public void testSuppressCloseErrors() throws Exception { - createTask(initialState); - expectInitializeTask(); - expectTaskGetTopic(true); - - expectPollInitialAssignment(); - - // Put one message through the task to get some offsets to commit - expectConsumerPoll(1); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - PowerMock.expectLastCall().andVoid(); - - // Throw an exception on the next put to trigger shutdown behavior - // This exception is the true "cause" of the failure - expectConsumerPoll(1); - expectConversionAndTransformation(1); - Throwable putException = new RuntimeException(); - sinkTask.put(EasyMock.anyObject()); - PowerMock.expectLastCall().andThrow(putException); - - // Throw another exception while closing the task's assignment - EasyMock.expect(sinkTask.preCommit(EasyMock.anyObject())) - .andStubReturn(Collections.emptyMap()); - Throwable closeException = new RuntimeException(); - sinkTask.close(EasyMock.anyObject()); - PowerMock.expectLastCall().andThrow(closeException); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - try { - workerTask.execute(); - fail("workerTask.execute should have thrown an exception"); - } catch (ConnectException e) { - PowerMock.verifyAll(); - assertSame("Exception from put should be the cause", putException, e.getCause()); - assertTrue("Exception from close should be suppressed", e.getSuppressed().length > 0); - assertSame(closeException, e.getSuppressed()[0]); - } - } - - // Verify that when commitAsync is called but the supplied callback is not called by the consumer before a - // rebalance occurs, the async callback does not reset the last committed offset from the rebalance. - // See KAFKA-5731 for more information. - @Test - public void testCommitWithOutOfOrderCallback() throws Exception { - createTask(initialState); - - expectInitializeTask(); - expectTaskGetTopic(true); - - // iter 1 - expectPollInitialAssignment(); - - // iter 2 - expectConsumerPoll(1); - expectConversionAndTransformation(4); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - final Map workerStartingOffsets = new HashMap<>(); - workerStartingOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET)); - workerStartingOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - - final Map workerCurrentOffsets = new HashMap<>(); - workerCurrentOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); - workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - - final List originalPartitions = new ArrayList<>(INITIAL_ASSIGNMENT); - final List rebalancedPartitions = asList(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3); - final Map rebalanceOffsets = new HashMap<>(); - rebalanceOffsets.put(TOPIC_PARTITION, workerCurrentOffsets.get(TOPIC_PARTITION)); - rebalanceOffsets.put(TOPIC_PARTITION2, workerCurrentOffsets.get(TOPIC_PARTITION2)); - rebalanceOffsets.put(TOPIC_PARTITION3, new OffsetAndMetadata(FIRST_OFFSET)); - - final Map postRebalanceCurrentOffsets = new HashMap<>(); - postRebalanceCurrentOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 3)); - postRebalanceCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - postRebalanceCurrentOffsets.put(TOPIC_PARTITION3, new OffsetAndMetadata(FIRST_OFFSET + 2)); - - EasyMock.expect(consumer.assignment()).andReturn(new HashSet<>(originalPartitions)).times(2); - - // iter 3 - note that we return the current offset to indicate they should be committed - sinkTask.preCommit(workerCurrentOffsets); - EasyMock.expectLastCall().andReturn(workerCurrentOffsets); - - // We need to delay the result of trying to commit offsets to Kafka via the consumer.commitAsync - // method. We do this so that we can test that the callback is not called until after the rebalance - // changes the lastCommittedOffsets. To fake this for tests we have the commitAsync build a function - // that will call the callback with the appropriate parameters, and we'll run that function later. - final AtomicReference asyncCallbackRunner = new AtomicReference<>(); - final AtomicBoolean asyncCallbackRan = new AtomicBoolean(); - - consumer.commitAsync(EasyMock.eq(workerCurrentOffsets), EasyMock.anyObject()); - EasyMock.expectLastCall().andAnswer(() -> { - // Grab the arguments passed to the consumer.commitAsync method - final Object[] args = EasyMock.getCurrentArguments(); - @SuppressWarnings("unchecked") - final Map offsets = (Map) args[0]; - final OffsetCommitCallback callback = (OffsetCommitCallback) args[1]; - asyncCallbackRunner.set(() -> { - callback.onComplete(offsets, null); - asyncCallbackRan.set(true); - }); - return null; - }); - - // Expect the next poll to discover and perform the rebalance, THEN complete the previous callback handler, - // and then return one record for TP1 and one for TP3. - final AtomicBoolean rebalanced = new AtomicBoolean(); - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - () -> { - // Rebalance always begins with revoking current partitions ... - rebalanceListener.getValue().onPartitionsRevoked(originalPartitions); - // Respond to the rebalance - Map offsets = new HashMap<>(); - offsets.put(TOPIC_PARTITION, rebalanceOffsets.get(TOPIC_PARTITION).offset()); - offsets.put(TOPIC_PARTITION2, rebalanceOffsets.get(TOPIC_PARTITION2).offset()); - offsets.put(TOPIC_PARTITION3, rebalanceOffsets.get(TOPIC_PARTITION3).offset()); - sinkTaskContext.getValue().offset(offsets); - rebalanceListener.getValue().onPartitionsAssigned(rebalancedPartitions); - rebalanced.set(true); - - // Run the previous async commit handler - asyncCallbackRunner.get().run(); - - // And prep the two records to return - long timestamp = RecordBatch.NO_TIMESTAMP; - TimestampType timestampType = TimestampType.NO_TIMESTAMP_TYPE; - List> records = new ArrayList<>(); - records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + 1, timestamp, timestampType, - 0, 0, RAW_KEY, RAW_VALUE, new RecordHeaders(), Optional.empty())); - records.add(new ConsumerRecord<>(TOPIC, PARTITION3, FIRST_OFFSET + recordsReturnedTp3 + 1, timestamp, timestampType, - 0, 0, RAW_KEY, RAW_VALUE, new RecordHeaders(), Optional.empty())); - recordsReturnedTp1 += 1; - recordsReturnedTp3 += 1; - return new ConsumerRecords<>(Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records)); - }); - - // onPartitionsRevoked - sinkTask.preCommit(workerCurrentOffsets); - EasyMock.expectLastCall().andReturn(workerCurrentOffsets); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); - sinkTask.close(new ArrayList<>(workerCurrentOffsets.keySet())); - EasyMock.expectLastCall(); - consumer.commitSync(workerCurrentOffsets); - EasyMock.expectLastCall(); - - // onPartitionsAssigned - step 1 - final long offsetTp1 = rebalanceOffsets.get(TOPIC_PARTITION).offset(); - final long offsetTp2 = rebalanceOffsets.get(TOPIC_PARTITION2).offset(); - final long offsetTp3 = rebalanceOffsets.get(TOPIC_PARTITION3).offset(); - EasyMock.expect(consumer.position(TOPIC_PARTITION)).andReturn(offsetTp1); - EasyMock.expect(consumer.position(TOPIC_PARTITION2)).andReturn(offsetTp2); - EasyMock.expect(consumer.position(TOPIC_PARTITION3)).andReturn(offsetTp3); - EasyMock.expect(consumer.assignment()).andReturn(new HashSet<>(rebalancedPartitions)).times(5); - - // onPartitionsAssigned - step 2 - sinkTask.open(EasyMock.eq(rebalancedPartitions)); - EasyMock.expectLastCall(); - - // onPartitionsAssigned - step 3 rewind - consumer.seek(TOPIC_PARTITION, offsetTp1); - EasyMock.expectLastCall(); - consumer.seek(TOPIC_PARTITION2, offsetTp2); - EasyMock.expectLastCall(); - consumer.seek(TOPIC_PARTITION3, offsetTp3); - EasyMock.expectLastCall(); - - // iter 4 - note that we return the current offset to indicate they should be committed - sinkTask.preCommit(postRebalanceCurrentOffsets); - EasyMock.expectLastCall().andReturn(postRebalanceCurrentOffsets); - - final Capture callback = EasyMock.newCapture(); - consumer.commitAsync(EasyMock.eq(postRebalanceCurrentOffsets), EasyMock.capture(callback)); - EasyMock.expectLastCall().andAnswer(() -> { - callback.getValue().onComplete(postRebalanceCurrentOffsets, null); - return null; - }); - - // no actual consumer.commit() triggered - expectConsumerPoll(1); - - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - workerTask.iteration(); // iter 1 -- initial assignment - - assertEquals(workerStartingOffsets, Whitebox.getInternalState(workerTask, "currentOffsets")); - assertEquals(workerStartingOffsets, Whitebox.getInternalState(workerTask, "lastCommittedOffsets")); - - time.sleep(WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_DEFAULT); - workerTask.iteration(); // iter 2 -- deliver 2 records - - sinkTaskContext.getValue().requestCommit(); - workerTask.iteration(); // iter 3 -- commit in progress - - assertSinkMetricValue("partition-count", 3); - assertSinkMetricValue("sink-record-read-total", 3.0); - assertSinkMetricValue("sink-record-send-total", 3.0); - assertSinkMetricValue("sink-record-active-count", 4.0); - assertSinkMetricValue("sink-record-active-count-max", 4.0); - assertSinkMetricValue("sink-record-active-count-avg", 0.71429); - assertSinkMetricValue("offset-commit-seq-no", 2.0); - assertSinkMetricValue("offset-commit-completion-total", 1.0); - assertSinkMetricValue("offset-commit-skip-total", 1.0); - assertTaskMetricValue("status", "running"); - assertTaskMetricValue("running-ratio", 1.0); - assertTaskMetricValue("pause-ratio", 0.0); - assertTaskMetricValue("batch-size-max", 2.0); - assertTaskMetricValue("batch-size-avg", 1.0); - assertTaskMetricValue("offset-commit-max-time-ms", 0.0); - assertTaskMetricValue("offset-commit-avg-time-ms", 0.0); - assertTaskMetricValue("offset-commit-failure-percentage", 0.0); - assertTaskMetricValue("offset-commit-success-percentage", 1.0); - - assertTrue(asyncCallbackRan.get()); - assertTrue(rebalanced.get()); - - // Check that the offsets were not reset by the out-of-order async commit callback - assertEquals(postRebalanceCurrentOffsets, Whitebox.getInternalState(workerTask, "currentOffsets")); - assertEquals(rebalanceOffsets, Whitebox.getInternalState(workerTask, "lastCommittedOffsets")); - - time.sleep(WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_DEFAULT); - sinkTaskContext.getValue().requestCommit(); - workerTask.iteration(); // iter 4 -- commit in progress - - // Check that the offsets were not reset by the out-of-order async commit callback - assertEquals(postRebalanceCurrentOffsets, Whitebox.getInternalState(workerTask, "currentOffsets")); - assertEquals(postRebalanceCurrentOffsets, Whitebox.getInternalState(workerTask, "lastCommittedOffsets")); - - assertSinkMetricValue("partition-count", 3); - assertSinkMetricValue("sink-record-read-total", 4.0); - assertSinkMetricValue("sink-record-send-total", 4.0); - assertSinkMetricValue("sink-record-active-count", 0.0); - assertSinkMetricValue("sink-record-active-count-max", 4.0); - assertSinkMetricValue("sink-record-active-count-avg", 0.5555555); - assertSinkMetricValue("offset-commit-seq-no", 3.0); - assertSinkMetricValue("offset-commit-completion-total", 2.0); - assertSinkMetricValue("offset-commit-skip-total", 1.0); - assertTaskMetricValue("status", "running"); - assertTaskMetricValue("running-ratio", 1.0); - assertTaskMetricValue("pause-ratio", 0.0); - assertTaskMetricValue("batch-size-max", 2.0); - assertTaskMetricValue("batch-size-avg", 1.0); - assertTaskMetricValue("offset-commit-max-time-ms", 0.0); - assertTaskMetricValue("offset-commit-avg-time-ms", 0.0); - assertTaskMetricValue("offset-commit-failure-percentage", 0.0); - assertTaskMetricValue("offset-commit-success-percentage", 1.0); - - PowerMock.verifyAll(); - } - - private void expectInitializeTask() { - consumer.subscribe(EasyMock.eq(asList(TOPIC)), EasyMock.capture(rebalanceListener)); - PowerMock.expectLastCall(); - - sinkTask.initialize(EasyMock.capture(sinkTaskContext)); - PowerMock.expectLastCall(); - sinkTask.start(TASK_PROPS); - PowerMock.expectLastCall(); - } - - private void expectPollInitialAssignment() { - sinkTask.open(INITIAL_ASSIGNMENT); - EasyMock.expectLastCall(); - - EasyMock.expect(consumer.assignment()).andReturn(INITIAL_ASSIGNMENT).times(2); - - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(() -> { - rebalanceListener.getValue().onPartitionsAssigned(INITIAL_ASSIGNMENT); - return ConsumerRecords.empty(); - }); - INITIAL_ASSIGNMENT.forEach(tp -> EasyMock.expect(consumer.position(tp)).andReturn(FIRST_OFFSET)); - - sinkTask.put(Collections.emptyList()); - EasyMock.expectLastCall(); - } - - private void expectConsumerPoll(final int numMessages) { - expectConsumerPoll(numMessages, RecordBatch.NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE, emptyHeaders()); - } - - private void expectConsumerPoll(final int numMessages, final long timestamp, final TimestampType timestampType, Headers headers) { - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - () -> { - List> records = new ArrayList<>(); - for (int i = 0; i < numMessages; i++) - records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + i, timestamp, timestampType, - 0, 0, RAW_KEY, RAW_VALUE, headers, Optional.empty())); - recordsReturnedTp1 += numMessages; - return new ConsumerRecords<>( - numMessages > 0 ? - Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records) : - Collections.emptyMap() - ); - }); - } - - private void expectConversionAndTransformation(final int numMessages) { - expectConversionAndTransformation(numMessages, null); - } - - private void expectConversionAndTransformation(final int numMessages, final String topicPrefix) { - expectConversionAndTransformation(numMessages, topicPrefix, emptyHeaders()); - } - - private void expectConversionAndTransformation(final int numMessages, final String topicPrefix, final Headers headers) { - EasyMock.expect(keyConverter.toConnectData(TOPIC, headers, RAW_KEY)).andReturn(new SchemaAndValue(KEY_SCHEMA, KEY)).times(numMessages); - EasyMock.expect(valueConverter.toConnectData(TOPIC, headers, RAW_VALUE)).andReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE)).times(numMessages); - - for (Header header : headers) { - EasyMock.expect(headerConverter.toConnectHeader(TOPIC, header.key(), header.value())).andReturn(new SchemaAndValue(VALUE_SCHEMA, new String(header.value()))).times(1); - } - - expectTransformation(numMessages, topicPrefix); - } - - private void expectTransformation(final int numMessages, final String topicPrefix) { - final Capture recordCapture = EasyMock.newCapture(); - EasyMock.expect(transformationChain.apply(EasyMock.anyObject(), EasyMock.capture(recordCapture))) - .andAnswer(() -> { - SinkRecord origRecord = recordCapture.getValue(); - return topicPrefix != null && !topicPrefix.isEmpty() - ? origRecord.newRecord( - topicPrefix + origRecord.topic(), - origRecord.kafkaPartition(), - origRecord.keySchema(), - origRecord.key(), - origRecord.valueSchema(), - origRecord.value(), - origRecord.timestamp(), - origRecord.headers() - ) - : origRecord; - }).times(numMessages); - } - - private void expectTaskGetTopic(boolean anyTimes) { - final Capture connectorCapture = EasyMock.newCapture(); - final Capture topicCapture = EasyMock.newCapture(); - IExpectationSetters expect = EasyMock.expect(statusBackingStore.getTopic( - EasyMock.capture(connectorCapture), - EasyMock.capture(topicCapture))); - if (anyTimes) { - expect.andStubAnswer(() -> new TopicStatus( - topicCapture.getValue(), - new ConnectorTaskId(connectorCapture.getValue(), 0), - Time.SYSTEM.milliseconds())); - } else { - expect.andAnswer(() -> new TopicStatus( - topicCapture.getValue(), - new ConnectorTaskId(connectorCapture.getValue(), 0), - Time.SYSTEM.milliseconds())); - } - if (connectorCapture.hasCaptured() && topicCapture.hasCaptured()) { - assertEquals("job", connectorCapture.getValue()); - assertEquals(TOPIC, topicCapture.getValue()); - } - } - - private void assertSinkMetricValue(String name, double expected) { - MetricGroup sinkTaskGroup = workerTask.sinkTaskMetricsGroup().metricGroup(); - double measured = metrics.currentMetricValueAsDouble(sinkTaskGroup, name); - assertEquals(expected, measured, 0.001d); - } - - private void assertTaskMetricValue(String name, double expected) { - MetricGroup taskGroup = workerTask.taskMetricsGroup().metricGroup(); - double measured = metrics.currentMetricValueAsDouble(taskGroup, name); - assertEquals(expected, measured, 0.001d); - } - - private void assertTaskMetricValue(String name, String expected) { - MetricGroup taskGroup = workerTask.taskMetricsGroup().metricGroup(); - String measured = metrics.currentMetricValueAsString(taskGroup, name); - assertEquals(expected, measured); - } - - private RecordHeaders emptyHeaders() { - return new RecordHeaders(); - } - - private abstract static class TestSinkTask extends SinkTask { - } -} From ccf4bd5f4621b80f5b1d3700df30e3b444381927 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Wed, 6 Mar 2024 12:02:58 -0800 Subject: [PATCH 035/521] MINOR: Add 3.7 to Kafka Streams system tests (#15443) Reviewers: Bruno Cadonna --- build.gradle | 15 +- settings.gradle | 1 + .../apache/kafka/streams/StreamsConfig.java | 8 +- .../streams/internals/UpgradeFromValues.java | 3 +- .../assignment/AssignorConfiguration.java | 2 + .../kafka/streams/tests/SmokeTestClient.java | 299 ++++++++ .../kafka/streams/tests/SmokeTestDriver.java | 670 ++++++++++++++++++ .../kafka/streams/tests/SmokeTestUtil.java | 131 ++++ .../kafka/streams/tests/StreamsSmokeTest.java | 100 +++ .../streams/tests/StreamsUpgradeTest.java | 120 ++++ .../streams_application_upgrade_test.py | 4 +- .../streams_broker_compatibility_test.py | 64 +- .../tests/streams/streams_upgrade_test.py | 6 +- 13 files changed, 1365 insertions(+), 58 deletions(-) create mode 100644 streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java create mode 100644 streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java create mode 100644 streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java create mode 100644 streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java create mode 100644 streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java diff --git a/build.gradle b/build.gradle index cba4381aae..786c731dcc 100644 --- a/build.gradle +++ b/build.gradle @@ -2303,6 +2303,7 @@ project(':streams') { ':streams:upgrade-system-tests-34:test', ':streams:upgrade-system-tests-35:test', ':streams:upgrade-system-tests-36:test', + ':streams:upgrade-system-tests-37:test', ':streams:examples:test' ] ) @@ -2721,7 +2722,6 @@ project(':streams:upgrade-system-tests-35') { } } - project(':streams:upgrade-system-tests-36') { archivesBaseName = "kafka-streams-upgrade-system-tests-36" @@ -2735,6 +2735,19 @@ project(':streams:upgrade-system-tests-36') { } } +project(':streams:upgrade-system-tests-37') { + archivesBaseName = "kafka-streams-upgrade-system-tests-37" + + dependencies { + testImplementation libs.kafkaStreams_37 + testRuntimeOnly libs.junitJupiter + } + + systemTestLibs { + dependsOn testJar + } +} + project(':jmh-benchmarks') { apply plugin: 'com.github.johnrengelman.shadow' diff --git a/settings.gradle b/settings.gradle index 4a4141f1b7..c2d66475ad 100644 --- a/settings.gradle +++ b/settings.gradle @@ -96,6 +96,7 @@ include 'clients', 'streams:upgrade-system-tests-34', 'streams:upgrade-system-tests-35', 'streams:upgrade-system-tests-36', + 'streams:upgrade-system-tests-37', 'tools', 'tools:tools-api', 'transaction-coordinator', diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 2f4841269b..54b30fea4d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -418,6 +418,12 @@ public class StreamsConfig extends AbstractConfig { @SuppressWarnings("WeakerAccess") public static final String UPGRADE_FROM_36 = UpgradeFromValues.UPGRADE_FROM_36.toString(); + /** + * Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 3.7.x}. + */ + @SuppressWarnings("WeakerAccess") + public static final String UPGRADE_FROM_37 = UpgradeFromValues.UPGRADE_FROM_37.toString(); + /** * Config value for parameter {@link #PROCESSING_GUARANTEE_CONFIG "processing.guarantee"} for at-least-once processing guarantees. */ @@ -774,7 +780,7 @@ public class StreamsConfig extends AbstractConfig { UPGRADE_FROM_25 + "\", \"" + UPGRADE_FROM_26 + "\", \"" + UPGRADE_FROM_27 + "\", \"" + UPGRADE_FROM_28 + "\", \"" + UPGRADE_FROM_30 + "\", \"" + UPGRADE_FROM_31 + "\", \"" + UPGRADE_FROM_32 + "\", \"" + UPGRADE_FROM_33 + "\", \"" + UPGRADE_FROM_34 + "\", \"" + - UPGRADE_FROM_35 + "\", \"" + UPGRADE_FROM_36 + "(for upgrading from the corresponding old version)."; + UPGRADE_FROM_35 + "\", \"" + UPGRADE_FROM_36 + "\", \"" + UPGRADE_FROM_37 + "(for upgrading from the corresponding old version)."; /** {@code windowstore.changelog.additional.retention.ms} */ @SuppressWarnings("WeakerAccess") diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/UpgradeFromValues.java b/streams/src/main/java/org/apache/kafka/streams/internals/UpgradeFromValues.java index abc2c7cb45..2bf19da39b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/internals/UpgradeFromValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/internals/UpgradeFromValues.java @@ -38,7 +38,8 @@ public enum UpgradeFromValues { UPGRADE_FROM_33("3.3"), UPGRADE_FROM_34("3.4"), UPGRADE_FROM_35("3.5"), - UPGRADE_FROM_36("3.6"); + UPGRADE_FROM_36("3.6"), + UPGRADE_FROM_37("3.7"); private final String value; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java index 039fd2b258..6d99d93536 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java @@ -127,6 +127,7 @@ public RebalanceProtocol rebalanceProtocol() { case UPGRADE_FROM_34: case UPGRADE_FROM_35: case UPGRADE_FROM_36: + case UPGRADE_FROM_37: // we need to add new version when new "upgrade.from" values become available // This config is for explicitly sending FK response to a requested partition @@ -187,6 +188,7 @@ public int configuredMetadataVersion(final int priorVersion) { case UPGRADE_FROM_34: case UPGRADE_FROM_35: case UPGRADE_FROM_36: + case UPGRADE_FROM_37: // we need to add new version when new "upgrade.from" values become available // This config is for explicitly sending FK response to a requested partition diff --git a/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java new file mode 100644 index 0000000000..dc0ad4d560 --- /dev/null +++ b/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java @@ -0,0 +1,299 @@ +/* + * 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.streams.tests; + +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.KafkaThread; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.Topology; +import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.Grouped; +import org.apache.kafka.streams.kstream.KGroupedStream; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.kstream.Suppressed.BufferConfig; +import org.apache.kafka.streams.kstream.TimeWindows; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.WindowStore; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.time.Duration; +import java.time.Instant; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; + +public class SmokeTestClient extends SmokeTestUtil { + + private final String name; + + private KafkaStreams streams; + private boolean uncaughtException = false; + private boolean started; + private volatile boolean closed; + + private static void addShutdownHook(final String name, final Runnable runnable) { + if (name != null) { + Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable)); + } else { + Runtime.getRuntime().addShutdownHook(new Thread(runnable)); + } + } + + private static File tempDirectory() { + final String prefix = "kafka-"; + final File file; + try { + file = Files.createTempDirectory(prefix).toFile(); + } catch (final IOException ex) { + throw new RuntimeException("Failed to create a temp dir", ex); + } + file.deleteOnExit(); + + addShutdownHook("delete-temp-file-shutdown-hook", () -> { + try { + Utils.delete(file); + } catch (final IOException e) { + System.out.println("Error deleting " + file.getAbsolutePath()); + e.printStackTrace(System.out); + } + }); + + return file; + } + + public SmokeTestClient(final String name) { + this.name = name; + } + + public boolean started() { + return started; + } + + public boolean closed() { + return closed; + } + + public void start(final Properties streamsProperties) { + final Topology build = getTopology(); + streams = new KafkaStreams(build, getStreamsConfig(streamsProperties)); + + final CountDownLatch countDownLatch = new CountDownLatch(1); + streams.setStateListener((newState, oldState) -> { + System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); + if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { + started = true; + countDownLatch.countDown(); + } + + if (newState == KafkaStreams.State.NOT_RUNNING) { + closed = true; + } + }); + + streams.setUncaughtExceptionHandler(e -> { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); + System.out.println(name + ": FATAL: An unexpected exception is encountered: " + e); + e.printStackTrace(System.out); + uncaughtException = true; + return StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT; + }); + + addShutdownHook("streams-shutdown-hook", this::close); + + streams.start(); + try { + if (!countDownLatch.await(1, TimeUnit.MINUTES)) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't start in one minute"); + } + } catch (final InterruptedException e) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: " + e); + e.printStackTrace(System.out); + } + System.out.println(name + ": SMOKE-TEST-CLIENT-STARTED"); + System.out.println(name + " started at " + Instant.now()); + } + + public void closeAsync() { + streams.close(Duration.ZERO); + } + + public void close() { + final boolean closed = streams.close(Duration.ofMinutes(1)); + + if (closed && !uncaughtException) { + System.out.println(name + ": SMOKE-TEST-CLIENT-CLOSED"); + } else if (closed) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); + } else { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't close"); + } + } + + private Properties getStreamsConfig(final Properties props) { + final Properties fullProps = new Properties(props); + fullProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest"); + fullProps.put(StreamsConfig.CLIENT_ID_CONFIG, "SmokeTest-" + name); + fullProps.put(StreamsConfig.STATE_DIR_CONFIG, tempDirectory().getAbsolutePath()); + fullProps.putAll(props); + return fullProps; + } + + public Topology getTopology() { + final StreamsBuilder builder = new StreamsBuilder(); + final Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde); + final KStream source = builder.stream("data", stringIntConsumed); + source.filterNot((k, v) -> k.equals("flush")) + .to("echo", Produced.with(stringSerde, intSerde)); + final KStream data = source.filter((key, value) -> value == null || value != END); + data.process(SmokeTestUtil.printProcessorSupplier("data", name)); + + // min + final KGroupedStream groupedData = data.groupByKey(Grouped.with(stringSerde, intSerde)); + + final KTable, Integer> minAggregation = groupedData + .windowedBy(TimeWindows.ofSizeAndGrace(Duration.ofDays(1), Duration.ofMinutes(1))) + .aggregate( + () -> Integer.MAX_VALUE, + (aggKey, value, aggregate) -> (value < aggregate) ? value : aggregate, + Materialized + .>as("uwin-min") + .withValueSerde(intSerde) + .withRetention(Duration.ofHours(25)) + ); + + streamify(minAggregation, "min-raw"); + + streamify(minAggregation.suppress(untilWindowCloses(BufferConfig.unbounded())), "min-suppressed"); + + minAggregation + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("min", Produced.with(stringSerde, intSerde)); + + final KTable, Integer> smallWindowSum = groupedData + .windowedBy(TimeWindows.ofSizeAndGrace(Duration.ofSeconds(2), Duration.ofSeconds(30)).advanceBy(Duration.ofSeconds(1))) + .reduce(Integer::sum); + + streamify(smallWindowSum, "sws-raw"); + streamify(smallWindowSum.suppress(untilWindowCloses(BufferConfig.unbounded())), "sws-suppressed"); + + final KTable minTable = builder.table( + "min", + Consumed.with(stringSerde, intSerde), + Materialized.as("minStoreName")); + + minTable.toStream().process(SmokeTestUtil.printProcessorSupplier("min", name)); + + // max + groupedData + .windowedBy(TimeWindows.ofSizeWithNoGrace(Duration.ofDays(2))) + .aggregate( + () -> Integer.MIN_VALUE, + (aggKey, value, aggregate) -> (value > aggregate) ? value : aggregate, + Materialized.>as("uwin-max").withValueSerde(intSerde)) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("max", Produced.with(stringSerde, intSerde)); + + final KTable maxTable = builder.table( + "max", + Consumed.with(stringSerde, intSerde), + Materialized.as("maxStoreName")); + maxTable.toStream().process(SmokeTestUtil.printProcessorSupplier("max", name)); + + // sum + groupedData + .windowedBy(TimeWindows.ofSizeWithNoGrace(Duration.ofDays(2))) + .aggregate( + () -> 0L, + (aggKey, value, aggregate) -> (long) value + aggregate, + Materialized.>as("win-sum").withValueSerde(longSerde)) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("sum", Produced.with(stringSerde, longSerde)); + + final Consumed stringLongConsumed = Consumed.with(stringSerde, longSerde); + final KTable sumTable = builder.table("sum", stringLongConsumed); + sumTable.toStream().process(SmokeTestUtil.printProcessorSupplier("sum", name)); + + // cnt + groupedData + .windowedBy(TimeWindows.ofSizeWithNoGrace(Duration.ofDays(2))) + .count(Materialized.as("uwin-cnt")) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("cnt", Produced.with(stringSerde, longSerde)); + + final KTable cntTable = builder.table( + "cnt", + Consumed.with(stringSerde, longSerde), + Materialized.as("cntStoreName")); + cntTable.toStream().process(SmokeTestUtil.printProcessorSupplier("cnt", name)); + + // dif + maxTable + .join( + minTable, + (value1, value2) -> value1 - value2) + .toStream() + .filterNot((k, v) -> k.equals("flush")) + .to("dif", Produced.with(stringSerde, intSerde)); + + // avg + sumTable + .join( + cntTable, + (value1, value2) -> (double) value1 / (double) value2) + .toStream() + .filterNot((k, v) -> k.equals("flush")) + .to("avg", Produced.with(stringSerde, doubleSerde)); + + // test repartition + final Agg agg = new Agg(); + cntTable.groupBy(agg.selector(), Grouped.with(stringSerde, longSerde)) + .aggregate(agg.init(), agg.adder(), agg.remover(), + Materialized.as(Stores.inMemoryKeyValueStore("cntByCnt")) + .withKeySerde(Serdes.String()) + .withValueSerde(Serdes.Long())) + .toStream() + .to("tagg", Produced.with(stringSerde, longSerde)); + + return builder.build(); + } + + private static void streamify(final KTable, Integer> windowedTable, final String topic) { + windowedTable + .toStream() + .filterNot((k, v) -> k.key().equals("flush")) + .map((key, value) -> new KeyValue<>(key.toString(), value)) + .to(topic, Produced.with(stringSerde, intSerde)); + } +} diff --git a/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java new file mode 100644 index 0000000000..dbacbb9625 --- /dev/null +++ b/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java @@ -0,0 +1,670 @@ +/* + * 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.streams.tests; + +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.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Utils; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static java.util.Collections.emptyMap; +import static org.apache.kafka.common.utils.Utils.mkEntry; + +public class SmokeTestDriver extends SmokeTestUtil { + private static final String[] NUMERIC_VALUE_TOPICS = { + "data", + "echo", + "max", + "min", "min-suppressed", "min-raw", + "dif", + "sum", + "sws-raw", "sws-suppressed", + "cnt", + "avg", + "tagg" + }; + private static final String[] STRING_VALUE_TOPICS = { + "fk" + }; + + private static final String[] TOPICS = new String[NUMERIC_VALUE_TOPICS.length + STRING_VALUE_TOPICS.length]; + static { + System.arraycopy(NUMERIC_VALUE_TOPICS, 0, TOPICS, 0, NUMERIC_VALUE_TOPICS.length); + System.arraycopy(STRING_VALUE_TOPICS, 0, TOPICS, NUMERIC_VALUE_TOPICS.length, STRING_VALUE_TOPICS.length); + } + + private static final int MAX_RECORD_EMPTY_RETRIES = 30; + + private static class ValueList { + public final String key; + private final int[] values; + private int index; + + ValueList(final int min, final int max) { + key = min + "-" + max; + + values = new int[max - min + 1]; + for (int i = 0; i < values.length; i++) { + values[i] = min + i; + } + // We want to randomize the order of data to test not completely predictable processing order + // However, values are also use as a timestamp of the record. (TODO: separate data and timestamp) + // We keep some correlation of time and order. Thus, the shuffling is done with a sliding window + shuffle(values, 10); + + index = 0; + } + + int next() { + return (index < values.length) ? values[index++] : -1; + } + } + + public static String[] topics() { + return Arrays.copyOf(TOPICS, TOPICS.length); + } + + static void generatePerpetually(final String kafka, + final int numKeys, + final int maxRecordsPerKey) { + final Properties producerProps = generatorProperties(kafka); + + int numRecordsProduced = 0; + + final ValueList[] data = new ValueList[numKeys]; + for (int i = 0; i < numKeys; i++) { + data[i] = new ValueList(i, i + maxRecordsPerKey - 1); + } + + final Random rand = new Random(); + + try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { + while (true) { + final int index = rand.nextInt(numKeys); + final String key = data[index].key; + final int value = data[index].next(); + + final ProducerRecord record = + new ProducerRecord<>( + "data", + stringSerde.serializer().serialize("", key), + intSerde.serializer().serialize("", value) + ); + producer.send(record); + + final ProducerRecord fkRecord = + new ProducerRecord<>( + "fk", + intSerde.serializer().serialize("", value), + stringSerde.serializer().serialize("", key) + ); + producer.send(fkRecord); + + numRecordsProduced++; + if (numRecordsProduced % 100 == 0) { + System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); + } + Utils.sleep(2); + } + } + } + + public static Map> generate(final String kafka, + final int numKeys, + final int maxRecordsPerKey, + final Duration timeToSpend) { + final Properties producerProps = generatorProperties(kafka); + + int numRecordsProduced = 0; + + final Map> allData = new HashMap<>(); + final ValueList[] data = new ValueList[numKeys]; + for (int i = 0; i < numKeys; i++) { + data[i] = new ValueList(i, i + maxRecordsPerKey - 1); + allData.put(data[i].key, new HashSet<>()); + } + final Random rand = new Random(); + + int remaining = data.length; + + final long recordPauseTime = timeToSpend.toMillis() / numKeys / maxRecordsPerKey; + + final List> dataNeedRetry = new ArrayList<>(); + final List> fkNeedRetry = new ArrayList<>(); + + try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { + while (remaining > 0) { + final int index = rand.nextInt(remaining); + final String key = data[index].key; + final int value = data[index].next(); + + if (value < 0) { + remaining--; + data[index] = data[remaining]; + } else { + final ProducerRecord record = + new ProducerRecord<>( + "data", + stringSerde.serializer().serialize("", key), + intSerde.serializer().serialize("", value) + ); + + producer.send(record, new TestCallback(record, dataNeedRetry)); + + final ProducerRecord fkRecord = + new ProducerRecord<>( + "fk", + intSerde.serializer().serialize("", value), + stringSerde.serializer().serialize("", key) + ); + + producer.send(fkRecord, new TestCallback(fkRecord, fkNeedRetry)); + + numRecordsProduced++; + allData.get(key).add(value); + if (numRecordsProduced % 100 == 0) { + System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); + } + Utils.sleep(Math.max(recordPauseTime, 2)); + } + } + producer.flush(); + + retry(producer, dataNeedRetry, stringSerde); + retry(producer, fkNeedRetry, intSerde); + + flush(producer, + "data", + stringSerde.serializer().serialize("", "flush"), + intSerde.serializer().serialize("", 0) + ); + flush(producer, + "fk", + intSerde.serializer().serialize("", 0), + stringSerde.serializer().serialize("", "flush") + ); + } + return Collections.unmodifiableMap(allData); + } + + private static void retry(final KafkaProducer producer, + List> needRetry, + final Serde keySerde) { + int remainingRetries = 5; + while (!needRetry.isEmpty()) { + final List> needRetry2 = new ArrayList<>(); + for (final ProducerRecord record : needRetry) { + System.out.println( + "retry producing " + keySerde.deserializer().deserialize("", record.key())); + producer.send(record, new TestCallback(record, needRetry2)); + } + producer.flush(); + needRetry = needRetry2; + if (--remainingRetries == 0 && !needRetry.isEmpty()) { + System.err.println("Failed to produce all records after multiple retries"); + Exit.exit(1); + } + } + } + + private static void flush(final KafkaProducer producer, + final String topic, + final byte[] keyBytes, + final byte[] valBytes) { + // now that we've sent everything, we'll send some final records with a timestamp high enough to flush out + // all suppressed records. + final List partitions = producer.partitionsFor(topic); + for (final PartitionInfo partition : partitions) { + producer.send(new ProducerRecord<>( + partition.topic(), + partition.partition(), + System.currentTimeMillis() + Duration.ofDays(2).toMillis(), + keyBytes, + valBytes + )); + } + } + + private static Properties generatorProperties(final String kafka) { + final Properties producerProps = new Properties(); + producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, "SmokeTest"); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + producerProps.put(ProducerConfig.ACKS_CONFIG, "all"); + return producerProps; + } + + private static class TestCallback implements Callback { + private final ProducerRecord originalRecord; + private final List> needRetry; + + TestCallback(final ProducerRecord originalRecord, + final List> needRetry) { + this.originalRecord = originalRecord; + this.needRetry = needRetry; + } + + @Override + public void onCompletion(final RecordMetadata metadata, final Exception exception) { + if (exception != null) { + if (exception instanceof TimeoutException) { + needRetry.add(originalRecord); + } else { + exception.printStackTrace(); + Exit.exit(1); + } + } + } + } + + private static void shuffle(final int[] data, @SuppressWarnings("SameParameterValue") final int windowSize) { + final Random rand = new Random(); + for (int i = 0; i < data.length; i++) { + // we shuffle data within windowSize + final int j = rand.nextInt(Math.min(data.length - i, windowSize)) + i; + + // swap + final int tmp = data[i]; + data[i] = data[j]; + data[j] = tmp; + } + } + + public static class NumberDeserializer implements Deserializer { + @Override + public Number deserialize(final String topic, final byte[] data) { + final Number value; + switch (topic) { + case "data": + case "echo": + case "min": + case "min-raw": + case "min-suppressed": + case "sws-raw": + case "sws-suppressed": + case "max": + case "dif": + value = intSerde.deserializer().deserialize(topic, data); + break; + case "sum": + case "cnt": + case "tagg": + value = longSerde.deserializer().deserialize(topic, data); + break; + case "avg": + value = doubleSerde.deserializer().deserialize(topic, data); + break; + default: + throw new RuntimeException("unknown topic: " + topic); + } + return value; + } + } + + public static VerificationResult verify(final String kafka, + final Map> inputs, + final int maxRecordsPerKey) { + final Properties props = new Properties(); + props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier"); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, NumberDeserializer.class); + props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); + + final KafkaConsumer consumer = new KafkaConsumer<>(props); + final List partitions = getAllPartitions(consumer, NUMERIC_VALUE_TOPICS); + consumer.assign(partitions); + consumer.seekToBeginning(partitions); + + final int recordsGenerated = inputs.size() * maxRecordsPerKey; + int recordsProcessed = 0; + final Map processed = + Stream.of(NUMERIC_VALUE_TOPICS) + .collect(Collectors.toMap(t -> t, t -> new AtomicInteger(0))); + + final Map>>> events = new HashMap<>(); + + VerificationResult verificationResult = new VerificationResult(false, "no results yet"); + int retry = 0; + final long start = System.currentTimeMillis(); + while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(6)) { + final ConsumerRecords records = consumer.poll(Duration.ofSeconds(5)); + if (records.isEmpty() && recordsProcessed >= recordsGenerated) { + verificationResult = verifyAll(inputs, events, false); + if (verificationResult.passed()) { + break; + } else if (retry++ > MAX_RECORD_EMPTY_RETRIES) { + System.out.println(Instant.now() + " Didn't get any more results, verification hasn't passed, and out of retries."); + break; + } else { + System.out.println(Instant.now() + " Didn't get any more results, but verification hasn't passed (yet). Retrying..." + retry); + } + } else { + System.out.println(Instant.now() + " Get some more results from " + records.partitions() + ", resetting retry."); + + retry = 0; + for (final ConsumerRecord record : records) { + final String key = record.key(); + + final String topic = record.topic(); + processed.get(topic).incrementAndGet(); + + if (topic.equals("echo")) { + recordsProcessed++; + if (recordsProcessed % 100 == 0) { + System.out.println("Echo records processed = " + recordsProcessed); + } + } + + events.computeIfAbsent(topic, t -> new HashMap<>()) + .computeIfAbsent(key, k -> new LinkedList<>()) + .add(record); + } + + System.out.println(processed); + } + } + consumer.close(); + final long finished = System.currentTimeMillis() - start; + System.out.println("Verification time=" + finished); + System.out.println("-------------------"); + System.out.println("Result Verification"); + System.out.println("-------------------"); + System.out.println("recordGenerated=" + recordsGenerated); + System.out.println("recordProcessed=" + recordsProcessed); + + if (recordsProcessed > recordsGenerated) { + System.out.println("PROCESSED-MORE-THAN-GENERATED"); + } else if (recordsProcessed < recordsGenerated) { + System.out.println("PROCESSED-LESS-THAN-GENERATED"); + } + + boolean success; + + final Map> received = + events.get("echo") + .entrySet() + .stream() + .map(entry -> mkEntry( + entry.getKey(), + entry.getValue().stream().map(ConsumerRecord::value).collect(Collectors.toSet())) + ) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + success = inputs.equals(received); + + if (success) { + System.out.println("ALL-RECORDS-DELIVERED"); + } else { + int missedCount = 0; + for (final Map.Entry> entry : inputs.entrySet()) { + missedCount += received.get(entry.getKey()).size(); + } + System.out.println("missedRecords=" + missedCount); + } + + // give it one more try if it's not already passing. + if (!verificationResult.passed()) { + verificationResult = verifyAll(inputs, events, true); + } + success &= verificationResult.passed(); + + System.out.println(verificationResult.result()); + + System.out.println(success ? "SUCCESS" : "FAILURE"); + return verificationResult; + } + + public static class VerificationResult { + private final boolean passed; + private final String result; + + VerificationResult(final boolean passed, final String result) { + this.passed = passed; + this.result = result; + } + + public boolean passed() { + return passed; + } + + public String result() { + return result; + } + } + + private static VerificationResult verifyAll(final Map> inputs, + final Map>>> events, + final boolean printResults) { + final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + boolean pass; + try (final PrintStream resultStream = new PrintStream(byteArrayOutputStream)) { + pass = verifyTAgg(resultStream, inputs, events.get("tagg"), printResults); + pass &= verifySuppressed(resultStream, "min-suppressed", events, printResults); + pass &= verify(resultStream, "min-suppressed", inputs, events, windowedKey -> { + final String unwindowedKey = windowedKey.substring(1, windowedKey.length() - 1).replaceAll("@.*", ""); + return getMin(unwindowedKey); + }, printResults); + pass &= verifySuppressed(resultStream, "sws-suppressed", events, printResults); + pass &= verify(resultStream, "min", inputs, events, SmokeTestDriver::getMin, printResults); + pass &= verify(resultStream, "max", inputs, events, SmokeTestDriver::getMax, printResults); + pass &= verify(resultStream, "dif", inputs, events, key -> getMax(key).intValue() - getMin(key).intValue(), printResults); + pass &= verify(resultStream, "sum", inputs, events, SmokeTestDriver::getSum, printResults); + pass &= verify(resultStream, "cnt", inputs, events, key1 -> getMax(key1).intValue() - getMin(key1).intValue() + 1L, printResults); + pass &= verify(resultStream, "avg", inputs, events, SmokeTestDriver::getAvg, printResults); + } + return new VerificationResult(pass, new String(byteArrayOutputStream.toByteArray(), StandardCharsets.UTF_8)); + } + + private static boolean verify(final PrintStream resultStream, + final String topic, + final Map> inputData, + final Map>>> events, + final Function keyToExpectation, + final boolean printResults) { + final Map>> observedInputEvents = events.get("data"); + final Map>> outputEvents = events.getOrDefault(topic, emptyMap()); + if (outputEvents.isEmpty()) { + resultStream.println(topic + " is empty"); + return false; + } else { + resultStream.printf("verifying %s with %d keys%n", topic, outputEvents.size()); + + if (outputEvents.size() != inputData.size()) { + resultStream.printf("fail: resultCount=%d expectedCount=%s%n\tresult=%s%n\texpected=%s%n", + outputEvents.size(), inputData.size(), outputEvents.keySet(), inputData.keySet()); + return false; + } + for (final Map.Entry>> entry : outputEvents.entrySet()) { + final String key = entry.getKey(); + final Number expected = keyToExpectation.apply(key); + final Number actual = entry.getValue().getLast().value(); + if (!expected.equals(actual)) { + resultStream.printf("%s fail: key=%s actual=%s expected=%s%n", topic, key, actual, expected); + + if (printResults) { + resultStream.printf("\t inputEvents=%n%s%n\t" + + "echoEvents=%n%s%n\tmaxEvents=%n%s%n\tminEvents=%n%s%n\tdifEvents=%n%s%n\tcntEvents=%n%s%n\ttaggEvents=%n%s%n", + indent("\t\t", observedInputEvents.get(key)), + indent("\t\t", events.getOrDefault("echo", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("max", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("min", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("dif", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("cnt", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("tagg", emptyMap()).getOrDefault(key, new LinkedList<>()))); + + if (!Utils.mkSet("echo", "max", "min", "dif", "cnt", "tagg").contains(topic)) + resultStream.printf("%sEvents=%n%s%n", topic, indent("\t\t", entry.getValue())); + } + + return false; + } + } + return true; + } + } + + + private static boolean verifySuppressed(final PrintStream resultStream, + @SuppressWarnings("SameParameterValue") final String topic, + final Map>>> events, + final boolean printResults) { + resultStream.println("verifying suppressed " + topic); + final Map>> topicEvents = events.getOrDefault(topic, emptyMap()); + for (final Map.Entry>> entry : topicEvents.entrySet()) { + if (entry.getValue().size() != 1) { + final String unsuppressedTopic = topic.replace("-suppressed", "-raw"); + final String key = entry.getKey(); + final String unwindowedKey = key.substring(1, key.length() - 1).replaceAll("@.*", ""); + resultStream.printf("fail: key=%s%n\tnon-unique result:%n%s%n", + key, + indent("\t\t", entry.getValue())); + + if (printResults) + resultStream.printf("\tresultEvents:%n%s%n\tinputEvents:%n%s%n", + indent("\t\t", events.get(unsuppressedTopic).get(key)), + indent("\t\t", events.get("data").get(unwindowedKey))); + + return false; + } + } + return true; + } + + private static String indent(@SuppressWarnings("SameParameterValue") final String prefix, + final Iterable> list) { + final StringBuilder stringBuilder = new StringBuilder(); + for (final ConsumerRecord record : list) { + stringBuilder.append(prefix).append(record).append('\n'); + } + return stringBuilder.toString(); + } + + private static Long getSum(final String key) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + return ((long) min + max) * (max - min + 1L) / 2L; + } + + private static Double getAvg(final String key) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + return ((long) min + max) / 2.0; + } + + + private static boolean verifyTAgg(final PrintStream resultStream, + final Map> allData, + final Map>> taggEvents, + final boolean printResults) { + if (taggEvents == null) { + resultStream.println("tagg is missing"); + return false; + } else if (taggEvents.isEmpty()) { + resultStream.println("tagg is empty"); + return false; + } else { + resultStream.println("verifying tagg"); + + // generate expected answer + final Map expected = new HashMap<>(); + for (final String key : allData.keySet()) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + final String cnt = Long.toString(max - min + 1L); + + expected.put(cnt, expected.getOrDefault(cnt, 0L) + 1); + } + + // check the result + for (final Map.Entry>> entry : taggEvents.entrySet()) { + final String key = entry.getKey(); + Long expectedCount = expected.remove(key); + if (expectedCount == null) { + expectedCount = 0L; + } + + if (entry.getValue().getLast().value().longValue() != expectedCount) { + resultStream.println("fail: key=" + key + " tagg=" + entry.getValue() + " expected=" + expectedCount); + + if (printResults) + resultStream.println("\t taggEvents: " + entry.getValue()); + return false; + } + } + + } + return true; + } + + private static Number getMin(final String key) { + return Integer.parseInt(key.split("-")[0]); + } + + private static Number getMax(final String key) { + return Integer.parseInt(key.split("-")[1]); + } + + private static List getAllPartitions(final KafkaConsumer consumer, final String... topics) { + final List partitions = new ArrayList<>(); + + for (final String topic : topics) { + for (final PartitionInfo info : consumer.partitionsFor(topic)) { + partitions.add(new TopicPartition(info.topic(), info.partition())); + } + } + return partitions; + } + +} diff --git a/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java new file mode 100644 index 0000000000..7cb34056e0 --- /dev/null +++ b/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java @@ -0,0 +1,131 @@ +/* + * 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.streams.tests; + +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.Aggregator; +import org.apache.kafka.streams.kstream.Initializer; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; + +import java.time.Instant; + +public class SmokeTestUtil { + + final static int END = Integer.MAX_VALUE; + + static ProcessorSupplier printProcessorSupplier(final String topic) { + return printProcessorSupplier(topic, ""); + } + + static ProcessorSupplier printProcessorSupplier(final String topic, final String name) { + return () -> new ContextualProcessor() { + private int numRecordsProcessed = 0; + private long smallestOffset = Long.MAX_VALUE; + private long largestOffset = Long.MIN_VALUE; + + @Override + public void init(final ProcessorContext context) { + super.init(context); + System.out.println("[3.7] initializing processor: topic=" + topic + " taskId=" + context.taskId()); + System.out.flush(); + numRecordsProcessed = 0; + smallestOffset = Long.MAX_VALUE; + largestOffset = Long.MIN_VALUE; + } + + @Override + public void process(final Record record) { + numRecordsProcessed++; + if (numRecordsProcessed % 100 == 0) { + System.out.printf("%s: %s%n", name, Instant.now()); + System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); + } + + context().recordMetadata().ifPresent(recordMetadata -> { + if (smallestOffset > recordMetadata.offset()) { + smallestOffset = recordMetadata.offset(); + } + if (largestOffset < recordMetadata.offset()) { + largestOffset = recordMetadata.offset(); + } + }); + } + + @Override + public void close() { + System.out.printf("Close processor for task %s%n", context().taskId()); + System.out.println("processed " + numRecordsProcessed + " records"); + final long processed; + if (largestOffset >= smallestOffset) { + processed = 1L + largestOffset - smallestOffset; + } else { + processed = 0L; + } + System.out.println("offset " + smallestOffset + " to " + largestOffset + " -> processed " + processed); + System.out.flush(); + } + }; + } + + public static final class Unwindow implements KeyValueMapper, V, K> { + @Override + public K apply(final Windowed winKey, final V value) { + return winKey.key(); + } + } + + public static class Agg { + + KeyValueMapper> selector() { + return (key, value) -> new KeyValue<>(value == null ? null : Long.toString(value), 1L); + } + + public Initializer init() { + return () -> 0L; + } + + Aggregator adder() { + return (aggKey, value, aggregate) -> aggregate + value; + } + + Aggregator remover() { + return (aggKey, value, aggregate) -> aggregate - value; + } + } + + public static Serde stringSerde = Serdes.String(); + + public static Serde intSerde = Serdes.Integer(); + + static Serde longSerde = Serdes.Long(); + + static Serde doubleSerde = Serdes.Double(); + + public static void sleep(final long duration) { + try { + Thread.sleep(duration); + } catch (final Exception ignore) { } + } + +} diff --git a/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java b/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java new file mode 100644 index 0000000000..5803b2fbd0 --- /dev/null +++ b/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java @@ -0,0 +1,100 @@ +/* + * 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.streams.tests; + +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.StreamsConfig; + +import java.io.IOException; +import java.time.Duration; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.UUID; + +import static org.apache.kafka.streams.tests.SmokeTestDriver.generate; +import static org.apache.kafka.streams.tests.SmokeTestDriver.generatePerpetually; + +public class StreamsSmokeTest { + + /** + * args ::= kafka propFileName command disableAutoTerminate + * command := "run" | "process" + * + * @param args + */ + public static void main(final String[] args) throws IOException { + if (args.length < 2) { + System.err.println("StreamsSmokeTest are expecting two parameters: propFile, command; but only see " + args.length + " parameter"); + Exit.exit(1); + } + + final String propFileName = args[0]; + final String command = args[1]; + final boolean disableAutoTerminate = args.length > 2; + + final Properties streamsProperties = Utils.loadProps(propFileName); + final String kafka = streamsProperties.getProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); + final String processingGuarantee = streamsProperties.getProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG); + + if (kafka == null) { + System.err.println("No bootstrap kafka servers specified in " + StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); + Exit.exit(1); + } + + if ("process".equals(command)) { + if (!StreamsConfig.AT_LEAST_ONCE.equals(processingGuarantee) && + !StreamsConfig.EXACTLY_ONCE_V2.equals(processingGuarantee)) { + + System.err.println("processingGuarantee must be either " + StreamsConfig.AT_LEAST_ONCE + " or " + + StreamsConfig.EXACTLY_ONCE_V2); + + Exit.exit(1); + } + } + + System.out.println("StreamsTest instance started (StreamsSmokeTest)"); + System.out.println("command=" + command); + System.out.println("props=" + streamsProperties); + System.out.println("disableAutoTerminate=" + disableAutoTerminate); + + switch (command) { + case "run": + // this starts the driver (data generation and result verification) + final int numKeys = 10; + final int maxRecordsPerKey = 500; + if (disableAutoTerminate) { + generatePerpetually(kafka, numKeys, maxRecordsPerKey); + } else { + // slow down data production to span 30 seconds so that system tests have time to + // do their bounces, etc. + final Map> allData = + generate(kafka, numKeys, maxRecordsPerKey, Duration.ofSeconds(30)); + SmokeTestDriver.verify(kafka, allData, maxRecordsPerKey); + } + break; + case "process": + // this starts the stream processing app + new SmokeTestClient(UUID.randomUUID().toString()).start(streamsProperties); + break; + default: + System.out.println("unknown command: " + command); + } + } + +} diff --git a/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java b/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java new file mode 100644 index 0000000000..15769bf16c --- /dev/null +++ b/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java @@ -0,0 +1,120 @@ +/* + * 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.streams.tests; + +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; + +import java.util.Properties; + +import static org.apache.kafka.streams.tests.SmokeTestUtil.intSerde; +import static org.apache.kafka.streams.tests.SmokeTestUtil.stringSerde; + + +public class StreamsUpgradeTest { + + @SuppressWarnings("unchecked") + public static void main(final String[] args) throws Exception { + if (args.length < 1) { + System.err.println("StreamsUpgradeTest requires one argument (properties-file) but provided none"); + } + final String propFileName = args[0]; + + final Properties streamsProperties = Utils.loadProps(propFileName); + + System.out.println("StreamsTest instance started (StreamsUpgradeTest v3.7)"); + System.out.println("props=" + streamsProperties); + + final StreamsBuilder builder = new StreamsBuilder(); + final KTable dataTable = builder.table( + "data", Consumed.with(stringSerde, intSerde)); + final KStream dataStream = dataTable.toStream(); + dataStream.process(printProcessorSupplier("data")); + dataStream.to("echo"); + + final boolean runFkJoin = Boolean.parseBoolean(streamsProperties.getProperty( + "test.run_fk_join", + "false")); + if (runFkJoin) { + try { + final KTable fkTable = builder.table( + "fk", Consumed.with(intSerde, stringSerde)); + buildFKTable(dataStream, fkTable); + } catch (final Exception e) { + System.err.println("Caught " + e.getMessage()); + } + } + + final Properties config = new Properties(); + config.setProperty( + StreamsConfig.APPLICATION_ID_CONFIG, + "StreamsUpgradeTest"); + config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); + config.putAll(streamsProperties); + + final KafkaStreams streams = new KafkaStreams(builder.build(), config); + streams.start(); + + Runtime.getRuntime().addShutdownHook(new Thread(() -> { + streams.close(); + System.out.println("UPGRADE-TEST-CLIENT-CLOSED"); + System.out.flush(); + })); + } + + private static void buildFKTable(final KStream primaryTable, + final KTable otherTable) { + final KStream kStream = primaryTable.toTable() + .join(otherTable, v -> v, (k0, v0) -> v0) + .toStream(); + kStream.process(printProcessorSupplier("fk")); + kStream.to("fk-result", Produced.with(stringSerde, stringSerde)); + } + + private static ProcessorSupplier printProcessorSupplier(final String topic) { + return () -> new ContextualProcessor() { + private int numRecordsProcessed = 0; + + @Override + public void init(final ProcessorContext context) { + System.out.println("[3.7] initializing processor: topic=" + topic + "taskId=" + context.taskId()); + numRecordsProcessed = 0; + } + + @Override + public void process(final Record record) { + numRecordsProcessed++; + if (numRecordsProcessed % 100 == 0) { + System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); + } + } + + @Override + public void close() {} + }; + } +} diff --git a/tests/kafkatest/tests/streams/streams_application_upgrade_test.py b/tests/kafkatest/tests/streams/streams_application_upgrade_test.py index 057f301df6..6719a59178 100644 --- a/tests/kafkatest/tests/streams/streams_application_upgrade_test.py +++ b/tests/kafkatest/tests/streams/streams_application_upgrade_test.py @@ -22,13 +22,13 @@ from kafkatest.services.streams import StreamsSmokeTestDriverService, StreamsSmokeTestJobRunnerService from kafkatest.services.zookeeper import ZookeeperService from kafkatest.version import LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, LATEST_2_8, \ - LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, DEV_VERSION, KafkaVersion + LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, DEV_VERSION, KafkaVersion smoke_test_versions = [str(LATEST_2_2), str(LATEST_2_3), str(LATEST_2_4), str(LATEST_2_5), str(LATEST_2_6), str(LATEST_2_7), str(LATEST_2_8), str(LATEST_3_0), str(LATEST_3_1), str(LATEST_3_2), str(LATEST_3_3), str(LATEST_3_4), - str(LATEST_3_5), str(LATEST_3_6)] + str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7)] dev_version = [str(DEV_VERSION)] class StreamsUpgradeTest(Test): diff --git a/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py b/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py index 82647687f8..1262c6d9f4 100644 --- a/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py +++ b/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py @@ -23,7 +23,7 @@ from kafkatest.services.zookeeper import ZookeeperService from kafkatest.version import LATEST_0_11_0, LATEST_0_10_2, LATEST_0_10_1, LATEST_0_10_0, LATEST_1_0, LATEST_1_1, \ LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, LATEST_2_8, \ - LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, KafkaVersion + LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, KafkaVersion class StreamsBrokerCompatibility(Test): @@ -64,25 +64,11 @@ def setUp(self): @cluster(num_nodes=4) - @parametrize(broker_version=str(LATEST_3_6)) - @parametrize(broker_version=str(LATEST_3_5)) - @parametrize(broker_version=str(LATEST_3_4)) - @parametrize(broker_version=str(LATEST_3_3)) - @parametrize(broker_version=str(LATEST_3_2)) - @parametrize(broker_version=str(LATEST_3_1)) - @parametrize(broker_version=str(LATEST_3_0)) - @parametrize(broker_version=str(LATEST_2_8)) - @parametrize(broker_version=str(LATEST_2_7)) - @parametrize(broker_version=str(LATEST_2_6)) - @parametrize(broker_version=str(LATEST_2_5)) - @parametrize(broker_version=str(LATEST_2_4)) - @parametrize(broker_version=str(LATEST_2_3)) - @parametrize(broker_version=str(LATEST_2_2)) - @parametrize(broker_version=str(LATEST_2_1)) - @parametrize(broker_version=str(LATEST_2_0)) - @parametrize(broker_version=str(LATEST_1_1)) - @parametrize(broker_version=str(LATEST_1_0)) - @parametrize(broker_version=str(LATEST_0_11_0)) + @matrix(broker_version=[str(LATEST_0_11_0),str(LATEST_1_0),str(LATEST_1_1),str(LATEST_2_0), + str(LATEST_2_1),str(LATEST_2_2),str(LATEST_2_3),str(LATEST_2_4), + str(LATEST_2_5),str(LATEST_2_6),str(LATEST_2_7),str(LATEST_2_8), + str(LATEST_3_0),str(LATEST_3_1),str(LATEST_3_2),str(LATEST_3_3), + str(LATEST_3_4),str(LATEST_3_5),str(LATEST_3_6),str(LATEST_3_7)]) def test_compatible_brokers_eos_disabled(self, broker_version): self.kafka.set_version(KafkaVersion(broker_version)) self.kafka.start() @@ -100,25 +86,11 @@ def test_compatible_brokers_eos_disabled(self, broker_version): self.kafka.stop() @cluster(num_nodes=4) - @parametrize(broker_version=str(LATEST_3_6)) - @parametrize(broker_version=str(LATEST_3_5)) - @parametrize(broker_version=str(LATEST_3_4)) - @parametrize(broker_version=str(LATEST_3_3)) - @parametrize(broker_version=str(LATEST_3_2)) - @parametrize(broker_version=str(LATEST_3_1)) - @parametrize(broker_version=str(LATEST_3_0)) - @parametrize(broker_version=str(LATEST_2_8)) - @parametrize(broker_version=str(LATEST_2_7)) - @parametrize(broker_version=str(LATEST_2_6)) - @parametrize(broker_version=str(LATEST_2_5)) - @parametrize(broker_version=str(LATEST_2_4)) - @parametrize(broker_version=str(LATEST_2_3)) - @parametrize(broker_version=str(LATEST_2_2)) - @parametrize(broker_version=str(LATEST_2_1)) - @parametrize(broker_version=str(LATEST_2_0)) - @parametrize(broker_version=str(LATEST_1_1)) - @parametrize(broker_version=str(LATEST_1_0)) - @parametrize(broker_version=str(LATEST_0_11_0)) + @matrix(broker_version=[str(LATEST_0_11_0),str(LATEST_1_0),str(LATEST_1_1),str(LATEST_2_0), + str(LATEST_2_1),str(LATEST_2_2),str(LATEST_2_3),str(LATEST_2_4), + str(LATEST_2_5),str(LATEST_2_6),str(LATEST_2_7),str(LATEST_2_8), + str(LATEST_3_0),str(LATEST_3_1),str(LATEST_3_2),str(LATEST_3_3), + str(LATEST_3_4),str(LATEST_3_5),str(LATEST_3_6),str(LATEST_3_7)]) def test_compatible_brokers_eos_alpha_enabled(self, broker_version): self.kafka.set_version(KafkaVersion(broker_version)) self.kafka.start() @@ -136,17 +108,9 @@ def test_compatible_brokers_eos_alpha_enabled(self, broker_version): self.kafka.stop() @cluster(num_nodes=4) - @parametrize(broker_version=str(LATEST_3_6)) - @parametrize(broker_version=str(LATEST_3_5)) - @parametrize(broker_version=str(LATEST_3_4)) - @parametrize(broker_version=str(LATEST_3_3)) - @parametrize(broker_version=str(LATEST_3_2)) - @parametrize(broker_version=str(LATEST_3_1)) - @parametrize(broker_version=str(LATEST_3_0)) - @parametrize(broker_version=str(LATEST_2_8)) - @parametrize(broker_version=str(LATEST_2_7)) - @parametrize(broker_version=str(LATEST_2_6)) - @parametrize(broker_version=str(LATEST_2_5)) + @matrix(broker_version=[str(LATEST_2_5),str(LATEST_2_6),str(LATEST_2_7),str(LATEST_2_8), + str(LATEST_3_0),str(LATEST_3_1),str(LATEST_3_2),str(LATEST_3_3), + str(LATEST_3_4),str(LATEST_3_5),str(LATEST_3_6),str(LATEST_3_7)]) def test_compatible_brokers_eos_v2_enabled(self, broker_version): self.kafka.set_version(KafkaVersion(broker_version)) self.kafka.start() diff --git a/tests/kafkatest/tests/streams/streams_upgrade_test.py b/tests/kafkatest/tests/streams/streams_upgrade_test.py index bd04a0bf07..371feaf13c 100644 --- a/tests/kafkatest/tests/streams/streams_upgrade_test.py +++ b/tests/kafkatest/tests/streams/streams_upgrade_test.py @@ -26,7 +26,7 @@ from kafkatest.tests.streams.utils import extract_generation_from_logs, extract_generation_id from kafkatest.version import LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, LATEST_0_11_0, LATEST_1_0, LATEST_1_1, \ LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, LATEST_2_8, \ - LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, DEV_BRANCH, DEV_VERSION, \ + LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, DEV_BRANCH, DEV_VERSION, \ KafkaVersion # broker 0.10.0 is not compatible with newer Kafka Streams versions @@ -36,7 +36,7 @@ str(LATEST_2_4), str(LATEST_2_5), str(LATEST_2_6), str(LATEST_2_7), str(LATEST_2_8), str(LATEST_3_0), str(LATEST_3_1), str(LATEST_3_2), str(LATEST_3_3), str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), - str(DEV_BRANCH)] + str(LATEST_3_7), str(DEV_BRANCH)] metadata_1_versions = [str(LATEST_0_10_0)] metadata_2_versions = [str(LATEST_0_10_1), str(LATEST_0_10_2), str(LATEST_0_11_0), str(LATEST_1_0), str(LATEST_1_1), @@ -46,7 +46,7 @@ # -> https://issues.apache.org/jira/browse/KAFKA-14646 # thus, we cannot test two bounce rolling upgrade because we know it's broken # instead we add version 2.4...3.3 to the `metadata_2_versions` upgrade list -fk_join_versions = [str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6)] +fk_join_versions = [str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7)] """ From 5f4806fd1c0eb0ef67885a5a7f12de282f494933 Mon Sep 17 00:00:00 2001 From: Nikolay Date: Thu, 7 Mar 2024 02:44:17 +0300 Subject: [PATCH 036/521] KAFKA-14589 [2/4] Tests of ConsoleGroupCommand rewritten in java (#15363) This PR is part of #14471 It contains some of ConsoleGroupCommand tests rewritten in java. Intention of separate PR is to reduce changes and simplify review. Reviewers: Chia-Ping Tsai --- .../admin/DescribeConsumerGroupTest.scala | 747 ---------------- .../group/DescribeConsumerGroupTest.java | 830 ++++++++++++++++++ 2 files changed, 830 insertions(+), 747 deletions(-) delete mode 100644 core/src/test/scala/unit/kafka/admin/DescribeConsumerGroupTest.scala create mode 100644 tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java diff --git a/core/src/test/scala/unit/kafka/admin/DescribeConsumerGroupTest.scala b/core/src/test/scala/unit/kafka/admin/DescribeConsumerGroupTest.scala deleted file mode 100644 index e98404f496..0000000000 --- a/core/src/test/scala/unit/kafka/admin/DescribeConsumerGroupTest.scala +++ /dev/null @@ -1,747 +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.admin - -import java.util.Properties -import kafka.utils.{Exit, TestInfoUtils, TestUtils} -import org.apache.kafka.clients.consumer.{ConsumerConfig, RoundRobinAssignor} -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.errors.TimeoutException -import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.{MethodSource, ValueSource} - -import scala.concurrent.ExecutionException -import scala.util.Random - -class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { - private val describeTypeOffsets = Array(Array(""), Array("--offsets")) - private val describeTypeMembers = Array(Array("--members"), Array("--members", "--verbose")) - private val describeTypeState = Array(Array("--state")) - private val describeTypes = describeTypeOffsets ++ describeTypeMembers ++ describeTypeState - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeNonExistingGroup(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - val missingGroup = "missing.group" - - for (describeType <- describeTypes) { - // note the group to be queried is a different (non-existing) group - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", missingGroup) ++ describeType - val service = getConsumerGroupService(cgcArgs) - - val output = TestUtils.grabConsoleOutput(service.describeGroups()) - assertTrue(output.contains(s"Consumer group '$missingGroup' does not exist."), - s"Expected error was not detected for describe option '${describeType.mkString(" ")}'") - } - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeWithMultipleSubActions(quorum: String): Unit = { - var exitStatus: Option[Int] = None - var exitMessage: Option[String] = None - Exit.setExitProcedure { (status, err) => - exitStatus = Some(status) - exitMessage = err - throw new RuntimeException - } - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group, "--members", "--state") - try { - ConsumerGroupCommand.main(cgcArgs) - } catch { - case e: RuntimeException => //expected - } finally { - Exit.resetExitProcedure() - } - assertEquals(Some(1), exitStatus) - assertTrue(exitMessage.get.contains("Option [describe] takes at most one of these options")) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeWithStateValue(quorum: String): Unit = { - var exitStatus: Option[Int] = None - var exitMessage: Option[String] = None - Exit.setExitProcedure { (status, err) => - exitStatus = Some(status) - exitMessage = err - throw new RuntimeException - } - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--all-groups", "--state", "Stable") - try { - ConsumerGroupCommand.main(cgcArgs) - } catch { - case e: RuntimeException => //expected - } finally { - Exit.resetExitProcedure() - } - assertEquals(Some(1), exitStatus) - assertTrue(exitMessage.get.contains("Option [describe] does not take a value for [state]")) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeOffsetsOfNonExistingGroup(quorum: String, groupProtocol: String): Unit = { - val group = "missing.group" - createOffsetsTopic() - - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - // note the group to be queried is a different (non-existing) group - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - val (state, assignments) = service.collectGroupOffsets(group) - assertTrue(state.contains("Dead") && assignments.contains(List()), - s"Expected the state to be 'Dead', with no members in the group '$group'.") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeMembersOfNonExistingGroup(quorum: String, groupProtocol: String): Unit = { - val group = "missing.group" - createOffsetsTopic() - - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - // note the group to be queried is a different (non-existing) group - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - val (state, assignments) = service.collectGroupMembers(group, false) - assertTrue(state.contains("Dead") && assignments.contains(List()), - s"Expected the state to be 'Dead', with no members in the group '$group'.") - - val (state2, assignments2) = service.collectGroupMembers(group, true) - assertTrue(state2.contains("Dead") && assignments2.contains(List()), - s"Expected the state to be 'Dead', with no members in the group '$group' (verbose option).") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeStateOfNonExistingGroup(quorum: String, groupProtocol: String): Unit = { - val group = "missing.group" - createOffsetsTopic() - - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - // note the group to be queried is a different (non-existing) group - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - val state = service.collectGroupState(group) - assertTrue(state.state == "Dead" && state.numMembers == 0 && - state.coordinator != null && brokers.map(_.config.brokerId).toList.contains(state.coordinator.id), - s"Expected the state to be 'Dead', with no members in the group '$group'." - ) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeExistingGroup(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - for (describeType <- describeTypes) { - val group = this.group + describeType.mkString("") - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, group = group, groupProtocol = groupProtocol) - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) ++ describeType - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (output, error) = TestUtils.grabConsoleOutputAndError(service.describeGroups()) - output.trim.split("\n").length == 2 && error.isEmpty - }, s"Expected a data row and no error in describe results with describe type ${describeType.mkString(" ")}.") - } - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeExistingGroups(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // Create N single-threaded consumer groups from a single-partition topic - val groups = (for (describeType <- describeTypes) yield { - val group = this.group + describeType.mkString("") - addConsumerGroupExecutor(numConsumers = 1, group = group, groupProtocol = groupProtocol) - Array("--group", group) - }).flatten - - val expectedNumLines = describeTypes.length * 2 - - for (describeType <- describeTypes) { - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe") ++ groups ++ describeType - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (output, error) = TestUtils.grabConsoleOutputAndError(service.describeGroups()) - val numLines = output.trim.split("\n").count(line => line.nonEmpty) - (numLines == expectedNumLines) && error.isEmpty - }, s"Expected a data row and no error in describe results with describe type ${describeType.mkString(" ")}.") - } - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeAllExistingGroups(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // Create N single-threaded consumer groups from a single-partition topic - for (describeType <- describeTypes) { - val group = this.group + describeType.mkString("") - addConsumerGroupExecutor(numConsumers = 1, group = group, groupProtocol = groupProtocol) - } - - val expectedNumLines = describeTypes.length * 2 - - for (describeType <- describeTypes) { - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--all-groups") ++ describeType - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (output, error) = TestUtils.grabConsoleOutputAndError(service.describeGroups()) - val numLines = output.trim.split("\n").count(line => line.nonEmpty) - (numLines == expectedNumLines) && error.isEmpty - }, s"Expected a data row and no error in describe results with describe type ${describeType.mkString(" ")}.") - } - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeOffsetsOfExistingGroup(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupOffsets(group) - state.contains("Stable") && - assignments.isDefined && - assignments.get.count(_.group == group) == 1 && - assignments.get.filter(_.group == group).head.consumerId.exists(_.trim != ConsumerGroupCommand.MISSING_COLUMN_VALUE) && - assignments.get.filter(_.group == group).head.clientId.exists(_.trim != ConsumerGroupCommand.MISSING_COLUMN_VALUE) && - assignments.get.filter(_.group == group).head.host.exists(_.trim != ConsumerGroupCommand.MISSING_COLUMN_VALUE) - }, s"Expected a 'Stable' group status, rows and valid values for consumer id / client id / host columns in describe results for group $group.") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeMembersOfExistingGroup(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupMembers(group, false) - state.contains("Stable") && - (assignments match { - case Some(memberAssignments) => - memberAssignments.count(_.group == group) == 1 && - memberAssignments.filter(_.group == group).head.consumerId != ConsumerGroupCommand.MISSING_COLUMN_VALUE && - memberAssignments.filter(_.group == group).head.clientId != ConsumerGroupCommand.MISSING_COLUMN_VALUE && - memberAssignments.filter(_.group == group).head.host != ConsumerGroupCommand.MISSING_COLUMN_VALUE - case None => - false - }) - }, s"Expected a 'Stable' group status, rows and valid member information for group $group.") - - val (_, assignments) = service.collectGroupMembers(group, true) - assignments match { - case None => - fail(s"Expected partition assignments for members of group $group") - case Some(memberAssignments) => - assertTrue(memberAssignments.size == 1 && memberAssignments.head.assignment.size == 1, - s"Expected a topic partition assigned to the single group member for group $group") - } - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeStateOfExistingGroup(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor( - numConsumers = 1, - groupProtocol = groupProtocol, - // This is only effective when new protocol is used. - remoteAssignor = Some("range") - ) - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val state = service.collectGroupState(group) - state.state == "Stable" && - state.numMembers == 1 && - state.assignmentStrategy == "range" && - state.coordinator != null && - brokers.map(_.config.brokerId).toList.contains(state.coordinator.id) - }, s"Expected a 'Stable' group status, with one member and round robin assignment strategy for group $group.") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeStateOfExistingGroupWithNonDefaultAssignor(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // run one consumer in the group consuming from a single-partition topic - val expectedName = if (groupProtocol == "consumer") { - addConsumerGroupExecutor(numConsumers = 1, remoteAssignor = Some("range"), groupProtocol = groupProtocol) - "range" - } else { - addConsumerGroupExecutor(numConsumers = 1, strategy = classOf[RoundRobinAssignor].getName, groupProtocol = groupProtocol) - "roundrobin" - } - - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val state = service.collectGroupState(group) - state.state == "Stable" && - state.numMembers == 1 && - state.assignmentStrategy == expectedName && - state.coordinator != null && - brokers.map(_.config.brokerId).toList.contains(state.coordinator.id) - }, s"Expected a 'Stable' group status, with one member and $expectedName assignment strategy for group $group.") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeExistingGroupWithNoMembers(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - for (describeType <- describeTypes) { - val group = this.group + describeType.mkString("") - // run one consumer in the group consuming from a single-partition topic - val executor = addConsumerGroupExecutor(numConsumers = 1, group = group, groupProtocol = groupProtocol) - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) ++ describeType - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (output, error) = TestUtils.grabConsoleOutputAndError(service.describeGroups()) - output.trim.split("\n").length == 2 && error.isEmpty - }, s"Expected describe group results with one data row for describe type '${describeType.mkString(" ")}'") - - // stop the consumer so the group has no active member anymore - executor.shutdown() - TestUtils.waitUntilTrue(() => { - TestUtils.grabConsoleError(service.describeGroups()).contains(s"Consumer group '$group' has no active members.") - }, s"Expected no active member in describe group results with describe type ${describeType.mkString(" ")}") - } - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeOffsetsOfExistingGroupWithNoMembers(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // run one consumer in the group consuming from a single-partition topic - val executor = addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol, syncCommit = true) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupOffsets(group) - state.contains("Stable") && assignments.exists(_.exists(assignment => assignment.group == group && assignment.offset.isDefined)) - }, "Expected the group to initially become stable, and to find group in assignments after initial offset commit.") - - // stop the consumer so the group has no active member anymore - executor.shutdown() - - val (result, succeeded) = TestUtils.computeUntilTrue(service.collectGroupOffsets(group)) { - case (state, assignments) => - val testGroupAssignments = assignments.toSeq.flatMap(_.filter(_.group == group)) - def assignment = testGroupAssignments.head - state.contains("Empty") && - testGroupAssignments.size == 1 && - assignment.consumerId.exists(_.trim == ConsumerGroupCommand.MISSING_COLUMN_VALUE) && // the member should be gone - assignment.clientId.exists(_.trim == ConsumerGroupCommand.MISSING_COLUMN_VALUE) && - assignment.host.exists(_.trim == ConsumerGroupCommand.MISSING_COLUMN_VALUE) - } - val (state, assignments) = result - assertTrue(succeeded, s"Expected no active member in describe group results, state: $state, assignments: $assignments") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeMembersOfExistingGroupWithNoMembers(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // run one consumer in the group consuming from a single-partition topic - val executor = addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupMembers(group, false) - state.contains("Stable") && assignments.exists(_.exists(_.group == group)) - }, "Expected the group to initially become stable, and to find group in assignments after initial offset commit.") - - // stop the consumer so the group has no active member anymore - executor.shutdown() - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupMembers(group, false) - state.contains("Empty") && assignments.isDefined && assignments.get.isEmpty - }, s"Expected no member in describe group members results for group '$group'") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeStateOfExistingGroupWithNoMembers(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // run one consumer in the group consuming from a single-partition topic - val executor = addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val state = service.collectGroupState(group) - state.state == "Stable" && - state.numMembers == 1 && - state.coordinator != null && - brokers.map(_.config.brokerId).toList.contains(state.coordinator.id) - }, s"Expected the group '$group' to initially become stable, and have a single member.") - - // stop the consumer so the group has no active member anymore - executor.shutdown() - - TestUtils.waitUntilTrue(() => { - val state = service.collectGroupState(group) - state.state == "Empty" && state.numMembers == 0 - }, s"Expected the group '$group' to become empty after the only member leaving.") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeWithConsumersWithoutAssignedPartitions(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - for (describeType <- describeTypes) { - val group = this.group + describeType.mkString("") - // run two consumers in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 2, group = group, groupProtocol = groupProtocol) - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) ++ describeType - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (output, error) = TestUtils.grabConsoleOutputAndError(service.describeGroups()) - val expectedNumRows = if (describeTypeMembers.contains(describeType)) 3 else 2 - error.isEmpty && output.trim.split("\n").size == expectedNumRows - }, s"Expected a single data row in describe group result with describe type '${describeType.mkString(" ")}'") - } - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeOffsetsWithConsumersWithoutAssignedPartitions(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // run two consumers in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 2, groupProtocol = groupProtocol) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupOffsets(group) - state.contains("Stable") && - assignments.isDefined && - assignments.get.count(_.group == group) == 1 && - assignments.get.count { x => x.group == group && x.partition.isDefined } == 1 - }, "Expected rows for consumers with no assigned partitions in describe group results") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeMembersWithConsumersWithoutAssignedPartitions(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // run two consumers in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 2, groupProtocol = groupProtocol) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupMembers(group, false) - state.contains("Stable") && - assignments.isDefined && - assignments.get.count(_.group == group) == 2 && - assignments.get.count { x => x.group == group && x.numPartitions == 1 } == 1 && - assignments.get.count { x => x.group == group && x.numPartitions == 0 } == 1 && - !assignments.get.exists(_.assignment.nonEmpty) - }, "Expected rows for consumers with no assigned partitions in describe group results") - - val (state, assignments) = service.collectGroupMembers(group, true) - assertTrue(state.contains("Stable") && assignments.get.count(_.assignment.nonEmpty) > 0, - "Expected additional columns in verbose version of describe members") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeStateWithConsumersWithoutAssignedPartitions(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // run two consumers in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 2, groupProtocol = groupProtocol) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val state = service.collectGroupState(group) - state.state == "Stable" && state.numMembers == 2 - }, "Expected two consumers in describe group results") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeWithMultiPartitionTopicAndMultipleConsumers(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - val topic2 = "foo2" - createTopic(topic2, 2, 1) - - for (describeType <- describeTypes) { - val group = this.group + describeType.mkString("") - // run two consumers in the group consuming from a two-partition topic - addConsumerGroupExecutor(2, topic2, group = group, groupProtocol = groupProtocol) - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) ++ describeType - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (output, error) = TestUtils.grabConsoleOutputAndError(service.describeGroups()) - val expectedNumRows = if (describeTypeState.contains(describeType)) 2 else 3 - error.isEmpty && output.trim.split("\n").size == expectedNumRows - }, s"Expected a single data row in describe group result with describe type '${describeType.mkString(" ")}'") - } - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeOffsetsWithMultiPartitionTopicAndMultipleConsumers(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - val topic2 = "foo2" - createTopic(topic2, 2, 1) - - // run two consumers in the group consuming from a two-partition topic - addConsumerGroupExecutor(numConsumers = 2, topic2, groupProtocol = groupProtocol) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupOffsets(group) - state.contains("Stable") && - assignments.isDefined && - assignments.get.count(_.group == group) == 2 && - assignments.get.count { x => x.group == group && x.partition.isDefined } == 2 && - assignments.get.count { x => x.group == group && x.partition.isEmpty } == 0 - }, "Expected two rows (one row per consumer) in describe group results.") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeMembersWithMultiPartitionTopicAndMultipleConsumers(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - val topic2 = "foo2" - createTopic(topic2, 2, 1) - - // run two consumers in the group consuming from a two-partition topic - addConsumerGroupExecutor(numConsumers = 2, topic2, groupProtocol = groupProtocol) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupMembers(group, false) - state.contains("Stable") && - assignments.isDefined && - assignments.get.count(_.group == group) == 2 && - assignments.get.count { x => x.group == group && x.numPartitions == 1 } == 2 && - assignments.get.count { x => x.group == group && x.numPartitions == 0 } == 0 - }, "Expected two rows (one row per consumer) in describe group members results.") - - val (state, assignments) = service.collectGroupMembers(group, true) - assertTrue(state.contains("Stable") && assignments.get.count(_.assignment.isEmpty) == 0, - "Expected additional columns in verbose version of describe members") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeStateWithMultiPartitionTopicAndMultipleConsumers(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - val topic2 = "foo2" - createTopic(topic2, 2, 1) - - // run two consumers in the group consuming from a two-partition topic - addConsumerGroupExecutor(numConsumers = 2, topic2, groupProtocol = groupProtocol) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val state = service.collectGroupState(group) - state.state == "Stable" && state.group == group && state.numMembers == 2 - }, "Expected a stable group with two members in describe group state result.") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft", "kraft+kip848")) - def testDescribeSimpleConsumerGroup(quorum: String): Unit = { - // Ensure that the offsets of consumers which don't use group management are still displayed - - createOffsetsTopic() - val topic2 = "foo2" - createTopic(topic2, 2, 1) - addSimpleGroupExecutor(Seq(new TopicPartition(topic2, 0), new TopicPartition(topic2, 1))) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupOffsets(group) - state.contains("Empty") && assignments.isDefined && assignments.get.count(_.group == group) == 2 - }, "Expected a stable group with two members in describe group state result.") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeGroupWithShortInitializationTimeout(quorum: String, groupProtocol: String): Unit = { - // Let creation of the offsets topic happen during group initialization to ensure that initialization doesn't - // complete before the timeout expires - - val describeType = describeTypes(Random.nextInt(describeTypes.length)) - val group = this.group + describeType.mkString("") - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - // set the group initialization timeout too low for the group to stabilize - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--timeout", "1", "--group", group) ++ describeType - val service = getConsumerGroupService(cgcArgs) - - val e = assertThrows(classOf[ExecutionException], () => TestUtils.grabConsoleOutputAndError(service.describeGroups())) - assertEquals(classOf[TimeoutException], e.getCause.getClass) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeGroupOffsetsWithShortInitializationTimeout(quorum: String, groupProtocol: String): Unit = { - // Let creation of the offsets topic happen during group initialization to ensure that initialization doesn't - // complete before the timeout expires - - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - - // set the group initialization timeout too low for the group to stabilize - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group, "--timeout", "1") - val service = getConsumerGroupService(cgcArgs) - - val e = assertThrows(classOf[ExecutionException], () => service.collectGroupOffsets(group)) - assertEquals(classOf[TimeoutException], e.getCause.getClass) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeGroupMembersWithShortInitializationTimeout(quorum: String, groupProtocol: String): Unit = { - // Let creation of the offsets topic happen during group initialization to ensure that initialization doesn't - // complete before the timeout expires - - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - - // set the group initialization timeout too low for the group to stabilize - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group, "--timeout", "1") - val service = getConsumerGroupService(cgcArgs) - - var e = assertThrows(classOf[ExecutionException], () => service.collectGroupMembers(group, false)) - assertEquals(classOf[TimeoutException], e.getCause.getClass) - e = assertThrows(classOf[ExecutionException], () => service.collectGroupMembers(group, true)) - assertEquals(classOf[TimeoutException], e.getCause.getClass) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeGroupStateWithShortInitializationTimeout(quorum: String, groupProtocol: String): Unit = { - // Let creation of the offsets topic happen during group initialization to ensure that initialization doesn't - // complete before the timeout expires - - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - - // set the group initialization timeout too low for the group to stabilize - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group, "--timeout", "1") - val service = getConsumerGroupService(cgcArgs) - - val e = assertThrows(classOf[ExecutionException], () => service.collectGroupState(group)) - assertEquals(classOf[TimeoutException], e.getCause.getClass) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeWithUnrecognizedNewConsumerOption(quorum: String): Unit = { - val cgcArgs = Array("--new-consumer", "--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - assertThrows(classOf[joptsimple.OptionException], () => getConsumerGroupService(cgcArgs)) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeNonOffsetCommitGroup(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - val customProps = new Properties - // create a consumer group that never commits offsets - customProps.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, customPropsOpt = Some(customProps), groupProtocol = groupProtocol) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupOffsets(group) - state.contains("Stable") && - assignments.isDefined && - assignments.get.count(_.group == group) == 1 && - assignments.get.filter(_.group == group).head.consumerId.exists(_.trim != ConsumerGroupCommand.MISSING_COLUMN_VALUE) && - assignments.get.filter(_.group == group).head.clientId.exists(_.trim != ConsumerGroupCommand.MISSING_COLUMN_VALUE) && - assignments.get.filter(_.group == group).head.host.exists(_.trim != ConsumerGroupCommand.MISSING_COLUMN_VALUE) - }, s"Expected a 'Stable' group status, rows and valid values for consumer id / client id / host columns in describe results for non-offset-committing group $group.") - } - -} - diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java new file mode 100644 index 0000000000..f0277d18cd --- /dev/null +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java @@ -0,0 +1,830 @@ +/* + * 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.tools.consumer.group; + +import kafka.admin.ConsumerGroupCommand; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.RangeAssignor; +import org.apache.kafka.clients.consumer.RoundRobinAssignor; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.test.TestUtils; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; +import scala.Function0; +import scala.Function1; +import scala.Option; +import scala.collection.Seq; +import scala.runtime.BoxedUnit; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.Properties; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +import static org.apache.kafka.test.TestUtils.RANDOM; +import static org.apache.kafka.tools.ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES; +import static org.apache.kafka.tools.ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + +public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { + private static final List> DESCRIBE_TYPE_OFFSETS = Arrays.asList(Collections.singletonList(""), Collections.singletonList("--offsets")); + private static final List> DESCRIBE_TYPE_MEMBERS = Arrays.asList(Collections.singletonList("--members"), Arrays.asList("--members", "--verbose")); + private static final List> DESCRIBE_TYPE_STATE = Collections.singletonList(Collections.singletonList("--state")); + private static final List> DESCRIBE_TYPES; + + static { + List> describeTypes = new ArrayList<>(); + + describeTypes.addAll(DESCRIBE_TYPE_OFFSETS); + describeTypes.addAll(DESCRIBE_TYPE_MEMBERS); + describeTypes.addAll(DESCRIBE_TYPE_STATE); + + DESCRIBE_TYPES = describeTypes; + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeNonExistingGroup(String quorum, String groupProtocol) { + createOffsetsTopic(listenerName(), new Properties()); + String missingGroup = "missing.group"; + + for (List describeType : DESCRIBE_TYPES) { + // note the group to be queried is a different (non-existing) group + List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", missingGroup)); + cgcArgs.addAll(describeType); + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0])); + + String output = kafka.utils.TestUtils.grabConsoleOutput(describeGroups(service)); + assertTrue(output.contains("Consumer group '" + missingGroup + "' does not exist."), + "Expected error was not detected for describe option '" + String.join(" ", describeType) + "'"); + } + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME) + @ValueSource(strings = {"zk", "kraft"}) + public void testDescribeWithMultipleSubActions(String quorum) { + AtomicInteger exitStatus = new AtomicInteger(0); + AtomicReference exitMessage = new AtomicReference<>(""); + Exit.setExitProcedure((status, err) -> { + exitStatus.set(status); + exitMessage.set(err); + throw new RuntimeException(); + }); + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP, "--members", "--state"}; + try { + assertThrows(RuntimeException.class, () -> ConsumerGroupCommand.main(cgcArgs)); + } finally { + Exit.resetExitProcedure(); + } + assertEquals(1, exitStatus.get()); + assertTrue(exitMessage.get().contains("Option [describe] takes at most one of these options")); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME) + @ValueSource(strings = {"zk", "kraft"}) + public void testDescribeWithStateValue(String quorum) { + AtomicInteger exitStatus = new AtomicInteger(0); + AtomicReference exitMessage = new AtomicReference<>(""); + Exit.setExitProcedure((status, err) -> { + exitStatus.set(status); + exitMessage.set(err); + throw new RuntimeException(); + }); + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--all-groups", "--state", "Stable"}; + try { + assertThrows(RuntimeException.class, () -> ConsumerGroupCommand.main(cgcArgs)); + } finally { + Exit.resetExitProcedure(); + } + assertEquals(1, exitStatus.get()); + assertTrue(exitMessage.get().contains("Option [describe] does not take a value for [state]")); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeOffsetsOfNonExistingGroup(String quorum, String groupProtocol) { + String group = "missing.group"; + createOffsetsTopic(listenerName(), new Properties()); + + // run one consumer in the group consuming from a single-partition topic + addConsumerGroupExecutor(1, groupProtocol); + // note the group to be queried is a different (non-existing) group + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + scala.Tuple2, Option>> res = service.collectGroupOffsets(group); + assertTrue(res._1.map(s -> s.contains("Dead")).getOrElse(() -> false) && res._2.map(Seq::isEmpty).getOrElse(() -> false), + "Expected the state to be 'Dead', with no members in the group '" + group + "'."); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeMembersOfNonExistingGroup(String quorum, String groupProtocol) { + String group = "missing.group"; + createOffsetsTopic(listenerName(), new Properties()); + + // run one consumer in the group consuming from a single-partition topic + addConsumerGroupExecutor(1, groupProtocol); + // note the group to be queried is a different (non-existing) group + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + scala.Tuple2, Option>> res = service.collectGroupMembers(group, false); + assertTrue(res._1.map(s -> s.contains("Dead")).getOrElse(() -> false) && res._2.map(Seq::isEmpty).getOrElse(() -> false), + "Expected the state to be 'Dead', with no members in the group '" + group + "'."); + + scala.Tuple2, Option>> res2 = service.collectGroupMembers(group, true); + assertTrue(res2._1.map(s -> s.contains("Dead")).getOrElse(() -> false) && res2._2.map(Seq::isEmpty).getOrElse(() -> false), + "Expected the state to be 'Dead', with no members in the group '" + group + "' (verbose option)."); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeStateOfNonExistingGroup(String quorum, String groupProtocol) { + String group = "missing.group"; + createOffsetsTopic(listenerName(), new Properties()); + + // run one consumer in the group consuming from a single-partition topic + addConsumerGroupExecutor(1, groupProtocol); + // note the group to be queried is a different (non-existing) group + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + ConsumerGroupCommand.GroupState state = service.collectGroupState(group); + assertTrue(Objects.equals(state.state(), "Dead") && state.numMembers() == 0 && + state.coordinator() != null && !brokers().filter(s -> s.config().brokerId() == state.coordinator().id()).isEmpty(), + "Expected the state to be 'Dead', with no members in the group '" + group + "'." + ); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeExistingGroup(String quorum, String groupProtocol) throws Exception { + createOffsetsTopic(listenerName(), new Properties()); + + for (List describeType : DESCRIBE_TYPES) { + String group = GROUP + String.join("", describeType); + // run one consumer in the group consuming from a single-partition topic + addConsumerGroupExecutor(1, TOPIC, group, groupProtocol); + List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group)); + cgcArgs.addAll(describeType); + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0])); + + TestUtils.waitForCondition(() -> { + scala.Tuple2 res = kafka.utils.TestUtils.grabConsoleOutputAndError(describeGroups(service)); + return res._1.trim().split("\n").length == 2 && res._2.isEmpty(); + }, "Expected a data row and no error in describe results with describe type " + String.join(" ", describeType) + "."); + } + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeExistingGroups(String quorum, String groupProtocol) throws Exception { + createOffsetsTopic(listenerName(), new Properties()); + + // Create N single-threaded consumer groups from a single-partition topic + List groups = new ArrayList<>(); + + for (List describeType : DESCRIBE_TYPES) { + String group = GROUP + String.join("", describeType); + addConsumerGroupExecutor(1, TOPIC, group, groupProtocol); + groups.addAll(Arrays.asList("--group", group)); + } + + int expectedNumLines = DESCRIBE_TYPES.size() * 2; + + for (List describeType : DESCRIBE_TYPES) { + List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--describe")); + cgcArgs.addAll(groups); + cgcArgs.addAll(describeType); + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0])); + + TestUtils.waitForCondition(() -> { + scala.Tuple2 res = kafka.utils.TestUtils.grabConsoleOutputAndError(describeGroups(service)); + long numLines = Arrays.stream(res._1.trim().split("\n")).filter(line -> !line.isEmpty()).count(); + return (numLines == expectedNumLines) && res._2.isEmpty(); + }, "Expected a data row and no error in describe results with describe type " + String.join(" ", describeType) + "."); + } + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeAllExistingGroups(String quorum, String groupProtocol) throws Exception { + createOffsetsTopic(listenerName(), new Properties()); + + // Create N single-threaded consumer groups from a single-partition topic + for (List describeType : DESCRIBE_TYPES) { + String group = GROUP + String.join("", describeType); + addConsumerGroupExecutor(1, TOPIC, group, groupProtocol); + } + + int expectedNumLines = DESCRIBE_TYPES.size() * 2; + + for (List describeType : DESCRIBE_TYPES) { + List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--all-groups")); + cgcArgs.addAll(describeType); + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0])); + + TestUtils.waitForCondition(() -> { + scala.Tuple2 res = kafka.utils.TestUtils.grabConsoleOutputAndError(describeGroups(service)); + long numLines = Arrays.stream(res._1.trim().split("\n")).filter(s -> !s.isEmpty()).count(); + return (numLines == expectedNumLines) && res._2.isEmpty(); + }, "Expected a data row and no error in describe results with describe type " + String.join(" ", describeType) + "."); + } + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeOffsetsOfExistingGroup(String quorum, String groupProtocol) throws Exception { + createOffsetsTopic(listenerName(), new Properties()); + + // run one consumer in the group consuming from a single-partition topic + addConsumerGroupExecutor(1, groupProtocol); + + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + TestUtils.waitForCondition(() -> { + scala.Tuple2, Option>> groupOffsets = service.collectGroupOffsets(GROUP); + Option state = groupOffsets._1; + Option> assignments = groupOffsets._2; + + Function1 isGrp = s -> Objects.equals(s.group(), GROUP); + + boolean res = state.map(s -> s.contains("Stable")).getOrElse(() -> false) && + assignments.isDefined() && + assignments.get().count(isGrp) == 1; + + if (!res) + return false; + + @SuppressWarnings("cast") + ConsumerGroupCommand.PartitionAssignmentState partitionState = + (ConsumerGroupCommand.PartitionAssignmentState) assignments.get().filter(isGrp).head(); + + return !partitionState.consumerId().map(s0 -> s0.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE())).getOrElse(() -> false) && + !partitionState.clientId().map(s0 -> s0.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE())).getOrElse(() -> false) && + !partitionState.host().map(h -> h.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE())).getOrElse(() -> false); + }, "Expected a 'Stable' group status, rows and valid values for consumer id / client id / host columns in describe results for group " + GROUP + "."); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeMembersOfExistingGroup(String quorum, String groupProtocol) throws Exception { + createOffsetsTopic(listenerName(), new Properties()); + + // run one consumer in the group consuming from a single-partition topic + addConsumerGroupExecutor(1, groupProtocol); + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + TestUtils.waitForCondition(() -> { + scala.Tuple2, Option>> groupMembers = service.collectGroupMembers(GROUP, false); + Option state = groupMembers._1; + Option> assignments = groupMembers._2; + + Function1 isGrp = s -> Objects.equals(s.group(), GROUP); + + boolean res = state.map(s -> s.contains("Stable")).getOrElse(() -> false) && + assignments.isDefined() && + assignments.get().count(s -> Objects.equals(s.group(), GROUP)) == 1; + + if (!res) + return false; + + @SuppressWarnings("cast") + ConsumerGroupCommand.MemberAssignmentState assignmentState = + (ConsumerGroupCommand.MemberAssignmentState) assignments.get().filter(isGrp).head(); + + return !Objects.equals(assignmentState.consumerId(), ConsumerGroupCommand.MISSING_COLUMN_VALUE()) && + !Objects.equals(assignmentState.clientId(), ConsumerGroupCommand.MISSING_COLUMN_VALUE()) && + !Objects.equals(assignmentState.host(), ConsumerGroupCommand.MISSING_COLUMN_VALUE()); + }, "Expected a 'Stable' group status, rows and valid member information for group " + GROUP + "."); + + scala.Tuple2, Option>> res = service.collectGroupMembers(GROUP, true); + + if (res._2.isDefined()) { + assertTrue(res._2.get().size() == 1 && res._2.get().iterator().next().assignment().size() == 1, + "Expected a topic partition assigned to the single group member for group " + GROUP); + } else { + fail("Expected partition assignments for members of group " + GROUP); + } + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeStateOfExistingGroup(String quorum, String groupProtocol) throws Exception { + createOffsetsTopic(listenerName(), new Properties()); + + // run one consumer in the group consuming from a single-partition topic + addConsumerGroupExecutor( + 1, + groupProtocol, + // This is only effective when new protocol is used. + Optional.of("range") + ); + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + TestUtils.waitForCondition(() -> { + ConsumerGroupCommand.GroupState state = service.collectGroupState(GROUP); + return Objects.equals(state.state(), "Stable") && + state.numMembers() == 1 && + Objects.equals(state.assignmentStrategy(), "range") && + state.coordinator() != null && + brokers().count(s -> s.config().brokerId() == state.coordinator().id()) > 0; + }, "Expected a 'Stable' group status, with one member and round robin assignment strategy for group " + GROUP + "."); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeStateOfExistingGroupWithNonDefaultAssignor(String quorum, String groupProtocol) throws Exception { + createOffsetsTopic(listenerName(), new Properties()); + + // run one consumer in the group consuming from a single-partition topic + String expectedName; + if (groupProtocol.equals("consumer")) { + addConsumerGroupExecutor(1, groupProtocol, Optional.of("range")); + expectedName = "range"; + } else { + addConsumerGroupExecutor(1, TOPIC, GROUP, RoundRobinAssignor.class.getName(), Optional.empty(), Optional.empty(), false, groupProtocol); + expectedName = "roundrobin"; + } + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + TestUtils.waitForCondition(() -> { + ConsumerGroupCommand.GroupState state = service.collectGroupState(GROUP); + return Objects.equals(state.state(), "Stable") && + state.numMembers() == 1 && + Objects.equals(state.assignmentStrategy(), expectedName) && + state.coordinator() != null && + brokers().count(s -> s.config().brokerId() == state.coordinator().id()) > 0; + }, "Expected a 'Stable' group status, with one member and " + expectedName + " assignment strategy for group " + GROUP + "."); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeExistingGroupWithNoMembers(String quorum, String groupProtocol) throws Exception { + createOffsetsTopic(listenerName(), new Properties()); + + for (List describeType : DESCRIBE_TYPES) { + String group = GROUP + String.join("", describeType); + // run one consumer in the group consuming from a single-partition topic + ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, TOPIC, group, groupProtocol); + List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group)); + cgcArgs.addAll(describeType); + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0])); + + TestUtils.waitForCondition(() -> { + scala.Tuple2 res = kafka.utils.TestUtils.grabConsoleOutputAndError(describeGroups(service)); + return res._1.trim().split("\n").length == 2 && res._2.isEmpty(); + }, "Expected describe group results with one data row for describe type '" + String.join(" ", describeType) + "'"); + + // stop the consumer so the group has no active member anymore + executor.shutdown(); + TestUtils.waitForCondition( + () -> kafka.utils.TestUtils.grabConsoleError(describeGroups(service)).contains("Consumer group '" + group + "' has no active members."), + "Expected no active member in describe group results with describe type " + String.join(" ", describeType)); + } + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeOffsetsOfExistingGroupWithNoMembers(String quorum, String groupProtocol) throws Exception { + createOffsetsTopic(listenerName(), new Properties()); + + // run one consumer in the group consuming from a single-partition topic + ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, TOPIC, GROUP, RangeAssignor.class.getName(), Optional.empty(), Optional.empty(), true, groupProtocol); + + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + TestUtils.waitForCondition(() -> { + scala.Tuple2, Option>> res = service.collectGroupOffsets(GROUP); + return res._1.map(s -> s.contains("Stable")).getOrElse(() -> false) + && res._2.map(c -> c.exists(assignment -> Objects.equals(assignment.group(), GROUP) && assignment.offset().isDefined())).getOrElse(() -> false); + }, "Expected the group to initially become stable, and to find group in assignments after initial offset commit."); + + // stop the consumer so the group has no active member anymore + executor.shutdown(); + + TestUtils.waitForCondition(() -> { + scala.Tuple2, Option>> offsets = service.collectGroupOffsets(GROUP); + Option state = offsets._1; + Option> assignments = offsets._2; + @SuppressWarnings("unchecked") + Seq testGroupAssignments = assignments.get().filter(a -> Objects.equals(a.group(), GROUP)).toSeq(); + ConsumerGroupCommand.PartitionAssignmentState assignment = testGroupAssignments.head(); + return state.map(s -> s.contains("Empty")).getOrElse(() -> false) && + testGroupAssignments.size() == 1 && + assignment.consumerId().map(c -> c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE())).getOrElse(() -> false) && // the member should be gone + assignment.clientId().map(c -> c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE())).getOrElse(() -> false) && + assignment.host().map(c -> c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE())).getOrElse(() -> false); + }, "failed to collect group offsets"); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeMembersOfExistingGroupWithNoMembers(String quorum, String groupProtocol) throws Exception { + createOffsetsTopic(listenerName(), new Properties()); + + // run one consumer in the group consuming from a single-partition topic + ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, groupProtocol); + + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + TestUtils.waitForCondition(() -> { + scala.Tuple2, Option>> res = service.collectGroupMembers(GROUP, false); + return res._1.map(s -> s.contains("Stable")).getOrElse(() -> false) + && res._2.map(c -> c.exists(m -> Objects.equals(m.group(), GROUP))).getOrElse(() -> false); + }, "Expected the group to initially become stable, and to find group in assignments after initial offset commit."); + + // stop the consumer so the group has no active member anymore + executor.shutdown(); + + TestUtils.waitForCondition(() -> { + scala.Tuple2, Option>> res = service.collectGroupMembers(GROUP, false); + return res._1.map(s -> s.contains("Empty")).getOrElse(() -> false) && res._2.isDefined() && res._2.get().isEmpty(); + }, "Expected no member in describe group members results for group '" + GROUP + "'"); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeStateOfExistingGroupWithNoMembers(String quorum, String groupProtocol) throws Exception { + createOffsetsTopic(listenerName(), new Properties()); + + // run one consumer in the group consuming from a single-partition topic + ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, groupProtocol); + + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + TestUtils.waitForCondition(() -> { + ConsumerGroupCommand.GroupState state = service.collectGroupState(GROUP); + return Objects.equals(state.state(), "Stable") && + state.numMembers() == 1 && + state.coordinator() != null && + brokers().count(s -> s.config().brokerId() == state.coordinator().id()) > 0; + }, "Expected the group '" + GROUP + "' to initially become stable, and have a single member."); + + // stop the consumer so the group has no active member anymore + executor.shutdown(); + + TestUtils.waitForCondition(() -> { + ConsumerGroupCommand.GroupState state = service.collectGroupState(GROUP); + return Objects.equals(state.state(), "Empty") && state.numMembers() == 0; + }, "Expected the group '" + GROUP + "' to become empty after the only member leaving."); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeWithConsumersWithoutAssignedPartitions(String quorum, String groupProtocol) throws Exception { + createOffsetsTopic(listenerName(), new Properties()); + + for (List describeType : DESCRIBE_TYPES) { + String group = GROUP + String.join("", describeType); + // run two consumers in the group consuming from a single-partition topic + addConsumerGroupExecutor(2, TOPIC, group, groupProtocol); + List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group)); + cgcArgs.addAll(describeType); + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0])); + + TestUtils.waitForCondition(() -> { + scala.Tuple2 res = kafka.utils.TestUtils.grabConsoleOutputAndError(describeGroups(service)); + int expectedNumRows = DESCRIBE_TYPE_MEMBERS.contains(describeType) ? 3 : 2; + return res._2.isEmpty() && res._1.trim().split("\n").length == expectedNumRows; + }, "Expected a single data row in describe group result with describe type '" + String.join(" ", describeType) + "'"); + } + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeOffsetsWithConsumersWithoutAssignedPartitions(String quorum, String groupProtocol) throws Exception { + createOffsetsTopic(listenerName(), new Properties()); + + // run two consumers in the group consuming from a single-partition topic + addConsumerGroupExecutor(2, groupProtocol); + + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + TestUtils.waitForCondition(() -> { + scala.Tuple2, Option>> res = service.collectGroupOffsets(GROUP); + return res._1.map(s -> s.contains("Stable")).getOrElse(() -> false) && + res._2.isDefined() && + res._2.get().count(s -> Objects.equals(s.group(), GROUP)) == 1 && + res._2.get().count(x -> Objects.equals(x.group(), GROUP) && x.partition().isDefined()) == 1; + }, "Expected rows for consumers with no assigned partitions in describe group results"); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeMembersWithConsumersWithoutAssignedPartitions(String quorum, String groupProtocol) throws Exception { + createOffsetsTopic(listenerName(), new Properties()); + + // run two consumers in the group consuming from a single-partition topic + addConsumerGroupExecutor(2, groupProtocol); + + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + TestUtils.waitForCondition(() -> { + scala.Tuple2, Option>> res = service.collectGroupMembers(GROUP, false); + return res._1.map(s -> s.contains("Stable")).getOrElse(() -> false) && + res._2.isDefined() && + res._2.get().count(s -> Objects.equals(s.group(), GROUP)) == 2 && + res._2.get().count(x -> Objects.equals(x.group(), GROUP) && x.numPartitions() == 1) == 1 && + res._2.get().count(x -> Objects.equals(x.group(), GROUP) && x.numPartitions() == 0) == 1 && + res._2.get().forall(s -> s.assignment().isEmpty()); + }, "Expected rows for consumers with no assigned partitions in describe group results"); + + scala.Tuple2, Option>> res = service.collectGroupMembers(GROUP, true); + assertTrue(res._1.map(s -> s.contains("Stable")).getOrElse(() -> false) + && res._2.map(c -> c.exists(s -> !s.assignment().isEmpty())).getOrElse(() -> false), + "Expected additional columns in verbose version of describe members"); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeStateWithConsumersWithoutAssignedPartitions(String quorum, String groupProtocol) throws Exception { + createOffsetsTopic(listenerName(), new Properties()); + + // run two consumers in the group consuming from a single-partition topic + addConsumerGroupExecutor(2, groupProtocol); + + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + TestUtils.waitForCondition(() -> { + ConsumerGroupCommand.GroupState state = service.collectGroupState(GROUP); + return Objects.equals(state.state(), "Stable") && state.numMembers() == 2; + }, "Expected two consumers in describe group results"); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeWithMultiPartitionTopicAndMultipleConsumers(String quorum, String groupProtocol) throws Exception { + createOffsetsTopic(listenerName(), new Properties()); + String topic2 = "foo2"; + createTopic(topic2, 2, 1, new Properties(), listenerName(), new Properties()); + + for (List describeType : DESCRIBE_TYPES) { + String group = GROUP + String.join("", describeType); + // run two consumers in the group consuming from a two-partition topic + addConsumerGroupExecutor(2, topic2, group, groupProtocol); + List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group)); + cgcArgs.addAll(describeType); + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0])); + + TestUtils.waitForCondition(() -> { + scala.Tuple2 res = kafka.utils.TestUtils.grabConsoleOutputAndError(describeGroups(service)); + int expectedNumRows = DESCRIBE_TYPE_STATE.contains(describeType) ? 2 : 3; + return res._2.isEmpty() && res._1.trim().split("\n").length == expectedNumRows; + }, "Expected a single data row in describe group result with describe type '" + String.join(" ", describeType) + "'"); + } + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeOffsetsWithMultiPartitionTopicAndMultipleConsumers(String quorum, String groupProtocol) throws Exception { + createOffsetsTopic(listenerName(), new Properties()); + String topic2 = "foo2"; + createTopic(topic2, 2, 1, new Properties(), listenerName(), new Properties()); + + // run two consumers in the group consuming from a two-partition topic + addConsumerGroupExecutor(2, topic2, GROUP, groupProtocol); + + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + TestUtils.waitForCondition(() -> { + scala.Tuple2, Option>> res = service.collectGroupOffsets(GROUP); + return res._1.map(s -> s.contains("Stable")).getOrElse(() -> false) && + res._2.isDefined() && + res._2.get().count(s -> Objects.equals(s.group(), GROUP)) == 2 && + res._2.get().count(x -> Objects.equals(x.group(), GROUP) && x.partition().isDefined()) == 2 && + res._2.get().count(x -> Objects.equals(x.group(), GROUP) && !x.partition().isDefined()) == 0; + }, "Expected two rows (one row per consumer) in describe group results."); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeMembersWithMultiPartitionTopicAndMultipleConsumers(String quorum, String groupProtocol) throws Exception { + createOffsetsTopic(listenerName(), new Properties()); + String topic2 = "foo2"; + createTopic(topic2, 2, 1, new Properties(), listenerName(), new Properties()); + + // run two consumers in the group consuming from a two-partition topic + addConsumerGroupExecutor(2, topic2, GROUP, groupProtocol); + + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + TestUtils.waitForCondition(() -> { + scala.Tuple2, Option>> res = service.collectGroupMembers(GROUP, false); + return res._1.map(s -> s.contains("Stable")).getOrElse(() -> false) && + res._2.isDefined() && + res._2.get().count(s -> Objects.equals(s.group(), GROUP)) == 2 && + res._2.get().count(x -> Objects.equals(x.group(), GROUP) && x.numPartitions() == 1) == 2 && + res._2.get().count(x -> Objects.equals(x.group(), GROUP) && x.numPartitions() == 0) == 0; + }, "Expected two rows (one row per consumer) in describe group members results."); + + scala.Tuple2, Option>> res = service.collectGroupMembers(GROUP, true); + assertTrue(res._1.map(s -> s.contains("Stable")).getOrElse(() -> false) && res._2.map(s -> s.count(x -> x.assignment().isEmpty())).getOrElse(() -> 0) == 0, + "Expected additional columns in verbose version of describe members"); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeStateWithMultiPartitionTopicAndMultipleConsumers(String quorum, String groupProtocol) throws Exception { + createOffsetsTopic(listenerName(), new Properties()); + String topic2 = "foo2"; + createTopic(topic2, 2, 1, new Properties(), listenerName(), new Properties()); + + // run two consumers in the group consuming from a two-partition topic + addConsumerGroupExecutor(2, topic2, GROUP, groupProtocol); + + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + TestUtils.waitForCondition(() -> { + ConsumerGroupCommand.GroupState state = service.collectGroupState(GROUP); + return Objects.equals(state.state(), "Stable") && Objects.equals(state.group(), GROUP) && state.numMembers() == 2; + }, "Expected a stable group with two members in describe group state result."); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME) + @ValueSource(strings = {"zk", "kraft", "kraft+kip848"}) + public void testDescribeSimpleConsumerGroup(String quorum) throws Exception { + // Ensure that the offsets of consumers which don't use group management are still displayed + + createOffsetsTopic(listenerName(), new Properties()); + String topic2 = "foo2"; + createTopic(topic2, 2, 1, new Properties(), listenerName(), new Properties()); + addSimpleGroupExecutor(Arrays.asList(new TopicPartition(topic2, 0), new TopicPartition(topic2, 1)), GROUP); + + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + TestUtils.waitForCondition(() -> { + scala.Tuple2, Option>> res = service.collectGroupOffsets(GROUP); + return res._1.map(s -> s.contains("Empty")).getOrElse(() -> false) + && res._2.isDefined() && res._2.get().count(s -> Objects.equals(s.group(), GROUP)) == 2; + }, "Expected a stable group with two members in describe group state result."); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeGroupWithShortInitializationTimeout(String quorum, String groupProtocol) { + // Let creation of the offsets topic happen during group initialization to ensure that initialization doesn't + // complete before the timeout expires + + List describeType = DESCRIBE_TYPES.get(RANDOM.nextInt(DESCRIBE_TYPES.size())); + String group = GROUP + String.join("", describeType); + // run one consumer in the group consuming from a single-partition topic + addConsumerGroupExecutor(1, groupProtocol); + // set the group initialization timeout too low for the group to stabilize + List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--timeout", "1", "--group", group)); + cgcArgs.addAll(describeType); + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0])); + + ExecutionException e = assertThrows(ExecutionException.class, service::describeGroups); + assertInstanceOf(TimeoutException.class, e.getCause()); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeGroupOffsetsWithShortInitializationTimeout(String quorum, String groupProtocol) { + // Let creation of the offsets topic happen during group initialization to ensure that initialization doesn't + // complete before the timeout expires + + // run one consumer in the group consuming from a single-partition topic + addConsumerGroupExecutor(1, groupProtocol); + + // set the group initialization timeout too low for the group to stabilize + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP, "--timeout", "1"}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + Throwable e = assertThrows(ExecutionException.class, () -> service.collectGroupOffsets(GROUP)); + assertEquals(TimeoutException.class, e.getCause().getClass()); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeGroupMembersWithShortInitializationTimeout(String quorum, String groupProtocol) { + // Let creation of the offsets topic happen during group initialization to ensure that initialization doesn't + // complete before the timeout expires + + // run one consumer in the group consuming from a single-partition topic + addConsumerGroupExecutor(1, groupProtocol); + + // set the group initialization timeout too low for the group to stabilize + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP, "--timeout", "1"}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + Throwable e = assertThrows(ExecutionException.class, () -> service.collectGroupMembers(GROUP, false)); + assertEquals(TimeoutException.class, e.getCause().getClass()); + e = assertThrows(ExecutionException.class, () -> service.collectGroupMembers(GROUP, true)); + assertEquals(TimeoutException.class, e.getCause().getClass()); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeGroupStateWithShortInitializationTimeout(String quorum, String groupProtocol) { + // Let creation of the offsets topic happen during group initialization to ensure that initialization doesn't + // complete before the timeout expires + + // run one consumer in the group consuming from a single-partition topic + addConsumerGroupExecutor(1, groupProtocol); + + // set the group initialization timeout too low for the group to stabilize + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP, "--timeout", "1"}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + Throwable e = assertThrows(ExecutionException.class, () -> service.collectGroupState(GROUP)); + assertEquals(TimeoutException.class, e.getCause().getClass()); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME) + @ValueSource(strings = {"zk", "kraft"}) + public void testDescribeWithUnrecognizedNewConsumerOption(String quorum) { + String[] cgcArgs = new String[]{"--new-consumer", "--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP}; + assertThrows(joptsimple.OptionException.class, () -> getConsumerGroupService(cgcArgs)); + } + + @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) + @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"}) + public void testDescribeNonOffsetCommitGroup(String quorum, String groupProtocol) throws Exception { + createOffsetsTopic(listenerName(), new Properties()); + + Properties customProps = new Properties(); + // create a consumer group that never commits offsets + customProps.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + // run one consumer in the group consuming from a single-partition topic + addConsumerGroupExecutor(1, TOPIC, GROUP, RangeAssignor.class.getName(), Optional.empty(), Optional.of(customProps), false, groupProtocol); + + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP}; + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + + TestUtils.waitForCondition(() -> { + scala.Tuple2, Option>> groupOffsets = service.collectGroupOffsets(GROUP); + + Function1 isGrp = s -> Objects.equals(s.group(), GROUP); + + boolean res = groupOffsets._1.map(s -> s.contains("Stable")).getOrElse(() -> false) && + groupOffsets._2.isDefined() && + groupOffsets._2.get().count(isGrp) == 1; + + if (!res) + return false; + + @SuppressWarnings("cast") + ConsumerGroupCommand.PartitionAssignmentState assignmentState = + (ConsumerGroupCommand.PartitionAssignmentState) groupOffsets._2.get().filter(isGrp).head(); + + return assignmentState.consumerId().map(c -> !c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE())).getOrElse(() -> false) && + assignmentState.clientId().map(c -> !c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE())).getOrElse(() -> false) && + assignmentState.host().map(h -> !h.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE())).getOrElse(() -> false); + }, "Expected a 'Stable' group status, rows and valid values for consumer id / client id / host columns in describe results for non-offset-committing group " + GROUP + "."); + } + + private Function0 describeGroups(ConsumerGroupCommand.ConsumerGroupService service) { + return () -> { + try { + service.describeGroups(); + return null; + } catch (Exception e) { + throw new RuntimeException(e); + } + }; + } +} From ba0db81e5307cf090dc5876f3c61ddbe5fef2284 Mon Sep 17 00:00:00 2001 From: Dmitry Werner Date: Thu, 7 Mar 2024 13:39:16 +0500 Subject: [PATCH 037/521] KAFKA-16246: Cleanups in ConsoleConsumer (#15457) Reviewers: Mickael Maison , Omnia Ibrahim --- .../kafka/tools/consumer/ConsoleConsumer.java | 51 +++----- .../consumer/ConsoleConsumerOptions.java | 31 +++-- .../consumer/ConsoleConsumerOptionsTest.java | 52 ++++++--- .../tools/consumer/ConsoleConsumerTest.java | 110 +++++++++++------- 4 files changed, 142 insertions(+), 102 deletions(-) diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumer.java b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumer.java index f84fb88c23..bb5ab1443e 100644 --- a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumer.java +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumer.java @@ -22,8 +22,6 @@ import java.util.Iterator; import java.util.Map; import java.util.Optional; -import java.util.OptionalInt; -import java.util.OptionalLong; import java.util.concurrent.CountDownLatch; import java.util.regex.Pattern; import java.util.Collections; @@ -68,11 +66,8 @@ public static void main(String[] args) throws Exception { public static void run(ConsoleConsumerOptions opts) { messageCount = 0; - long timeoutMs = opts.timeoutMs() >= 0 ? opts.timeoutMs() : Long.MAX_VALUE; Consumer consumer = new KafkaConsumer<>(opts.consumerProps(), new ByteArrayDeserializer(), new ByteArrayDeserializer()); - ConsumerWrapper consumerWrapper = opts.partitionArg().isPresent() - ? new ConsumerWrapper(Optional.of(opts.topicArg()), opts.partitionArg(), OptionalLong.of(opts.offsetArg()), Optional.empty(), consumer, timeoutMs) - : new ConsumerWrapper(Optional.of(opts.topicArg()), OptionalInt.empty(), OptionalLong.empty(), Optional.ofNullable(opts.includedTopicsArg()), consumer, timeoutMs); + ConsumerWrapper consumerWrapper = new ConsumerWrapper(opts, consumer); addShutdownHook(consumerWrapper, opts); @@ -148,43 +143,25 @@ static boolean checkErr(PrintStream output) { } public static class ConsumerWrapper { - final Optional topic; - final OptionalInt partitionId; - final OptionalLong offset; - final Optional includedTopics; - final Consumer consumer; - final long timeoutMs; final Time time = Time.SYSTEM; + final long timeoutMs; + final Consumer consumer; Iterator> recordIter = Collections.emptyIterator(); - public ConsumerWrapper(Optional topic, - OptionalInt partitionId, - OptionalLong offset, - Optional includedTopics, - Consumer consumer, - long timeoutMs) { - this.topic = topic; - this.partitionId = partitionId; - this.offset = offset; - this.includedTopics = includedTopics; + public ConsumerWrapper(ConsoleConsumerOptions opts, Consumer consumer) { this.consumer = consumer; - this.timeoutMs = timeoutMs; - - if (topic.isPresent() && partitionId.isPresent() && offset.isPresent() && !includedTopics.isPresent()) { - seek(topic.get(), partitionId.getAsInt(), offset.getAsLong()); - } else if (topic.isPresent() && partitionId.isPresent() && !offset.isPresent() && !includedTopics.isPresent()) { - // default to latest if no offset is provided - seek(topic.get(), partitionId.getAsInt(), ListOffsetsRequest.LATEST_TIMESTAMP); - } else if (topic.isPresent() && !partitionId.isPresent() && !offset.isPresent() && !includedTopics.isPresent()) { - consumer.subscribe(Collections.singletonList(topic.get())); - } else if (!topic.isPresent() && !partitionId.isPresent() && !offset.isPresent() && includedTopics.isPresent()) { - consumer.subscribe(Pattern.compile(includedTopics.get())); + timeoutMs = opts.timeoutMs(); + Optional topic = opts.topicArg(); + + if (topic.isPresent()) { + if (opts.partitionArg().isPresent()) { + seek(topic.get(), opts.partitionArg().getAsInt(), opts.offsetArg()); + } else { + consumer.subscribe(Collections.singletonList(topic.get())); + } } else { - throw new IllegalArgumentException("An invalid combination of arguments is provided. " + - "Exactly one of 'topic' or 'include' must be provided. " + - "If 'topic' is provided, an optional 'partition' may also be provided. " + - "If 'partition' is provided, an optional 'offset' may also be provided, otherwise, consumption starts from the end of the partition."); + opts.includedTopicsArg().ifPresent(topics -> consumer.subscribe(Pattern.compile(topics))); } } diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java index a713afb2bf..aa37919515 100644 --- a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java @@ -34,7 +34,7 @@ import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.Objects; +import java.util.Optional; import java.util.OptionalInt; import java.util.Properties; import java.util.Random; @@ -55,7 +55,7 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions { private final OptionSpec messageFormatterConfigOpt; private final OptionSpec resetBeginningOpt; private final OptionSpec maxMessagesOpt; - private final OptionSpec timeoutMsOpt; + private final OptionSpec timeoutMsOpt; private final OptionSpec skipMessageOnErrorOpt; private final OptionSpec bootstrapServerOpt; private final OptionSpec keyDeserializerOpt; @@ -66,6 +66,7 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions { private final Properties consumerProps; private final long offset; + private final long timeoutMs; private final MessageFormatter formatter; public ConsoleConsumerOptions(String[] args) throws IOException { @@ -139,7 +140,7 @@ public ConsoleConsumerOptions(String[] args) throws IOException { timeoutMsOpt = parser.accepts("timeout-ms", "If specified, exit if no message is available for consumption for the specified interval.") .withRequiredArg() .describedAs("timeout_ms") - .ofType(Integer.class); + .ofType(Long.class); skipMessageOnErrorOpt = parser.accepts("skip-message-on-error", "If there is an error when processing a message, " + "skip it instead of halt."); bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED: The server(s) to connect to.") @@ -184,12 +185,13 @@ public ConsoleConsumerOptions(String[] args) throws IOException { Set groupIdsProvided = checkConsumerGroup(consumerPropsFromFile, extraConsumerProps); consumerProps = buildConsumerProps(consumerPropsFromFile, extraConsumerProps, groupIdsProvided); offset = parseOffset(); + timeoutMs = parseTimeoutMs(); formatter = buildFormatter(); } private void checkRequiredArgs() { - List topicOrFilterArgs = new ArrayList<>(Arrays.asList(topicArg(), includedTopicsArg())); - topicOrFilterArgs.removeIf(Objects::isNull); + List> topicOrFilterArgs = new ArrayList<>(Arrays.asList(topicArg(), includedTopicsArg())); + topicOrFilterArgs.removeIf(arg -> !arg.isPresent()); // user need to specify value for either --topic or one of the include filters options (--include or --whitelist) if (topicOrFilterArgs.size() != 1) { CommandLineUtils.printUsageAndExit(parser, "Exactly one of --include/--topic is required. " + @@ -322,6 +324,11 @@ private void invalidOffset(String offset) { "'earliest', 'latest', or a non-negative long."); } + private long parseTimeoutMs() { + long timeout = options.has(timeoutMsOpt) ? options.valueOf(timeoutMsOpt) : -1; + return timeout >= 0 ? timeout : Long.MAX_VALUE; + } + private MessageFormatter buildFormatter() { MessageFormatter formatter = null; try { @@ -365,16 +372,16 @@ OptionalInt partitionArg() { return OptionalInt.empty(); } - String topicArg() { - return options.valueOf(topicOpt); + Optional topicArg() { + return options.has(topicOpt) ? Optional.of(options.valueOf(topicOpt)) : Optional.empty(); } int maxMessages() { return options.has(maxMessagesOpt) ? options.valueOf(maxMessagesOpt) : -1; } - int timeoutMs() { - return options.has(timeoutMsOpt) ? options.valueOf(timeoutMsOpt) : -1; + long timeoutMs() { + return timeoutMs; } boolean enableSystestEventsLogging() { @@ -385,10 +392,10 @@ String bootstrapServer() { return options.valueOf(bootstrapServerOpt); } - String includedTopicsArg() { + Optional includedTopicsArg() { return options.has(includeOpt) - ? options.valueOf(includeOpt) - : options.valueOf(whitelistOpt); + ? Optional.of(options.valueOf(includeOpt)) + : Optional.ofNullable(options.valueOf(whitelistOpt)); } Properties formatterArgs() throws IOException { diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java index 523122c4cd..3242b642cd 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java @@ -48,12 +48,12 @@ public void shouldParseValidConsumerValidConfig() throws IOException { ConsoleConsumerOptions config = new ConsoleConsumerOptions(args); assertEquals("localhost:9092", config.bootstrapServer()); - assertEquals("test", config.topicArg()); + assertEquals("test", config.topicArg().orElse("")); assertTrue(config.fromBeginning()); assertFalse(config.enableSystestEventsLogging()); assertFalse(config.skipMessageOnError()); assertEquals(-1, config.maxMessages()); - assertEquals(-1, config.timeoutMs()); + assertEquals(Long.MAX_VALUE, config.timeoutMs()); } @Test @@ -67,7 +67,7 @@ public void shouldParseIncludeArgument() throws IOException { ConsoleConsumerOptions config = new ConsoleConsumerOptions(args); assertEquals("localhost:9092", config.bootstrapServer()); - assertEquals("includeTest*", config.includedTopicsArg()); + assertEquals("includeTest*", config.includedTopicsArg().orElse("")); assertTrue(config.fromBeginning()); } @@ -82,7 +82,7 @@ public void shouldParseWhitelistArgument() throws IOException { ConsoleConsumerOptions config = new ConsoleConsumerOptions(args); assertEquals("localhost:9092", config.bootstrapServer()); - assertEquals("whitelistTest*", config.includedTopicsArg()); + assertEquals("whitelistTest*", config.includedTopicsArg().orElse("")); assertTrue(config.fromBeginning()); } @@ -96,7 +96,7 @@ public void shouldIgnoreWhitelistArgumentIfIncludeSpecified() throws IOException }; ConsoleConsumerOptions config = new ConsoleConsumerOptions(args); assertEquals("localhost:9092", config.bootstrapServer()); - assertEquals("includeTest*", config.includedTopicsArg()); + assertEquals("includeTest*", config.includedTopicsArg().orElse("")); assertTrue(config.fromBeginning()); } @@ -112,7 +112,7 @@ public void shouldParseValidSimpleConsumerValidConfigWithNumericOffset() throws ConsoleConsumerOptions config = new ConsoleConsumerOptions(args); assertEquals("localhost:9092", config.bootstrapServer()); - assertEquals("test", config.topicArg()); + assertEquals("test", config.topicArg().orElse("")); assertTrue(config.partitionArg().isPresent()); assertEquals(0, config.partitionArg().getAsInt()); assertEquals(3, config.offsetArg()); @@ -191,7 +191,7 @@ public void shouldParseValidSimpleConsumerValidConfigWithStringOffset() throws E ConsoleConsumerOptions config = new ConsoleConsumerOptions(args); assertEquals("localhost:9092", config.bootstrapServer()); - assertEquals("test", config.topicArg()); + assertEquals("test", config.topicArg().orElse("")); assertTrue(config.partitionArg().isPresent()); assertEquals(0, config.partitionArg().getAsInt()); assertEquals(-1, config.offsetArg()); @@ -211,7 +211,7 @@ public void shouldParseValidConsumerConfigWithAutoOffsetResetLatest() throws IOE Properties consumerProperties = config.consumerProps(); assertEquals("localhost:9092", config.bootstrapServer()); - assertEquals("test", config.topicArg()); + assertEquals("test", config.topicArg().orElse("")); assertFalse(config.fromBeginning()); assertEquals("latest", consumerProperties.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)); } @@ -228,7 +228,7 @@ public void shouldParseValidConsumerConfigWithAutoOffsetResetEarliest() throws I Properties consumerProperties = config.consumerProps(); assertEquals("localhost:9092", config.bootstrapServer()); - assertEquals("test", config.topicArg()); + assertEquals("test", config.topicArg().orElse("")); assertFalse(config.fromBeginning()); assertEquals("earliest", consumerProperties.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)); } @@ -246,7 +246,7 @@ public void shouldParseValidConsumerConfigWithAutoOffsetResetAndMatchingFromBegi Properties consumerProperties = config.consumerProps(); assertEquals("localhost:9092", config.bootstrapServer()); - assertEquals("test", config.topicArg()); + assertEquals("test", config.topicArg().orElse("")); assertTrue(config.fromBeginning()); assertEquals("earliest", consumerProperties.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)); } @@ -262,7 +262,7 @@ public void shouldParseValidConsumerConfigWithNoOffsetReset() throws IOException Properties consumerProperties = config.consumerProps(); assertEquals("localhost:9092", config.bootstrapServer()); - assertEquals("test", config.topicArg()); + assertEquals("test", config.topicArg().orElse("")); assertFalse(config.fromBeginning()); assertEquals("latest", consumerProperties.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)); } @@ -442,7 +442,7 @@ public void shouldParseGroupIdFromBeginningGivenTogether() throws IOException { ConsoleConsumerOptions config = new ConsoleConsumerOptions(args); assertEquals("localhost:9092", config.bootstrapServer()); - assertEquals("test", config.topicArg()); + assertEquals("test", config.topicArg().orElse("")); assertEquals(-2, config.offsetArg()); assertTrue(config.fromBeginning()); @@ -455,7 +455,7 @@ public void shouldParseGroupIdFromBeginningGivenTogether() throws IOException { config = new ConsoleConsumerOptions(args); assertEquals("localhost:9092", config.bootstrapServer()); - assertEquals("test", config.topicArg()); + assertEquals("test", config.topicArg().orElse("")); assertEquals(-1, config.offsetArg()); assertFalse(config.fromBeginning()); } @@ -618,4 +618,30 @@ public void testParseOffset() throws Exception { Exit.resetExitProcedure(); } } + + @Test + public void testParseTimeoutMs() throws Exception { + String[] withoutTimeoutMs = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--partition", "0" + }; + assertEquals(Long.MAX_VALUE, new ConsoleConsumerOptions(withoutTimeoutMs).timeoutMs()); + + String[] negativeTimeoutMs = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--partition", "0", + "--timeout-ms", "-100" + }; + assertEquals(Long.MAX_VALUE, new ConsoleConsumerOptions(negativeTimeoutMs).timeoutMs()); + + String[] validTimeoutMs = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--partition", "0", + "--timeout-ms", "100" + }; + assertEquals(100, new ConsoleConsumerOptions(validTimeoutMs).timeoutMs()); + } } diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java index 008893f9c5..f4fa6ac3be 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java @@ -24,21 +24,19 @@ import org.apache.kafka.common.MessageFormatter; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.TimeoutException; -import org.apache.kafka.common.requests.ListOffsetsRequest; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.util.MockTime; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.io.IOException; import java.io.PrintStream; import java.time.Duration; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.Map; -import java.util.Optional; -import java.util.OptionalInt; -import java.util.OptionalLong; +import java.util.regex.Pattern; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -58,8 +56,7 @@ public void setup() { } @Test - public void shouldThrowTimeoutExceptionWhenTimeoutIsReached() { - String topic = "test"; + public void shouldThrowTimeoutExceptionWhenTimeoutIsReached() throws IOException { final Time time = new MockTime(); final int timeoutMs = 1000; @@ -71,20 +68,22 @@ public void shouldThrowTimeoutExceptionWhenTimeoutIsReached() { return ConsumerRecords.EMPTY; }); + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--timeout-ms", String.valueOf(timeoutMs) + }; + ConsoleConsumer.ConsumerWrapper consumer = new ConsoleConsumer.ConsumerWrapper( - Optional.of(topic), - OptionalInt.empty(), - OptionalLong.empty(), - Optional.empty(), - mockConsumer, - timeoutMs + new ConsoleConsumerOptions(args), + mockConsumer ); assertThrows(TimeoutException.class, consumer::receive); } @Test - public void shouldResetUnConsumedOffsetsBeforeExit() { + public void shouldResetUnConsumedOffsetsBeforeExit() throws IOException { String topic = "test"; int maxMessages = 123; int totalMessages = 700; @@ -94,13 +93,16 @@ public void shouldResetUnConsumedOffsetsBeforeExit() { TopicPartition tp1 = new TopicPartition(topic, 0); TopicPartition tp2 = new TopicPartition(topic, 1); + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", topic, + "--timeout-ms", "1000" + }; + ConsoleConsumer.ConsumerWrapper consumer = new ConsoleConsumer.ConsumerWrapper( - Optional.of(topic), - OptionalInt.empty(), - OptionalLong.empty(), - Optional.empty(), - mockConsumer, - 1000L); + new ConsoleConsumerOptions(args), + mockConsumer + ); mockConsumer.rebalance(Arrays.asList(tp1, tp2)); Map offsets = new HashMap<>(); @@ -165,47 +167,75 @@ public void shouldStopWhenOutputCheckErrorFails() { @Test @SuppressWarnings("unchecked") - public void shouldSeekWhenOffsetIsSet() { + public void shouldSeekWhenOffsetIsSet() throws IOException { Consumer mockConsumer = mock(Consumer.class); TopicPartition tp0 = new TopicPartition("test", 0); + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", tp0.topic(), + "--partition", String.valueOf(tp0.partition()), + "--timeout-ms", "1000" + }; + ConsoleConsumer.ConsumerWrapper consumer = new ConsoleConsumer.ConsumerWrapper( - Optional.of(tp0.topic()), - OptionalInt.of(tp0.partition()), - OptionalLong.empty(), - Optional.empty(), - mockConsumer, - 1000L); + new ConsoleConsumerOptions(args), + mockConsumer + ); verify(mockConsumer).assign(eq(Collections.singletonList(tp0))); verify(mockConsumer).seekToEnd(eq(Collections.singletonList(tp0))); consumer.cleanup(); reset(mockConsumer); - consumer = new ConsoleConsumer.ConsumerWrapper( - Optional.of(tp0.topic()), - OptionalInt.of(tp0.partition()), - OptionalLong.of(123L), - Optional.empty(), - mockConsumer, - 1000L); + args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", tp0.topic(), + "--partition", String.valueOf(tp0.partition()), + "--offset", "123", + "--timeout-ms", "1000" + }; + + consumer = new ConsoleConsumer.ConsumerWrapper(new ConsoleConsumerOptions(args), mockConsumer); verify(mockConsumer).assign(eq(Collections.singletonList(tp0))); verify(mockConsumer).seek(eq(tp0), eq(123L)); consumer.cleanup(); reset(mockConsumer); - consumer = new ConsoleConsumer.ConsumerWrapper( - Optional.of(tp0.topic()), - OptionalInt.of(tp0.partition()), - OptionalLong.of(ListOffsetsRequest.EARLIEST_TIMESTAMP), - Optional.empty(), - mockConsumer, - 1000L); + args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", tp0.topic(), + "--partition", String.valueOf(tp0.partition()), + "--offset", "earliest", + "--timeout-ms", "1000" + }; + + consumer = new ConsoleConsumer.ConsumerWrapper(new ConsoleConsumerOptions(args), mockConsumer); verify(mockConsumer).assign(eq(Collections.singletonList(tp0))); verify(mockConsumer).seekToBeginning(eq(Collections.singletonList(tp0))); consumer.cleanup(); reset(mockConsumer); } + + @Test + @SuppressWarnings("unchecked") + public void shouldWorkWithoutTopicOption() throws IOException { + Consumer mockConsumer = mock(Consumer.class); + + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--include", "includeTest*", + "--from-beginning" + }; + + ConsoleConsumer.ConsumerWrapper consumer = new ConsoleConsumer.ConsumerWrapper( + new ConsoleConsumerOptions(args), + mockConsumer + ); + + verify(mockConsumer).subscribe(any(Pattern.class)); + consumer.cleanup(); + } } From a33c47ea4ddc810d66b6ed17ab74e40c5b7668fb Mon Sep 17 00:00:00 2001 From: Christo Lolov Date: Thu, 7 Mar 2024 09:33:31 +0000 Subject: [PATCH 038/521] KAFKA-14133: Move consumer mock in TaskManagerTest to Mockito - part 2 (#15261) The previous pull request in this series was #15112. This pull request continues the migration of the consumer mock in TaskManagerTest test by test for easier reviews. I envision there will be at least 1 more pull request to clean things up. For example, all calls to taskManager.setMainConsumer should be removed. Reviewer: Bruno Cadonna --- .../processor/internals/TaskManagerTest.java | 379 ++++++++---------- 1 file changed, 177 insertions(+), 202 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java index ba1c91e7f7..681e69d300 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java @@ -58,7 +58,6 @@ import java.time.Duration; import java.util.ArrayList; -import org.easymock.EasyMock; import org.easymock.EasyMockRunner; import org.easymock.Mock; import org.easymock.MockType; @@ -112,7 +111,6 @@ import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.expectLastCall; import static org.easymock.EasyMock.replay; -import static org.easymock.EasyMock.reset; import static org.easymock.EasyMock.verify; import static org.hamcrest.CoreMatchers.hasItem; import static org.hamcrest.MatcherAssert.assertThat; @@ -133,6 +131,7 @@ import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.inOrder; +import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.when; @@ -185,6 +184,7 @@ public class TaskManagerTest { private final TaskId taskId10 = new TaskId(1, 0); private final TopicPartition t2p0 = new TopicPartition(topic2, 0); private final Set taskId10Partitions = mkSet(t2p0); + private final Set assignment = singleton(new TopicPartition("assignment", 0)); final java.util.function.Consumer> noOpResetter = partitions -> { }; @@ -2016,13 +2016,12 @@ public void shouldReleaseLockForUnassignedTasksAfterRebalance() throws Exception assertThat(taskManager.lockedTaskDirectories(), is(mkSet(taskId00, taskId01, taskId02))); handleAssignment(taskId00Assignment, taskId01Assignment, emptyMap()); - reset(consumer); - expectConsumerAssignmentPaused(consumer); - replay(consumer); taskManager.handleRebalanceComplete(); assertThat(taskManager.lockedTaskDirectories(), is(mkSet(taskId00, taskId01))); verify(stateDirectory); + + Mockito.verify(mockitoConsumer).pause(assignment); } @Test @@ -2332,19 +2331,11 @@ public void shouldCloseActiveUnassignedSuspendedTasksWhenClosingRevokedTasks() { task00.setCommittableOffsetsAndMetadata(offsets); // first `handleAssignment` - expectRestoreToBeCompleted(consumer); - when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00)); - expectLastCall(); - - // `handleRevocation` - consumer.commitSync(offsets); - expectLastCall(); + when(mockitoConsumer.assignment()).thenReturn(assignment); - // second `handleAssignment` - consumer.commitSync(offsets); - expectLastCall(); + when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00)); - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(taskId00Assignment, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -2369,12 +2360,9 @@ public void closeClean() { } }; - // first `handleAssignment` - expectRestoreToBeCompleted(consumer); when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00)); - expectLastCall(); - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(taskId00Assignment, emptyMap()); taskManager.handleRevocation(taskId00Partitions); @@ -2399,7 +2387,7 @@ public void shouldCloseActiveTasksWhenHandlingLostTasks() throws Exception { final StateMachineTask task01 = new StateMachineTask(taskId01, taskId01Partitions, false, stateManager); // `handleAssignment` - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00)); when(standbyTaskCreator.createTasks(taskId01Assignment)).thenReturn(singletonList(task01)); @@ -2412,11 +2400,11 @@ public void shouldCloseActiveTasksWhenHandlingLostTasks() throws Exception { expectLockObtainedFor(); replay(stateDirectory); + taskManager.setMainConsumer(mockitoConsumer); + taskManager.handleRebalanceStart(emptySet()); assertThat(taskManager.lockedTaskDirectories(), Matchers.is(mkSet(taskId00, taskId01))); - replay(consumer); - taskManager.handleAssignment(taskId00Assignment, taskId01Assignment); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); assertThat(task00.state(), is(Task.State.RUNNING)); @@ -2455,15 +2443,13 @@ public void shouldThrowWhenHandlingClosingTasksOnProducerCloseError() { task00.setCommittableOffsetsAndMetadata(offsets); // `handleAssignment` - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00)); // `handleAssignment` - consumer.commitSync(offsets); - expectLastCall(); doThrow(new RuntimeException("KABOOM!")).when(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(taskId00); - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(taskId00Assignment, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -2529,10 +2515,12 @@ public void postCommit(final boolean enforceCheckpoint) { }; // `handleAssignment` - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()) + .thenReturn(assignment) + .thenReturn(taskId00Partitions); when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00)); - expect(consumer.assignment()).andReturn(taskId00Partitions); - replay(consumer); + + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(taskId00Assignment, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), tp -> assertThat(tp, is(empty()))), is(true)); @@ -2548,7 +2536,6 @@ public void postCommit(final boolean enforceCheckpoint) { assertThat(taskManager.activeTaskMap(), is(singletonMap(taskId00, task00))); assertThat(taskManager.standbyTaskMap(), Matchers.anEmptyMap()); - verify(consumer); Mockito.verify(stateManager).markChangelogAsCorrupted(taskId00Partitions); } @@ -2564,10 +2551,12 @@ public void suspend() { } }; - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()) + .thenReturn(assignment) + .thenReturn(taskId00Partitions); when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00)); - expect(consumer.assignment()).andReturn(taskId00Partitions); - replay(consumer); + + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(taskId00Assignment, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), tp -> assertThat(tp, is(empty()))), is(true)); @@ -2581,7 +2570,6 @@ public void suspend() { assertThat(taskManager.activeTaskMap(), is(singletonMap(taskId00, task00))); assertThat(taskManager.standbyTaskMap(), Matchers.anEmptyMap()); - verify(consumer); Mockito.verify(stateManager).markChangelogAsCorrupted(taskId00Partitions); } @@ -2592,20 +2580,20 @@ public void shouldCommitNonCorruptedTasksOnTaskCorruptedException() { final StateMachineTask corruptedTask = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager); final StateMachineTask nonCorruptedTask = new StateMachineTask(taskId01, taskId01Partitions, true, stateManager); - final Map> assignment = new HashMap<>(taskId00Assignment); - assignment.putAll(taskId01Assignment); + final Map> firstAssignment = new HashMap<>(taskId00Assignment); + firstAssignment.putAll(taskId01Assignment); // `handleAssignment` - when(activeTaskCreator.createTasks(any(), Mockito.eq(assignment))) + when(activeTaskCreator.createTasks(any(), Mockito.eq(firstAssignment))) .thenReturn(asList(corruptedTask, nonCorruptedTask)); - expectRestoreToBeCompleted(consumer); - expect(consumer.assignment()).andReturn(taskId00Partitions); - // check that we should not commit empty map either - consumer.commitSync(eq(emptyMap())); - expectLastCall().andStubThrow(new AssertionError("should not invoke commitSync when offset map is empty")); - replay(consumer); - taskManager.handleAssignment(assignment, emptyMap()); + when(mockitoConsumer.assignment()) + .thenReturn(assignment) + .thenReturn(taskId00Partitions); + + taskManager.setMainConsumer(mockitoConsumer); + + taskManager.handleAssignment(firstAssignment, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), tp -> assertThat(tp, is(empty()))), is(true)); assertThat(nonCorruptedTask.state(), is(Task.State.RUNNING)); @@ -2618,7 +2606,8 @@ public void shouldCommitNonCorruptedTasksOnTaskCorruptedException() { assertThat(nonCorruptedTask.partitionsForOffsetReset, equalTo(Collections.emptySet())); assertThat(corruptedTask.partitionsForOffsetReset, equalTo(taskId00Partitions)); - verify(consumer); + // check that we should not commit empty map either + Mockito.verify(mockitoConsumer, never()).commitSync(emptyMap()); Mockito.verify(stateManager).markChangelogAsCorrupted(taskId00Partitions); } @@ -2637,8 +2626,9 @@ public void shouldNotCommitNonRunningNonCorruptedTasks() { // `handleAssignment` when(activeTaskCreator.createTasks(any(), Mockito.eq(assignment))) .thenReturn(asList(corruptedTask, nonRunningNonCorruptedTask)); - expect(consumer.assignment()).andReturn(taskId00Partitions); - replay(consumer); + when(mockitoConsumer.assignment()).thenReturn(taskId00Partitions); + + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(assignment, emptyMap()); @@ -2650,7 +2640,6 @@ public void shouldNotCommitNonRunningNonCorruptedTasks() { assertThat(corruptedTask.partitionsForOffsetReset, equalTo(taskId00Partitions)); assertFalse(nonRunningNonCorruptedTask.commitPrepared); - verify(consumer); Mockito.verify(stateManager).markChangelogAsCorrupted(taskId00Partitions); } @@ -2732,9 +2721,9 @@ public Map prepareCommit() { .thenReturn(singleton(runningNonCorruptedActive)); when(standbyTaskCreator.createTasks(taskId00Assignment)).thenReturn(singleton(corruptedStandby)); - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(taskId01Assignment, taskId00Assignment); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -2751,7 +2740,6 @@ public Map prepareCommit() { assertThat(corruptedStandby.commitPrepared, is(true)); assertThat(corruptedStandby.state(), is(Task.State.CREATED)); - verify(consumer); Mockito.verify(stateManager).markChangelogAsCorrupted(taskId00Partitions); } @@ -2768,21 +2756,23 @@ public void shouldNotAttemptToCommitInHandleCorruptedDuringARebalance() { uncorruptedActive.setCommitNeeded(); // handleAssignment - final Map> assignment = new HashMap<>(); - assignment.putAll(taskId00Assignment); - assignment.putAll(taskId01Assignment); - when(activeTaskCreator.createTasks(any(), Mockito.eq(assignment))) + final Map> firstAssignement = new HashMap<>(); + firstAssignement.putAll(taskId00Assignment); + firstAssignement.putAll(taskId01Assignment); + when(activeTaskCreator.createTasks(any(), Mockito.eq(firstAssignement))) .thenReturn(asList(corruptedActive, uncorruptedActive)); - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()) + .thenReturn(assignment) + .thenReturn(union(HashSet::new, taskId00Partitions, taskId01Partitions)); - expect(consumer.assignment()).andStubReturn(union(HashSet::new, taskId00Partitions, taskId01Partitions)); - - replay(consumer, stateDirectory); + replay(stateDirectory); uncorruptedActive.setCommittableOffsetsAndMetadata(offsets); - taskManager.handleAssignment(assignment, emptyMap()); + taskManager.setMainConsumer(mockitoConsumer); + + taskManager.handleAssignment(firstAssignement, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); assertThat(uncorruptedActive.state(), is(Task.State.RUNNING)); @@ -2800,7 +2790,6 @@ public void shouldNotAttemptToCommitInHandleCorruptedDuringARebalance() { assertThat(uncorruptedActive.commitCompleted, is(false)); assertThat(uncorruptedActive.state(), is(State.RUNNING)); - verify(consumer); } @Test @@ -2818,22 +2807,21 @@ public void markChangelogAsCorrupted(final Collection partitions uncorruptedActive.setCommittableOffsetsAndMetadata(offsets); // handleAssignment - final Map> assignment = new HashMap<>(); - assignment.putAll(taskId00Assignment); - assignment.putAll(taskId01Assignment); - when(activeTaskCreator.createTasks(any(), Mockito.eq(assignment))) + final Map> firstAssignment = new HashMap<>(); + firstAssignment.putAll(taskId00Assignment); + firstAssignment.putAll(taskId01Assignment); + when(activeTaskCreator.createTasks(any(), Mockito.eq(firstAssignment))) .thenReturn(asList(corruptedActive, uncorruptedActive)); - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()) + .thenReturn(assignment) + .thenReturn(union(HashSet::new, taskId00Partitions, taskId01Partitions)); - consumer.commitSync(offsets); - expectLastCall().andThrow(new TimeoutException()); + doThrow(new TimeoutException()).when(mockitoConsumer).commitSync(offsets); - expect(consumer.assignment()).andStubReturn(union(HashSet::new, taskId00Partitions, taskId01Partitions)); - - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); - taskManager.handleAssignment(assignment, emptyMap()); + taskManager.handleAssignment(firstAssignment, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); assertThat(uncorruptedActive.state(), is(Task.State.RUNNING)); @@ -2861,7 +2849,6 @@ public void markChangelogAsCorrupted(final Collection partitions assertThat(corruptedActive.state(), is(Task.State.CREATED)); assertThat(uncorruptedActive.state(), is(Task.State.CREATED)); - verify(consumer); Mockito.verify(stateManager).markChangelogAsCorrupted(taskId00Partitions); } @@ -2893,24 +2880,24 @@ public void markChangelogAsCorrupted(final Collection partitions uncorruptedActiveTask.setCommittableOffsetsAndMetadata(offsets); // handleAssignment - final Map> assignment = new HashMap<>(); - assignment.putAll(taskId00Assignment); - assignment.putAll(taskId01Assignment); - when(activeTaskCreator.createTasks(any(), Mockito.eq(assignment))) + final Map> firstAssignment = new HashMap<>(); + firstAssignment.putAll(taskId00Assignment); + firstAssignment.putAll(taskId01Assignment); + when(activeTaskCreator.createTasks(any(), Mockito.eq(firstAssignment))) .thenReturn(asList(corruptedActiveTask, uncorruptedActiveTask)); - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()) + .thenReturn(assignment) + .thenReturn(union(HashSet::new, taskId00Partitions, taskId01Partitions)); final ConsumerGroupMetadata groupMetadata = new ConsumerGroupMetadata("appId"); - expect(consumer.groupMetadata()).andReturn(groupMetadata); + when(mockitoConsumer.groupMetadata()).thenReturn(groupMetadata); doThrow(new TimeoutException()).when(producer).commitTransaction(offsets, groupMetadata); - expect(consumer.assignment()).andStubReturn(union(HashSet::new, taskId00Partitions, taskId01Partitions)); - - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); - taskManager.handleAssignment(assignment, emptyMap()); + taskManager.handleAssignment(firstAssignment, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); assertThat(uncorruptedActiveTask.state(), is(Task.State.RUNNING)); @@ -2944,7 +2931,6 @@ public void markChangelogAsCorrupted(final Collection partitions assertThat(uncorruptedActiveTask.state(), is(Task.State.CREATED)); assertThat(corruptedTaskChangelogMarkedAsCorrupted.get(), is(true)); assertThat(uncorruptedTaskChangelogMarkedAsCorrupted.get(), is(true)); - verify(consumer); Mockito.verify(stateManager).markChangelogAsCorrupted(taskId00ChangelogPartitions); Mockito.verify(stateManager).markChangelogAsCorrupted(taskId01ChangelogPartitions); } @@ -2978,16 +2964,16 @@ public void markChangelogAsCorrupted(final Collection partitions mkEntry(taskId02, taskId02Partitions) ); - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()) + .thenReturn(assignment) + .thenReturn(union(HashSet::new, taskId00Partitions, taskId01Partitions, taskId02Partitions)); when(activeTaskCreator.createTasks(any(), Mockito.eq(assignmentActive))) .thenReturn(asList(revokedActiveTask, unrevokedActiveTaskWithCommitNeeded, unrevokedActiveTaskWithoutCommitNeeded)); - expectLastCall(); - consumer.commitSync(expectedCommittedOffsets); - expectLastCall().andThrow(new TimeoutException()); - expect(consumer.assignment()).andStubReturn(union(HashSet::new, taskId00Partitions, taskId01Partitions, taskId02Partitions)); - replay(consumer); + doThrow(new TimeoutException()).when(mockitoConsumer).commitSync(expectedCommittedOffsets); + + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(assignmentActive, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -3038,19 +3024,19 @@ public void markChangelogAsCorrupted(final Collection partitions mkEntry(taskId02, taskId02Partitions) ); - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()) + .thenReturn(assignment) + .thenReturn(union(HashSet::new, taskId00Partitions, taskId01Partitions, taskId02Partitions)); when(activeTaskCreator.createTasks(any(), Mockito.eq(assignmentActive))) .thenReturn(asList(revokedActiveTask, unrevokedActiveTask, unrevokedActiveTaskWithoutCommitNeeded)); final ConsumerGroupMetadata groupMetadata = new ConsumerGroupMetadata("appId"); - expect(consumer.groupMetadata()).andReturn(groupMetadata); + when(mockitoConsumer.groupMetadata()).thenReturn(groupMetadata); doThrow(new TimeoutException()).when(producer).commitTransaction(expectedCommittedOffsets, groupMetadata); - expect(consumer.assignment()).andStubReturn(union(HashSet::new, taskId00Partitions, taskId01Partitions, taskId02Partitions)); - - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(assignmentActive, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -3077,11 +3063,10 @@ public void markChangelogAsCorrupted(final Collection partitions public void shouldCloseStandbyUnassignedTasksWhenCreatingNewTasks() { final Task task00 = new StateMachineTask(taskId00, taskId00Partitions, false, stateManager); - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(standbyTaskCreator.createTasks(taskId00Assignment)).thenReturn(singletonList(task00)); - consumer.commitSync(Collections.emptyMap()); - expectLastCall(); - replay(consumer); + + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(emptyMap(), taskId00Assignment); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -3098,12 +3083,11 @@ public void shouldAddNonResumedSuspendedTasks() { final Task task00 = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager); final Task task01 = new StateMachineTask(taskId01, taskId01Partitions, false, stateManager); - expectRestoreToBeCompleted(consumer); - // expect these calls twice (because we're going to tryToCompleteRestoration twice) - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00)); when(standbyTaskCreator.createTasks(taskId01Assignment)).thenReturn(singletonList(task01)); - replay(consumer); + + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(taskId00Assignment, taskId01Assignment); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -3115,18 +3099,20 @@ public void shouldAddNonResumedSuspendedTasks() { assertThat(task00.state(), is(Task.State.RUNNING)); assertThat(task01.state(), is(Task.State.RUNNING)); + // expect these calls twice (because we're going to tryToCompleteRestoration twice) Mockito.verify(activeTaskCreator).createTasks(any(), Mockito.eq(emptyMap())); + Mockito.verify(mockitoConsumer, times(2)).assignment(); + Mockito.verify(mockitoConsumer, times(2)).resume(assignment); } @Test public void shouldUpdateInputPartitionsAfterRebalance() { final Task task00 = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager); - expectRestoreToBeCompleted(consumer); - // expect these calls twice (because we're going to tryToCompleteRestoration twice) - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00)); - replay(consumer); + + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(taskId00Assignment, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -3138,7 +3124,9 @@ public void shouldUpdateInputPartitionsAfterRebalance() { assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); assertThat(task00.state(), is(Task.State.RUNNING)); assertEquals(newPartitionsSet, task00.inputPartitions()); - verify(consumer); + // expect these calls twice (because we're going to tryToCompleteRestoration twice) + Mockito.verify(mockitoConsumer, times(2)).resume(assignment); + Mockito.verify(mockitoConsumer, times(2)).assignment(); Mockito.verify(activeTaskCreator).createTasks(any(), Mockito.eq(emptyMap())); } @@ -3244,12 +3232,10 @@ public void shouldSuspendActiveTasksDuringRevocation() { final Map offsets = singletonMap(t1p0, new OffsetAndMetadata(0L, null)); task00.setCommittableOffsetsAndMetadata(offsets); - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00)); - consumer.commitSync(offsets); - expectLastCall(); - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(taskId00Assignment, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -3293,7 +3279,7 @@ public void shouldCommitAllActiveTasksThatNeedCommittingOnHandleRevocationWithEo final Map> assignmentStandby = mkMap( mkEntry(taskId10, taskId10Partitions) ); - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(assignmentActive))) .thenReturn(asList(task00, task01, task02)); @@ -3303,20 +3289,14 @@ public void shouldCommitAllActiveTasksThatNeedCommittingOnHandleRevocationWithEo .thenReturn(singletonList(task10)); final ConsumerGroupMetadata groupMetadata = new ConsumerGroupMetadata("appId"); - expect(consumer.groupMetadata()).andReturn(groupMetadata); - producer.commitTransaction(expectedCommittedOffsets, groupMetadata); - expectLastCall(); + when(mockitoConsumer.groupMetadata()).thenReturn(groupMetadata); task00.committedOffsets(); - EasyMock.expectLastCall(); task01.committedOffsets(); - EasyMock.expectLastCall(); task02.committedOffsets(); - EasyMock.expectLastCall(); task10.committedOffsets(); - EasyMock.expectLastCall(); - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(assignmentActive, assignmentStandby); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -3331,6 +3311,8 @@ public void shouldCommitAllActiveTasksThatNeedCommittingOnHandleRevocationWithEo assertThat(task01.commitNeeded, is(false)); assertThat(task02.commitPrepared, is(false)); assertThat(task10.commitPrepared, is(false)); + + Mockito.verify(producer).commitTransaction(expectedCommittedOffsets, groupMetadata); } @Test @@ -3364,16 +3346,14 @@ public void shouldCommitAllNeededTasksOnHandleRevocation() { final Map> assignmentStandby = mkMap( mkEntry(taskId10, taskId10Partitions) ); - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(assignmentActive))) .thenReturn(asList(task00, task01, task02)); when(standbyTaskCreator.createTasks(assignmentStandby)) .thenReturn(singletonList(task10)); - consumer.commitSync(expectedCommittedOffsets); - expectLastCall(); - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(assignmentActive, assignmentStandby); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -3390,6 +3370,8 @@ public void shouldCommitAllNeededTasksOnHandleRevocation() { assertThat(task01.commitPrepared, is(true)); assertThat(task02.commitPrepared, is(false)); assertThat(task10.commitPrepared, is(false)); + + Mockito.verify(mockitoConsumer).commitSync(expectedCommittedOffsets); } @Test @@ -3404,12 +3386,12 @@ public void shouldNotCommitOnHandleAssignmentIfNoTaskClosed() { final Map> assignmentActive = singletonMap(taskId00, taskId00Partitions); final Map> assignmentStandby = singletonMap(taskId10, taskId10Partitions); - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(assignmentActive))).thenReturn(singleton(task00)); when(standbyTaskCreator.createTasks(assignmentStandby)).thenReturn(singletonList(task10)); - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(assignmentActive, assignmentStandby); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -3434,12 +3416,12 @@ public void shouldNotCommitOnHandleAssignmentIfOnlyStandbyTaskClosed() { final Map> assignmentActive = singletonMap(taskId00, taskId00Partitions); final Map> assignmentStandby = singletonMap(taskId10, taskId10Partitions); - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(assignmentActive))).thenReturn(singleton(task00)); when(standbyTaskCreator.createTasks(assignmentStandby)).thenReturn(singletonList(task10)); - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(assignmentActive, assignmentStandby); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -3478,17 +3460,15 @@ public void suspend() { } }; - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00)); - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(taskId00Assignment, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); assertThat(task00.state(), is(Task.State.RUNNING)); assertThrows(RuntimeException.class, () -> taskManager.handleRevocation(taskId00Partitions)); assertThat(task00.state(), is(Task.State.SUSPENDED)); - - verify(consumer); } @Test @@ -3938,10 +3918,12 @@ public void shouldShutDownStateUpdaterAndAddRemovedTasksToTaskRegistry() { @Test public void shouldInitializeNewActiveTasks() { final StateMachineTask task00 = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager); - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); + when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))) .thenReturn(singletonList(task00)); - replay(consumer); + + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(taskId00Assignment, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -3950,17 +3932,17 @@ public void shouldInitializeNewActiveTasks() { assertThat(taskManager.activeTaskMap(), Matchers.equalTo(singletonMap(taskId00, task00))); assertThat(taskManager.standbyTaskMap(), Matchers.anEmptyMap()); // verifies that we actually resume the assignment at the end of restoration. - verify(consumer); + Mockito.verify(mockitoConsumer).resume(assignment); } @Test public void shouldInitialiseNewStandbyTasks() { final StateMachineTask task01 = new StateMachineTask(taskId01, taskId01Partitions, false, stateManager); - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(standbyTaskCreator.createTasks(taskId01Assignment)).thenReturn(singletonList(task01)); - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(emptyMap(), taskId01Assignment); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -3972,7 +3954,6 @@ public void shouldInitialiseNewStandbyTasks() { @Test public void shouldHandleRebalanceEvents() { - final Set assignment = singleton(new TopicPartition("assignment", 0)); taskManager.setMainConsumer(mockitoConsumer); when(mockitoConsumer.assignment()).thenReturn(assignment); expect(stateDirectory.listNonEmptyTaskDirectories()).andReturn(new ArrayList<>()); @@ -3992,15 +3973,13 @@ public void shouldCommitActiveAndStandbyTasks() { task00.setCommittableOffsetsAndMetadata(offsets); final StateMachineTask task01 = new StateMachineTask(taskId01, taskId01Partitions, false, stateManager); - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))) .thenReturn(singletonList(task00)); when(standbyTaskCreator.createTasks(taskId01Assignment)) .thenReturn(singletonList(task01)); - consumer.commitSync(offsets); - expectLastCall(); - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(taskId00Assignment, taskId01Assignment); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -4014,6 +3993,8 @@ public void shouldCommitActiveAndStandbyTasks() { assertThat(taskManager.commitAll(), equalTo(2)); assertThat(task00.commitNeeded, is(false)); assertThat(task01.commitNeeded, is(false)); + + Mockito.verify(mockitoConsumer).commitSync(offsets); } @Test @@ -4036,15 +4017,13 @@ public void shouldCommitProvidedTasksIfNeeded() { mkEntry(taskId05, taskId05Partitions) ); - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(assignmentActive))) .thenReturn(Arrays.asList(task00, task01, task02)); when(standbyTaskCreator.createTasks(assignmentStandby)) .thenReturn(Arrays.asList(task03, task04, task05)); - consumer.commitSync(eq(emptyMap())); - - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(assignmentActive, assignmentStandby); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -4070,10 +4049,10 @@ public void shouldCommitProvidedTasksIfNeeded() { public void shouldNotCommitOffsetsIfOnlyStandbyTasksAssigned() { final StateMachineTask task00 = new StateMachineTask(taskId00, taskId00Partitions, false, stateManager); - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(standbyTaskCreator.createTasks(taskId00Assignment)).thenReturn(singletonList(task00)); - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(Collections.emptyMap(), taskId00Assignment); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -4094,13 +4073,15 @@ public void shouldNotCommitActiveAndStandbyTasksWhileRebalanceInProgress() throw makeTaskFolders(taskId00.toString(), taskId01.toString()); expectDirectoryNotEmpty(taskId00, taskId01); expectLockObtainedFor(taskId00, taskId01); - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))) .thenReturn(singletonList(task00)); when(standbyTaskCreator.createTasks(taskId01Assignment)) .thenReturn(singletonList(task01)); - replay(stateDirectory, consumer); + replay(stateDirectory); + + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(taskId00Assignment, taskId01Assignment); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -4200,9 +4181,10 @@ public Map prepareCommit() { } }; - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00)); - replay(consumer); + + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(taskId00Assignment, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -4225,10 +4207,10 @@ public Map prepareCommit() { } }; - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(standbyTaskCreator.createTasks(taskId01Assignment)).thenReturn(singletonList(task01)); - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(emptyMap(), taskId01Assignment); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -4259,10 +4241,10 @@ public Map purgeableOffsets() { } }; - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00)); - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(taskId00Assignment, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -4294,9 +4276,10 @@ public Map purgeableOffsets() { } }; - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00)); - replay(consumer); + + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(taskId00Assignment, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -4317,14 +4300,14 @@ public Map purgeableOffsets() { public void shouldIgnorePurgeDataErrors() { final StateMachineTask task00 = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager); - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); final KafkaFutureImpl futureDeletedRecords = new KafkaFutureImpl<>(); final DeleteRecordsResult deleteRecordsResult = new DeleteRecordsResult(singletonMap(t1p1, futureDeletedRecords)); futureDeletedRecords.completeExceptionally(new Exception("KABOOM!")); when(adminClient.deleteRecords(any())).thenReturn(deleteRecordsResult); - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); taskManager.addTask(task00); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -4366,15 +4349,13 @@ public void shouldMaybeCommitAllActiveTasksThatNeedCommit() { mkEntry(taskId10, taskId10Partitions) ); - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(assignmentActive))) .thenReturn(asList(task00, task01, task02, task03)); when(standbyTaskCreator.createTasks(assignmentStandby)) .thenReturn(singletonList(task04)); - consumer.commitSync(expectedCommittedOffsets); - expectLastCall(); - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(assignmentActive, assignmentStandby); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -4399,6 +4380,8 @@ public void shouldMaybeCommitAllActiveTasksThatNeedCommit() { task04.setCommitRequested(); assertThat(taskManager.maybeCommitActiveTasksPerUserRequested(), equalTo(3)); + + Mockito.verify(mockitoConsumer).commitSync(expectedCommittedOffsets); } @Test @@ -4406,16 +4389,17 @@ public void shouldProcessActiveTasks() { final StateMachineTask task00 = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager); final StateMachineTask task01 = new StateMachineTask(taskId01, taskId01Partitions, true, stateManager); - final Map> assignment = new HashMap<>(); - assignment.put(taskId00, taskId00Partitions); - assignment.put(taskId01, taskId01Partitions); + final Map> firstAssignment = new HashMap<>(); + firstAssignment.put(taskId00, taskId00Partitions); + firstAssignment.put(taskId01, taskId01Partitions); - expectRestoreToBeCompleted(consumer); - when(activeTaskCreator.createTasks(any(), Mockito.eq(assignment))) + when(mockitoConsumer.assignment()).thenReturn(assignment); + when(activeTaskCreator.createTasks(any(), Mockito.eq(firstAssignment))) .thenReturn(Arrays.asList(task00, task01)); - replay(consumer); - taskManager.handleAssignment(assignment, emptyMap()); + taskManager.setMainConsumer(mockitoConsumer); + + taskManager.handleAssignment(firstAssignment, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); assertThat(task00.state(), is(Task.State.RUNNING)); @@ -4523,9 +4507,10 @@ public boolean process(final long wallClockTime) { } }; - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00)); - replay(consumer); + + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(taskId00Assignment, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -4547,10 +4532,11 @@ public boolean process(final long wallClockTime) { } }; - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))) .thenReturn(singletonList(task00)); - replay(consumer); + + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(taskId00Assignment, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -4575,9 +4561,10 @@ public boolean maybePunctuateStreamTime() { } }; - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00)); - replay(consumer); + + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(taskId00Assignment, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -4596,9 +4583,10 @@ public boolean maybePunctuateStreamTime() { } }; - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00)); - replay(consumer); + + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(taskId00Assignment, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -4622,9 +4610,10 @@ public boolean maybePunctuateSystemTime() { } }; - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00)); - replay(consumer); + + taskManager.setMainConsumer(mockitoConsumer); taskManager.handleAssignment(taskId00Assignment, emptyMap()); assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true)); @@ -4660,12 +4649,10 @@ public void shouldHaveRemainingPartitionsUncleared() { final Map offsets = singletonMap(t1p0, new OffsetAndMetadata(0L, null)); task00.setCommittableOffsetsAndMetadata(offsets); - expectRestoreToBeCompleted(consumer); + when(mockitoConsumer.assignment()).thenReturn(assignment); when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00)); - consumer.commitSync(offsets); - expectLastCall(); - replay(consumer); + taskManager.setMainConsumer(mockitoConsumer); try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(TaskManager.class)) { appender.setClassLoggerToDebug(TaskManager.class); @@ -4821,8 +4808,9 @@ private Map handleAssignment(final Map consumer) { - final Set assignment = singleton(new TopicPartition("assignment", 0)); - expect(consumer.assignment()).andReturn(assignment); - consumer.pause(assignment); - } - @Test public void shouldThrowTaskMigratedExceptionOnCommitFailed() { final StateMachineTask task01 = new StateMachineTask(taskId01, taskId01Partitions, true, stateManager); @@ -5135,13 +5117,6 @@ public void shouldListNotPausedTasks() { assertEquals(taskManager.notPausedTasks().size(), 0); } - private static void expectRestoreToBeCompleted(final Consumer consumer) { - final Set assignment = singleton(new TopicPartition("assignment", 0)); - expect(consumer.assignment()).andReturn(assignment); - consumer.resume(assignment); - expectLastCall(); - } - private static KafkaFutureImpl completedFuture() { final KafkaFutureImpl futureDeletedRecords = new KafkaFutureImpl<>(); futureDeletedRecords.complete(null); From 86e7885a81c7132e522d2c59dc6fcf81026cc60d Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 7 Mar 2024 06:00:21 -0800 Subject: [PATCH 039/521] KAFKA-16100: Add timeout to all the CompletableApplicationEvents (#15455) This is part of the larger task of enforcing the timeouts for application events, per KAFKA-15974. This takes a first step by adding a Timer to all of the CompletableApplicationEvent subclasses. For the few classes that already included a timeout, this refactors them to use the Timer mechanism instead. Reviewers: Andrew Schofield , Bruno Cadonna --- .../internals/AsyncKafkaConsumer.java | 41 +++++++++++-------- .../events/AbstractTopicMetadataEvent.java | 17 ++------ .../events/AllTopicsMetadataEvent.java | 6 ++- .../events/ApplicationEventProcessor.java | 25 ++--------- .../internals/events/AsyncCommitEvent.java | 2 +- .../internals/events/CommitEvent.java | 20 +++++++-- .../events/CompletableApplicationEvent.java | 20 ++++++++- .../events/FetchCommittedOffsetsEvent.java | 17 ++------ .../internals/events/LeaveOnCloseEvent.java | 6 ++- .../internals/events/ListOffsetsEvent.java | 5 ++- .../internals/events/ResetPositionsEvent.java | 6 ++- .../internals/events/SyncCommitEvent.java | 21 ++-------- .../internals/events/TopicMetadataEvent.java | 6 ++- .../internals/events/UnsubscribeEvent.java | 6 ++- .../events/ValidatePositionsEvent.java | 6 ++- .../internals/ConsumerNetworkThreadTest.java | 19 ++++++--- .../events/ApplicationEventProcessorTest.java | 17 +++----- 17 files changed, 120 insertions(+), 120 deletions(-) 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 5354503c01..fcd57469c2 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 @@ -938,13 +938,14 @@ public Map committed(final Set committedOffsets = applicationEventHandler.addAndGet(event, - time.timer(timeout)); + timer); committedOffsets.forEach(this::updateLastSeenEpochIfNewer); return committedOffsets; } catch (TimeoutException e) { @@ -990,11 +991,12 @@ public List partitionsFor(String topic, Duration timeout) { throw new TimeoutException(); } - final TopicMetadataEvent topicMetadataEvent = new TopicMetadataEvent(topic, timeout.toMillis()); + final Timer timer = time.timer(timeout); + final TopicMetadataEvent topicMetadataEvent = new TopicMetadataEvent(topic, timer); wakeupTrigger.setActiveTask(topicMetadataEvent.future()); try { Map> topicMetadata = - applicationEventHandler.addAndGet(topicMetadataEvent, time.timer(timeout)); + applicationEventHandler.addAndGet(topicMetadataEvent, timer); return topicMetadata.getOrDefault(topic, Collections.emptyList()); } finally { @@ -1018,10 +1020,11 @@ public Map> listTopics(Duration timeout) { throw new TimeoutException(); } - final AllTopicsMetadataEvent topicMetadataEvent = new AllTopicsMetadataEvent(timeout.toMillis()); + final Timer timer = time.timer(timeout); + final AllTopicsMetadataEvent topicMetadataEvent = new AllTopicsMetadataEvent(timer); wakeupTrigger.setActiveTask(topicMetadataEvent.future()); try { - return applicationEventHandler.addAndGet(topicMetadataEvent, time.timer(timeout)); + return applicationEventHandler.addAndGet(topicMetadataEvent, timer); } finally { wakeupTrigger.clearTask(); } @@ -1089,16 +1092,18 @@ public Map offsetsForTimes(Map beginningOrEndOffset(Collection timestampToSearch = partitions .stream() .collect(Collectors.toMap(Function.identity(), tp -> timestamp)); + Timer timer = time.timer(timeout); ListOffsetsEvent listOffsetsEvent = new ListOffsetsEvent( timestampToSearch, - false); + false, + timer); Map offsetAndTimestampMap = applicationEventHandler.addAndGet( listOffsetsEvent, - time.timer(timeout)); + timer); return offsetAndTimestampMap .entrySet() .stream() @@ -1274,7 +1281,7 @@ void prepareShutdown(final Timer timer, final AtomicReference firstEx completeQuietly( () -> { maybeRevokePartitions(); - applicationEventHandler.addAndGet(new LeaveOnCloseEvent(), timer); + applicationEventHandler.addAndGet(new LeaveOnCloseEvent(timer), timer); }, "Failed to send leaveGroup heartbeat with a timeout(ms)=" + timer.timeoutMs(), firstException); } @@ -1351,7 +1358,7 @@ public void commitSync(Map offsets, Duration long commitStart = time.nanoseconds(); try { Timer requestTimer = time.timer(timeout.toMillis()); - SyncCommitEvent syncCommitEvent = new SyncCommitEvent(offsets, timeout.toMillis()); + SyncCommitEvent syncCommitEvent = new SyncCommitEvent(offsets, requestTimer); CompletableFuture commitFuture = commit(syncCommitEvent); wakeupTrigger.setActiveTask(commitFuture); ConsumerUtils.getResult(commitFuture, requestTimer); @@ -1465,10 +1472,10 @@ public void unsubscribe() { try { fetchBuffer.retainAll(Collections.emptySet()); if (groupMetadata.get().isPresent()) { - UnsubscribeEvent unsubscribeEvent = new UnsubscribeEvent(); + Timer timer = time.timer(Long.MAX_VALUE); + UnsubscribeEvent unsubscribeEvent = new UnsubscribeEvent(timer); applicationEventHandler.add(unsubscribeEvent); log.info("Unsubscribing all topics or patterns and assigned partitions"); - Timer timer = time.timer(Long.MAX_VALUE); try { processBackgroundEvents(backgroundEventProcessor, unsubscribeEvent.future(), timer); @@ -1579,7 +1586,7 @@ private boolean updateFetchPositions(final Timer timer) { // Validate positions using the partition leader end offsets, to detect if any partition // has been truncated due to a leader change. This will trigger an OffsetForLeaderEpoch // request, retrieve the partition end offsets, and validate the current position against it. - applicationEventHandler.addAndGet(new ValidatePositionsEvent(), timer); + applicationEventHandler.addAndGet(new ValidatePositionsEvent(timer), timer); cachedSubscriptionHasAllFetchPositions = subscriptions.hasAllFetchPositions(); if (cachedSubscriptionHasAllFetchPositions) return true; @@ -1602,7 +1609,7 @@ private boolean updateFetchPositions(final Timer timer) { // which are awaiting reset. This will trigger a ListOffset request, retrieve the // partition offsets according to the strategy (ex. earliest, latest), and update the // positions. - applicationEventHandler.addAndGet(new ResetPositionsEvent(), timer); + applicationEventHandler.addAndGet(new ResetPositionsEvent(timer), timer); return true; } catch (TimeoutException e) { return false; @@ -1635,7 +1642,7 @@ private boolean initWithCommittedOffsetsIfNeeded(Timer timer) { final FetchCommittedOffsetsEvent event = new FetchCommittedOffsetsEvent( initializingPartitions, - timer.remainingMs()); + timer); final Map offsets = applicationEventHandler.addAndGet(event, timer); refreshCommittedOffsets(offsets, metadata, subscriptions); return true; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java index 31c21817d8..3347002cc6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java @@ -17,25 +17,14 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.utils.Timer; import java.util.List; import java.util.Map; public abstract class AbstractTopicMetadataEvent extends CompletableApplicationEvent>> { - private final long timeoutMs; - - protected AbstractTopicMetadataEvent(final Type type, final long timeoutMs) { - super(type); - this.timeoutMs = timeoutMs; - } - - public long timeoutMs() { - return timeoutMs; - } - - @Override - public String toStringBase() { - return super.toStringBase() + ", timeoutMs=" + timeoutMs; + protected AbstractTopicMetadataEvent(final Type type, final Timer timer) { + super(type, timer); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AllTopicsMetadataEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AllTopicsMetadataEvent.java index 154703aaee..bda18e6421 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AllTopicsMetadataEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AllTopicsMetadataEvent.java @@ -16,9 +16,11 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + public class AllTopicsMetadataEvent extends AbstractTopicMetadataEvent { - public AllTopicsMetadataEvent(final long timeoutMs) { - super(Type.ALL_TOPICS_METADATA, timeoutMs); + public AllTopicsMetadataEvent(final Timer timer) { + super(Type.ALL_TOPICS_METADATA, timer); } } 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 c86aa8815f..3382530746 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 @@ -165,8 +165,7 @@ private void process(final SyncCommitEvent event) { } CommitRequestManager manager = requestManagers.commitRequestManager.get(); - long expirationTimeoutMs = getExpirationTimeForTimeout(event.retryTimeoutMs()); - CompletableFuture future = manager.commitSync(event.offsets(), expirationTimeoutMs); + CompletableFuture future = manager.commitSync(event.offsets(), event.deadlineMs()); future.whenComplete(complete(event.future())); } @@ -177,8 +176,7 @@ private void process(final FetchCommittedOffsetsEvent event) { return; } CommitRequestManager manager = requestManagers.commitRequestManager.get(); - long expirationTimeMs = getExpirationTimeForTimeout(event.timeout()); - CompletableFuture> future = manager.fetchOffsets(event.partitions(), expirationTimeMs); + CompletableFuture> future = manager.fetchOffsets(event.partitions(), event.deadlineMs()); future.whenComplete(complete(event.future())); } @@ -250,16 +248,14 @@ private void process(final ValidatePositionsEvent event) { } private void process(final TopicMetadataEvent event) { - final long expirationTimeMs = getExpirationTimeForTimeout(event.timeoutMs()); final CompletableFuture>> future = - requestManagers.topicMetadataRequestManager.requestTopicMetadata(event.topic(), expirationTimeMs); + requestManagers.topicMetadataRequestManager.requestTopicMetadata(event.topic(), event.deadlineMs()); future.whenComplete(complete(event.future())); } private void process(final AllTopicsMetadataEvent event) { - final long expirationTimeMs = getExpirationTimeForTimeout(event.timeoutMs()); final CompletableFuture>> future = - requestManagers.topicMetadataRequestManager.requestAllTopicsMetadata(expirationTimeMs); + requestManagers.topicMetadataRequestManager.requestAllTopicsMetadata(event.deadlineMs()); future.whenComplete(complete(event.future())); } @@ -296,19 +292,6 @@ private void process(final LeaveOnCloseEvent event) { future.whenComplete(complete(event.future())); } - /** - * @return Expiration time in milliseconds calculated with the current time plus the given - * timeout. Returns Long.MAX_VALUE if the expiration overflows it. - * Visible for testing. - */ - long getExpirationTimeForTimeout(final long timeoutMs) { - long expiration = System.currentTimeMillis() + timeoutMs; - if (expiration < 0) { - return Long.MAX_VALUE; - } - return expiration; - } - 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/AsyncCommitEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java index 2f03fdfb1e..c36f0534b3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java @@ -27,6 +27,6 @@ public class AsyncCommitEvent extends CommitEvent { public AsyncCommitEvent(final Map offsets) { - super(Type.COMMIT_ASYNC, offsets); + super(Type.COMMIT_ASYNC, offsets, Long.MAX_VALUE); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java index 253d27e257..1da7b84039 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; import java.util.Collections; import java.util.Map; @@ -29,15 +30,28 @@ public abstract class CommitEvent extends CompletableApplicationEvent { */ private final Map offsets; - protected CommitEvent(final Type type, final Map offsets) { - super(type); - this.offsets = Collections.unmodifiableMap(offsets); + protected CommitEvent(final Type type, final Map offsets, final Timer timer) { + super(type, timer); + this.offsets = validate(offsets); + } + + protected CommitEvent(final Type type, final Map offsets, final long deadlineMs) { + super(type, deadlineMs); + this.offsets = validate(offsets); + } + /** + * Validates the offsets are not negative and then returns the given offset map as + * {@link Collections#unmodifiableMap(Map) as unmodifiable}. + */ + private static Map validate(final Map offsets) { for (OffsetAndMetadata offsetAndMetadata : offsets.values()) { if (offsetAndMetadata.offset() < 0) { throw new IllegalArgumentException("Invalid offset: " + offsetAndMetadata.offset()); } } + + return Collections.unmodifiableMap(offsets); } public Map offsets() { 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 a62c3aaa4c..dae9e9f101 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 @@ -16,6 +16,9 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + +import java.util.Objects; import java.util.concurrent.CompletableFuture; /** @@ -27,10 +30,19 @@ public abstract class CompletableApplicationEvent extends ApplicationEvent implements CompletableEvent { private final CompletableFuture future; + private final long deadlineMs; - protected CompletableApplicationEvent(final Type type) { + protected CompletableApplicationEvent(final Type type, final Timer timer) { super(type); this.future = new CompletableFuture<>(); + Objects.requireNonNull(timer); + this.deadlineMs = timer.remainingMs() + timer.currentTimeMs(); + } + + protected CompletableApplicationEvent(final Type type, final long deadlineMs) { + super(type); + this.future = new CompletableFuture<>(); + this.deadlineMs = deadlineMs; } @Override @@ -38,8 +50,12 @@ public CompletableFuture future() { return future; } + public long deadlineMs() { + return deadlineMs; + } + @Override protected String toStringBase() { - return super.toStringBase() + ", future=" + future; + return super.toStringBase() + ", future=" + future + ", deadlineMs=" + deadlineMs; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsEvent.java index 7cf56b990b..980a8f1104 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsEvent.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; import java.util.Collections; import java.util.Map; @@ -30,27 +31,17 @@ public class FetchCommittedOffsetsEvent extends CompletableApplicationEvent partitions; - /** - * Time until which the request will be retried if it fails with a retriable error. - */ - private final long timeoutMs; - - public FetchCommittedOffsetsEvent(final Set partitions, final long timeoutMs) { - super(Type.FETCH_COMMITTED_OFFSETS); + public FetchCommittedOffsetsEvent(final Set partitions, final Timer timer) { + super(Type.FETCH_COMMITTED_OFFSETS, timer); this.partitions = Collections.unmodifiableSet(partitions); - this.timeoutMs = timeoutMs; } public Set partitions() { return partitions; } - public long timeout() { - return timeoutMs; - } - @Override public String toStringBase() { - return super.toStringBase() + ", partitions=" + partitions + ", partitions=" + partitions; + return super.toStringBase() + ", partitions=" + partitions; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseEvent.java index 5ee19a7cc0..e77b4dfb28 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseEvent.java @@ -16,9 +16,11 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + public class LeaveOnCloseEvent extends CompletableApplicationEvent { - public LeaveOnCloseEvent() { - super(Type.LEAVE_ON_CLOSE); + public LeaveOnCloseEvent(final Timer timer) { + super(Type.LEAVE_ON_CLOSE, timer); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java index fd3b321173..e218705846 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; import java.util.Collections; import java.util.HashMap; @@ -36,8 +37,8 @@ public class ListOffsetsEvent extends CompletableApplicationEvent timestampsToSearch; private final boolean requireTimestamps; - public ListOffsetsEvent(final Map timestampToSearch, final boolean requireTimestamps) { - super(Type.LIST_OFFSETS); + public ListOffsetsEvent(final Map timestampToSearch, final boolean requireTimestamps, final Timer timer) { + super(Type.LIST_OFFSETS, timer); this.timestampsToSearch = Collections.unmodifiableMap(timestampToSearch); this.requireTimestamps = requireTimestamps; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsEvent.java index 06f6ebbb68..65893b62ec 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsEvent.java @@ -17,6 +17,8 @@ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + /** * Event for resetting offsets for all assigned partitions that require it. This is an * asynchronous event that generates ListOffsets requests, and completes by updating in-memory @@ -24,7 +26,7 @@ */ public class ResetPositionsEvent extends CompletableApplicationEvent { - public ResetPositionsEvent() { - super(Type.RESET_POSITIONS); + public ResetPositionsEvent(final Timer timer) { + super(Type.RESET_POSITIONS, timer); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitEvent.java index 7e00e0da59..87945616ea 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitEvent.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; import java.util.Map; @@ -27,23 +28,7 @@ */ public class SyncCommitEvent extends CommitEvent { - /** - * Time to wait for a response, retrying on retriable errors. - */ - private final long retryTimeoutMs; - - public SyncCommitEvent(final Map offsets, - final long retryTimeoutMs) { - super(Type.COMMIT_SYNC, offsets); - this.retryTimeoutMs = retryTimeoutMs; - } - - public Long retryTimeoutMs() { - return retryTimeoutMs; - } - - @Override - public String toStringBase() { - return super.toStringBase() + ", offsets=" + offsets() + ", retryTimeoutMs=" + retryTimeoutMs; + public SyncCommitEvent(final Map offsets, final Timer timer) { + super(Type.COMMIT_SYNC, offsets, timer); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataEvent.java index ebbb2a6c46..33e1270ce6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataEvent.java @@ -16,14 +16,16 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + import java.util.Objects; public class TopicMetadataEvent extends AbstractTopicMetadataEvent { private final String topic; - public TopicMetadataEvent(final String topic, final long timeoutMs) { - super(Type.TOPIC_METADATA, timeoutMs); + public TopicMetadataEvent(final String topic, final Timer timer) { + super(Type.TOPIC_METADATA, timer); this.topic = Objects.requireNonNull(topic); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java index 07af36e5fe..0b98837001 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java @@ -17,6 +17,8 @@ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + /** * Application event triggered when a user calls the unsubscribe API. This will make the consumer * release all its assignments and send a heartbeat request to leave the consumer group. @@ -26,8 +28,8 @@ */ public class UnsubscribeEvent extends CompletableApplicationEvent { - public UnsubscribeEvent() { - super(Type.UNSUBSCRIBE); + public UnsubscribeEvent(final Timer timer) { + super(Type.UNSUBSCRIBE, timer); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsEvent.java index efa358b4c7..21e7f3cf6e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsEvent.java @@ -17,6 +17,8 @@ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + /** * Event for validating offsets for all assigned partitions for which a leader change has been * detected. This is an asynchronous event that generates OffsetForLeaderEpoch requests, and @@ -24,7 +26,7 @@ */ public class ValidatePositionsEvent extends CompletableApplicationEvent { - public ValidatePositionsEvent() { - super(Type.VALIDATE_POSITIONS); + public ValidatePositionsEvent(final Timer timer) { + super(Type.VALIDATE_POSITIONS, timer); } } \ No newline at end of file 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 cbd56d8b5e..e4d492fb58 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 @@ -41,6 +41,7 @@ import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.RequestTestUtils; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; import org.apache.kafka.test.TestCondition; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; @@ -161,7 +162,8 @@ public void testAsyncCommitEvent() { @Test public void testSyncCommitEvent() { - ApplicationEvent e = new SyncCommitEvent(new HashMap<>(), 100L); + Timer timer = time.timer(100); + ApplicationEvent e = new SyncCommitEvent(new HashMap<>(), timer); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(SyncCommitEvent.class)); @@ -170,7 +172,8 @@ public void testSyncCommitEvent() { @Test public void testListOffsetsEventIsProcessed() { Map timestamps = Collections.singletonMap(new TopicPartition("topic1", 1), 5L); - ApplicationEvent e = new ListOffsetsEvent(timestamps, true); + Timer timer = time.timer(100); + ApplicationEvent e = new ListOffsetsEvent(timestamps, true, timer); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ListOffsetsEvent.class)); @@ -179,7 +182,8 @@ public void testListOffsetsEventIsProcessed() { @Test public void testResetPositionsEventIsProcessed() { - ResetPositionsEvent e = new ResetPositionsEvent(); + Timer timer = time.timer(100); + ResetPositionsEvent e = new ResetPositionsEvent(timer); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ResetPositionsEvent.class)); @@ -190,7 +194,8 @@ public void testResetPositionsEventIsProcessed() { public void testResetPositionsProcessFailureIsIgnored() { doThrow(new NullPointerException()).when(offsetsRequestManager).resetPositionsIfNeeded(); - ResetPositionsEvent event = new ResetPositionsEvent(); + Timer timer = time.timer(100); + ResetPositionsEvent event = new ResetPositionsEvent(timer); applicationEventsQueue.add(event); assertDoesNotThrow(() -> consumerNetworkThread.runOnce()); @@ -199,7 +204,8 @@ public void testResetPositionsProcessFailureIsIgnored() { @Test public void testValidatePositionsEventIsProcessed() { - ValidatePositionsEvent e = new ValidatePositionsEvent(); + Timer timer = time.timer(100); + ValidatePositionsEvent e = new ValidatePositionsEvent(timer); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ValidatePositionsEvent.class)); @@ -224,7 +230,8 @@ public void testAssignmentChangeEvent() { @Test void testFetchTopicMetadata() { - applicationEventsQueue.add(new TopicMetadataEvent("topic", Long.MAX_VALUE)); + Timer timer = time.timer(Long.MAX_VALUE); + applicationEventsQueue.add(new TopicMetadataEvent("topic", timer)); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(TopicMetadataEvent.class)); } 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 f3e2557ae9..b23660e546 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 @@ -27,6 +27,9 @@ import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.clients.consumer.internals.TopicMetadataRequestManager; 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.BeforeEach; import org.junit.jupiter.api.Test; @@ -36,7 +39,6 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; -import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -44,6 +46,7 @@ import static org.mockito.Mockito.when; public class ApplicationEventProcessorTest { + private final Time time = new MockTime(1); private ApplicationEventProcessor processor; private BlockingQueue applicationEventQueue = mock(BlockingQueue.class); private RequestManagers requestManagers; @@ -96,18 +99,10 @@ public void testPrepClosingCommitEvents() { verify(commitRequestManager).signalClose(); } - @Test - public void testExpirationCalculation() { - assertEquals(Long.MAX_VALUE, processor.getExpirationTimeForTimeout(Long.MAX_VALUE)); - assertEquals(Long.MAX_VALUE, processor.getExpirationTimeForTimeout(Long.MAX_VALUE - 1)); - long timeout = processor.getExpirationTimeForTimeout(1000); - assertTrue(timeout > 0); - assertTrue(timeout < Long.MAX_VALUE); - } - @Test public void testPrepClosingLeaveGroupEvent() { - LeaveOnCloseEvent event = new LeaveOnCloseEvent(); + Timer timer = time.timer(100); + LeaveOnCloseEvent event = new LeaveOnCloseEvent(timer); when(heartbeatRequestManager.membershipManager()).thenReturn(membershipManager); when(membershipManager.leaveGroup()).thenReturn(CompletableFuture.completedFuture(null)); processor.process(event); From 652537f28e5102bfe197528551cd244ce0345319 Mon Sep 17 00:00:00 2001 From: Stanislav Kozlovski Date: Thu, 7 Mar 2024 15:20:54 +0100 Subject: [PATCH 040/521] MINOR: Add 3.7.0 to core and client's upgrade compatibility tests (#15452) Reviewers: Mickael Maison , Chia-Ping Tsai --- .../tests/client/client_compatibility_features_test.py | 5 +++-- .../client/client_compatibility_produce_consume_test.py | 4 +++- .../tests/core/compatibility_test_new_broker_test.py | 3 ++- tests/kafkatest/tests/core/downgrade_test.py | 5 ++++- tests/kafkatest/tests/core/upgrade_test.py | 5 ++++- 5 files changed, 16 insertions(+), 6 deletions(-) diff --git a/tests/kafkatest/tests/client/client_compatibility_features_test.py b/tests/kafkatest/tests/client/client_compatibility_features_test.py index b82122b3cd..f29f1df0b4 100644 --- a/tests/kafkatest/tests/client/client_compatibility_features_test.py +++ b/tests/kafkatest/tests/client/client_compatibility_features_test.py @@ -28,8 +28,8 @@ from ducktape.tests.test import Test from kafkatest.version import DEV_BRANCH, LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, LATEST_0_11_0, LATEST_1_0, \ LATEST_1_1, LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, \ - LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, V_0_11_0_0, \ - V_0_10_1_0, KafkaVersion + LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, \ + V_0_11_0_0, V_0_10_1_0, KafkaVersion def get_broker_features(broker_version): features = {} @@ -140,6 +140,7 @@ def invoke_compatibility_program(self, features): @parametrize(broker_version=str(LATEST_3_4)) @parametrize(broker_version=str(LATEST_3_5)) @parametrize(broker_version=str(LATEST_3_6)) + @parametrize(broker_version=str(LATEST_3_7)) def run_compatibility_test(self, broker_version, metadata_quorum=quorum.zk): if self.zk: self.zk.start() diff --git a/tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py b/tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py index 33df4d2fdc..afa69eb660 100644 --- a/tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py +++ b/tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py @@ -25,7 +25,8 @@ from kafkatest.utils import is_int_with_prefix from kafkatest.version import DEV_BRANCH, LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, LATEST_0_11_0, LATEST_1_0, \ LATEST_1_1, LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, \ - LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, KafkaVersion + LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, \ + KafkaVersion class ClientCompatibilityProduceConsumeTest(ProduceConsumeValidateTest): """ @@ -80,6 +81,7 @@ def min_cluster_size(self): @parametrize(broker_version=str(LATEST_3_4)) @parametrize(broker_version=str(LATEST_3_5)) @parametrize(broker_version=str(LATEST_3_6)) + @parametrize(broker_version=str(LATEST_3_7)) def test_produce_consume(self, broker_version, metadata_quorum=quorum.zk): print("running producer_consumer_compat with broker_version = %s" % broker_version, flush=True) self.kafka.set_version(KafkaVersion(broker_version)) diff --git a/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py b/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py index 6d25b0b8e0..d25f9992c6 100644 --- a/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py +++ b/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py @@ -24,7 +24,7 @@ from kafkatest.version import LATEST_0_8_2, LATEST_0_9, LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, LATEST_0_11_0, \ LATEST_1_0, LATEST_1_1, LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, \ LATEST_2_7, LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, \ - DEV_BRANCH, KafkaVersion + LATEST_3_7, DEV_BRANCH, KafkaVersion # Compatibility tests for moving to a new broker (e.g., 0.10.x) and using a mix of old and new clients (e.g., 0.9.x) class ClientCompatibilityTestNewBroker(ProduceConsumeValidateTest): @@ -64,6 +64,7 @@ def setUp(self): @matrix(producer_version=[str(LATEST_3_4)], consumer_version=[str(LATEST_3_4)], compression_types=[["none"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade) @matrix(producer_version=[str(LATEST_3_5)], consumer_version=[str(LATEST_3_5)], compression_types=[["none"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade) @matrix(producer_version=[str(LATEST_3_6)], consumer_version=[str(LATEST_3_6)], compression_types=[["none"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade) + @matrix(producer_version=[str(LATEST_3_7)], consumer_version=[str(LATEST_3_7)], compression_types=[["none"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade) @matrix(producer_version=[str(LATEST_2_1)], consumer_version=[str(LATEST_2_1)], compression_types=[["zstd"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade) @matrix(producer_version=[str(LATEST_2_0)], consumer_version=[str(LATEST_2_0)], compression_types=[["snappy"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade) @matrix(producer_version=[str(LATEST_1_1)], consumer_version=[str(LATEST_1_1)], compression_types=[["lz4"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade) diff --git a/tests/kafkatest/tests/core/downgrade_test.py b/tests/kafkatest/tests/core/downgrade_test.py index afdb009c5e..a2ada3d868 100644 --- a/tests/kafkatest/tests/core/downgrade_test.py +++ b/tests/kafkatest/tests/core/downgrade_test.py @@ -21,7 +21,7 @@ from kafkatest.tests.end_to_end import EndToEndTest from kafkatest.version import LATEST_1_1, LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, \ LATEST_2_6, LATEST_2_7, LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, \ - LATEST_3_6, DEV_BRANCH, KafkaVersion + LATEST_3_6, LATEST_3_7, DEV_BRANCH, KafkaVersion class TestDowngrade(EndToEndTest): PARTITIONS = 3 @@ -81,6 +81,9 @@ def wait_until_rejoin(self): timeout_sec=60, backoff_sec=1, err_msg="Replicas did not rejoin the ISR in a reasonable amount of time") @cluster(num_nodes=7) + @parametrize(version=str(LATEST_3_7), compression_types=["snappy"]) + @parametrize(version=str(LATEST_3_7), compression_types=["zstd"], security_protocol="SASL_SSL") + @matrix(version=[str(LATEST_3_7)], compression_types=[["none"]], static_membership=[False, True]) @parametrize(version=str(LATEST_3_6), compression_types=["snappy"]) @parametrize(version=str(LATEST_3_6), compression_types=["zstd"], security_protocol="SASL_SSL") @matrix(version=[str(LATEST_3_6)], compression_types=[["none"]], static_membership=[False, True]) diff --git a/tests/kafkatest/tests/core/upgrade_test.py b/tests/kafkatest/tests/core/upgrade_test.py index 62855b78c8..8ba1d0a02d 100644 --- a/tests/kafkatest/tests/core/upgrade_test.py +++ b/tests/kafkatest/tests/core/upgrade_test.py @@ -27,7 +27,7 @@ from kafkatest.version import LATEST_0_8_2, LATEST_0_9, LATEST_0_10, LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, \ LATEST_0_11_0, LATEST_1_0, LATEST_1_1, LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, \ LATEST_2_6, LATEST_2_7, LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, \ - LATEST_3_6, V_0_11_0_0, V_2_8_0, V_3_0_0, DEV_BRANCH, KafkaVersion + LATEST_3_6, LATEST_3_7, V_0_11_0_0, V_2_8_0, V_3_0_0, DEV_BRANCH, KafkaVersion from kafkatest.services.kafka.util import new_jdk_not_supported class TestUpgrade(ProduceConsumeValidateTest): @@ -94,6 +94,9 @@ def perform_upgrade(self, from_kafka_version, to_message_format_version=None): self.wait_until_rejoin() @cluster(num_nodes=6) + @parametrize(from_kafka_version=str(LATEST_3_7), to_message_format_version=None, compression_types=["none"]) + @parametrize(from_kafka_version=str(LATEST_3_7), to_message_format_version=None, compression_types=["lz4"]) + @parametrize(from_kafka_version=str(LATEST_3_7), to_message_format_version=None, compression_types=["snappy"]) @parametrize(from_kafka_version=str(LATEST_3_6), to_message_format_version=None, compression_types=["none"]) @parametrize(from_kafka_version=str(LATEST_3_6), to_message_format_version=None, compression_types=["lz4"]) @parametrize(from_kafka_version=str(LATEST_3_6), to_message_format_version=None, compression_types=["snappy"]) From 1d50cbeda85bd3409cff17fd5114e73f25cf865c Mon Sep 17 00:00:00 2001 From: Andrew Schofield Date: Thu, 7 Mar 2024 15:24:11 +0000 Subject: [PATCH 041/521] KAFKA-16319: Divide DeleteTopics requests by leader node (#15479) Reviewers: Reviewers: Mickael Maison , Kirk True , Daniel Gospodinow --- .../admin/internals/DeleteRecordsHandler.java | 6 +- .../internals/DeleteRecordsHandlerTest.java | 73 +++++++++++++++++-- 2 files changed, 69 insertions(+), 10 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteRecordsHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteRecordsHandler.java index 2daad22603..9f40d19f00 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteRecordsHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteRecordsHandler.java @@ -79,15 +79,15 @@ public static SimpleAdminApiFuture newFuture( @Override public DeleteRecordsRequest.Builder buildBatchedRequest(int brokerId, Set keys) { Map deletionsForTopic = new HashMap<>(); - for (Map.Entry entry: recordsToDelete.entrySet()) { - TopicPartition topicPartition = entry.getKey(); + for (TopicPartition topicPartition : keys) { + RecordsToDelete toDelete = recordsToDelete.get(topicPartition); DeleteRecordsRequestData.DeleteRecordsTopic deleteRecords = deletionsForTopic.computeIfAbsent( topicPartition.topic(), key -> new DeleteRecordsRequestData.DeleteRecordsTopic().setName(topicPartition.topic()) ); deleteRecords.partitions().add(new DeleteRecordsRequestData.DeleteRecordsPartition() .setPartitionIndex(topicPartition.partition()) - .setOffset(entry.getValue().beforeOffset())); + .setOffset(toDelete.beforeOffset())); } DeleteRecordsRequestData data = new DeleteRecordsRequestData() diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteRecordsHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteRecordsHandlerTest.java index c39747f1fb..58492696c4 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteRecordsHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteRecordsHandlerTest.java @@ -22,15 +22,24 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; + import org.apache.kafka.clients.admin.DeletedRecords; import org.apache.kafka.clients.admin.RecordsToDelete; +import org.apache.kafka.clients.admin.internals.AdminApiLookupStrategy.LookupResult; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.message.DeleteRecordsRequestData; import org.apache.kafka.common.message.DeleteRecordsResponseData; +import org.apache.kafka.common.message.MetadataResponseData; +import org.apache.kafka.common.message.MetadataResponseData.MetadataResponsePartition; +import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic; +import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.DeleteRecordsRequest; import org.apache.kafka.common.requests.DeleteRecordsResponse; +import org.apache.kafka.common.requests.MetadataRequest; +import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.utils.LogContext; import org.junit.jupiter.api.Test; @@ -41,6 +50,7 @@ import static org.apache.kafka.common.utils.Utils.mkSet; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertTrue; public class DeleteRecordsHandlerTest { @@ -50,7 +60,8 @@ public class DeleteRecordsHandlerTest { private final TopicPartition t0p1 = new TopicPartition("t0", 1); private final TopicPartition t0p2 = new TopicPartition("t0", 2); private final TopicPartition t0p3 = new TopicPartition("t0", 3); - private final Node node = new Node(1, "host", 1234); + private final Node node1 = new Node(1, "host", 1234); + private final Node node2 = new Node(2, "host", 1235); private final Map recordsToDelete = new HashMap() { { put(t0p0, RecordsToDelete.beforeOffset(10L)); @@ -63,11 +74,11 @@ public class DeleteRecordsHandlerTest { @Test public void testBuildRequestSimple() { DeleteRecordsHandler handler = new DeleteRecordsHandler(recordsToDelete, logContext, timeout); - DeleteRecordsRequest request = handler.buildBatchedRequest(node.id(), mkSet(t0p0, t0p1)).build(); - List topicPartitions = request.data().topics(); - assertEquals(1, topicPartitions.size()); - DeleteRecordsRequestData.DeleteRecordsTopic topic = topicPartitions.get(0); - assertEquals(4, topic.partitions().size()); + DeleteRecordsRequest request = handler.buildBatchedRequest(node1.id(), mkSet(t0p0, t0p1)).build(); + List topics = request.data().topics(); + assertEquals(1, topics.size()); + DeleteRecordsRequestData.DeleteRecordsTopic topic = topics.get(0); + assertEquals(2, topic.partitions().size()); } @Test @@ -199,6 +210,54 @@ public void testHandleResponseSanityCheck() { assertTrue(result.unmappedKeys.isEmpty()); } + // This is a more complicated test which ensures that DeleteRecords requests for multiple + // leader nodes are correctly divided up among the nodes based on leadership. + // node1 leads t0p0 and t0p2, while node2 leads t0p1 and t0p3. + @Test + public void testBuildRequestMultipleLeaders() { + MetadataResponseData metadataResponseData = new MetadataResponseData(); + MetadataResponseTopic topicMetadata = new MetadataResponseTopic(); + topicMetadata.setName("t0").setErrorCode(Errors.NONE.code()); + topicMetadata.partitions().add(new MetadataResponsePartition() + .setPartitionIndex(0).setLeaderId(node1.id()).setErrorCode(Errors.NONE.code())); + topicMetadata.partitions().add(new MetadataResponsePartition() + .setPartitionIndex(1).setLeaderId(node2.id()).setErrorCode(Errors.NONE.code())); + topicMetadata.partitions().add(new MetadataResponsePartition() + .setPartitionIndex(2).setLeaderId(node1.id()).setErrorCode(Errors.NONE.code())); + topicMetadata.partitions().add(new MetadataResponsePartition() + .setPartitionIndex(3).setLeaderId(node2.id()).setErrorCode(Errors.NONE.code())); + metadataResponseData.topics().add(topicMetadata); + MetadataResponse metadataResponse = new MetadataResponse(metadataResponseData, ApiKeys.METADATA.latestVersion()); + + DeleteRecordsHandler handler = new DeleteRecordsHandler(recordsToDelete, logContext, timeout); + AdminApiLookupStrategy strategy = handler.lookupStrategy(); + assertInstanceOf(PartitionLeaderStrategy.class, strategy); + PartitionLeaderStrategy specificStrategy = (PartitionLeaderStrategy) strategy; + MetadataRequest request = specificStrategy.buildRequest(mkSet(t0p0, t0p1, t0p2, t0p3)).build(); + assertEquals(mkSet("t0"), new HashSet<>(request.topics())); + + Set tpSet = mkSet(t0p0, t0p1, t0p2, t0p3); + LookupResult lookupResult = strategy.handleResponse(tpSet, metadataResponse); + assertEquals(emptyMap(), lookupResult.failedKeys); + assertEquals(tpSet, lookupResult.mappedKeys.keySet()); + + Map> partitionsPerBroker = new HashMap<>(); + lookupResult.mappedKeys.forEach((tp, node) -> partitionsPerBroker.computeIfAbsent(node, key -> new HashSet<>()).add(tp)); + + DeleteRecordsRequest deleteRequest = handler.buildBatchedRequest(node1.id(), partitionsPerBroker.get(node1.id())).build(); + assertEquals(2, deleteRequest.data().topics().get(0).partitions().size()); + assertEquals(mkSet(t0p0, t0p2), + deleteRequest.data().topics().get(0).partitions().stream() + .map(drp -> new TopicPartition("t0", drp.partitionIndex())) + .collect(Collectors.toSet())); + deleteRequest = handler.buildBatchedRequest(node2.id(), partitionsPerBroker.get(node2.id())).build(); + assertEquals(2, deleteRequest.data().topics().get(0).partitions().size()); + assertEquals(mkSet(t0p1, t0p3), + deleteRequest.data().topics().get(0).partitions().stream() + .map(drp -> new TopicPartition("t0", drp.partitionIndex())) + .collect(Collectors.toSet())); + } + private DeleteRecordsResponse createResponse(Map errorsByPartition) { return createResponse(errorsByPartition, recordsToDelete.keySet()); } @@ -227,7 +286,7 @@ private DeleteRecordsResponse createResponse( private AdminApiHandler.ApiResult handleResponse(DeleteRecordsResponse response) { DeleteRecordsHandler handler = new DeleteRecordsHandler(recordsToDelete, logContext, timeout); - return handler.handleResponse(node, recordsToDelete.keySet(), response); + return handler.handleResponse(node1, recordsToDelete.keySet(), response); } private void assertResult( From f5c4d522fd79775692441cd75f6f733324f2e7e9 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Thu, 7 Mar 2024 07:51:04 -0800 Subject: [PATCH 042/521] MINOR: Add read/write all operation (#15462) There are a few cases in the group coordinator service where we want to read from or write to each of the known coordinators (each of __consumer_offsets partitions). The current implementation needs to get the list of the known coordinators then schedules the operation and finally aggregate the results. This patch is an attempt to streamline this by adding multi read/write to the runtime. Reviewers: Omnia Ibrahim , Chia-Ping Tsai --- .../group/GroupCoordinatorService.java | 60 +++---- .../group/runtime/CoordinatorRuntime.java | 67 ++++++-- .../group/GroupCoordinatorServiceTest.java | 148 ++++++------------ .../group/runtime/CoordinatorRuntimeTest.java | 110 +++++++++++++ .../apache/kafka/server/util/FutureUtils.java | 19 +++ 5 files changed, 254 insertions(+), 150 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java index a2363b4822..2fdc128c7b 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java @@ -80,7 +80,6 @@ import java.util.Map; import java.util.OptionalInt; import java.util.Properties; -import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; @@ -498,29 +497,24 @@ public CompletableFuture listGroups( ); } - final Set existingPartitionSet = runtime.partitions(); - - if (existingPartitionSet.isEmpty()) { - return CompletableFuture.completedFuture(new ListGroupsResponseData()); - } - - final List>> futures = - new ArrayList<>(); - - for (TopicPartition tp : existingPartitionSet) { - futures.add(runtime.scheduleReadOperation( + final List>> futures = FutureUtils.mapExceptionally( + runtime.scheduleReadAllOperation( "list-groups", - tp, - (coordinator, lastCommittedOffset) -> coordinator.listGroups(request.statesFilter(), request.typesFilter(), lastCommittedOffset) - ).exceptionally(exception -> { + (coordinator, lastCommittedOffset) -> coordinator.listGroups( + request.statesFilter(), + request.typesFilter(), + lastCommittedOffset + ) + ), + exception -> { exception = Errors.maybeUnwrapException(exception); if (exception instanceof NotCoordinatorException) { return Collections.emptyList(); } else { throw new CompletionException(exception); } - })); - } + } + ); return FutureUtils .combineFutures(futures, ArrayList::new, List::addAll) @@ -963,23 +957,21 @@ public void onPartitionsDeleted( ) throws ExecutionException, InterruptedException { throwIfNotActive(); - final Set existingPartitionSet = runtime.partitions(); - final List> futures = new ArrayList<>(existingPartitionSet.size()); - - existingPartitionSet.forEach(partition -> futures.add( - runtime.scheduleWriteOperation( - "on-partition-deleted", - partition, - Duration.ofMillis(config.offsetCommitTimeoutMs), - coordinator -> coordinator.onPartitionsDeleted(topicPartitions) - ).exceptionally(exception -> { - log.error("Could not delete offsets for deleted partitions {} in coordinator {} due to: {}.", - partition, partition, exception.getMessage(), exception); - return null; - }) - )); - - CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).get(); + CompletableFuture.allOf( + FutureUtils.mapExceptionally( + runtime.scheduleWriteAllOperation( + "on-partition-deleted", + Duration.ofMillis(config.offsetCommitTimeoutMs), + coordinator -> coordinator.onPartitionsDeleted(topicPartitions) + ), + exception -> { + log.error("Could not delete offsets for deleted partitions {} due to: {}.", + topicPartitions, exception.getMessage(), exception + ); + return null; + } + ).toArray(new CompletableFuture[0]) + ).get(); } /** diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java index ccb4caf04f..6b98a51dd4 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java @@ -41,12 +41,11 @@ import java.time.Duration; import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.OptionalInt; import java.util.OptionalLong; -import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.RejectedExecutionException; @@ -54,6 +53,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; +import java.util.stream.Collectors; /** * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator @@ -1446,6 +1446,32 @@ public CompletableFuture scheduleWriteOperation( return event.future; } + /** + * Schedule a write operation for each coordinator. + * + * @param name The name of the write operation. + * @param timeout The write operation timeout. + * @param op The write operation. + * + * @return A list of futures where each future will be completed with the result of the write operation + * when the operation is completed or an exception if the write operation failed. + * + * @param The type of the result. + */ + public List> scheduleWriteAllOperation( + String name, + Duration timeout, + CoordinatorWriteOperation op + ) { + throwIfNotRunning(); + log.debug("Scheduled execution of write all operation {}.", name); + return coordinators + .keySet() + .stream() + .map(tp -> scheduleWriteOperation(name, tp, timeout, op)) + .collect(Collectors.toList()); + } + /** * Schedules a transactional write operation. * @@ -1535,12 +1561,12 @@ public CompletableFuture scheduleTransactionCompletion( /** * Schedules a read operation. * - * @param name The name of the write operation. + * @param name The name of the read operation. * @param tp The address of the coordinator (aka its topic-partitions). * @param op The read operation. * * @return A future that will be completed with the result of the read operation - * when the operation is completed or an exception if the write operation failed. + * when the operation is completed or an exception if the read operation failed. * * @param The type of the result. */ @@ -1556,6 +1582,30 @@ public CompletableFuture scheduleReadOperation( return event.future; } + /** + * Schedules a read operation for each coordinator. + * + * @param name The name of the read operation. + * @param op The read operation. + * + * @return A list of futures where each future will be completed with the result of the read operation + * when the operation is completed or an exception if the read operation failed. + * + * @param The type of the result. + */ + public List> scheduleReadAllOperation( + String name, + CoordinatorReadOperation op + ) { + throwIfNotRunning(); + log.debug("Scheduled execution of read all operation {}.", name); + return coordinators + .keySet() + .stream() + .map(tp -> scheduleReadOperation(name, tp, op)) + .collect(Collectors.toList()); + } + /** * Schedules an internal event. * @@ -1572,15 +1622,6 @@ private void scheduleInternalOperation( enqueue(new CoordinatorInternalEvent(name, tp, op)); } - /** - * @return The topic partitions of the coordinators currently registered in the - * runtime. - */ - public Set partitions() { - throwIfNotRunning(); - return new HashSet<>(coordinators.keySet()); - } - /** * Schedules the loading of a coordinator. This is called when the broker is elected as * the leader for a partition. diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java index 2404d304be..0d93f45068 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java @@ -77,7 +77,6 @@ import org.junit.jupiter.params.provider.NullSource; import org.junit.jupiter.params.provider.ValueSource; import org.mockito.ArgumentMatchers; -import org.mockito.internal.util.collections.Sets; import java.net.InetAddress; import java.time.Duration; @@ -91,8 +90,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.stream.Collectors; -import java.util.stream.IntStream; import java.util.stream.Stream; import static org.apache.kafka.common.requests.JoinGroupRequest.UNKNOWN_MEMBER_ID; @@ -759,10 +756,7 @@ public void testListGroups() throws ExecutionException, InterruptedException, Ti runtime, new GroupCoordinatorMetrics() ); - int partitionCount = 3; - service.startup(() -> partitionCount); - - ListGroupsRequestData request = new ListGroupsRequestData(); + service.startup(() -> 3); List expectedResults = Arrays.asList( new ListGroupsResponseData.ListedGroup() @@ -781,26 +775,22 @@ public void testListGroups() throws ExecutionException, InterruptedException, Ti .setGroupState("Dead") .setGroupType("consumer") ); - when(runtime.partitions()).thenReturn(Sets.newSet( - new TopicPartition("__consumer_offsets", 0), - new TopicPartition("__consumer_offsets", 1), - new TopicPartition("__consumer_offsets", 2) + + when(runtime.scheduleReadAllOperation( + ArgumentMatchers.eq("list-groups"), + ArgumentMatchers.any() + )).thenReturn(Arrays.asList( + CompletableFuture.completedFuture(Collections.singletonList(expectedResults.get(0))), + CompletableFuture.completedFuture(Collections.singletonList(expectedResults.get(1))), + CompletableFuture.completedFuture(Collections.singletonList(expectedResults.get(2))) )); - for (int i = 0; i < partitionCount; i++) { - when(runtime.scheduleReadOperation( - ArgumentMatchers.eq("list-groups"), - ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", i)), - ArgumentMatchers.any() - )).thenReturn(CompletableFuture.completedFuture(Collections.singletonList(expectedResults.get(i)))); - } CompletableFuture responseFuture = service.listGroups( requestContext(ApiKeys.LIST_GROUPS), - request + new ListGroupsRequestData() ); - List actualResults = responseFuture.get(5, TimeUnit.SECONDS).groups(); - assertEquals(expectedResults, actualResults); + assertEquals(expectedResults, responseFuture.get(5, TimeUnit.SECONDS).groups()); } @Test @@ -813,8 +803,7 @@ public void testListGroupsFailedWithNotCoordinatorException() runtime, new GroupCoordinatorMetrics() ); - int partitionCount = 3; - service.startup(() -> partitionCount); + service.startup(() -> 3); List expectedResults = Arrays.asList( new ListGroupsResponseData.ListedGroup() @@ -829,36 +818,25 @@ public void testListGroupsFailedWithNotCoordinatorException() .setGroupType("consumer") ); - ListGroupsRequestData request = new ListGroupsRequestData(); - when(runtime.partitions()).thenReturn(Sets.newSet( - new TopicPartition("__consumer_offsets", 0), - new TopicPartition("__consumer_offsets", 1), - new TopicPartition("__consumer_offsets", 2) - )); - for (int i = 0; i < 2; i++) { - when(runtime.scheduleReadOperation( - ArgumentMatchers.eq("list-groups"), - ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", i)), - ArgumentMatchers.any() - )).thenReturn(CompletableFuture.completedFuture(Collections.singletonList(expectedResults.get(i)))); - } - - when(runtime.scheduleReadOperation( + when(runtime.scheduleReadAllOperation( ArgumentMatchers.eq("list-groups"), - ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 2)), ArgumentMatchers.any() - )).thenReturn(FutureUtils.failedFuture(new NotCoordinatorException(""))); + )).thenReturn(Arrays.asList( + CompletableFuture.completedFuture(Collections.singletonList(expectedResults.get(0))), + CompletableFuture.completedFuture(Collections.singletonList(expectedResults.get(1))), + FutureUtils.failedFuture(new NotCoordinatorException("")) + )); CompletableFuture responseFuture = service.listGroups( requestContext(ApiKeys.LIST_GROUPS), - request + new ListGroupsRequestData() ); - List actualResults = responseFuture.get(5, TimeUnit.SECONDS).groups(); - assertEquals(expectedResults, actualResults); + + assertEquals(expectedResults, responseFuture.get(5, TimeUnit.SECONDS).groups()); } @Test - public void testListGroupsFailedImmediately() + public void testListGroupsWithFailure() throws InterruptedException, ExecutionException, TimeoutException { CoordinatorRuntime runtime = mockRuntime(); GroupCoordinatorService service = new GroupCoordinatorService( @@ -867,37 +845,27 @@ public void testListGroupsFailedImmediately() runtime, new GroupCoordinatorMetrics() ); - int partitionCount = 3; - service.startup(() -> partitionCount); - - ListGroupsRequestData request = new ListGroupsRequestData(); - when(runtime.partitions()).thenReturn(Sets.newSet( - new TopicPartition("__consumer_offsets", 0), - new TopicPartition("__consumer_offsets", 1), - new TopicPartition("__consumer_offsets", 2) - )); - for (int i = 0; i < 2; i++) { - when(runtime.scheduleReadOperation( - ArgumentMatchers.eq("list-groups"), - ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", i)), - ArgumentMatchers.any() - )).thenReturn(CompletableFuture.completedFuture(Collections.emptyList())); - } + service.startup(() -> 3); - when(runtime.scheduleReadOperation( + when(runtime.scheduleReadAllOperation( ArgumentMatchers.eq("list-groups"), - ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 2)), ArgumentMatchers.any() - )).thenReturn(FutureUtils.failedFuture(new CoordinatorLoadInProgressException(""))); + )).thenReturn(Arrays.asList( + CompletableFuture.completedFuture(Collections.emptyList()), + CompletableFuture.completedFuture(Collections.emptyList()), + FutureUtils.failedFuture(new CoordinatorLoadInProgressException("")) + )); CompletableFuture responseFuture = service.listGroups( requestContext(ApiKeys.LIST_GROUPS), - request + new ListGroupsRequestData() ); - ListGroupsResponseData listGroupsResponseData = responseFuture.get(5, TimeUnit.SECONDS); - assertEquals(Errors.COORDINATOR_LOAD_IN_PROGRESS.code(), listGroupsResponseData.errorCode()); - assertEquals(Collections.emptyList(), listGroupsResponseData.groups()); + assertEquals( + new ListGroupsResponseData() + .setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code()), + responseFuture.get(5, TimeUnit.SECONDS) + ); } @Test @@ -1703,12 +1671,6 @@ public void testDeleteGroups() throws Exception { result1.duplicate() )); - when(runtime.partitions()).thenReturn(Sets.newSet( - new TopicPartition("__consumer_offsets", 0), - new TopicPartition("__consumer_offsets", 1), - new TopicPartition("__consumer_offsets", 2) - )); - when(runtime.scheduleWriteOperation( ArgumentMatchers.eq("delete-groups"), ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 2)), @@ -2067,7 +2029,6 @@ public void testCompleteTransactionWithUnexpectedPartition() { @Test public void testOnPartitionsDeleted() { - int partitionCount = 3; CoordinatorRuntime runtime = mockRuntime(); GroupCoordinatorService service = new GroupCoordinatorService( new LogContext(), @@ -2075,36 +2036,17 @@ public void testOnPartitionsDeleted() { runtime, new GroupCoordinatorMetrics() ); + service.startup(() -> 3); - service.startup(() -> partitionCount); - - when(runtime.partitions()).thenReturn( - IntStream - .range(0, partitionCount) - .mapToObj(i -> new TopicPartition("__consumer_offsets", i)) - .collect(Collectors.toSet()) - ); - - List> futures = IntStream - .range(0, partitionCount) - .mapToObj(__ -> new CompletableFuture()) - .collect(Collectors.toList()); - - IntStream.range(0, partitionCount).forEach(i -> { - CompletableFuture future = futures.get(i); - when(runtime.scheduleWriteOperation( - ArgumentMatchers.eq("on-partition-deleted"), - ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", i)), - ArgumentMatchers.eq(Duration.ofMillis(5000)), - ArgumentMatchers.any() - )).thenAnswer(__ -> future); - }); - - IntStream.range(0, partitionCount - 1).forEach(i -> { - futures.get(i).complete(null); - }); - - futures.get(partitionCount - 1).completeExceptionally(Errors.COORDINATOR_LOAD_IN_PROGRESS.exception()); + when(runtime.scheduleWriteAllOperation( + ArgumentMatchers.eq("on-partition-deleted"), + ArgumentMatchers.eq(Duration.ofMillis(5000)), + ArgumentMatchers.any() + )).thenReturn(Arrays.asList( + CompletableFuture.completedFuture(null), + CompletableFuture.completedFuture(null), + FutureUtils.failedFuture(Errors.COORDINATOR_LOAD_IN_PROGRESS.exception()) + )); // The exception is logged and swallowed. assertDoesNotThrow(() -> diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java index ba1a340e8e..4e10978d35 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java @@ -43,6 +43,7 @@ import org.mockito.ArgumentMatcher; import java.time.Duration; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashSet; @@ -1145,6 +1146,63 @@ public void testScheduleWriteOpWhenWriteTimesOut() throws InterruptedException { assertFutureThrows(timedOutWrite, org.apache.kafka.common.errors.TimeoutException.class); } + @Test + public void testScheduleWriteAllOperation() throws ExecutionException, InterruptedException, TimeoutException { + MockTimer timer = new MockTimer(); + MockPartitionWriter writer = new MockPartitionWriter(); + + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(DEFAULT_WRITE_TIMEOUT) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(new DirectEventProcessor()) + .withPartitionWriter(writer) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(GroupCoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(GroupCoordinatorMetrics.class)) + .build(); + + TopicPartition coordinator0 = new TopicPartition("__consumer_offsets", 0); + TopicPartition coordinator1 = new TopicPartition("__consumer_offsets", 1); + TopicPartition coordinator2 = new TopicPartition("__consumer_offsets", 2); + + // Load coordinators. + runtime.scheduleLoadOperation(coordinator0, 10); + runtime.scheduleLoadOperation(coordinator1, 10); + runtime.scheduleLoadOperation(coordinator2, 10); + + // Writes. + AtomicInteger cnt = new AtomicInteger(0); + List>> writes = runtime.scheduleWriteAllOperation("write", DEFAULT_WRITE_TIMEOUT, state -> { + int counter = cnt.getAndIncrement(); + return new CoordinatorResult<>( + Collections.singletonList("record#" + counter), + Collections.singletonList("response#" + counter) + ); + }); + + assertEquals(1L, runtime.contextOrThrow(coordinator0).coordinator.lastWrittenOffset()); + assertEquals(1L, runtime.contextOrThrow(coordinator1).coordinator.lastWrittenOffset()); + assertEquals(1L, runtime.contextOrThrow(coordinator2).coordinator.lastWrittenOffset()); + + assertEquals(Collections.singletonList(InMemoryPartitionWriter.LogEntry.value("record#0")), writer.entries(coordinator0)); + assertEquals(Collections.singletonList(InMemoryPartitionWriter.LogEntry.value("record#1")), writer.entries(coordinator1)); + assertEquals(Collections.singletonList(InMemoryPartitionWriter.LogEntry.value("record#2")), writer.entries(coordinator2)); + + // Commit. + writer.commit(coordinator0); + writer.commit(coordinator1); + writer.commit(coordinator2); + + // Verify. + assertEquals( + Arrays.asList("response#0", "response#1", "response#2"), + FutureUtils.combineFutures(writes, ArrayList::new, List::addAll).get(5, TimeUnit.SECONDS) + ); + } + @Test public void testScheduleTransactionalWriteOp() { MockTimer timer = new MockTimer(); @@ -1743,6 +1801,58 @@ public void testScheduleReadOpWhenOpsFails() { assertFutureThrows(read, IllegalArgumentException.class); } + @Test + public void testScheduleReadAllOp() throws ExecutionException, InterruptedException, TimeoutException { + MockTimer timer = new MockTimer(); + MockPartitionWriter writer = new MockPartitionWriter(); + + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(DEFAULT_WRITE_TIMEOUT) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(new DirectEventProcessor()) + .withPartitionWriter(writer) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(GroupCoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(GroupCoordinatorMetrics.class)) + .build(); + + TopicPartition coordinator0 = new TopicPartition("__consumer_offsets", 0); + TopicPartition coordinator1 = new TopicPartition("__consumer_offsets", 1); + TopicPartition coordinator2 = new TopicPartition("__consumer_offsets", 2); + + // Loads the coordinators. + runtime.scheduleLoadOperation(coordinator0, 10); + runtime.scheduleLoadOperation(coordinator1, 10); + runtime.scheduleLoadOperation(coordinator2, 10); + + // Writes + runtime.scheduleWriteOperation("write#0", coordinator0, DEFAULT_WRITE_TIMEOUT, + state -> new CoordinatorResult<>(Collections.singletonList("record0"), "response0")); + runtime.scheduleWriteOperation("write#1", coordinator1, DEFAULT_WRITE_TIMEOUT, + state -> new CoordinatorResult<>(Collections.singletonList("record1"), "response1")); + runtime.scheduleWriteOperation("write#2", coordinator2, DEFAULT_WRITE_TIMEOUT, + state -> new CoordinatorResult<>(Collections.singletonList("record2"), "response2")); + + // Commit writes. + writer.commit(coordinator0); + writer.commit(coordinator1); + writer.commit(coordinator2); + + // Read. + List>> responses = runtime.scheduleReadAllOperation( + "read", + (state, offset) -> new ArrayList<>(state.records) + ); + + assertEquals( + Arrays.asList("record0", "record1", "record2"), + FutureUtils.combineFutures(responses, ArrayList::new, List::addAll).get(5, TimeUnit.SECONDS) + ); + } + @Test public void testClose() throws Exception { MockCoordinatorLoader loader = spy(new MockCoordinatorLoader()); diff --git a/server-common/src/main/java/org/apache/kafka/server/util/FutureUtils.java b/server-common/src/main/java/org/apache/kafka/server/util/FutureUtils.java index a3f78a84b1..a95716407b 100644 --- a/server-common/src/main/java/org/apache/kafka/server/util/FutureUtils.java +++ b/server-common/src/main/java/org/apache/kafka/server/util/FutureUtils.java @@ -19,11 +19,13 @@ import org.apache.kafka.common.utils.Time; import org.slf4j.Logger; +import java.util.ArrayList; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; import java.util.function.BiConsumer; +import java.util.function.Function; import java.util.function.Supplier; @@ -125,4 +127,21 @@ public static CompletableFuture combineFutures( return res; }); } + + /** + * Applies the given exception handler to all the futures provided in the list + * and returns a new list of futures. + * + * @param futures A list of futures. + * @param fn A function taking an exception to handle it. + * @return A list of futures. + */ + public static List> mapExceptionally( + List> futures, + Function fn + ) { + final List> results = new ArrayList<>(futures.size()); + futures.forEach(future -> results.add(future.exceptionally(fn))); + return results; + } } From d8dd068a626dcab538c2b234ffd8799a94b2f0ed Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Thu, 7 Mar 2024 09:17:52 -0800 Subject: [PATCH 043/521] KAFKA-15964: fix flaky StreamsAssignmentScaleTest (#15485) This PR bumps some timeouts due to slow Jenkins builds. Reviewers: Bruno Cadonna --- .../internals/StreamsPartitionAssignor.java | 2 +- .../internals/StreamsAssignmentScaleTest.java | 26 +++++++++---------- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java index 3f16838dfc..fb4e45cdfb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java @@ -628,7 +628,7 @@ private boolean assignTasksToClients(final Cluster fullMetadata, log.info("{} client nodes and {} consumers participating in this rebalance: \n{}.", clientStates.size(), - clientStates.values().stream().map(ClientState::capacity).reduce(Integer::sum), + clientStates.values().stream().map(ClientState::capacity).reduce(Integer::sum).orElse(0), clientStates.entrySet().stream() .sorted(comparingByKey()) .map(entry -> entry.getKey() + ": " + entry.getValue().consumers()) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsAssignmentScaleTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsAssignmentScaleTest.java index 834b624219..e14e033363 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsAssignmentScaleTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsAssignmentScaleTest.java @@ -68,73 +68,73 @@ @Category({IntegrationTest.class}) @RunWith(MockitoJUnitRunner.StrictStubs.class) public class StreamsAssignmentScaleTest { - final static long MAX_ASSIGNMENT_DURATION = 60 * 1000L; //each individual assignment should complete within 20s + final static long MAX_ASSIGNMENT_DURATION = 120 * 1000L; // we should stay below `max.poll.interval.ms` final static String APPLICATION_ID = "streams-assignment-scale-test"; private final Logger log = LoggerFactory.getLogger(StreamsAssignmentScaleTest.class); /* HighAvailabilityTaskAssignor tests */ - @Test(timeout = 120 * 1000) + @Test(timeout = 300 * 1000) public void testHighAvailabilityTaskAssignorLargePartitionCount() { completeLargeAssignment(6_000, 2, 1, 1, HighAvailabilityTaskAssignor.class); } - @Test(timeout = 120 * 1000) + @Test(timeout = 300 * 1000) public void testHighAvailabilityTaskAssignorLargeNumConsumers() { completeLargeAssignment(1_000, 1_000, 1, 1, HighAvailabilityTaskAssignor.class); } - @Test(timeout = 120 * 1000) + @Test(timeout = 300 * 1000) public void testHighAvailabilityTaskAssignorManyStandbys() { completeLargeAssignment(1_000, 100, 1, 50, HighAvailabilityTaskAssignor.class); } - @Test(timeout = 120 * 1000) + @Test(timeout = 300 * 1000) public void testHighAvailabilityTaskAssignorManyThreadsPerClient() { completeLargeAssignment(1_000, 10, 1000, 1, HighAvailabilityTaskAssignor.class); } /* StickyTaskAssignor tests */ - @Test(timeout = 120 * 1000) + @Test(timeout = 300 * 1000) public void testStickyTaskAssignorLargePartitionCount() { completeLargeAssignment(2_000, 2, 1, 1, StickyTaskAssignor.class); } - @Test(timeout = 120 * 1000) + @Test(timeout = 300 * 1000) public void testStickyTaskAssignorLargeNumConsumers() { completeLargeAssignment(1_000, 1_000, 1, 1, StickyTaskAssignor.class); } - @Test(timeout = 120 * 1000) + @Test(timeout = 300 * 1000) public void testStickyTaskAssignorManyStandbys() { completeLargeAssignment(1_000, 100, 1, 20, StickyTaskAssignor.class); } - @Test(timeout = 120 * 1000) + @Test(timeout = 300 * 1000) public void testStickyTaskAssignorManyThreadsPerClient() { completeLargeAssignment(1_000, 10, 1000, 1, StickyTaskAssignor.class); } /* FallbackPriorTaskAssignor tests */ - @Test(timeout = 120 * 1000) + @Test(timeout = 300 * 1000) public void testFallbackPriorTaskAssignorLargePartitionCount() { completeLargeAssignment(2_000, 2, 1, 1, FallbackPriorTaskAssignor.class); } - @Test(timeout = 120 * 1000) + @Test(timeout = 300 * 1000) public void testFallbackPriorTaskAssignorLargeNumConsumers() { completeLargeAssignment(1_000, 1_000, 1, 1, FallbackPriorTaskAssignor.class); } - @Test(timeout = 120 * 1000) + @Test(timeout = 300 * 1000) public void testFallbackPriorTaskAssignorManyStandbys() { completeLargeAssignment(1_000, 100, 1, 20, FallbackPriorTaskAssignor.class); } - @Test(timeout = 120 * 1000) + @Test(timeout = 300 * 1000) public void testFallbackPriorTaskAssignorManyThreadsPerClient() { completeLargeAssignment(1_000, 10, 1000, 1, FallbackPriorTaskAssignor.class); } From 5dd382ccbd1fada55cc44a5223dcae56e0e8811d Mon Sep 17 00:00:00 2001 From: PoAn Yang Date: Fri, 8 Mar 2024 03:02:22 +0800 Subject: [PATCH 044/521] MINOR: Use INFO logging for tools tests (#15487) Reviewers: Luke Chen , Chia-Ping Tsai --- tools/src/test/resources/log4j.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/src/test/resources/log4j.properties b/tools/src/test/resources/log4j.properties index 5291604d49..3aca07dc53 100644 --- a/tools/src/test/resources/log4j.properties +++ b/tools/src/test/resources/log4j.properties @@ -12,7 +12,7 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. -log4j.rootLogger=TRACE, stdout +log4j.rootLogger=INFO, stdout log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout From 96d026bc994fb305c366ce4de27804fcd1f7899a Mon Sep 17 00:00:00 2001 From: Dung Ha <60119105+infantlikesprogramming@users.noreply.github.com> Date: Thu, 7 Mar 2024 17:43:57 -0500 Subject: [PATCH 045/521] KAFKA-16202 Extra dot in error message in producer (#15296) The author of KAFKA-16202 noticed that there is an extra dot in the error message for KafkaStorageException message. Looking into org.apache.kafka.clients.producer.internals.Sender, it turns out that the string for the message to be sent in completeBatch() added an extra dot. I think that the formatted component (error.exception(response.errorMessage).toString())) of the error message already has a dot in the end of its string. Thus the dot after the "{}" sign caused the extra dot. Reviewers: "Gyeongwon, Do" , Chia-Ping Tsai --- .../kafka/clients/producer/internals/Sender.java | 5 ++--- .../org/apache/kafka/common/protocol/Errors.java | 12 ++++++------ 2 files changed, 8 insertions(+), 9 deletions(-) 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 2cdc3b876d..99bc1d68b0 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 @@ -706,9 +706,8 @@ private void completeBatch(ProducerBatch batch, ProduceResponse.PartitionRespons "topic-partition may not exist or the user may not have Describe access to it", batch.topicPartition); } else { - log.warn("Received invalid metadata error in produce request on partition {} due to {}. Going " + - "to request metadata update now", batch.topicPartition, - error.exception(response.errorMessage).toString()); + log.warn("Received invalid metadata error in produce request on partition {} due to {} Going " + + "to request metadata update now", batch.topicPartition, error.exception(response.errorMessage).toString()); } if (error.exception() instanceof NotLeaderOrFollowerException || error.exception() instanceof FencedLeaderEpochException) { log.debug("For {}, received error {}, with leaderIdAndEpoch {}", batch.topicPartition, error, response.currentLeader); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index 838be869f6..610b1b66e7 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -363,12 +363,12 @@ public enum Errors { DUPLICATE_RESOURCE(92, "A request illegally referred to the same resource twice.", DuplicateResourceException::new), UNACCEPTABLE_CREDENTIAL(93, "Requested credential would not meet criteria for acceptability.", UnacceptableCredentialException::new), INCONSISTENT_VOTER_SET(94, "Indicates that the either the sender or recipient of a " + - "voter-only request is not one of the expected voters", InconsistentVoterSetException::new), + "voter-only request is not one of the expected voters.", InconsistentVoterSetException::new), INVALID_UPDATE_VERSION(95, "The given update version was invalid.", InvalidUpdateVersionException::new), FEATURE_UPDATE_FAILED(96, "Unable to update finalized features due to an unexpected server error.", FeatureUpdateFailedException::new), PRINCIPAL_DESERIALIZATION_FAILURE(97, "Request principal deserialization failed during forwarding. " + "This indicates an internal error on the broker cluster security setup.", PrincipalDeserializationException::new), - SNAPSHOT_NOT_FOUND(98, "Requested snapshot was not found", SnapshotNotFoundException::new), + SNAPSHOT_NOT_FOUND(98, "Requested snapshot was not found.", SnapshotNotFoundException::new), POSITION_OUT_OF_RANGE( 99, "Requested position is not greater than or equal to zero, and less than the size of the snapshot.", @@ -376,10 +376,10 @@ public enum Errors { UNKNOWN_TOPIC_ID(100, "This server does not host this topic ID.", UnknownTopicIdException::new), DUPLICATE_BROKER_REGISTRATION(101, "This broker ID is already in use.", DuplicateBrokerRegistrationException::new), BROKER_ID_NOT_REGISTERED(102, "The given broker ID was not registered.", BrokerIdNotRegisteredException::new), - INCONSISTENT_TOPIC_ID(103, "The log's topic ID did not match the topic ID in the request", InconsistentTopicIdException::new), - INCONSISTENT_CLUSTER_ID(104, "The clusterId in the request does not match that found on the server", InconsistentClusterIdException::new), - TRANSACTIONAL_ID_NOT_FOUND(105, "The transactionalId could not be found", TransactionalIdNotFoundException::new), - FETCH_SESSION_TOPIC_ID_ERROR(106, "The fetch session encountered inconsistent topic ID usage", FetchSessionTopicIdException::new), + INCONSISTENT_TOPIC_ID(103, "The log's topic ID did not match the topic ID in the request.", InconsistentTopicIdException::new), + INCONSISTENT_CLUSTER_ID(104, "The clusterId in the request does not match that found on the server.", InconsistentClusterIdException::new), + TRANSACTIONAL_ID_NOT_FOUND(105, "The transactionalId could not be found.", TransactionalIdNotFoundException::new), + FETCH_SESSION_TOPIC_ID_ERROR(106, "The fetch session encountered inconsistent topic ID usage.", FetchSessionTopicIdException::new), INELIGIBLE_REPLICA(107, "The new ISR contains at least one ineligible replica.", IneligibleReplicaException::new), NEW_LEADER_ELECTED(108, "The AlterPartition request successfully updated the partition state but the leader has changed.", NewLeaderElectedException::new), OFFSET_MOVED_TO_TIERED_STORAGE(109, "The requested offset is moved to tiered storage.", OffsetMovedToTieredStorageException::new), From 3f3cee1b21f71bcc3f478514bc996e6c9ef6ccaa Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Fri, 8 Mar 2024 07:07:21 +0800 Subject: [PATCH 046/521] KAFKA-16325 Add missing producer metrics to documentatio (#15466) Add `buffer-exhausted-rate`, `buffer-exhausted-total`, `bufferpool-wait-ratio` and `metadata-wait-time-ns-total` Reviewers: Chia-Ping Tsai --- docs/ops.html | 21 ++++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) diff --git a/docs/ops.html b/docs/ops.html index 92f6761555..9f49b5c1b2 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -2537,11 +2537,26 @@

< The total amount of buffer memory that is not being used (either unallocated or in the free list). kafka.producer:type=producer-metrics,client-id=([-.\w]+) + + buffer-exhausted-rate + The average per-second number of record sends that are dropped due to buffer exhaustion + kafka.producer:type=producer-metrics,client-id=([-.\w]+) + + + buffer-exhausted-total + The total number of record sends that are dropped due to buffer exhaustion + kafka.producer:type=producer-metrics,client-id=([-.\w]+) + bufferpool-wait-time The fraction of time an appender waits for space allocation. kafka.producer:type=producer-metrics,client-id=([-.\w]+) + + bufferpool-wait-ratio + The fraction of time an appender waits for space allocation. + kafka.producer:type=producer-metrics,client-id=([-.\w]+) + bufferpool-wait-time-total *Deprecated* The total time an appender waits for space allocation in nanoseconds. Replacement is bufferpool-wait-time-ns-total @@ -2582,7 +2597,11 @@

< The total time the Producer spent aborting transactions in nanoseconds (for EOS). kafka.producer:type=producer-metrics,client-id=([-.\w]+) - + + metadata-wait-time-ns-total + the total time in nanoseconds that has spent waiting for metadata from the Kafka broker + kafka.producer:type=producer-metrics,client-id=([-.\w]+) +

Producer Sender Metrics
From 80def43a3438f74344f034f6a326c02d76151c80 Mon Sep 17 00:00:00 2001 From: testn Date: Fri, 8 Mar 2024 23:43:44 +0700 Subject: [PATCH 047/521] MINOR: Reduce memory allocation in ClientTelemetryReporter (#15402) Reviewers: Divij Vaidya --- checkstyle/suppressions.xml | 2 +- .../internals/ClientTelemetryReporter.java | 19 +++++++++++-------- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 32916fe660..e384ad8c82 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -93,7 +93,7 @@ files="(Utils|Topic|KafkaLZ4BlockOutputStream|AclData|JoinGroupRequest).java"/> + files="(AbstractFetch|ClientTelemetryReporter|ConsumerCoordinator|CommitRequestManager|FetchCollector|OffsetFetcherUtils|KafkaProducer|Sender|ConfigDef|KerberosLogin|AbstractRequest|AbstractResponse|Selector|SslFactory|SslTransportLayer|SaslClientAuthenticator|SaslClientCallbackHandler|SaslServerAuthenticator|AbstractCoordinator|TransactionManager|AbstractStickyAssignor|DefaultSslEngineFactory|Authorizer|RecordAccumulator|MemoryRecords|FetchSessionHandler|MockAdminClient).java"/> diff --git a/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java b/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java index ae92c539da..483179d4c4 100644 --- a/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java +++ b/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java @@ -265,8 +265,8 @@ class DefaultClientTelemetrySender implements ClientTelemetrySender { These are the lower and upper bounds of the jitter that we apply to the initial push telemetry API call. This helps to avoid a flood of requests all coming at the same time. */ - private final static double INITIAL_PUSH_JITTER_LOWER = 0.5; - private final static double INITIAL_PUSH_JITTER_UPPER = 1.5; + private static final double INITIAL_PUSH_JITTER_LOWER = 0.5; + private static final double INITIAL_PUSH_JITTER_UPPER = 1.5; private final ReadWriteLock lock = new ReentrantReadWriteLock(); private final Condition subscriptionLoaded = lock.writeLock().newCondition(); @@ -325,6 +325,7 @@ public long timeToNextUpdate(long requestTimeoutMs) { final long timeMs; final String apiName; final String msg; + final boolean isTraceEnabled = log.isTraceEnabled(); switch (localState) { case SUBSCRIPTION_IN_PROGRESS: @@ -336,15 +337,15 @@ public long timeToNextUpdate(long requestTimeoutMs) { */ apiName = (localState == ClientTelemetryState.SUBSCRIPTION_IN_PROGRESS) ? ApiKeys.GET_TELEMETRY_SUBSCRIPTIONS.name : ApiKeys.PUSH_TELEMETRY.name; timeMs = requestTimeoutMs; - msg = String.format("the remaining wait time for the %s network API request, as specified by %s", apiName, CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG); + msg = isTraceEnabled ? "" : String.format("the remaining wait time for the %s network API request, as specified by %s", apiName, CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG); break; case TERMINATING_PUSH_IN_PROGRESS: timeMs = Long.MAX_VALUE; - msg = String.format("the terminating push is in progress, disabling telemetry for further requests"); + msg = isTraceEnabled ? "" : "the terminating push is in progress, disabling telemetry for further requests"; break; case TERMINATING_PUSH_NEEDED: timeMs = 0; - msg = String.format("the client should try to submit the final %s network API request ASAP before closing", ApiKeys.PUSH_TELEMETRY.name); + msg = isTraceEnabled ? "" : String.format("the client should try to submit the final %s network API request ASAP before closing", ApiKeys.PUSH_TELEMETRY.name); break; case SUBSCRIPTION_NEEDED: case PUSH_NEEDED: @@ -352,17 +353,19 @@ public long timeToNextUpdate(long requestTimeoutMs) { long timeRemainingBeforeRequest = localLastRequestMs + localIntervalMs - nowMs; if (timeRemainingBeforeRequest <= 0) { timeMs = 0; - msg = String.format("the wait time before submitting the next %s network API request has elapsed", apiName); + msg = isTraceEnabled ? "" : String.format("the wait time before submitting the next %s network API request has elapsed", apiName); } else { timeMs = timeRemainingBeforeRequest; - msg = String.format("the client will wait before submitting the next %s network API request", apiName); + msg = isTraceEnabled ? "" : String.format("the client will wait before submitting the next %s network API request", apiName); } break; default: throw new IllegalStateException("Unknown telemetry state: " + localState); } - log.trace("For telemetry state {}, returning the value {} ms; {}", localState, timeMs, msg); + if (isTraceEnabled) { + log.trace("For telemetry state {}, returning the value {} ms; {}", localState, timeMs, msg); + } return timeMs; } From b9a5b4a8053c1fa65e27a9f93440194b0dd5eec4 Mon Sep 17 00:00:00 2001 From: Daan Gerits Date: Fri, 8 Mar 2024 19:57:56 +0100 Subject: [PATCH 048/521] KAFKA-10892: Shared Readonly State Stores ( revisited ) (#12742) Implements KIP-813. Reviewers: Matthias J. Sax , Walker Carlson --- checkstyle/suppressions.xml | 2 +- .../developer-guide/processor-api.html | 13 ++ docs/streams/upgrade-guide.html | 8 + .../org/apache/kafka/streams/Topology.java | 82 ++++++++++ .../apache/kafka/streams/TopologyTest.java | 63 +++++++- .../integration/EosIntegrationTest.java | 24 ++- .../integration/RestoreIntegrationTest.java | 145 ++++++++++++------ .../utils/IntegrationTestUtils.java | 12 ++ .../streams/processor/ReadOnlyStoreTest.java | 132 ++++++++++++++++ .../org/apache/kafka/test/MockProcessor.java | 5 +- 10 files changed, 422 insertions(+), 64 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/ReadOnlyStoreTest.java diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index e384ad8c82..27c252f839 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -234,7 +234,7 @@ + files="(RecordCollectorTest|StreamsPartitionAssignorTest|StreamThreadTest|StreamTaskTest|TaskManagerTest|TopologyTestDriverTest|KafkaStreamsTest|EosIntegrationTest|RestoreIntegrationTest).java"/> diff --git a/docs/streams/developer-guide/processor-api.html b/docs/streams/developer-guide/processor-api.html index ccb03ce7b5..e25ecd601a 100644 --- a/docs/streams/developer-guide/processor-api.html +++ b/docs/streams/developer-guide/processor-api.html @@ -51,6 +51,7 @@
  • Enable or Disable Fault Tolerance of State Stores (Store Changelogs)
  • Timestamped State Stores
  • Versioned Key-Value State Stores
  • +
  • Readonly State Stores
  • Implementing Custom State Stores
  • @@ -466,6 +467,18 @@

    stores to rebuild state from changelog.

    +
    +

    ReadOnly State Stores

    +

    A read-only state store materialized the data from its input topic. It also uses the input topic + for fault-tolerance, and thus does not have an additional changelog topic (the input topic is + re-used as changelog). Thus, the input topic should be configured with log compaction. + Note that no other processor should modify the content of the state store, and the only writer + should be the associated "state update processor"; other processors may read the content of the + read-only store.

    + +

    note: beware of the partitioning requirements when using read-only state stores for lookups during + processing. You might want to make sure the original changelog topic is co-partitioned with the processors + reading the read-only statestore.

    Implementing Custom State Stores

    diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html index 1454407740..0f819cb384 100644 --- a/docs/streams/upgrade-guide.html +++ b/docs/streams/upgrade-guide.html @@ -133,6 +133,14 @@

    < More details about the new config StreamsConfig#TOPOLOGY_OPTIMIZATION can be found in KIP-295.

    +

    Streams API changes in 3.8.0

    +

    + The Processor API now support so-called read-only state stores, added via + KIP-813. + These stores don't have a dedicated changelog topic, but use their source topic for fault-tolerance, + simlar to KTables with source-topic optimization enabled. +

    +

    Streams API changes in 3.7.0

    We added a new method to KafkaStreams, namely KafkaStreams#setStandbyUpdateListener() in diff --git a/streams/src/main/java/org/apache/kafka/streams/Topology.java b/streams/src/main/java/org/apache/kafka/streams/Topology.java index d9c810aa42..d1f0d1eb8b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/Topology.java +++ b/streams/src/main/java/org/apache/kafka/streams/Topology.java @@ -738,6 +738,88 @@ public synchronized Topology addStateStore(final StoreBuilder storeBuilder, return this; } + /** + * Adds a read-only {@link StateStore} to the topology. + *

    + * A read-only {@link StateStore} does not create a dedicated changelog topic but uses it's input topic as + * changelog; thus, the used topic should be configured with log compaction. + *

    + * The auto.offset.reset property will be set to earliest for this topic. + *

    + * The provided {@link ProcessorSupplier} will be used to create a processor for all messages received + * from the given topic. This processor should contain logic to keep the {@link StateStore} up-to-date. + * + * @param storeBuilder user defined store builder + * @param sourceName name of the {@link SourceNode} that will be automatically added + * @param timestampExtractor the stateless timestamp extractor used for this source, + * if not specified the default extractor defined in the configs will be used + * @param keyDeserializer the {@link Deserializer} to deserialize keys with + * @param valueDeserializer the {@link Deserializer} to deserialize values with + * @param topic the topic to source the data from + * @param processorName the name of the {@link ProcessorSupplier} + * @param stateUpdateSupplier the instance of {@link ProcessorSupplier} + * @return itself + * @throws TopologyException if the processor of state is already registered + */ + public synchronized Topology addReadOnlyStateStore(final StoreBuilder storeBuilder, + final String sourceName, + final TimestampExtractor timestampExtractor, + final Deserializer keyDeserializer, + final Deserializer valueDeserializer, + final String topic, + final String processorName, + final ProcessorSupplier stateUpdateSupplier) { + storeBuilder.withLoggingDisabled(); + + internalTopologyBuilder.addSource(AutoOffsetReset.EARLIEST, sourceName, timestampExtractor, keyDeserializer, valueDeserializer, topic); + internalTopologyBuilder.addProcessor(processorName, stateUpdateSupplier, sourceName); + internalTopologyBuilder.addStateStore(storeBuilder, processorName); + internalTopologyBuilder.connectSourceStoreAndTopic(storeBuilder.name(), topic); + + return this; + } + + /** + * Adds a read-only {@link StateStore} to the topology. + *

    + * A read-only {@link StateStore} does not create a dedicated changelog topic but uses it's input topic as + * changelog; thus, the used topic should be configured with log compaction. + *

    + * The auto.offset.reset property will be set to earliest for this topic. + *

    + * The provided {@link ProcessorSupplier} will be used to create a processor for all messages received + * from the given topic. This processor should contain logic to keep the {@link StateStore} up-to-date. + * The default {@link TimestampExtractor} as specified in the {@link StreamsConfig config} is used. + * + * @param storeBuilder user defined store builder + * @param sourceName name of the {@link SourceNode} that will be automatically added + * @param keyDeserializer the {@link Deserializer} to deserialize keys with + * @param valueDeserializer the {@link Deserializer} to deserialize values with + * @param topic the topic to source the data from + * @param processorName the name of the {@link ProcessorSupplier} + * @param stateUpdateSupplier the instance of {@link ProcessorSupplier} + * @return itself + * @throws TopologyException if the processor of state is already registered + */ + public synchronized Topology addReadOnlyStateStore(final StoreBuilder storeBuilder, + final String sourceName, + final Deserializer keyDeserializer, + final Deserializer valueDeserializer, + final String topic, + final String processorName, + final ProcessorSupplier stateUpdateSupplier) { + return addReadOnlyStateStore( + storeBuilder, + sourceName, + null, + keyDeserializer, + valueDeserializer, + topic, + processorName, + stateUpdateSupplier + ); + } + /** * Adds a global {@link StateStore} to the topology. * The {@link StateStore} sources its data from all partitions of the provided input topic. diff --git a/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java index 5fdf5c220c..ceecbc3bc2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java @@ -38,6 +38,7 @@ import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.SubtopologyDescription; import org.apache.kafka.streams.processor.internals.ProcessorTopology; +import org.apache.kafka.streams.processor.internals.StoreFactory; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.SessionStore; import org.apache.kafka.streams.state.StoreBuilder; @@ -58,6 +59,7 @@ import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.junit.MockitoJUnitRunner; +import org.mockito.internal.util.collections.Sets; import java.time.Duration; import java.util.Arrays; @@ -337,7 +339,7 @@ public void shouldNotAllowToAddStoreWithSameNameAndDifferentInstance() { mockStoreBuilder(); topology.addStateStore(storeBuilder); - final StoreBuilder otherStoreBuilder = mock(StoreBuilder.class); + final StoreBuilder otherStoreBuilder = mock(StoreBuilder.class); when(otherStoreBuilder.name()).thenReturn("store"); when(otherStoreBuilder.logConfig()).thenReturn(Collections.emptyMap()); when(otherStoreBuilder.loggingEnabled()).thenReturn(false); @@ -2313,7 +2315,7 @@ private TopologyDescription.Sink addSink(final String sinkName, topology.addSink(sinkName, sinkTopic, null, null, null, parentNames); final TopologyDescription.Sink expectedSinkNode = - new InternalTopologyBuilder.Sink(sinkName, sinkTopic); + new InternalTopologyBuilder.Sink<>(sinkName, sinkTopic); for (final TopologyDescription.Node parent : parents) { ((InternalTopologyBuilder.AbstractNode) parent).addSuccessor(expectedSinkNode); @@ -2351,6 +2353,63 @@ private void addGlobalStoreToTopologyAndExpectedDescription(final String globalS expectedDescription.addGlobalStore(expectedGlobalStore); } + @Test + public void readOnlyStateStoresShouldHaveTheirOwnSubTopology() { + final String sourceName = "source"; + final String storeName = "store"; + final String topicName = "topic"; + final String processorName = "processor"; + + final KeyValueStoreBuilder storeBuilder = mock(KeyValueStoreBuilder.class); + when(storeBuilder.name()).thenReturn(storeName); + topology.addReadOnlyStateStore( + storeBuilder, + sourceName, + null, + null, + null, + topicName, + processorName, + new MockProcessorSupplier<>()); + + final TopologyDescription.Source expectedSource = new InternalTopologyBuilder.Source(sourceName, Sets.newSet(topicName), null); + final TopologyDescription.Processor expectedProcessor = new InternalTopologyBuilder.Processor(processorName, Sets.newSet(storeName)); + + ((InternalTopologyBuilder.AbstractNode) expectedSource).addSuccessor(expectedProcessor); + ((InternalTopologyBuilder.AbstractNode) expectedProcessor).addPredecessor(expectedSource); + + final Set allNodes = new HashSet<>(); + allNodes.add(expectedSource); + allNodes.add(expectedProcessor); + expectedDescription.addSubtopology(new SubtopologyDescription(0, allNodes)); + + assertThat(topology.describe(), equalTo(expectedDescription)); + assertThat(topology.describe().hashCode(), equalTo(expectedDescription.hashCode())); + } + + @Test + public void readOnlyStateStoresShouldNotLog() { + final String sourceName = "source"; + final String storeName = "store"; + final String topicName = "topic"; + final String processorName = "processor"; + + final KeyValueStoreBuilder storeBuilder = mock(KeyValueStoreBuilder.class); + when(storeBuilder.name()).thenReturn(storeName); + topology.addReadOnlyStateStore( + storeBuilder, + sourceName, + null, + null, + null, + topicName, + processorName, + new MockProcessorSupplier<>()); + + final StoreFactory stateStoreFactory = topology.internalTopologyBuilder.stateStores().get(storeName); + assertThat(stateStoreFactory.loggingEnabled(), equalTo(false)); + } + private TopologyConfig overrideDefaultStore(final String defaultStore) { final Properties topologyOverrides = new Properties(); // change default store as in-memory diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java index d79631f8f7..7989de0e76 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java @@ -123,8 +123,8 @@ public class EosIntegrationTest { public Timeout globalTimeout = Timeout.seconds(600); private static final Logger LOG = LoggerFactory.getLogger(EosIntegrationTest.class); private static final int NUM_BROKERS = 3; - private static final int MAX_POLL_INTERVAL_MS = 5 * 1000; - private static final int MAX_WAIT_TIME_MS = 60 * 1000; + private static final int MAX_POLL_INTERVAL_MS = 30_1000; + private static final int MAX_WAIT_TIME_MS = 120_1000; public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster( NUM_BROKERS, @@ -403,7 +403,7 @@ public void shouldNotViolateEosIfOneTaskFails() throws Exception { // -> the failure only kills one thread // after fail over, we should read 40 committed records (even if 50 record got written) - try (final KafkaStreams streams = getKafkaStreams("dummy", false, "appDir", 2, eosConfig, MAX_POLL_INTERVAL_MS)) { + try (final KafkaStreams streams = getKafkaStreams("dummy", false, "appDir", 2, eosConfig)) { startApplicationAndWaitUntilRunning(streams); final List> committedDataBeforeFailure = prepareData(0L, 10L, 0L, 1L); @@ -511,7 +511,7 @@ public void shouldNotViolateEosIfOneTaskFailsWithState() throws Exception { // We need more processing time under "with state" situation, so increasing the max.poll.interval.ms // to avoid unexpected rebalance during test, which will cause unexpected fail over triggered - try (final KafkaStreams streams = getKafkaStreams("dummy", true, "appDir", 2, eosConfig, 3 * MAX_POLL_INTERVAL_MS)) { + try (final KafkaStreams streams = getKafkaStreams("dummy", true, "appDir", 2, eosConfig)) { startApplicationAndWaitUntilRunning(streams); final List> committedDataBeforeFailure = prepareData(0L, 10L, 0L, 1L); @@ -624,12 +624,12 @@ public void shouldNotViolateEosIfOneTaskGetsFencedUsingIsolatedAppInstances() th // -> the stall only affects one thread and should trigger a rebalance // after rebalancing, we should read 40 committed records (even if 50 record got written) // - // afterwards, the "stalling" thread resumes, and another rebalance should get triggered + // afterward, the "stalling" thread resumes, and another rebalance should get triggered // we write the remaining 20 records and verify to read 60 result records try ( - final KafkaStreams streams1 = getKafkaStreams("streams1", false, "appDir1", 1, eosConfig, MAX_POLL_INTERVAL_MS); - final KafkaStreams streams2 = getKafkaStreams("streams2", false, "appDir2", 1, eosConfig, MAX_POLL_INTERVAL_MS) + final KafkaStreams streams1 = getKafkaStreams("streams1", false, "appDir1", 1, eosConfig); + final KafkaStreams streams2 = getKafkaStreams("streams2", false, "appDir2", 1, eosConfig) ) { startApplicationAndWaitUntilRunning(streams1); startApplicationAndWaitUntilRunning(streams2); @@ -778,7 +778,7 @@ public void shouldWriteLatestOffsetsToCheckpointOnShutdown() throws Exception { final List> writtenData = prepareData(0L, 10, 0L, 1L); final List> expectedResult = computeExpectedResult(writtenData); - try (final KafkaStreams streams = getKafkaStreams("streams", true, "appDir", 1, eosConfig, MAX_POLL_INTERVAL_MS)) { + try (final KafkaStreams streams = getKafkaStreams("streams", true, "appDir", 1, eosConfig)) { writeInputData(writtenData); startApplicationAndWaitUntilRunning(streams); @@ -1004,8 +1004,7 @@ private KafkaStreams getKafkaStreams(final String dummyHostName, final boolean withState, final String appDir, final int numberOfStreamsThreads, - final String eosConfig, - final int maxPollIntervalMs) { + final String eosConfig) { commitRequested = new AtomicInteger(0); errorInjected = new AtomicBoolean(false); stallInjected = new AtomicBoolean(false); @@ -1112,9 +1111,8 @@ public void close() { } properties.put(StreamsConfig.producerPrefix(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG), (int) commitIntervalMs); properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.METADATA_MAX_AGE_CONFIG), "1000"); properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG), "earliest"); - properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG), maxPollIntervalMs); - properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), maxPollIntervalMs - 1); - properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG), maxPollIntervalMs); + properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), MAX_POLL_INTERVAL_MS - 1); + properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG), MAX_POLL_INTERVAL_MS); properties.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); properties.put(StreamsConfig.STATE_DIR_CONFIG, stateTmpDir + appDir); properties.put(StreamsConfig.APPLICATION_SERVER_CONFIG, dummyHostName + ":2142"); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java index 812da30074..b41e0c3d29 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java @@ -28,6 +28,7 @@ import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; @@ -155,8 +156,8 @@ private Properties props(final Properties extraProperties) { streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath()); - streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); - streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); + streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class); + streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class); streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L); streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); streamsConfiguration.putAll(extraProperties); @@ -196,8 +197,8 @@ public void shouldRestoreNullRecord() throws Exception { final Properties streamsConfiguration = StreamsTestUtils.getStreamsConfig( applicationId, CLUSTER.bootstrapServers(), - Serdes.Integer().getClass().getName(), - Serdes.ByteArray().getClass().getName(), + Serdes.IntegerSerde.class.getName(), + Serdes.BytesSerde.class.getName(), props); CLUSTER.createTopics(inputTopic); @@ -249,7 +250,63 @@ public void shouldRestoreNullRecord() throws Exception { @ParameterizedTest @MethodSource("parameters") - public void shouldRestoreStateFromSourceTopic(final boolean stateUpdaterEnabled) throws Exception { + public void shouldRestoreStateFromSourceTopicForReadOnlyStore(final boolean stateUpdaterEnabled) throws Exception { + final AtomicInteger numReceived = new AtomicInteger(0); + final Topology topology = new Topology(); + + final Properties props = props(stateUpdaterEnabled); + + // restoring from 1000 to 4000 (committed), and then process from 4000 to 5000 on each of the two partitions + final int offsetLimitDelta = 1000; + final int offsetCheckpointed = 1000; + createStateForRestoration(inputStream, 0); + setCommittedOffset(inputStream, offsetLimitDelta); + + final StateDirectory stateDirectory = new StateDirectory(new StreamsConfig(props), new MockTime(), true, false); + // note here the checkpointed offset is the last processed record's offset, so without control message we should write this offset - 1 + new OffsetCheckpoint(new File(stateDirectory.getOrCreateDirectoryForTask(new TaskId(0, 0)), ".checkpoint")) + .write(Collections.singletonMap(new TopicPartition(inputStream, 0), (long) offsetCheckpointed - 1)); + new OffsetCheckpoint(new File(stateDirectory.getOrCreateDirectoryForTask(new TaskId(0, 1)), ".checkpoint")) + .write(Collections.singletonMap(new TopicPartition(inputStream, 1), (long) offsetCheckpointed - 1)); + + final CountDownLatch startupLatch = new CountDownLatch(1); + final CountDownLatch shutdownLatch = new CountDownLatch(1); + + topology.addReadOnlyStateStore( + Stores.keyValueStoreBuilder( + Stores.persistentKeyValueStore("store"), + new Serdes.IntegerSerde(), + new Serdes.StringSerde() + ), + "readOnlySource", + new IntegerDeserializer(), + new StringDeserializer(), + inputStream, + "readOnlyProcessor", + () -> new ReadOnlyStoreProcessor(numReceived, offsetLimitDelta, shutdownLatch) + ); + + kafkaStreams = new KafkaStreams(topology, props); + kafkaStreams.setStateListener((newState, oldState) -> { + if (newState == KafkaStreams.State.RUNNING && oldState == KafkaStreams.State.REBALANCING) { + startupLatch.countDown(); + } + }); + + final AtomicLong restored = new AtomicLong(0); + kafkaStreams.setGlobalStateRestoreListener(new TrackingStateRestoreListener(restored)); + kafkaStreams.start(); + + assertTrue(startupLatch.await(30, TimeUnit.SECONDS)); + assertThat(restored.get(), equalTo((long) numberOfKeys - offsetLimitDelta * 2 - offsetCheckpointed * 2)); + + assertTrue(shutdownLatch.await(30, TimeUnit.SECONDS)); + assertThat(numReceived.get(), equalTo(offsetLimitDelta * 2)); + } + + @ParameterizedTest + @MethodSource("parameters") + public void shouldRestoreStateFromSourceTopicForGlobalTable(final boolean stateUpdaterEnabled) throws Exception { final AtomicInteger numReceived = new AtomicInteger(0); final StreamsBuilder builder = new StreamsBuilder(); @@ -265,9 +322,9 @@ public void shouldRestoreStateFromSourceTopic(final boolean stateUpdaterEnabled) final StateDirectory stateDirectory = new StateDirectory(new StreamsConfig(props), new MockTime(), true, false); // note here the checkpointed offset is the last processed record's offset, so without control message we should write this offset - 1 new OffsetCheckpoint(new File(stateDirectory.getOrCreateDirectoryForTask(new TaskId(0, 0)), ".checkpoint")) - .write(Collections.singletonMap(new TopicPartition(inputStream, 0), (long) offsetCheckpointed - 1)); + .write(Collections.singletonMap(new TopicPartition(inputStream, 0), (long) offsetCheckpointed - 1)); new OffsetCheckpoint(new File(stateDirectory.getOrCreateDirectoryForTask(new TaskId(0, 1)), ".checkpoint")) - .write(Collections.singletonMap(new TopicPartition(inputStream, 1), (long) offsetCheckpointed - 1)); + .write(Collections.singletonMap(new TopicPartition(inputStream, 1), (long) offsetCheckpointed - 1)); final CountDownLatch startupLatch = new CountDownLatch(1); final CountDownLatch shutdownLatch = new CountDownLatch(1); @@ -288,22 +345,7 @@ public void shouldRestoreStateFromSourceTopic(final boolean stateUpdaterEnabled) }); final AtomicLong restored = new AtomicLong(0); - kafkaStreams.setGlobalStateRestoreListener(new StateRestoreListener() { - @Override - public void onRestoreStart(final TopicPartition topicPartition, final String storeName, final long startingOffset, final long endingOffset) { - - } - - @Override - public void onBatchRestored(final TopicPartition topicPartition, final String storeName, final long batchEndOffset, final long numRestored) { - - } - - @Override - public void onRestoreEnd(final TopicPartition topicPartition, final String storeName, final long totalRestored) { - restored.addAndGet(totalRestored); - } - }); + kafkaStreams.setGlobalStateRestoreListener(new TrackingStateRestoreListener(restored)); kafkaStreams.start(); assertTrue(startupLatch.await(30, TimeUnit.SECONDS)); @@ -332,9 +374,9 @@ public void shouldRestoreStateFromChangelogTopic(final boolean stateUpdaterEnabl final StateDirectory stateDirectory = new StateDirectory(new StreamsConfig(props), new MockTime(), true, false); // note here the checkpointed offset is the last processed record's offset, so without control message we should write this offset - 1 new OffsetCheckpoint(new File(stateDirectory.getOrCreateDirectoryForTask(new TaskId(0, 0)), ".checkpoint")) - .write(Collections.singletonMap(new TopicPartition(changelog, 0), (long) offsetCheckpointed - 1)); + .write(Collections.singletonMap(new TopicPartition(changelog, 0), (long) offsetCheckpointed - 1)); new OffsetCheckpoint(new File(stateDirectory.getOrCreateDirectoryForTask(new TaskId(0, 1)), ".checkpoint")) - .write(Collections.singletonMap(new TopicPartition(changelog, 1), (long) offsetCheckpointed - 1)); + .write(Collections.singletonMap(new TopicPartition(changelog, 1), (long) offsetCheckpointed - 1)); final CountDownLatch startupLatch = new CountDownLatch(1); final CountDownLatch shutdownLatch = new CountDownLatch(1); @@ -355,22 +397,7 @@ public void shouldRestoreStateFromChangelogTopic(final boolean stateUpdaterEnabl }); final AtomicLong restored = new AtomicLong(0); - kafkaStreams.setGlobalStateRestoreListener(new StateRestoreListener() { - @Override - public void onRestoreStart(final TopicPartition topicPartition, final String storeName, final long startingOffset, final long endingOffset) { - - } - - @Override - public void onBatchRestored(final TopicPartition topicPartition, final String storeName, final long batchEndOffset, final long numRestored) { - - } - - @Override - public void onRestoreEnd(final TopicPartition topicPartition, final String storeName, final long totalRestored) { - restored.addAndGet(totalRestored); - } - }); + kafkaStreams.setGlobalStateRestoreListener(new TrackingStateRestoreListener(restored)); kafkaStreams.start(); assertTrue(startupLatch.await(30, TimeUnit.SECONDS)); @@ -386,10 +413,12 @@ public void shouldSuccessfullyStartWhenLoggingDisabled(final boolean stateUpdate final StreamsBuilder builder = new StreamsBuilder(); final KStream stream = builder.stream(inputStream); - stream.groupByKey() - .reduce( - (value1, value2) -> value1 + value2, - Materialized.>as("reduce-store").withLoggingDisabled()); + stream + .groupByKey() + .reduce( + Integer::sum, + Materialized.>as("reduce-store").withLoggingDisabled() + ); final CountDownLatch startupLatch = new CountDownLatch(1); kafkaStreams = new KafkaStreams(builder.build(), props(stateUpdaterEnabled)); @@ -821,4 +850,30 @@ private void waitForTransitionTo(final Set observed, final K () -> "Client did not transition to " + state + " on time. Observed transitions: " + observed ); } + + private static class ReadOnlyStoreProcessor implements Processor { + private final AtomicInteger numReceived; + private final int offsetLimitDelta; + private final CountDownLatch shutdownLatch; + KeyValueStore store; + + public ReadOnlyStoreProcessor(final AtomicInteger numReceived, final int offsetLimitDelta, final CountDownLatch shutdownLatch) { + this.numReceived = numReceived; + this.offsetLimitDelta = offsetLimitDelta; + this.shutdownLatch = shutdownLatch; + } + + @Override + public void init(final ProcessorContext context) { + store = context.getStateStore("store"); + } + + @Override + public void process(final Record record) { + store.put(record.key(), record.value()); + if (numReceived.incrementAndGet() == offsetLimitDelta * 2) { + shutdownLatch.countDown(); + } + } + } } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java index 4f1d8d3d42..7c5734d7b9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java @@ -1518,6 +1518,15 @@ public static class TrackingStateRestoreListener implements StateRestoreListener public final Map changelogToStartOffset = new ConcurrentHashMap<>(); public final Map changelogToEndOffset = new ConcurrentHashMap<>(); public final Map changelogToTotalNumRestored = new ConcurrentHashMap<>(); + private final AtomicLong restored; + + public TrackingStateRestoreListener() { + restored = null; + } + + public TrackingStateRestoreListener(final AtomicLong restored) { + this.restored = restored; + } @Override public void onRestoreStart(final TopicPartition topicPartition, @@ -1541,6 +1550,9 @@ public void onBatchRestored(final TopicPartition topicPartition, public void onRestoreEnd(final TopicPartition topicPartition, final String storeName, final long totalRestored) { + if (restored != null) { + restored.addAndGet(totalRestored); + } } public long totalNumRestored() { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/ReadOnlyStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/ReadOnlyStoreTest.java new file mode 100644 index 0000000000..a786c82bf2 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/ReadOnlyStoreTest.java @@ -0,0 +1,132 @@ +/* + * 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.streams.processor; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.TestInputTopic; +import org.apache.kafka.streams.TestOutputTopic; +import org.apache.kafka.streams.Topology; +import org.apache.kafka.streams.TopologyTestDriver; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.Stores; +import org.junit.Test; + +import java.util.LinkedList; +import java.util.List; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +public class ReadOnlyStoreTest { + + @Test + public void shouldConnectProcessorAndWriteDataToReadOnlyStore() { + final Topology topology = new Topology(); + topology.addReadOnlyStateStore( + Stores.keyValueStoreBuilder( + Stores.inMemoryKeyValueStore("readOnlyStore"), + new Serdes.IntegerSerde(), + new Serdes.StringSerde() + ), + "readOnlySource", + new IntegerDeserializer(), + new StringDeserializer(), + "storeTopic", + "readOnlyProcessor", + () -> new Processor() { + KeyValueStore store; + + @Override + public void init(final ProcessorContext context) { + store = context.getStateStore("readOnlyStore"); + } + @Override + public void process(final Record record) { + store.put(record.key(), record.value()); + } + } + ); + + topology.addSource("source", new IntegerDeserializer(), new StringDeserializer(), "inputTopic"); + topology.addProcessor( + "processor", + () -> new Processor() { + ProcessorContext context; + KeyValueStore store; + + @Override + public void init(final ProcessorContext context) { + this.context = context; + store = context.getStateStore("readOnlyStore"); + } + + @Override + public void process(final Record record) { + context.forward(record.withValue( + record.value() + " -- " + store.get(record.key()) + )); + } + }, + "source" + ); + topology.connectProcessorAndStateStores("processor", "readOnlyStore"); + topology.addSink("sink", "outputTopic", new IntegerSerializer(), new StringSerializer(), "processor"); + + try (final TopologyTestDriver driver = new TopologyTestDriver(topology)) { + final TestInputTopic readOnlyStoreTopic = + driver.createInputTopic("storeTopic", new IntegerSerializer(), new StringSerializer()); + final TestInputTopic input = + driver.createInputTopic("inputTopic", new IntegerSerializer(), new StringSerializer()); + final TestOutputTopic output = + driver.createOutputTopic("outputTopic", new IntegerDeserializer(), new StringDeserializer()); + + readOnlyStoreTopic.pipeInput(1, "foo"); + readOnlyStoreTopic.pipeInput(2, "bar"); + + input.pipeInput(1, "bar"); + input.pipeInput(2, "foo"); + + final KeyValueStore store = driver.getKeyValueStore("readOnlyStore"); + + try (final KeyValueIterator it = store.all()) { + final List> storeContent = new LinkedList<>(); + it.forEachRemaining(storeContent::add); + + final List> expectedResult = new LinkedList<>(); + expectedResult.add(KeyValue.pair(1, "foo")); + expectedResult.add(KeyValue.pair(2, "bar")); + + assertThat(storeContent, equalTo(expectedResult)); + } + + final List> expectedResult = new LinkedList<>(); + expectedResult.add(KeyValue.pair(1, "bar -- foo")); + expectedResult.add(KeyValue.pair(2, "foo -- bar")); + + assertThat(output.readKeyValuesToList(), equalTo(expectedResult)); + } + } +} diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessor.java b/streams/src/test/java/org/apache/kafka/test/MockProcessor.java index 9766d1c0fe..8c1812ccf1 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessor.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessor.java @@ -32,7 +32,6 @@ public class MockProcessor implements Processor { private final MockApiProcessor delegate; - public MockProcessor(final PunctuationType punctuationType, final long scheduleInterval) { delegate = new MockApiProcessor<>(punctuationType, scheduleInterval); @@ -43,12 +42,12 @@ public MockProcessor() { } @Override - public void init(ProcessorContext context) { + public void init(final ProcessorContext context) { delegate.init(context); } @Override - public void process(Record record) { + public void process(final Record record) { delegate.process(record); } From 414365979e960d0705f955579836faff6881f46a Mon Sep 17 00:00:00 2001 From: Nikolay Date: Fri, 8 Mar 2024 22:54:39 +0300 Subject: [PATCH 049/521] KAFKA-14589 [4/4] Tests of ConsoleGroupCommand rewritten in java (#15465) Reviewers: Chia-Ping Tsai --- .../admin/ConsumerGroupCommandTest.scala | 228 ------- .../admin/ResetConsumerGroupOffsetTest.scala | 514 --------------- .../group/ConsumerGroupCommandTest.java | 5 +- .../group/ResetConsumerGroupOffsetTest.java | 589 ++++++++++++++++++ 4 files changed, 590 insertions(+), 746 deletions(-) delete mode 100644 core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala delete mode 100644 core/src/test/scala/unit/kafka/admin/ResetConsumerGroupOffsetTest.scala create mode 100644 tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java diff --git a/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala deleted file mode 100644 index f682df1f1d..0000000000 --- a/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala +++ /dev/null @@ -1,228 +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.admin - -import java.time.Duration -import java.util.concurrent.{ExecutorService, Executors, TimeUnit} -import java.util.{Collections, Properties, stream} -import kafka.admin.ConsumerGroupCommand.{ConsumerGroupCommandOptions, ConsumerGroupService} -import kafka.api.BaseConsumerTest -import kafka.integration.KafkaServerTestHarness -import kafka.server.KafkaConfig -import kafka.utils.TestUtils -import org.apache.kafka.clients.admin.AdminClientConfig -import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, GroupProtocol, KafkaConsumer, RangeAssignor} -import org.apache.kafka.common.{PartitionInfo, TopicPartition} -import org.apache.kafka.common.errors.WakeupException -import org.apache.kafka.common.serialization.StringDeserializer -import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} -import org.junit.jupiter.params.provider.Arguments - -import scala.jdk.CollectionConverters._ -import scala.collection.mutable.ArrayBuffer - -class ConsumerGroupCommandTest extends KafkaServerTestHarness { - import ConsumerGroupCommandTest._ - - val topic = "foo" - val group = "test.group" - - private var consumerGroupService: List[ConsumerGroupService] = List() - private var consumerGroupExecutors: List[AbstractConsumerGroupExecutor] = List() - - // configure the servers and clients - override def generateConfigs = { - val configs = TestUtils.createBrokerConfigs(1, zkConnectOrNull, enableControlledShutdown = false) - - if (isNewGroupCoordinatorEnabled()) { - configs.foreach(_.setProperty(KafkaConfig.NewGroupCoordinatorEnableProp, "true")) - } - - configs.map(KafkaConfig.fromProps) - } - - @BeforeEach - override def setUp(testInfo: TestInfo): Unit = { - super.setUp(testInfo) - createTopic(topic, 1, 1) - } - - @AfterEach - override def tearDown(): Unit = { - consumerGroupService.foreach(_.close()) - consumerGroupExecutors.foreach(_.shutdown()) - super.tearDown() - } - - def committedOffsets(topic: String = topic, group: String = group): collection.Map[TopicPartition, Long] = { - val consumer = createNoAutoCommitConsumer(group) - try { - val partitions: Set[TopicPartition] = consumer.partitionsFor(topic) - .asScala.toSet.map {partitionInfo : PartitionInfo => new TopicPartition(partitionInfo.topic, partitionInfo.partition)} - consumer.committed(partitions.asJava).asScala.filter(_._2 != null).map { case (k, v) => k -> v.offset } - } finally { - consumer.close() - } - } - - def createNoAutoCommitConsumer(group: String): Consumer[String, String] = { - val props = new Properties - props.put("bootstrap.servers", bootstrapServers()) - props.put("group.id", group) - props.put("enable.auto.commit", "false") - new KafkaConsumer(props, new StringDeserializer, new StringDeserializer) - } - - def getConsumerGroupService(args: Array[String]): ConsumerGroupService = { - val opts = new ConsumerGroupCommandOptions(args) - val service = new ConsumerGroupService(opts, Map(AdminClientConfig.RETRIES_CONFIG -> Int.MaxValue.toString)) - consumerGroupService = service :: consumerGroupService - service - } - - def addConsumerGroupExecutor(numConsumers: Int, - topic: String = topic, - group: String = group, - strategy: String = classOf[RangeAssignor].getName, - remoteAssignor: Option[String] = None, - customPropsOpt: Option[Properties] = None, - syncCommit: Boolean = false, - groupProtocol: String = GroupProtocol.CLASSIC.toString): ConsumerGroupExecutor = { - val executor = new ConsumerGroupExecutor(bootstrapServers(), numConsumers, group, groupProtocol, topic, strategy, remoteAssignor, customPropsOpt, syncCommit) - addExecutor(executor) - executor - } - - def addSimpleGroupExecutor(partitions: Iterable[TopicPartition] = Seq(new TopicPartition(topic, 0)), - group: String = group): SimpleConsumerGroupExecutor = { - val executor = new SimpleConsumerGroupExecutor(bootstrapServers(), group, partitions) - addExecutor(executor) - executor - } - - private def addExecutor(executor: AbstractConsumerGroupExecutor): AbstractConsumerGroupExecutor = { - consumerGroupExecutors = executor :: consumerGroupExecutors - executor - } - -} - -object ConsumerGroupCommandTest { - def getTestQuorumAndGroupProtocolParametersAll(): stream.Stream[Arguments] = BaseConsumerTest.getTestQuorumAndGroupProtocolParametersAll() - def getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly(): stream.Stream[Arguments] = BaseConsumerTest.getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly() - def getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly(): stream.Stream[Arguments] = BaseConsumerTest.getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly() - - abstract class AbstractConsumerRunnable(broker: String, groupId: String, customPropsOpt: Option[Properties] = None, - syncCommit: Boolean = false) extends Runnable { - val props = new Properties - configure(props) - customPropsOpt.foreach(props.asScala ++= _.asScala) - val consumer = new KafkaConsumer(props) - - def configure(props: Properties): Unit = { - props.put("bootstrap.servers", broker) - props.put("group.id", groupId) - props.put("key.deserializer", classOf[StringDeserializer].getName) - props.put("value.deserializer", classOf[StringDeserializer].getName) - } - - def subscribe(): Unit - - def run(): Unit = { - try { - subscribe() - while (true) { - consumer.poll(Duration.ofMillis(Long.MaxValue)) - if (syncCommit) - consumer.commitSync() - } - } catch { - case _: WakeupException => // OK - } finally { - consumer.close() - } - } - - def shutdown(): Unit = { - consumer.wakeup() - } - } - - class ConsumerRunnable(broker: String, groupId: String, groupProtocol: String, topic: String, strategy: String, - remoteAssignor: Option[String], customPropsOpt: Option[Properties] = None, syncCommit: Boolean = false) - extends AbstractConsumerRunnable(broker, groupId, customPropsOpt, syncCommit) { - - override def configure(props: Properties): Unit = { - super.configure(props) - props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol) - if (groupProtocol.toUpperCase == GroupProtocol.CONSUMER.toString) { - remoteAssignor.foreach { assignor => - props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, assignor) - } - } else { - props.put("partition.assignment.strategy", strategy) - } - } - - override def subscribe(): Unit = { - consumer.subscribe(Collections.singleton(topic)) - } - } - - class SimpleConsumerRunnable(broker: String, groupId: String, partitions: Iterable[TopicPartition]) - extends AbstractConsumerRunnable(broker, groupId) { - - override def subscribe(): Unit = { - consumer.assign(partitions.toList.asJava) - } - } - - class AbstractConsumerGroupExecutor(numThreads: Int) { - private val executor: ExecutorService = Executors.newFixedThreadPool(numThreads) - private val consumers = new ArrayBuffer[AbstractConsumerRunnable]() - - def submit(consumerThread: AbstractConsumerRunnable): Unit = { - consumers += consumerThread - executor.submit(consumerThread) - } - - def shutdown(): Unit = { - consumers.foreach(_.shutdown()) - executor.shutdown() - executor.awaitTermination(5000, TimeUnit.MILLISECONDS) - } - } - - class ConsumerGroupExecutor(broker: String, numConsumers: Int, groupId: String, groupProtocol: String, topic: String, strategy: String, - remoteAssignor: Option[String], customPropsOpt: Option[Properties] = None, syncCommit: Boolean = false) - extends AbstractConsumerGroupExecutor(numConsumers) { - - for (_ <- 1 to numConsumers) { - submit(new ConsumerRunnable(broker, groupId, groupProtocol, topic, strategy, remoteAssignor, customPropsOpt, syncCommit)) - } - - } - - class SimpleConsumerGroupExecutor(broker: String, groupId: String, partitions: Iterable[TopicPartition]) - extends AbstractConsumerGroupExecutor(1) { - - submit(new SimpleConsumerRunnable(broker, groupId, partitions)) - } - -} - diff --git a/core/src/test/scala/unit/kafka/admin/ResetConsumerGroupOffsetTest.scala b/core/src/test/scala/unit/kafka/admin/ResetConsumerGroupOffsetTest.scala deleted file mode 100644 index 4aafedf0ab..0000000000 --- a/core/src/test/scala/unit/kafka/admin/ResetConsumerGroupOffsetTest.scala +++ /dev/null @@ -1,514 +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.admin - -import java.io.{BufferedWriter, FileWriter} -import java.text.{SimpleDateFormat} -import java.util.{Calendar, Date, Properties} - -import joptsimple.OptionException -import kafka.admin.ConsumerGroupCommand.ConsumerGroupService -import kafka.server.KafkaConfig -import kafka.utils.TestUtils -import org.apache.kafka.clients.producer.ProducerRecord -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.test -import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.Test - -import scala.jdk.CollectionConverters._ -import scala.collection.Seq - - - -/** - * Test cases by: - * - Non-existing consumer group - * - One for each scenario, with scope=all-topics - * - scope=one topic, scenario=to-earliest - * - scope=one topic+partitions, scenario=to-earliest - * - scope=topics, scenario=to-earliest - * - scope=topics+partitions, scenario=to-earliest - * - export/import - */ -class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest { - - val overridingProps = new Properties() - val topic1 = "foo1" - val topic2 = "foo2" - - override def generateConfigs: Seq[KafkaConfig] = { - TestUtils.createBrokerConfigs(1, zkConnect, enableControlledShutdown = false) - .map(KafkaConfig.fromProps(_, overridingProps)) - } - - private def basicArgs: Array[String] = { - Array("--reset-offsets", - "--bootstrap-server", bootstrapServers(), - "--timeout", test.TestUtils.DEFAULT_MAX_WAIT_MS.toString) - } - - private def buildArgsForGroups(groups: Seq[String], args: String*): Array[String] = { - val groupArgs = groups.flatMap(group => Seq("--group", group)).toArray - basicArgs ++ groupArgs ++ args - } - - private def buildArgsForGroup(group: String, args: String*): Array[String] = { - buildArgsForGroups(Seq(group), args: _*) - } - - private def buildArgsForAllGroups(args: String*): Array[String] = { - basicArgs ++ Array("--all-groups") ++ args - } - - @Test - def testResetOffsetsNotExistingGroup(): Unit = { - val group = "missing.group" - val args = buildArgsForGroup(group, "--all-topics", "--to-current", "--execute") - val consumerGroupCommand = getConsumerGroupService(args) - // Make sure we got a coordinator - TestUtils.waitUntilTrue(() => { - consumerGroupCommand.collectGroupState(group).coordinator.host() == "localhost" - }, "Can't find a coordinator") - val resetOffsets = consumerGroupCommand.resetOffsets()(group) - assertEquals(Map.empty, resetOffsets) - assertEquals(resetOffsets, committedOffsets(group = group)) - } - - @Test - def testResetOffsetsExistingTopic(): Unit = { - val group = "new.group" - val args = buildArgsForGroup(group, "--topic", topic, "--to-offset", "50") - produceMessages(topic, 100) - resetAndAssertOffsets(args, expectedOffset = 50, dryRun = true) - resetAndAssertOffsets(args ++ Array("--dry-run"), expectedOffset = 50, dryRun = true) - resetAndAssertOffsets(args ++ Array("--execute"), expectedOffset = 50) - } - - @Test - def testResetOffsetsExistingTopicSelectedGroups(): Unit = { - produceMessages(topic, 100) - val groups = - for (id <- 1 to 3) yield { - val group = this.group + id - val executor = addConsumerGroupExecutor(numConsumers = 1, topic = topic, group = group) - awaitConsumerProgress(count = 100L, group = group) - executor.shutdown() - group - } - val args = buildArgsForGroups(groups,"--topic", topic, "--to-offset", "50") - resetAndAssertOffsets(args, expectedOffset = 50, dryRun = true) - resetAndAssertOffsets(args ++ Array("--dry-run"), expectedOffset = 50, dryRun = true) - resetAndAssertOffsets(args ++ Array("--execute"), expectedOffset = 50) - } - - @Test - def testResetOffsetsExistingTopicAllGroups(): Unit = { - val args = buildArgsForAllGroups("--topic", topic, "--to-offset", "50") - produceMessages(topic, 100) - for (group <- 1 to 3 map (group + _)) { - val executor = addConsumerGroupExecutor(numConsumers = 1, topic = topic, group = group) - awaitConsumerProgress(count = 100L, group = group) - executor.shutdown() - } - resetAndAssertOffsets(args, expectedOffset = 50, dryRun = true) - resetAndAssertOffsets(args ++ Array("--dry-run"), expectedOffset = 50, dryRun = true) - resetAndAssertOffsets(args ++ Array("--execute"), expectedOffset = 50) - } - - @Test - def testResetOffsetsAllTopicsAllGroups(): Unit = { - val args = buildArgsForAllGroups("--all-topics", "--to-offset", "50") - val topics = 1 to 3 map (topic + _) - val groups = 1 to 3 map (group + _) - topics foreach (topic => produceMessages(topic, 100)) - for { - topic <- topics - group <- groups - } { - val executor = addConsumerGroupExecutor(numConsumers = 3, topic = topic, group = group) - awaitConsumerProgress(topic = topic, count = 100L, group = group) - executor.shutdown() - } - resetAndAssertOffsets(args, expectedOffset = 50, dryRun = true, topics = topics) - resetAndAssertOffsets(args ++ Array("--dry-run"), expectedOffset = 50, dryRun = true, topics = topics) - resetAndAssertOffsets(args ++ Array("--execute"), expectedOffset = 50, topics = topics) - } - - @Test - def testResetOffsetsToLocalDateTime(): Unit = { - val format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS") - val calendar = Calendar.getInstance() - calendar.add(Calendar.DATE, -1) - - produceMessages(topic, 100) - - val executor = addConsumerGroupExecutor(numConsumers = 1, topic) - awaitConsumerProgress(count = 100L) - executor.shutdown() - - val args = buildArgsForGroup(group, "--all-topics", "--to-datetime", format.format(calendar.getTime), "--execute") - resetAndAssertOffsets(args, expectedOffset = 0) - } - - @Test - def testResetOffsetsToZonedDateTime(): Unit = { - val format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX") - - produceMessages(topic, 50) - val checkpoint = new Date() - produceMessages(topic, 50) - - val executor = addConsumerGroupExecutor(numConsumers = 1, topic) - awaitConsumerProgress(count = 100L) - executor.shutdown() - - val args = buildArgsForGroup(group, "--all-topics", "--to-datetime", format.format(checkpoint), "--execute") - resetAndAssertOffsets(args, expectedOffset = 50) - } - - @Test - def testResetOffsetsByDuration(): Unit = { - val args = buildArgsForGroup(group, "--all-topics", "--by-duration", "PT1M", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - resetAndAssertOffsets(args, expectedOffset = 0) - } - - @Test - def testResetOffsetsByDurationToEarliest(): Unit = { - val args = buildArgsForGroup(group, "--all-topics", "--by-duration", "PT0.1S", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - resetAndAssertOffsets(args, expectedOffset = 100) - } - - @Test - def testResetOffsetsByDurationFallbackToLatestWhenNoRecords(): Unit = { - val topic = "foo2" - val args = buildArgsForGroup(group, "--topic", topic, "--by-duration", "PT1M", "--execute") - createTopic(topic) - resetAndAssertOffsets(args, expectedOffset = 0, topics = Seq("foo2")) - - adminZkClient.deleteTopic(topic) - } - - @Test - def testResetOffsetsToEarliest(): Unit = { - val args = buildArgsForGroup(group, "--all-topics", "--to-earliest", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - resetAndAssertOffsets(args, expectedOffset = 0) - } - - @Test - def testResetOffsetsToLatest(): Unit = { - val args = buildArgsForGroup(group, "--all-topics", "--to-latest", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - produceMessages(topic, 100) - resetAndAssertOffsets(args, expectedOffset = 200) - } - - @Test - def testResetOffsetsToCurrentOffset(): Unit = { - val args = buildArgsForGroup(group, "--all-topics", "--to-current", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - produceMessages(topic, 100) - resetAndAssertOffsets(args, expectedOffset = 100) - } - - @Test - def testResetOffsetsToSpecificOffset(): Unit = { - val args = buildArgsForGroup(group, "--all-topics", "--to-offset", "1", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - resetAndAssertOffsets(args, expectedOffset = 1) - } - - @Test - def testResetOffsetsShiftPlus(): Unit = { - val args = buildArgsForGroup(group, "--all-topics", "--shift-by", "50", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - produceMessages(topic, 100) - resetAndAssertOffsets(args, expectedOffset = 150) - } - - @Test - def testResetOffsetsShiftMinus(): Unit = { - val args = buildArgsForGroup(group, "--all-topics", "--shift-by", "-50", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - produceMessages(topic, 100) - resetAndAssertOffsets(args, expectedOffset = 50) - } - - @Test - def testResetOffsetsShiftByLowerThanEarliest(): Unit = { - val args = buildArgsForGroup(group, "--all-topics", "--shift-by", "-150", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - produceMessages(topic, 100) - resetAndAssertOffsets(args, expectedOffset = 0) - } - - @Test - def testResetOffsetsShiftByHigherThanLatest(): Unit = { - val args = buildArgsForGroup(group, "--all-topics", "--shift-by", "150", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - produceMessages(topic, 100) - resetAndAssertOffsets(args, expectedOffset = 200) - } - - @Test - def testResetOffsetsToEarliestOnOneTopic(): Unit = { - val args = buildArgsForGroup(group, "--topic", topic, "--to-earliest", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - resetAndAssertOffsets(args, expectedOffset = 0) - } - - @Test - def testResetOffsetsToEarliestOnOneTopicAndPartition(): Unit = { - val topic = "bar" - createTopic(topic, 2, 1) - - val args = buildArgsForGroup(group, "--topic", s"$topic:1", "--to-earliest", "--execute") - val consumerGroupCommand = getConsumerGroupService(args) - - produceConsumeAndShutdown(topic, group, totalMessages = 100, numConsumers = 2) - val priorCommittedOffsets = committedOffsets(topic = topic) - - val tp0 = new TopicPartition(topic, 0) - val tp1 = new TopicPartition(topic, 1) - val expectedOffsets = Map(tp0 -> priorCommittedOffsets(tp0), tp1 -> 0L) - resetAndAssertOffsetsCommitted(consumerGroupCommand, expectedOffsets, topic) - - adminZkClient.deleteTopic(topic) - } - - @Test - def testResetOffsetsToEarliestOnTopics(): Unit = { - val topic1 = "topic1" - val topic2 = "topic2" - createTopic(topic1, 1, 1) - createTopic(topic2, 1, 1) - - val args = buildArgsForGroup(group, "--topic", topic1, "--topic", topic2, "--to-earliest", "--execute") - val consumerGroupCommand = getConsumerGroupService(args) - - produceConsumeAndShutdown(topic1, group, 100, 1) - produceConsumeAndShutdown(topic2, group, 100, 1) - - val tp1 = new TopicPartition(topic1, 0) - val tp2 = new TopicPartition(topic2, 0) - - val allResetOffsets = resetOffsets(consumerGroupCommand)(group).map { case (k, v) => k -> v.offset } - assertEquals(Map(tp1 -> 0L, tp2 -> 0L), allResetOffsets) - assertEquals(Map(tp1 -> 0L), committedOffsets(topic1)) - assertEquals(Map(tp2 -> 0L), committedOffsets(topic2)) - - adminZkClient.deleteTopic(topic1) - adminZkClient.deleteTopic(topic2) - } - - @Test - def testResetOffsetsToEarliestOnTopicsAndPartitions(): Unit = { - val topic1 = "topic1" - val topic2 = "topic2" - - createTopic(topic1, 2, 1) - createTopic(topic2, 2, 1) - - val args = buildArgsForGroup(group, "--topic", s"$topic1:1", "--topic", s"$topic2:1", "--to-earliest", "--execute") - val consumerGroupCommand = getConsumerGroupService(args) - - produceConsumeAndShutdown(topic1, group, 100, 2) - produceConsumeAndShutdown(topic2, group, 100, 2) - - val priorCommittedOffsets1 = committedOffsets(topic1) - val priorCommittedOffsets2 = committedOffsets(topic2) - - val tp1 = new TopicPartition(topic1, 1) - val tp2 = new TopicPartition(topic2, 1) - val allResetOffsets = resetOffsets(consumerGroupCommand)(group).map { case (k, v) => k -> v.offset } - assertEquals(Map(tp1 -> 0, tp2 -> 0), allResetOffsets) - - assertEquals(priorCommittedOffsets1.toMap + (tp1 -> 0L), committedOffsets(topic1)) - assertEquals(priorCommittedOffsets2.toMap + (tp2 -> 0L), committedOffsets(topic2)) - - adminZkClient.deleteTopic(topic1) - adminZkClient.deleteTopic(topic2) - } - - @Test - // This one deals with old CSV export/import format for a single --group arg: "topic,partition,offset" to support old behavior - def testResetOffsetsExportImportPlanSingleGroupArg(): Unit = { - val topic = "bar" - val tp0 = new TopicPartition(topic, 0) - val tp1 = new TopicPartition(topic, 1) - createTopic(topic, 2, 1) - - val cgcArgs = buildArgsForGroup(group, "--all-topics", "--to-offset", "2", "--export") - val consumerGroupCommand = getConsumerGroupService(cgcArgs) - - produceConsumeAndShutdown(topic = topic, group = group, totalMessages = 100, numConsumers = 2) - - val file = TestUtils.tempFile("reset", ".csv") - - val exportedOffsets = consumerGroupCommand.resetOffsets() - val bw = new BufferedWriter(new FileWriter(file)) - bw.write(consumerGroupCommand.exportOffsetsToCsv(exportedOffsets)) - bw.close() - assertEquals(Map(tp0 -> 2L, tp1 -> 2L), exportedOffsets(group).map { case (k, v) => k -> v.offset }) - - val cgcArgsExec = buildArgsForGroup(group, "--all-topics", "--from-file", file.getCanonicalPath, "--dry-run") - val consumerGroupCommandExec = getConsumerGroupService(cgcArgsExec) - val importedOffsets = consumerGroupCommandExec.resetOffsets() - assertEquals(Map(tp0 -> 2L, tp1 -> 2L), importedOffsets(group).map { case (k, v) => k -> v.offset }) - - adminZkClient.deleteTopic(topic) - } - - @Test - // This one deals with universal CSV export/import file format "group,topic,partition,offset", - // supporting multiple --group args or --all-groups arg - def testResetOffsetsExportImportPlan(): Unit = { - val group1 = group + "1" - val group2 = group + "2" - val topic1 = "bar1" - val topic2 = "bar2" - val t1p0 = new TopicPartition(topic1, 0) - val t1p1 = new TopicPartition(topic1, 1) - val t2p0 = new TopicPartition(topic2, 0) - val t2p1 = new TopicPartition(topic2, 1) - createTopic(topic1, 2, 1) - createTopic(topic2, 2, 1) - - val cgcArgs = buildArgsForGroups(Seq(group1, group2), "--all-topics", "--to-offset", "2", "--export") - val consumerGroupCommand = getConsumerGroupService(cgcArgs) - - produceConsumeAndShutdown(topic = topic1, group = group1, totalMessages = 100) - produceConsumeAndShutdown(topic = topic2, group = group2, totalMessages = 100) - - awaitConsumerGroupInactive(consumerGroupCommand, group1) - awaitConsumerGroupInactive(consumerGroupCommand, group2) - - val file = TestUtils.tempFile("reset", ".csv") - - val exportedOffsets = consumerGroupCommand.resetOffsets() - val bw = new BufferedWriter(new FileWriter(file)) - bw.write(consumerGroupCommand.exportOffsetsToCsv(exportedOffsets)) - bw.close() - assertEquals(Map(t1p0 -> 2L, t1p1 -> 2L), exportedOffsets(group1).map { case (k, v) => k -> v.offset }) - assertEquals(Map(t2p0 -> 2L, t2p1 -> 2L), exportedOffsets(group2).map { case (k, v) => k -> v.offset }) - - // Multiple --group's offset import - val cgcArgsExec = buildArgsForGroups(Seq(group1, group2), "--all-topics", "--from-file", file.getCanonicalPath, "--dry-run") - val consumerGroupCommandExec = getConsumerGroupService(cgcArgsExec) - val importedOffsets = consumerGroupCommandExec.resetOffsets() - assertEquals(Map(t1p0 -> 2L, t1p1 -> 2L), importedOffsets(group1).map { case (k, v) => k -> v.offset }) - assertEquals(Map(t2p0 -> 2L, t2p1 -> 2L), importedOffsets(group2).map { case (k, v) => k -> v.offset }) - - // Single --group offset import using "group,topic,partition,offset" csv format - val cgcArgsExec2 = buildArgsForGroup(group1, "--all-topics", "--from-file", file.getCanonicalPath, "--dry-run") - val consumerGroupCommandExec2 = getConsumerGroupService(cgcArgsExec2) - val importedOffsets2 = consumerGroupCommandExec2.resetOffsets() - assertEquals(Map(t1p0 -> 2L, t1p1 -> 2L), importedOffsets2(group1).map { case (k, v) => k -> v.offset }) - - adminZkClient.deleteTopic(topic) - } - - @Test - def testResetWithUnrecognizedNewConsumerOption(): Unit = { - val cgcArgs = Array("--new-consumer", "--bootstrap-server", bootstrapServers(), "--reset-offsets", "--group", group, "--all-topics", - "--to-offset", "2", "--export") - assertThrows(classOf[OptionException], () => getConsumerGroupService(cgcArgs)) - } - - private def produceMessages(topic: String, numMessages: Int): Unit = { - val records = (0 until numMessages).map(_ => new ProducerRecord[Array[Byte], Array[Byte]](topic, - new Array[Byte](100 * 1000))) - TestUtils.produceMessages(servers, records, acks = 1) - } - - private def produceConsumeAndShutdown(topic: String, group: String, totalMessages: Int, numConsumers: Int = 1): Unit = { - produceMessages(topic, totalMessages) - val executor = addConsumerGroupExecutor(numConsumers = numConsumers, topic = topic, group = group) - awaitConsumerProgress(topic, group, totalMessages) - executor.shutdown() - } - - private def awaitConsumerProgress(topic: String = topic, - group: String = group, - count: Long): Unit = { - val consumer = createNoAutoCommitConsumer(group) - try { - val partitions = consumer.partitionsFor(topic).asScala.map { partitionInfo => - new TopicPartition(partitionInfo.topic, partitionInfo.partition) - }.toSet - - TestUtils.waitUntilTrue(() => { - val committed = consumer.committed(partitions.asJava).values.asScala - val total = committed.foldLeft(0L) { case (currentSum, offsetAndMetadata) => - currentSum + Option(offsetAndMetadata).map(_.offset).getOrElse(0L) - } - total == count - }, "Expected that consumer group has consumed all messages from topic/partition. " + - s"Expected offset: $count. Actual offset: ${committedOffsets(topic, group).values.sum}") - - } finally { - consumer.close() - } - - } - - private def awaitConsumerGroupInactive(consumerGroupService: ConsumerGroupService, group: String): Unit = { - TestUtils.waitUntilTrue(() => { - val state = consumerGroupService.collectGroupState(group).state - state == "Empty" || state == "Dead" - }, s"Expected that consumer group is inactive. Actual state: ${consumerGroupService.collectGroupState(group).state}") - } - - private def resetAndAssertOffsets(args: Array[String], - expectedOffset: Long, - dryRun: Boolean = false, - topics: Seq[String] = Seq(topic)): Unit = { - val consumerGroupCommand = getConsumerGroupService(args) - val expectedOffsets = topics.map(topic => topic -> Map(new TopicPartition(topic, 0) -> expectedOffset)).toMap - val resetOffsetsResultByGroup = resetOffsets(consumerGroupCommand) - - try { - for { - topic <- topics - (group, partitionInfo) <- resetOffsetsResultByGroup - } { - val priorOffsets = committedOffsets(topic = topic, group = group) - assertEquals(expectedOffsets(topic), - partitionInfo.filter(partitionInfo => partitionInfo._1.topic() == topic).map { case (k, v) => k -> v.offset }) - assertEquals(if (dryRun) priorOffsets else expectedOffsets(topic), committedOffsets(topic = topic, group = group)) - } - } finally { - consumerGroupCommand.close() - } - } - - private def resetAndAssertOffsetsCommitted(consumerGroupService: ConsumerGroupService, - expectedOffsets: Map[TopicPartition, Long], - topic: String): Unit = { - val allResetOffsets = resetOffsets(consumerGroupService) - for { - (group, offsetsInfo) <- allResetOffsets - (tp, offsetMetadata) <- offsetsInfo - } { - assertEquals(offsetMetadata.offset(), expectedOffsets(tp)) - assertEquals(expectedOffsets, committedOffsets(topic, group)) - } - } - - private def resetOffsets(consumerGroupService: ConsumerGroupService) = { - consumerGroupService.resetOffsets() - } -} diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java index bde3af37a1..e3ee39f6b1 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java @@ -87,10 +87,7 @@ public Seq generateConfigs() { 0, false ).foreach(props -> { - if (isNewGroupCoordinatorEnabled()) { - props.setProperty(KafkaConfig.NewGroupCoordinatorEnableProp(), "true"); - } - + props.setProperty(KafkaConfig.NewGroupCoordinatorEnableProp(), isNewGroupCoordinatorEnabled() + ""); cfgs.add(KafkaConfig.fromProps(props)); return null; }); diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java new file mode 100644 index 0000000000..dca2634522 --- /dev/null +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java @@ -0,0 +1,589 @@ +/* + * 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.tools.consumer.group; + +import joptsimple.OptionException; +import kafka.admin.ConsumerGroupCommand; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.GroupProtocol; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.test.TestUtils; +import org.junit.jupiter.api.Test; +import scala.Option; + +import java.io.BufferedWriter; +import java.io.File; +import java.io.FileWriter; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Calendar; +import java.util.Collection; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Test cases by: + * - Non-existing consumer group + * - One for each scenario, with scope=all-topics + * - scope=one topic, scenario=to-earliest + * - scope=one topic+partitions, scenario=to-earliest + * - scope=topics, scenario=to-earliest + * - scope=topics+partitions, scenario=to-earliest + * - export/import + */ +public class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest { + private String[] basicArgs() { + return new String[]{"--reset-offsets", + "--bootstrap-server", bootstrapServers(listenerName()), + "--timeout", Long.toString(DEFAULT_MAX_WAIT_MS)}; + } + + private String[] buildArgsForGroups(List groups, String...args) { + List res = new ArrayList<>(Arrays.asList(basicArgs())); + for (String group : groups) { + res.add("--group"); + res.add(group); + } + res.addAll(Arrays.asList(args)); + return res.toArray(new String[0]); + } + + private String[] buildArgsForGroup(String group, String...args) { + return buildArgsForGroups(Collections.singletonList(group), args); + } + + private String[] buildArgsForAllGroups(String...args) { + List res = new ArrayList<>(Arrays.asList(basicArgs())); + res.add("--all-groups"); + res.addAll(Arrays.asList(args)); + return res.toArray(new String[0]); + } + + @Test + public void testResetOffsetsNotExistingGroup() throws Exception { + String group = "missing.group"; + String[] args = buildArgsForGroup(group, "--all-topics", "--to-current", "--execute"); + ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(args); + // Make sure we got a coordinator + TestUtils.waitForCondition( + () -> Objects.equals(consumerGroupCommand.collectGroupState(group).coordinator().host(), "localhost"), + "Can't find a coordinator"); + Option> resetOffsets = consumerGroupCommand.resetOffsets().get(group); + assertTrue(resetOffsets.isDefined() && resetOffsets.get().isEmpty()); + assertTrue(committedOffsets(TOPIC, group).isEmpty()); + } + + @Test + public void testResetOffsetsExistingTopic() { + String group = "new.group"; + String[] args = buildArgsForGroup(group, "--topic", TOPIC, "--to-offset", "50"); + produceMessages(TOPIC, 100); + resetAndAssertOffsets(args, 50, true, Collections.singletonList(TOPIC)); + resetAndAssertOffsets(addTo(args, "--dry-run"), 50, true, Collections.singletonList(TOPIC)); + resetAndAssertOffsets(addTo(args, "--execute"), 50, false, Collections.singletonList(TOPIC)); + } + + @Test + public void testResetOffsetsExistingTopicSelectedGroups() throws Exception { + produceMessages(TOPIC, 100); + List groups = IntStream.rangeClosed(1, 3).mapToObj(id -> GROUP + id).collect(Collectors.toList()); + for (String group : groups) { + ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, TOPIC, group, GroupProtocol.CLASSIC.name); + awaitConsumerProgress(TOPIC, group, 100L); + executor.shutdown(); + } + String[] args = buildArgsForGroups(groups, "--topic", TOPIC, "--to-offset", "50"); + resetAndAssertOffsets(args, 50, true, Collections.singletonList(TOPIC)); + resetAndAssertOffsets(addTo(args, "--dry-run"), 50, true, Collections.singletonList(TOPIC)); + resetAndAssertOffsets(addTo(args, "--execute"), 50, false, Collections.singletonList(TOPIC)); + } + + @Test + public void testResetOffsetsExistingTopicAllGroups() throws Exception { + String[] args = buildArgsForAllGroups("--topic", TOPIC, "--to-offset", "50"); + produceMessages(TOPIC, 100); + for (int i = 1; i <= 3; i++) { + String group = GROUP + i; + ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, TOPIC, group, GroupProtocol.CLASSIC.name); + awaitConsumerProgress(TOPIC, group, 100L); + executor.shutdown(); + } + resetAndAssertOffsets(args, 50, true, Collections.singletonList(TOPIC)); + resetAndAssertOffsets(addTo(args, "--dry-run"), 50, true, Collections.singletonList(TOPIC)); + resetAndAssertOffsets(addTo(args, "--execute"), 50, false, Collections.singletonList(TOPIC)); + } + + @Test + public void testResetOffsetsAllTopicsAllGroups() throws Exception { + String[] args = buildArgsForAllGroups("--all-topics", "--to-offset", "50"); + List topics = IntStream.rangeClosed(1, 3).mapToObj(i -> TOPIC + i).collect(Collectors.toList()); + List groups = IntStream.rangeClosed(1, 3).mapToObj(i -> GROUP + i).collect(Collectors.toList()); + topics.forEach(topic -> produceMessages(topic, 100)); + + for (String topic : topics) { + for (String group : groups) { + ConsumerGroupExecutor executor = addConsumerGroupExecutor(3, topic, group, GroupProtocol.CLASSIC.name); + awaitConsumerProgress(topic, group, 100); + executor.shutdown(); + } + } + resetAndAssertOffsets(args, 50, true, topics); + resetAndAssertOffsets(addTo(args, "--dry-run"), 50, true, topics); + resetAndAssertOffsets(addTo(args, "--execute"), 50, false, topics); + } + + @Test + public void testResetOffsetsToLocalDateTime() throws Exception { + SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS"); + Calendar calendar = Calendar.getInstance(); + calendar.add(Calendar.DATE, -1); + + produceMessages(TOPIC, 100); + + ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, TOPIC, GROUP, GroupProtocol.CLASSIC.name); + awaitConsumerProgress(TOPIC, GROUP, 100L); + executor.shutdown(); + + String[] args = buildArgsForGroup(GROUP, "--all-topics", "--to-datetime", format.format(calendar.getTime()), "--execute"); + resetAndAssertOffsets(args, 0); + } + + @Test + public void testResetOffsetsToZonedDateTime() throws Exception { + SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX"); + + produceMessages(TOPIC, 50); + Date checkpoint = new Date(); + produceMessages(TOPIC, 50); + + ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, TOPIC, GROUP, GroupProtocol.CLASSIC.name); + awaitConsumerProgress(TOPIC, GROUP, 100L); + executor.shutdown(); + + String[] args = buildArgsForGroup(GROUP, "--all-topics", "--to-datetime", format.format(checkpoint), "--execute"); + resetAndAssertOffsets(args, 50); + } + + @Test + public void testResetOffsetsByDuration() throws Exception { + String[] args = buildArgsForGroup(GROUP, "--all-topics", "--by-duration", "PT1M", "--execute"); + produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); + resetAndAssertOffsets(args, 0); + } + + @Test + public void testResetOffsetsByDurationToEarliest() throws Exception { + String[] args = buildArgsForGroup(GROUP, "--all-topics", "--by-duration", "PT0.1S", "--execute"); + produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); + resetAndAssertOffsets(args, 100); + } + + @Test + public void testResetOffsetsByDurationFallbackToLatestWhenNoRecords() throws Exception { + String topic = "foo2"; + String[] args = buildArgsForGroup(GROUP, "--topic", topic, "--by-duration", "PT1M", "--execute"); + createTopic(topic, 1, 1, new Properties(), listenerName(), new Properties()); + resetAndAssertOffsets(args, 0, false, Collections.singletonList("foo2")); + + adminZkClient().deleteTopic(topic); + } + + @Test + public void testResetOffsetsToEarliest() throws Exception { + String[] args = buildArgsForGroup(GROUP, "--all-topics", "--to-earliest", "--execute"); + produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); + resetAndAssertOffsets(args, 0); + } + + @Test + public void testResetOffsetsToLatest() throws Exception { + String[] args = buildArgsForGroup(GROUP, "--all-topics", "--to-latest", "--execute"); + produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); + produceMessages(TOPIC, 100); + resetAndAssertOffsets(args, 200); + } + + @Test + public void testResetOffsetsToCurrentOffset() throws Exception { + String[] args = buildArgsForGroup(GROUP, "--all-topics", "--to-current", "--execute"); + produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); + produceMessages(TOPIC, 100); + resetAndAssertOffsets(args, 100); + } + + @Test + public void testResetOffsetsToSpecificOffset() throws Exception { + String[] args = buildArgsForGroup(GROUP, "--all-topics", "--to-offset", "1", "--execute"); + produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); + resetAndAssertOffsets(args, 1); + } + + @Test + public void testResetOffsetsShiftPlus() throws Exception { + String[] args = buildArgsForGroup(GROUP, "--all-topics", "--shift-by", "50", "--execute"); + produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); + produceMessages(TOPIC, 100); + resetAndAssertOffsets(args, 150); + } + + @Test + public void testResetOffsetsShiftMinus() throws Exception { + String[] args = buildArgsForGroup(GROUP, "--all-topics", "--shift-by", "-50", "--execute"); + produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); + produceMessages(TOPIC, 100); + resetAndAssertOffsets(args, 50); + } + + @Test + public void testResetOffsetsShiftByLowerThanEarliest() throws Exception { + String[] args = buildArgsForGroup(GROUP, "--all-topics", "--shift-by", "-150", "--execute"); + produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); + produceMessages(TOPIC, 100); + resetAndAssertOffsets(args, 0); + } + + @Test + public void testResetOffsetsShiftByHigherThanLatest() throws Exception { + String[] args = buildArgsForGroup(GROUP, "--all-topics", "--shift-by", "150", "--execute"); + produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); + produceMessages(TOPIC, 100); + resetAndAssertOffsets(args, 200); + } + + @Test + public void testResetOffsetsToEarliestOnOneTopic() throws Exception { + String[] args = buildArgsForGroup(GROUP, "--topic", TOPIC, "--to-earliest", "--execute"); + produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); + resetAndAssertOffsets(args, 0); + } + + @Test + public void testResetOffsetsToEarliestOnOneTopicAndPartition() throws Exception { + String topic = "bar"; + createTopic(topic, 2, 1, new Properties(), listenerName(), new Properties()); + + String[] args = buildArgsForGroup(GROUP, "--topic", topic + ":1", "--to-earliest", "--execute"); + ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(args); + + produceConsumeAndShutdown(topic, GROUP, 100, 2); + Map priorCommittedOffsets = committedOffsets(topic, GROUP); + + TopicPartition tp0 = new TopicPartition(topic, 0); + TopicPartition tp1 = new TopicPartition(topic, 1); + Map expectedOffsets = new HashMap<>(); + expectedOffsets.put(tp0, priorCommittedOffsets.get(tp0)); + expectedOffsets.put(tp1, 0L); + resetAndAssertOffsetsCommitted(consumerGroupCommand, expectedOffsets, topic); + + adminZkClient().deleteTopic(topic); + } + + @Test + public void testResetOffsetsToEarliestOnTopics() throws Exception { + String topic1 = "topic1"; + String topic2 = "topic2"; + createTopic(topic1, 1, 1, new Properties(), listenerName(), new Properties()); + createTopic(topic2, 1, 1, new Properties(), listenerName(), new Properties()); + + String[] args = buildArgsForGroup(GROUP, "--topic", topic1, "--topic", topic2, "--to-earliest", "--execute"); + ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(args); + + produceConsumeAndShutdown(topic1, GROUP, 100, 1); + produceConsumeAndShutdown(topic2, GROUP, 100, 1); + + TopicPartition tp1 = new TopicPartition(topic1, 0); + TopicPartition tp2 = new TopicPartition(topic2, 0); + + Map allResetOffsets = toOffsetMap(consumerGroupCommand.resetOffsets().get(GROUP)); + Map expMap = new HashMap<>(); + expMap.put(tp1, 0L); + expMap.put(tp2, 0L); + assertEquals(expMap, allResetOffsets); + assertEquals(Collections.singletonMap(tp1, 0L), committedOffsets(topic1, GROUP)); + assertEquals(Collections.singletonMap(tp2, 0L), committedOffsets(topic2, GROUP)); + + adminZkClient().deleteTopic(topic1); + adminZkClient().deleteTopic(topic2); + } + + @Test + public void testResetOffsetsToEarliestOnTopicsAndPartitions() throws Exception { + String topic1 = "topic1"; + String topic2 = "topic2"; + + createTopic(topic1, 2, 1, new Properties(), listenerName(), new Properties()); + createTopic(topic2, 2, 1, new Properties(), listenerName(), new Properties()); + + String[] args = buildArgsForGroup(GROUP, "--topic", topic1 + ":1", "--topic", topic2 + ":1", "--to-earliest", "--execute"); + ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(args); + + produceConsumeAndShutdown(topic1, GROUP, 100, 2); + produceConsumeAndShutdown(topic2, GROUP, 100, 2); + + Map priorCommittedOffsets1 = committedOffsets(topic1, GROUP); + Map priorCommittedOffsets2 = committedOffsets(topic2, GROUP); + + TopicPartition tp1 = new TopicPartition(topic1, 1); + TopicPartition tp2 = new TopicPartition(topic2, 1); + Map allResetOffsets = toOffsetMap(consumerGroupCommand.resetOffsets().get(GROUP)); + Map expMap = new HashMap<>(); + expMap.put(tp1, 0L); + expMap.put(tp2, 0L); + assertEquals(expMap, allResetOffsets); + priorCommittedOffsets1.put(tp1, 0L); + assertEquals(priorCommittedOffsets1, committedOffsets(topic1, GROUP)); + priorCommittedOffsets2.put(tp2, 0L); + assertEquals(priorCommittedOffsets2, committedOffsets(topic2, GROUP)); + + adminZkClient().deleteTopic(topic1); + adminZkClient().deleteTopic(topic2); + } + + @Test + // This one deals with old CSV export/import format for a single --group arg: "topic,partition,offset" to support old behavior + public void testResetOffsetsExportImportPlanSingleGroupArg() throws Exception { + String topic = "bar"; + TopicPartition tp0 = new TopicPartition(topic, 0); + TopicPartition tp1 = new TopicPartition(topic, 1); + createTopic(topic, 2, 1, new Properties(), listenerName(), new Properties()); + + String[] cgcArgs = buildArgsForGroup(GROUP, "--all-topics", "--to-offset", "2", "--export"); + ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(cgcArgs); + + produceConsumeAndShutdown(topic, GROUP, 100, 2); + + File file = TestUtils.tempFile("reset", ".csv"); + + scala.collection.Map> exportedOffsets = consumerGroupCommand.resetOffsets(); + BufferedWriter bw = new BufferedWriter(new FileWriter(file)); + bw.write(consumerGroupCommand.exportOffsetsToCsv(exportedOffsets)); + bw.close(); + + Map exp1 = new HashMap<>(); + exp1.put(tp0, 2L); + exp1.put(tp1, 2L); + assertEquals(exp1, toOffsetMap(exportedOffsets.get(GROUP))); + + String[] cgcArgsExec = buildArgsForGroup(GROUP, "--all-topics", "--from-file", file.getCanonicalPath(), "--dry-run"); + ConsumerGroupCommand.ConsumerGroupService consumerGroupCommandExec = getConsumerGroupService(cgcArgsExec); + scala.collection.Map> importedOffsets = consumerGroupCommandExec.resetOffsets(); + assertEquals(exp1, toOffsetMap(importedOffsets.get(GROUP))); + + adminZkClient().deleteTopic(topic); + } + + @Test + // This one deals with universal CSV export/import file format "group,topic,partition,offset", + // supporting multiple --group args or --all-groups arg + public void testResetOffsetsExportImportPlan() throws Exception { + String group1 = GROUP + "1"; + String group2 = GROUP + "2"; + String topic1 = "bar1"; + String topic2 = "bar2"; + TopicPartition t1p0 = new TopicPartition(topic1, 0); + TopicPartition t1p1 = new TopicPartition(topic1, 1); + TopicPartition t2p0 = new TopicPartition(topic2, 0); + TopicPartition t2p1 = new TopicPartition(topic2, 1); + createTopic(topic1, 2, 1, new Properties(), listenerName(), new Properties()); + createTopic(topic2, 2, 1, new Properties(), listenerName(), new Properties()); + + String[] cgcArgs = buildArgsForGroups(Arrays.asList(group1, group2), "--all-topics", "--to-offset", "2", "--export"); + ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(cgcArgs); + + produceConsumeAndShutdown(topic1, group1, 100, 1); + produceConsumeAndShutdown(topic2, group2, 100, 1); + + awaitConsumerGroupInactive(consumerGroupCommand, group1); + awaitConsumerGroupInactive(consumerGroupCommand, group2); + + File file = TestUtils.tempFile("reset", ".csv"); + + scala.collection.Map> exportedOffsets = consumerGroupCommand.resetOffsets(); + BufferedWriter bw = new BufferedWriter(new FileWriter(file)); + bw.write(consumerGroupCommand.exportOffsetsToCsv(exportedOffsets)); + bw.close(); + Map exp1 = new HashMap<>(); + exp1.put(t1p0, 2L); + exp1.put(t1p1, 2L); + Map exp2 = new HashMap<>(); + exp2.put(t2p0, 2L); + exp2.put(t2p1, 2L); + + assertEquals(exp1, toOffsetMap(exportedOffsets.get(group1))); + assertEquals(exp2, toOffsetMap(exportedOffsets.get(group2))); + + // Multiple --group's offset import + String[] cgcArgsExec = buildArgsForGroups(Arrays.asList(group1, group2), "--all-topics", "--from-file", file.getCanonicalPath(), "--dry-run"); + ConsumerGroupCommand.ConsumerGroupService consumerGroupCommandExec = getConsumerGroupService(cgcArgsExec); + scala.collection.Map> importedOffsets = consumerGroupCommandExec.resetOffsets(); + assertEquals(exp1, toOffsetMap(importedOffsets.get(group1))); + assertEquals(exp2, toOffsetMap(importedOffsets.get(group2))); + + // Single --group offset import using "group,topic,partition,offset" csv format + String[] cgcArgsExec2 = buildArgsForGroup(group1, "--all-topics", "--from-file", file.getCanonicalPath(), "--dry-run"); + ConsumerGroupCommand.ConsumerGroupService consumerGroupCommandExec2 = getConsumerGroupService(cgcArgsExec2); + scala.collection.Map> importedOffsets2 = consumerGroupCommandExec2.resetOffsets(); + assertEquals(exp1, toOffsetMap(importedOffsets2.get(group1))); + + adminZkClient().deleteTopic(TOPIC); + } + + @Test + public void testResetWithUnrecognizedNewConsumerOption() { + String[] cgcArgs = new String[]{"--new-consumer", "--bootstrap-server", bootstrapServers(listenerName()), "--reset-offsets", + "--group", GROUP, "--all-topics", "--to-offset", "2", "--export"}; + assertThrows(OptionException.class, () -> getConsumerGroupService(cgcArgs)); + } + + private void produceMessages(String topic, int numMessages) { + List> records = IntStream.range(0, numMessages) + .mapToObj(i -> new ProducerRecord(topic, new byte[100 * 1000])) + .collect(Collectors.toList()); + kafka.utils.TestUtils.produceMessages(servers(), seq(records), 1); + } + + private void produceConsumeAndShutdown(String topic, String group, int totalMessages, int numConsumers) throws Exception { + produceMessages(topic, totalMessages); + ConsumerGroupExecutor executor = addConsumerGroupExecutor(numConsumers, topic, group, GroupProtocol.CLASSIC.name); + awaitConsumerProgress(topic, group, totalMessages); + executor.shutdown(); + } + + private void awaitConsumerProgress(String topic, + String group, + long count) throws Exception { + try (Consumer consumer = createNoAutoCommitConsumer(group)) { + Set partitions = consumer.partitionsFor(topic).stream() + .map(partitionInfo -> new TopicPartition(partitionInfo.topic(), partitionInfo.partition())) + .collect(Collectors.toSet()); + + TestUtils.waitForCondition(() -> { + Collection committed = consumer.committed(partitions).values(); + long total = committed.stream() + .mapToLong(offsetAndMetadata -> Optional.ofNullable(offsetAndMetadata).map(OffsetAndMetadata::offset).orElse(0L)) + .sum(); + + return total == count; + }, "Expected that consumer group has consumed all messages from topic/partition. " + + "Expected offset: " + count + ". Actual offset: " + committedOffsets(topic, group).values().stream().mapToLong(Long::longValue).sum()); + } + } + + private void awaitConsumerGroupInactive(ConsumerGroupCommand.ConsumerGroupService consumerGroupService, String group) throws Exception { + TestUtils.waitForCondition(() -> { + String state = consumerGroupService.collectGroupState(group).state(); + return Objects.equals(state, "Empty") || Objects.equals(state, "Dead"); + }, "Expected that consumer group is inactive. Actual state: " + consumerGroupService.collectGroupState(group).state()); + } + + private void resetAndAssertOffsets(String[] args, + long expectedOffset) { + resetAndAssertOffsets(args, expectedOffset, false, Collections.singletonList(TOPIC)); + } + + private void resetAndAssertOffsets(String[] args, + long expectedOffset, + boolean dryRun, + List topics) { + ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(args); + Map> expectedOffsets = topics.stream().collect(Collectors.toMap( + Function.identity(), + topic -> Collections.singletonMap(new TopicPartition(topic, 0), expectedOffset))); + scala.collection.Map> resetOffsetsResultByGroup = consumerGroupCommand.resetOffsets(); + + try { + for (final String topic : topics) { + resetOffsetsResultByGroup.foreach(entry -> { + String group = entry._1; + scala.collection.Map partitionInfo = entry._2; + Map priorOffsets = committedOffsets(topic, group); + Map offsets = new HashMap<>(); + partitionInfo.foreach(partitionInfoEntry -> { + TopicPartition tp = partitionInfoEntry._1; + OffsetAndMetadata offsetAndMetadata = partitionInfoEntry._2; + if (Objects.equals(tp.topic(), topic)) + offsets.put(tp, offsetAndMetadata.offset()); + return null; + }); + assertEquals(expectedOffsets.get(topic), offsets); + assertEquals(dryRun ? priorOffsets : expectedOffsets.get(topic), committedOffsets(topic, group)); + return null; + }); + } + } finally { + consumerGroupCommand.close(); + } + } + + private void resetAndAssertOffsetsCommitted(ConsumerGroupCommand.ConsumerGroupService consumerGroupService, + Map expectedOffsets, + String topic) { + scala.collection.Map> allResetOffsets = consumerGroupService.resetOffsets(); + + allResetOffsets.foreach(entry -> { + String group = entry._1; + scala.collection.Map offsetsInfo = entry._2; + offsetsInfo.foreach(offsetInfoEntry -> { + TopicPartition tp = offsetInfoEntry._1; + OffsetAndMetadata offsetMetadata = offsetInfoEntry._2; + assertEquals(offsetMetadata.offset(), expectedOffsets.get(tp)); + assertEquals(expectedOffsets, committedOffsets(topic, group)); + return null; + }); + return null; + }); + } + + Map toOffsetMap(Option> map) { + assertTrue(map.isDefined()); + Map res = new HashMap<>(); + map.foreach(m -> { + m.foreach(entry -> { + TopicPartition tp = entry._1; + OffsetAndMetadata offsetAndMetadata = entry._2; + res.put(tp, offsetAndMetadata.offset()); + return null; + }); + return null; + }); + return res; + } + + private String[] addTo(String[] args, String...extra) { + List res = new ArrayList<>(Arrays.asList(args)); + res.addAll(Arrays.asList(extra)); + return res.toArray(new String[0]); + } +} From 861fe68cee5ae052531a34a7c40196d2ffdda055 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Fri, 8 Mar 2024 13:51:32 -0800 Subject: [PATCH 050/521] TRIVIAL: fix typo --- .../src/main/java/org/apache/kafka/streams/StreamsConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 54b30fea4d..9616ed81aa 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -519,7 +519,7 @@ public class StreamsConfig extends AbstractConfig { /** {@code enable.metrics.push} */ @SuppressWarnings("WeakerAccess") public static final String ENABLE_METRICS_PUSH_CONFIG = CommonClientConfigs.ENABLE_METRICS_PUSH_CONFIG; - public static final String ENABLE_METRICS_PUSH_DOC = "Whether to enable pushing of internal [main-|restore-|global]consumer, producer, and admin client metrics to the cluster, if the cluster has a client metrics subscription which matches a client."; + public static final String ENABLE_METRICS_PUSH_DOC = "Whether to enable pushing of internal [main-|restore-|global-]consumer, producer, and admin client metrics to the cluster, if the cluster has a client metrics subscription which matches a client."; /** {@code commit.interval.ms} */ @SuppressWarnings("WeakerAccess") From 2fcafbd497f1a49bcb9915147c2c7f2de05e5f59 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Fri, 8 Mar 2024 14:28:39 -0800 Subject: [PATCH 051/521] HOTFIX: fix html markup --- .../src/main/java/org/apache/kafka/streams/StreamsConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 9616ed81aa..c166b3a43f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -514,7 +514,7 @@ public class StreamsConfig extends AbstractConfig { @SuppressWarnings("WeakerAccess") public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG; private static final String CLIENT_ID_DOC = "An ID prefix string used for the client IDs of internal [main-|restore-|global-]consumer, producer, and admin clients" + - " with pattern <client.id>-[Global]StreamThread[-<threadSequenceNumber$gt;]-<consumer|producer|restore-consumer|global-consumer>."; + " with pattern <client.id>-[Global]StreamThread[-<threadSequenceNumber>]-<consumer|producer|restore-consumer|global-consumer>."; /** {@code enable.metrics.push} */ @SuppressWarnings("WeakerAccess") From 432825df95fe9824da05cf86cd40b7741210deb2 Mon Sep 17 00:00:00 2001 From: Joel Hamill <11722533+joel-hamill@users.noreply.github.com> Date: Fri, 8 Mar 2024 15:15:43 -0800 Subject: [PATCH 052/521] MINOR: Fix incorrect syntax for config (#15500) Fix incorrect syntax for config. Reviewers: Matthias J. Sax --- .../org/apache/kafka/clients/producer/ProducerConfig.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 d74b5e4c1d..9471b48aa4 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 @@ -294,8 +294,10 @@ public class ProducerConfig extends AbstractConfig { "