From da055f121e7cab109ee1573fe925c453d567c576 Mon Sep 17 00:00:00 2001 From: Eugene Tolbakov Date: Tue, 12 Jul 2022 08:31:16 +0100 Subject: [PATCH 001/175] KAFKA-14013: Limit the length of the `reason` field sent on the wire (#12388) KIP-800 added the `reason` field to the JoinGroupRequest and the LeaveGroupRequest as I mean to provide more information to the group coordinator. In https://issues.apache.org/jira/browse/KAFKA-13998, we discovered that the size of the field is limited to 32767 chars by our serialisation mechanism. At the moment, the field either provided directly by the user or constructed internally is directly set regardless of its length. This patch sends only the first 255 chars of the used provided or internally generated reason on the wire. Given the purpose of this field, that seems acceptable and that should still provide enough information to operators to understand the cause of a rebalance. Reviewers: David Jacot --- .../kafka/clients/admin/KafkaAdminClient.java | 3 +- .../internals/AbstractCoordinator.java | 10 +++--- .../common/requests/JoinGroupRequest.java | 14 ++++++++ .../clients/admin/KafkaAdminClientTest.java | 7 ++++ .../internals/AbstractCoordinatorTest.java | 32 +++++++++++++++++-- 5 files changed, 58 insertions(+), 8 deletions(-) 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 aad2610c94a45..2b2642e351862 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 @@ -218,6 +218,7 @@ import org.apache.kafka.common.requests.ExpireDelegationTokenResponse; import org.apache.kafka.common.requests.IncrementalAlterConfigsRequest; import org.apache.kafka.common.requests.IncrementalAlterConfigsResponse; +import org.apache.kafka.common.requests.JoinGroupRequest; import org.apache.kafka.common.requests.ListGroupsRequest; import org.apache.kafka.common.requests.ListGroupsResponse; import org.apache.kafka.common.requests.ListOffsetsRequest; @@ -3756,7 +3757,7 @@ private List getMembersFromGroup(String groupId, String reason) public RemoveMembersFromConsumerGroupResult removeMembersFromConsumerGroup(String groupId, RemoveMembersFromConsumerGroupOptions options) { String reason = options.reason() == null || options.reason().isEmpty() ? - DEFAULT_LEAVE_GROUP_REASON : options.reason(); + DEFAULT_LEAVE_GROUP_REASON : JoinGroupRequest.maybeTruncateReason(options.reason()); List members; if (options.removeAll()) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index fa80727d5af43..c9ad797ebeb73 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -478,11 +478,11 @@ boolean joinGroupIfNeeded(final Timer timer) { resetJoinGroupFuture(); synchronized (AbstractCoordinator.this) { - final String shortReason = String.format("rebalance failed due to %s", - exception.getClass().getSimpleName()); + final String simpleName = exception.getClass().getSimpleName(); + final String shortReason = String.format("rebalance failed due to %s", simpleName); final String fullReason = String.format("rebalance failed due to '%s' (%s)", exception.getMessage(), - exception.getClass().getSimpleName()); + simpleName); requestRejoin(shortReason, fullReason); } @@ -559,7 +559,7 @@ RequestFuture sendJoinGroupRequest() { .setProtocolType(protocolType()) .setProtocols(metadata()) .setRebalanceTimeoutMs(this.rebalanceConfig.rebalanceTimeoutMs) - .setReason(this.rejoinReason) + .setReason(JoinGroupRequest.maybeTruncateReason(this.rejoinReason)) ); log.debug("Sending JoinGroup ({}) to coordinator {}", requestBuilder, this.coordinator); @@ -1114,7 +1114,7 @@ public synchronized RequestFuture maybeLeaveGroup(String leaveReason) { generation.memberId, coordinator, leaveReason); LeaveGroupRequest.Builder request = new LeaveGroupRequest.Builder( rebalanceConfig.groupId, - Collections.singletonList(new MemberIdentity().setMemberId(generation.memberId).setReason(leaveReason)) + Collections.singletonList(new MemberIdentity().setMemberId(generation.memberId).setReason(JoinGroupRequest.maybeTruncateReason(leaveReason))) ); future = client.send(coordinator, request).compose(new LeaveGroupResponseHandler(generation)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java index 220a59d183428..774506357bbb9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java @@ -70,6 +70,20 @@ public static void validateGroupInstanceId(String id) { }); } + /** + * Ensures that the provided {@code reason} remains within a range of 255 chars. + * @param reason This is the reason that is sent to the broker over the wire + * as a part of {@code JoinGroupRequest} or {@code LeaveGroupRequest} messages. + * @return a provided reason as is or truncated reason if it exceeds the 255 chars threshold. + */ + public static String maybeTruncateReason(final String reason) { + if (reason.length() > 255) { + return reason.substring(0, 255); + } else { + return reason; + } + } + public JoinGroupRequest(JoinGroupRequestData data, short version) { super(ApiKeys.JOIN_GROUP, version); this.data = data; 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 523a961f96493..61a2aaa00b232 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 @@ -4117,6 +4117,13 @@ public void testRemoveMembersFromGroupReason() throws Exception { testRemoveMembersFromGroup("testing remove members reason", "testing remove members reason"); } + @Test + public void testRemoveMembersFromGroupTruncatesReason() throws Exception { + final String reason = "Very looooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooong reason that is 271 characters long to make sure that length limit logic handles the scenario nicely"; + final String truncatedReason = reason.substring(0, 255); + testRemoveMembersFromGroup(reason, truncatedReason); + } + @Test public void testRemoveMembersFromGroupDefaultReason() throws Exception { testRemoveMembersFromGroup(null, DEFAULT_LEAVE_GROUP_REASON); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java index 6812af29ce5de..ddbebb6dde63e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java @@ -571,6 +571,15 @@ public void testRejoinReason() { expectSyncGroup(generation, memberId); ensureActiveGroup(generation, memberId); assertEquals("", coordinator.rejoinReason()); + + // check limit length of reason field + final String reason = "Very looooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooong reason that is 271 characters long to make sure that length limit logic handles the scenario nicely"; + final String truncatedReason = reason.substring(0, 255); + expectJoinGroup(memberId, truncatedReason, generation, memberId); + expectSyncGroup(generation, memberId); + coordinator.requestRejoin(reason); + ensureActiveGroup(generation, memberId); + assertEquals("", coordinator.rejoinReason()); } private void ensureActiveGroup( @@ -1159,6 +1168,19 @@ public void testHandleNormalLeaveGroupResponse() { assertTrue(leaveGroupFuture.succeeded()); } + @Test + public void testHandleNormalLeaveGroupResponseAndTruncatedLeaveReason() { + MemberResponse memberResponse = new MemberResponse() + .setMemberId(memberId) + .setErrorCode(Errors.NONE.code()); + LeaveGroupResponse response = + leaveGroupResponse(Collections.singletonList(memberResponse)); + String leaveReason = "Very looooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooong leaveReason that is 271 characters long to make sure that length limit logic handles the scenario nicely"; + RequestFuture leaveGroupFuture = setupLeaveGroup(response, leaveReason, leaveReason.substring(0, 255)); + assertNotNull(leaveGroupFuture); + assertTrue(leaveGroupFuture.succeeded()); + } + @Test public void testHandleMultipleMembersLeaveGroupResponse() { MemberResponse memberResponse = new MemberResponse() @@ -1193,6 +1215,12 @@ public void testHandleLeaveGroupResponseWithException() { } private RequestFuture setupLeaveGroup(LeaveGroupResponse leaveGroupResponse) { + return setupLeaveGroup(leaveGroupResponse, "test maybe leave group", "test maybe leave group"); + } + + private RequestFuture setupLeaveGroup(LeaveGroupResponse leaveGroupResponse, + String leaveReason, + String expectedLeaveReason) { setupCoordinator(RETRY_BACKOFF_MS, Integer.MAX_VALUE, Optional.empty()); mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); @@ -1204,11 +1232,11 @@ private RequestFuture setupLeaveGroup(LeaveGroupResponse leaveGroupRespons } LeaveGroupRequestData leaveGroupRequest = ((LeaveGroupRequest) body).data(); return leaveGroupRequest.members().get(0).memberId().equals(memberId) && - leaveGroupRequest.members().get(0).reason().equals("test maybe leave group"); + leaveGroupRequest.members().get(0).reason().equals(expectedLeaveReason); }, leaveGroupResponse); coordinator.ensureActiveGroup(); - return coordinator.maybeLeaveGroup("test maybe leave group"); + return coordinator.maybeLeaveGroup(leaveReason); } @Test From 48f019a9e10b9ea48db2e936dbd191fed128931b Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 12 Jul 2022 01:58:19 -0700 Subject: [PATCH 002/175] KAFKA-14062: OAuth client token refresh fails with SASL extensions (#12398) - Different objects should be considered unique even with same content to support logout - Added comments for SaslExtension re: removal of equals and hashCode - Also swapped out the use of mocks in exchange for *real* SaslExtensions so that we exercise the use of default equals() and hashCode() methods. - Updates to implement equals and hashCode and add tests in SaslExtensionsTest to confirm Co-authored-by: Purshotam Chauhan Reviewers: Manikumar Reddy --- .../common/security/auth/SaslExtensions.java | 83 ++++++++++++++++--- .../security/auth/SaslExtensionsCallback.java | 4 +- .../OAuthBearerClientInitialResponse.java | 2 +- .../common/security/SaslExtensionsTest.java | 28 +++++++ .../OAuthBearerLoginModuleTest.java | 37 ++++++--- 5 files changed, 126 insertions(+), 28 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/security/auth/SaslExtensions.java b/clients/src/main/java/org/apache/kafka/common/security/auth/SaslExtensions.java index c129f1ec400f7..ca4c4df6079ba 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/auth/SaslExtensions.java +++ b/clients/src/main/java/org/apache/kafka/common/security/auth/SaslExtensions.java @@ -19,15 +19,34 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.Set; +import java.util.StringJoiner; +import javax.security.auth.Subject; /** - * A simple immutable value object class holding customizable SASL extensions + * A simple immutable value object class holding customizable SASL extensions. + * + *

+ * + * Note on object identity and equality: SaslExtensions intentionally + * overrides the standard {@link #equals(Object)} and {@link #hashCode()} methods calling their + * respective {@link Object#equals(Object)} and {@link Object#hashCode()} implementations. In so + * doing, it provides equality only via reference identity and will not base equality on + * the underlying values of its {@link #extensionsMap extentions map}. + * + *

+ * + * The reason for this approach to equality is based off of the manner in which + * credentials are stored in a {@link Subject}. SaslExtensions are added to and + * removed from a {@link Subject} via its {@link Subject#getPublicCredentials() public credentials}. + * The public credentials are stored in a {@link Set} in the {@link Subject}, so object equality + * therefore becomes a concern. With shallow, reference-based equality, distinct + * SaslExtensions instances with the same map values can be considered unique. This is + * critical to operations like token refresh. + * + * See KAFKA-14062 for more detail. */ public class SaslExtensions { - /** - * An "empty" instance indicating no SASL extensions - */ - public static final SaslExtensions NO_SASL_EXTENSIONS = new SaslExtensions(Collections.emptyMap()); private final Map extensionsMap; public SaslExtensions(Map extensionsMap) { @@ -41,21 +60,59 @@ public Map map() { return extensionsMap; } + /** + * Creates an "empty" instance indicating no SASL extensions. Do not cache the result of + * this method call for use by multiple {@link Subject}s as the references need to be + * unique. + * + *

+ * + * See the class-level documentation for details. + * @return Unique, but empty, SaslExtensions instance + */ + @SuppressWarnings("unchecked") + public static SaslExtensions empty() { + // It's ok to re-use the EMPTY_MAP instance as the object equality is on the outer + // SaslExtensions reference. + return new SaslExtensions(Collections.EMPTY_MAP); + } + + /** + * Implements equals using the reference comparison implementation from + * {@link Object#equals(Object)}. + * + *

+ * + * See the class-level documentation for details. + * + * @param o Other object to compare + * @return True if o == this + */ @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - return extensionsMap.equals(((SaslExtensions) o).extensionsMap); + public final boolean equals(Object o) { + return super.equals(o); } + /** + * Implements hashCode using the native implementation from + * {@link Object#hashCode()}. + * + *

+ * + * See the class-level documentation for details. + * + * @return Hash code of instance + */ @Override - public String toString() { - return extensionsMap.toString(); + public final int hashCode() { + return super.hashCode(); } @Override - public int hashCode() { - return extensionsMap.hashCode(); + public String toString() { + return new StringJoiner(", ", SaslExtensions.class.getSimpleName() + "[", "]") + .add("extensionsMap=" + extensionsMap) + .toString(); } } diff --git a/clients/src/main/java/org/apache/kafka/common/security/auth/SaslExtensionsCallback.java b/clients/src/main/java/org/apache/kafka/common/security/auth/SaslExtensionsCallback.java index c5bd449e0cc08..f2010afda67e4 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/auth/SaslExtensionsCallback.java +++ b/clients/src/main/java/org/apache/kafka/common/security/auth/SaslExtensionsCallback.java @@ -26,13 +26,13 @@ * in the SASL exchange. */ public class SaslExtensionsCallback implements Callback { - private SaslExtensions extensions = SaslExtensions.NO_SASL_EXTENSIONS; + private SaslExtensions extensions = SaslExtensions.empty(); /** * Returns always non-null {@link SaslExtensions} consisting of the extension * names and values that are sent by the client to the server in the initial * client SASL authentication message. The default value is - * {@link SaslExtensions#NO_SASL_EXTENSIONS} so that if this callback is + * {@link SaslExtensions#empty()} so that if this callback is * unhandled the client will see a non-null value. */ public SaslExtensions extensions() { 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 a356f0da3ddb9..52623ff9fd4f2 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 @@ -108,7 +108,7 @@ public OAuthBearerClientInitialResponse(String tokenValue, String authorizationI this.tokenValue = Objects.requireNonNull(tokenValue, "token value must not be null"); this.authorizationId = authorizationId == null ? "" : authorizationId; validateExtensions(extensions); - this.saslExtensions = extensions != null ? extensions : SaslExtensions.NO_SASL_EXTENSIONS; + this.saslExtensions = extensions != null ? extensions : SaslExtensions.empty(); } /** diff --git a/clients/src/test/java/org/apache/kafka/common/security/SaslExtensionsTest.java b/clients/src/test/java/org/apache/kafka/common/security/SaslExtensionsTest.java index 9acb78cf3ef50..085baf70d2a2a 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/SaslExtensionsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/SaslExtensionsTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.common.security; +import java.util.Collections; import org.apache.kafka.common.security.auth.SaslExtensions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -23,6 +24,7 @@ import java.util.HashMap; import java.util.Map; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -50,4 +52,30 @@ public void testCannotAddValueToMapReferenceAndGetFromExtensions() { this.map.put("hello", "42"); assertNull(extensions.map().get("hello")); } + + /** + * Tests that even when using the same underlying values in the map, two {@link SaslExtensions} + * are considered unique. + * + * @see SaslExtensions class-level documentation + */ + @Test + public void testExtensionsWithEqualValuesAreUnique() { + // If the maps are distinct objects but have the same underlying values, the SaslExtension + // objects should still be unique. + assertNotEquals(new SaslExtensions(Collections.singletonMap("key", "value")), + new SaslExtensions(Collections.singletonMap("key", "value")), + "SaslExtensions with unique maps should be unique"); + + // If the maps are the same object (with the same underlying values), the SaslExtension + // objects should still be unique. + assertNotEquals(new SaslExtensions(map), + new SaslExtensions(map), + "SaslExtensions with duplicate maps should be unique"); + + // If the maps are empty, the SaslExtension objects should still be unique. + assertNotEquals(SaslExtensions.empty(), + SaslExtensions.empty(), + "SaslExtensions returned from SaslExtensions.empty() should be unique"); + } } diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginModuleTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginModuleTest.java index ea03ec5bfa339..0dabeab1f433f 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginModuleTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginModuleTest.java @@ -127,8 +127,8 @@ public void login1Commit1Login2Commit2Logout1Login3Commit3Logout2() throws Login // Create callback handler OAuthBearerToken[] tokens = new OAuthBearerToken[] {mock(OAuthBearerToken.class), mock(OAuthBearerToken.class), mock(OAuthBearerToken.class)}; - SaslExtensions[] extensions = new SaslExtensions[] {mock(SaslExtensions.class), - mock(SaslExtensions.class), mock(SaslExtensions.class)}; + SaslExtensions[] extensions = new SaslExtensions[] {saslExtensions(), + saslExtensions(), saslExtensions()}; TestCallbackHandler testTokenCallbackHandler = new TestCallbackHandler(tokens, extensions); // Create login modules @@ -208,7 +208,6 @@ public void login1Commit1Login2Commit2Logout1Login3Commit3Logout2() throws Login assertSame(extensions[2], publicCredentials.iterator().next()); verifyNoInteractions((Object[]) tokens); - verifyNoInteractions((Object[]) extensions); } @Test @@ -224,8 +223,8 @@ public void login1Commit1Logout1Login2Commit2Logout2() throws LoginException { // Create callback handler OAuthBearerToken[] tokens = new OAuthBearerToken[] {mock(OAuthBearerToken.class), mock(OAuthBearerToken.class)}; - SaslExtensions[] extensions = new SaslExtensions[] {mock(SaslExtensions.class), - mock(SaslExtensions.class)}; + SaslExtensions[] extensions = new SaslExtensions[] {saslExtensions(), + saslExtensions()}; TestCallbackHandler testTokenCallbackHandler = new TestCallbackHandler(tokens, extensions); // Create login modules @@ -270,7 +269,6 @@ public void login1Commit1Logout1Login2Commit2Logout2() throws LoginException { assertEquals(0, publicCredentials.size()); verifyNoInteractions((Object[]) tokens); - verifyNoInteractions((Object[]) extensions); } @Test @@ -285,8 +283,8 @@ public void loginAbortLoginCommitLogout() throws LoginException { // Create callback handler OAuthBearerToken[] tokens = new OAuthBearerToken[] {mock(OAuthBearerToken.class), mock(OAuthBearerToken.class)}; - SaslExtensions[] extensions = new SaslExtensions[] {mock(SaslExtensions.class), - mock(SaslExtensions.class)}; + SaslExtensions[] extensions = new SaslExtensions[] {saslExtensions(), + saslExtensions()}; TestCallbackHandler testTokenCallbackHandler = new TestCallbackHandler(tokens, extensions); // Create login module @@ -322,7 +320,6 @@ public void loginAbortLoginCommitLogout() throws LoginException { assertEquals(0, publicCredentials.size()); verifyNoInteractions((Object[]) tokens); - verifyNoInteractions((Object[]) extensions); } @Test @@ -338,8 +335,8 @@ public void login1Commit1Login2Abort2Login3Commit3Logout3() throws LoginExceptio // Create callback handler OAuthBearerToken[] tokens = new OAuthBearerToken[] {mock(OAuthBearerToken.class), mock(OAuthBearerToken.class), mock(OAuthBearerToken.class)}; - SaslExtensions[] extensions = new SaslExtensions[] {mock(SaslExtensions.class), - mock(SaslExtensions.class), mock(SaslExtensions.class)}; + SaslExtensions[] extensions = new SaslExtensions[] {saslExtensions(), saslExtensions(), + saslExtensions()}; TestCallbackHandler testTokenCallbackHandler = new TestCallbackHandler(tokens, extensions); // Create login modules @@ -406,7 +403,6 @@ public void login1Commit1Login2Abort2Login3Commit3Logout3() throws LoginExceptio assertSame(extensions[2], publicCredentials.iterator().next()); verifyNoInteractions((Object[]) tokens); - verifyNoInteractions((Object[]) extensions); } /** @@ -436,4 +432,21 @@ public void commitDoesNotThrowOnUnsupportedExtensionsCallback() throws LoginExce verifyNoInteractions((Object[]) tokens); } + + /** + * We don't want to use mocks for our tests as we need to make sure to test + * {@link SaslExtensions}' {@link SaslExtensions#equals(Object)} and + * {@link SaslExtensions#hashCode()} methods. + * + *

+ * + * We need to make distinct calls to this method (vs. caching the result and reusing it + * multiple times) because we need to ensure the {@link SaslExtensions} instances are unique. + * This properly mimics the behavior that is used during the token refresh logic. + * + * @return Unique, newly-created {@link SaslExtensions} instance + */ + private SaslExtensions saslExtensions() { + return SaslExtensions.empty(); + } } From ca7a5ce0d92010b3f8f7963ed3fb7905031a5bdc Mon Sep 17 00:00:00 2001 From: dengziming Date: Tue, 12 Jul 2022 23:02:43 +0800 Subject: [PATCH 003/175] KAFKA-13968: Fix 3 major bugs of KRaft snapshot generating (#12265) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit There are 3 bugs when a broker generates a snapshot. 1. Broker should not generate snapshots until it starts publishing. Before a broker starts publishing, BrokerMetadataListener._publisher=None, so _publisher.foreach(publish) will do nothing, so featuresDelta.metadataVersionChange().isPresent is always true, so we will generating a snapshot on every commit since we believe metadata version has changed, here are the logs, note offset 1 is a LeaderChangeMessage so there is no snapshot: [2022-06-08 13:07:43,010] INFO [BrokerMetadataSnapshotter id=0] Creating a new snapshot at offset 0... (kafka.server.metadata.BrokerMetadataSnapshotter:66) [2022-06-08 13:07:43,222] INFO [BrokerMetadataSnapshotter id=0] Creating a new snapshot at offset 2... (kafka.server.metadata.BrokerMetadataSnapshotter:66) [2022-06-08 13:07:43,727] INFO [BrokerMetadataSnapshotter id=0] Creating a new snapshot at offset 3... (kafka.server.metadata.BrokerMetadataSnapshotter:66) [2022-06-08 13:07:44,228] INFO [BrokerMetadataSnapshotter id=0] Creating a new snapshot at offset 4... (kafka.server.metadata.BrokerMetadataSnapshotter:66) 2. We should compute metadataVersionChanged before _publisher.foreach(publish) After _publisher.foreach(publish) the BrokerMetadataListener_delta is always Empty, so metadataVersionChanged is always false, this means we will never trigger snapshot generating even metadata version has changed. 3. We should try to generate a snapshot when starting publishing When we started publishing, there may be a metadata version change, so we should try to generate a snapshot before first publishing. Reviewers: Jason Gustafson , Divij Vaidya , José Armando García Sancio --- .../scala/kafka/server/BrokerServer.scala | 9 +-- .../metadata/BrokerMetadataListener.scala | 39 +++++++----- .../metadata/BrokerMetadataSnapshotter.scala | 26 +++++--- .../metadata/BrokerMetadataListenerTest.scala | 61 ++++++++++++++++++- .../BrokerMetadataSnapshotterTest.scala | 6 +- 5 files changed, 104 insertions(+), 37 deletions(-) diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index d0d2a98b483f9..eb21c1ed25e45 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -62,13 +62,8 @@ class BrokerSnapshotWriterBuilder(raftClient: RaftClient[ApiMessageAndVersion]) extends SnapshotWriterBuilder { override def build(committedOffset: Long, committedEpoch: Int, - lastContainedLogTime: Long): SnapshotWriter[ApiMessageAndVersion] = { - raftClient.createSnapshot(committedOffset, committedEpoch, lastContainedLogTime). - asScala.getOrElse( - throw new RuntimeException("A snapshot already exists with " + - s"committedOffset=$committedOffset, committedEpoch=$committedEpoch, " + - s"lastContainedLogTime=$lastContainedLogTime") - ) + lastContainedLogTime: Long): Option[SnapshotWriter[ApiMessageAndVersion]] = { + raftClient.createSnapshot(committedOffset, committedEpoch, lastContainedLogTime).asScala } } diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala index fa0bc52d7aa01..3b79526a954e9 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala @@ -117,26 +117,34 @@ class BrokerMetadataListener( } finally { reader.close() } - _publisher.foreach(publish) - // If we detected a change in metadata.version, generate a local snapshot - val metadataVersionChanged = Option(_delta.featuresDelta()).exists { featuresDelta => - featuresDelta.metadataVersionChange().isPresent + _bytesSinceLastSnapshot = _bytesSinceLastSnapshot + results.numBytes + if (shouldSnapshot()) { + maybeStartSnapshot() } - snapshotter.foreach { snapshotter => - _bytesSinceLastSnapshot = _bytesSinceLastSnapshot + results.numBytes - if (shouldSnapshot() || metadataVersionChanged) { - if (snapshotter.maybeStartSnapshot(_highestTimestamp, _delta.apply())) { - _bytesSinceLastSnapshot = 0L - } - } - } + _publisher.foreach(publish) } } private def shouldSnapshot(): Boolean = { - _bytesSinceLastSnapshot >= maxBytesBetweenSnapshots + (_bytesSinceLastSnapshot >= maxBytesBetweenSnapshots) || metadataVersionChanged() + } + + private def metadataVersionChanged(): Boolean = { + // The _publisher is empty before starting publishing, and we won't compute feature delta + // until we starting publishing + _publisher.nonEmpty && Option(_delta.featuresDelta()).exists { featuresDelta => + featuresDelta.metadataVersionChange().isPresent + } + } + + private def maybeStartSnapshot(): Unit = { + snapshotter.foreach { snapshotter => + if (snapshotter.maybeStartSnapshot(_highestTimestamp, _delta.apply())) { + _bytesSinceLastSnapshot = 0L + } + } } /** @@ -213,7 +221,7 @@ class BrokerMetadataListener( s" ${messageAndVersion.message}") } - _highestOffset = lastCommittedOffset.getOrElse(batch.baseOffset() + index) + _highestOffset = lastCommittedOffset.getOrElse(batch.baseOffset() + index) delta.replay(highestMetadataOffset, epoch, messageAndVersion.message()) numRecords += 1 @@ -244,6 +252,9 @@ class BrokerMetadataListener( _publisher = Some(publisher) log.info(s"Starting to publish metadata events at offset $highestMetadataOffset.") try { + if (metadataVersionChanged()) { + maybeStartSnapshot() + } publish(publisher) future.complete(null) } catch { diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataSnapshotter.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataSnapshotter.scala index b5179c32f1416..dd77b277c8b61 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataSnapshotter.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataSnapshotter.scala @@ -27,7 +27,7 @@ import org.apache.kafka.snapshot.SnapshotWriter trait SnapshotWriterBuilder { def build(committedOffset: Long, committedEpoch: Int, - lastContainedLogTime: Long): SnapshotWriter[ApiMessageAndVersion] + lastContainedLogTime: Long): Option[SnapshotWriter[ApiMessageAndVersion]] } class BrokerMetadataSnapshotter( @@ -51,20 +51,26 @@ class BrokerMetadataSnapshotter( val eventQueue = new KafkaEventQueue(time, logContext, threadNamePrefix.getOrElse("")) override def maybeStartSnapshot(lastContainedLogTime: Long, image: MetadataImage): Boolean = synchronized { - if (_currentSnapshotOffset == -1L) { + if (_currentSnapshotOffset != -1) { + info(s"Declining to create a new snapshot at ${image.highestOffsetAndEpoch()} because " + + s"there is already a snapshot in progress at offset ${_currentSnapshotOffset}") + false + } else { val writer = writerBuilder.build( image.highestOffsetAndEpoch().offset, image.highestOffsetAndEpoch().epoch, lastContainedLogTime ) - _currentSnapshotOffset = image.highestOffsetAndEpoch().offset - info(s"Creating a new snapshot at offset ${_currentSnapshotOffset}...") - eventQueue.append(new CreateSnapshotEvent(image, writer)) - true - } else { - warn(s"Declining to create a new snapshot at ${image.highestOffsetAndEpoch()} because " + - s"there is already a snapshot in progress at offset ${_currentSnapshotOffset}") - false + if (writer.nonEmpty) { + _currentSnapshotOffset = image.highestOffsetAndEpoch().offset + info(s"Creating a new snapshot at offset ${_currentSnapshotOffset}...") + eventQueue.append(new CreateSnapshotEvent(image, writer.get)) + true + } else { + info(s"Declining to create a new snapshot at ${image.highestOffsetAndEpoch()} because " + + s"there is already a snapshot at offset ${image.highestOffsetAndEpoch().offset}") + false + } } } diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala index f75823a02928b..6de448f280294 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala @@ -20,14 +20,13 @@ package kafka.server.metadata import java.util import java.util.concurrent.atomic.AtomicReference import java.util.{Collections, Optional} - -import org.apache.kafka.common.metadata.{PartitionChangeRecord, PartitionRecord, RegisterBrokerRecord, TopicRecord} +import org.apache.kafka.common.metadata.{FeatureLevelRecord, PartitionChangeRecord, PartitionRecord, RegisterBrokerRecord, TopicRecord} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.utils.Time import org.apache.kafka.common.{Endpoint, Uuid} import org.apache.kafka.image.{MetadataDelta, MetadataImage} import org.apache.kafka.metadata.{BrokerRegistration, RecordTestUtils, VersionRange} -import org.apache.kafka.server.common.ApiMessageAndVersion +import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.Test @@ -240,6 +239,43 @@ class BrokerMetadataListenerTest { } } + @Test + def testNotSnapshotAfterMetadataVersionChangeBeforePublishing(): Unit = { + val snapshotter = new MockMetadataSnapshotter() + val listener = newBrokerMetadataListener(snapshotter = Some(snapshotter), + maxBytesBetweenSnapshots = 1000L) + + updateFeature(listener, feature = MetadataVersion.FEATURE_NAME, MetadataVersion.latest.featureLevel(), 100L) + listener.getImageRecords().get() + assertEquals(-1L, snapshotter.activeSnapshotOffset, "We won't generate snapshot on metadata version change before starting publishing") + } + + @Test + def testSnapshotAfterMetadataVersionChangeWhenStarting(): Unit = { + val snapshotter = new MockMetadataSnapshotter() + val listener = newBrokerMetadataListener(snapshotter = Some(snapshotter), + maxBytesBetweenSnapshots = 1000L) + + val endOffset = 100L + updateFeature(listener, feature = MetadataVersion.FEATURE_NAME, MetadataVersion.latest.featureLevel(), endOffset) + listener.startPublishing(new MockMetadataPublisher()).get() + assertEquals(endOffset, snapshotter.activeSnapshotOffset, "We should try to generate snapshot when starting publishing") + } + + @Test + def testSnapshotAfterMetadataVersionChange(): Unit = { + val snapshotter = new MockMetadataSnapshotter() + val listener = newBrokerMetadataListener(snapshotter = Some(snapshotter), + maxBytesBetweenSnapshots = 1000L) + listener.startPublishing(new MockMetadataPublisher()).get() + + val endOffset = 100L + updateFeature(listener, feature = MetadataVersion.FEATURE_NAME, (MetadataVersion.latest().featureLevel() - 1).toShort, endOffset) + // Waiting for the metadata version update to get processed + listener.getImageRecords().get() + assertEquals(endOffset, snapshotter.activeSnapshotOffset, "We should generate snapshot on feature update") + } + private def registerBrokers( listener: BrokerMetadataListener, brokerIds: Iterable[Int], @@ -285,4 +321,23 @@ class BrokerMetadataListenerTest { ) } + private def updateFeature( + listener: BrokerMetadataListener, + feature: String, + version: Short, + endOffset: Long + ): Unit = { + listener.handleCommit( + RecordTestUtils.mockBatchReader( + endOffset, + 0, + util.Arrays.asList( + new ApiMessageAndVersion(new FeatureLevelRecord(). + setName(feature). + setFeatureLevel(version), 0.toShort) + ) + ) + ) + } + } diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataSnapshotterTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataSnapshotterTest.scala index 82426611425a1..e6702ee287fef 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataSnapshotterTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataSnapshotterTest.scala @@ -20,7 +20,6 @@ package kafka.server.metadata import java.nio.ByteBuffer import java.util.Optional import java.util.concurrent.{CompletableFuture, CountDownLatch} - import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.protocol.ByteBufferAccessor import org.apache.kafka.common.record.{CompressionType, MemoryRecords} @@ -34,6 +33,7 @@ import org.apache.kafka.snapshot.{MockRawSnapshotWriter, RecordsSnapshotWriter, import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} import org.junit.jupiter.api.Test +import scala.compat.java8.OptionConverters._ class BrokerMetadataSnapshotterTest { @Test @@ -48,7 +48,7 @@ class BrokerMetadataSnapshotterTest { override def build(committedOffset: Long, committedEpoch: Int, - lastContainedLogTime: Long): SnapshotWriter[ApiMessageAndVersion] = { + lastContainedLogTime: Long): Option[SnapshotWriter[ApiMessageAndVersion]] = { val offsetAndEpoch = new OffsetAndEpoch(committedOffset, committedEpoch) RecordsSnapshotWriter.createWithHeader( () => { @@ -62,7 +62,7 @@ class BrokerMetadataSnapshotterTest { lastContainedLogTime, CompressionType.NONE, MetadataRecordSerde.INSTANCE - ).get(); + ).asScala } def consumeSnapshotBuffer(committedOffset: Long, committedEpoch: Int)(buffer: ByteBuffer): Unit = { From 1b52eeafe98620650492058f0898c54017bfefed Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Tue, 12 Jul 2022 23:46:04 -0700 Subject: [PATCH 004/175] MINOR: Run MessageFormatChangeTest in ZK mode only (#12395) KRaft mode will not support writing messages with an older message format (2.8) since the min supported IBP is 3.0 for KRaft. Testing support for reading older message formats will be covered by https://issues.apache.org/jira/browse/KAFKA-14056. Reviewers: David Jacot --- tests/kafkatest/tests/client/message_format_change_test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/kafkatest/tests/client/message_format_change_test.py b/tests/kafkatest/tests/client/message_format_change_test.py index cb6cf72d22e5d..b65ef24704a68 100644 --- a/tests/kafkatest/tests/client/message_format_change_test.py +++ b/tests/kafkatest/tests/client/message_format_change_test.py @@ -57,9 +57,9 @@ def produce_and_consume(self, producer_version, consumer_version, group): err_msg="Producer did not produce all messages in reasonable amount of time")) @cluster(num_nodes=12) - @matrix(producer_version=[str(DEV_BRANCH)], consumer_version=[str(DEV_BRANCH)], metadata_quorum=quorum.all_non_upgrade) - @matrix(producer_version=[str(LATEST_0_10)], consumer_version=[str(LATEST_0_10)], metadata_quorum=quorum.all_non_upgrade) - @matrix(producer_version=[str(LATEST_0_9)], consumer_version=[str(LATEST_0_9)], metadata_quorum=quorum.all_non_upgrade) + @matrix(producer_version=[str(DEV_BRANCH)], consumer_version=[str(DEV_BRANCH)], metadata_quorum=[quorum.zk]) + @matrix(producer_version=[str(LATEST_0_10)], consumer_version=[str(LATEST_0_10)], metadata_quorum=[quorum.zk]) + @matrix(producer_version=[str(LATEST_0_9)], consumer_version=[str(LATEST_0_9)], metadata_quorum=[quorum.zk]) def test_compatibility(self, producer_version, consumer_version, metadata_quorum=quorum.zk): """ This tests performs the following checks: The workload is a mix of 0.9.x, 0.10.x and 0.11.x producers and consumers From 8e92551f4294098a1d01121ce1ca01f5ff2e0499 Mon Sep 17 00:00:00 2001 From: Christo Lolov Date: Thu, 14 Jul 2022 01:47:34 +0100 Subject: [PATCH 005/175] KAFKA-14008: Add docs for Streams throughput metrics introduced in KIP-846 (#12377) Reviewers: Walker Carlson , Matthias J. Sax --- docs/ops.html | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/docs/ops.html b/docs/ops.html index 76fee9cc868f6..1854cf057c2fc 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -2604,6 +2604,16 @@

- * This is a convenience method for {@link #listConsumerGroupOffsets(String, ListConsumerGroupOffsetsOptions)} with default options. + * This is a convenience method for {@link #listConsumerGroupOffsets(Map, ListConsumerGroupOffsetsOptions)} + * to list offsets of all partitions of one group with default options. * * @return The ListGroupOffsetsResult. */ @@ -932,6 +941,29 @@ default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId) return listConsumerGroupOffsets(groupId, new ListConsumerGroupOffsetsOptions()); } + /** + * List the consumer group offsets available in the cluster for the specified consumer groups. + * + * @param groupSpecs Map of consumer group ids to a spec that specifies the topic partitions of the group to list offsets for. + * + * @param options The options to use when listing the consumer group offsets. + * @return The ListConsumerGroupOffsetsResult + */ + ListConsumerGroupOffsetsResult listConsumerGroupOffsets(Map groupSpecs, ListConsumerGroupOffsetsOptions options); + + /** + * List the consumer group offsets available in the cluster for the specified groups with the default options. + *

+ * This is a convenience method for + * {@link #listConsumerGroupOffsets(Map, ListConsumerGroupOffsetsOptions)} with default options. + * + * @param groupSpecs Map of consumer group ids to a spec that specifies the topic partitions of the group to list offsets for. + * @return The ListConsumerGroupOffsetsResult. + */ + default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(Map groupSpecs) { + return listConsumerGroupOffsets(groupSpecs, new ListConsumerGroupOffsetsOptions()); + } + /** * Delete consumer groups from the cluster. * 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 2b2642e351862..41eb27a1ddad8 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 @@ -3401,13 +3401,14 @@ void handleFailure(Throwable throwable) { } @Override - public ListConsumerGroupOffsetsResult listConsumerGroupOffsets(final String groupId, - final ListConsumerGroupOffsetsOptions options) { + public ListConsumerGroupOffsetsResult listConsumerGroupOffsets(Map groupSpecs, + ListConsumerGroupOffsetsOptions options) { SimpleAdminApiFuture> future = - ListConsumerGroupOffsetsHandler.newFuture(groupId); - ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(groupId, options.topicPartitions(), options.requireStable(), logContext); + ListConsumerGroupOffsetsHandler.newFuture(groupSpecs.keySet()); + ListConsumerGroupOffsetsHandler handler = + new ListConsumerGroupOffsetsHandler(groupSpecs, options.requireStable(), logContext); invokeDriver(handler, future, options.timeoutMs); - return new ListConsumerGroupOffsetsResult(future.get(CoordinatorKey.byGroupId(groupId))); + return new ListConsumerGroupOffsetsResult(future.all()); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsOptions.java index 292a47ef393c3..44d3a407327e1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsOptions.java @@ -23,23 +23,28 @@ import java.util.List; /** - * Options for {@link Admin#listConsumerGroupOffsets(String)}. + * Options for {@link Admin#listConsumerGroupOffsets(java.util.Map)} and {@link Admin#listConsumerGroupOffsets(String)}. *

* The API of this class is evolving, see {@link Admin} for details. */ @InterfaceStability.Evolving public class ListConsumerGroupOffsetsOptions extends AbstractOptions { - private List topicPartitions = null; + private List topicPartitions; private boolean requireStable = false; /** * Set the topic partitions to list as part of the result. * {@code null} includes all topic partitions. + *

+ * @deprecated Since 3.3. + * Use {@link Admin#listConsumerGroupOffsets(java.util.Map, ListConsumerGroupOffsetsOptions)} + * to specify topic partitions. * * @param topicPartitions List of topic partitions to include * @return This ListGroupOffsetsOptions */ + @Deprecated public ListConsumerGroupOffsetsOptions topicPartitions(List topicPartitions) { this.topicPartitions = topicPartitions; return this; @@ -55,7 +60,12 @@ public ListConsumerGroupOffsetsOptions requireStable(final boolean requireStable /** * Returns a list of topic partitions to add as part of the result. + *

+ * @deprecated Since 3.3. + * Use {@link Admin#listConsumerGroupOffsets(java.util.Map, ListConsumerGroupOffsetsOptions)} + * to specify topic partitions. */ + @Deprecated public List topicPartitions() { return topicPartitions; } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsResult.java index 48f4531418110..2136e33a401e1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsResult.java @@ -17,25 +17,32 @@ package org.apache.kafka.clients.admin; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; + +import org.apache.kafka.clients.admin.internals.CoordinatorKey; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.annotation.InterfaceStability; -import java.util.Map; - /** - * The result of the {@link Admin#listConsumerGroupOffsets(String)} call. + * The result of the {@link Admin#listConsumerGroupOffsets(Map)} and + * {@link Admin#listConsumerGroupOffsets(String)} call. *

* The API of this class is evolving, see {@link Admin} for details. */ @InterfaceStability.Evolving public class ListConsumerGroupOffsetsResult { - final KafkaFuture> future; + final Map>> futures; - ListConsumerGroupOffsetsResult(KafkaFuture> future) { - this.future = future; + ListConsumerGroupOffsetsResult(final Map>> futures) { + this.futures = futures.entrySet().stream() + .collect(Collectors.toMap(e -> e.getKey().idValue, Entry::getValue)); } /** @@ -43,7 +50,42 @@ public class ListConsumerGroupOffsetsResult { * If the group does not have a committed offset for this partition, the corresponding value in the returned map will be null. */ public KafkaFuture> partitionsToOffsetAndMetadata() { - return future; + if (futures.size() != 1) { + throw new IllegalStateException("Offsets from multiple consumer groups were requested. " + + "Use partitionsToOffsetAndMetadata(groupId) instead to get future for a specific group."); + } + return futures.values().iterator().next(); } + /** + * Return a future which yields a map of topic partitions to OffsetAndMetadata objects for + * the specified group. If the group doesn't have a committed offset for a specific + * partition, the corresponding value in the returned map will be null. + */ + public KafkaFuture> partitionsToOffsetAndMetadata(String groupId) { + if (!futures.containsKey(groupId)) + throw new IllegalArgumentException("Offsets for consumer group '" + groupId + "' were not requested."); + return futures.get(groupId); + } + + /** + * Return a future which yields all Map objects, + * if requests for all the groups succeed. + */ + public KafkaFuture>> all() { + return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])).thenApply( + nil -> { + Map> listedConsumerGroupOffsets = new HashMap<>(futures.size()); + futures.forEach((key, future) -> { + try { + listedConsumerGroupOffsets.put(key, future.get()); + } catch (InterruptedException | ExecutionException e) { + // This should be unreachable, since the KafkaFuture#allOf already ensured + // that all of the futures completed successfully. + throw new RuntimeException(e); + } + }); + return listedConsumerGroupOffsets; + }); + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsSpec.java b/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsSpec.java new file mode 100644 index 0000000000000..83858e49c8170 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsSpec.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.admin; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.annotation.InterfaceStability; + +import java.util.Collection; +import java.util.Objects; + +/** + * Specification of consumer group offsets to list using {@link Admin#listConsumerGroupOffsets(java.util.Map)}. + * + * The API of this class is evolving, see {@link Admin} for details. + */ +@InterfaceStability.Evolving +public class ListConsumerGroupOffsetsSpec { + + private Collection topicPartitions; + + /** + * Set the topic partitions whose offsets are to be listed for a consumer group. + * {@code null} includes all topic partitions. + * + * @param topicPartitions List of topic partitions to include + * @return This ListConsumerGroupOffsetSpec + */ + public ListConsumerGroupOffsetsSpec topicPartitions(Collection topicPartitions) { + this.topicPartitions = topicPartitions; + return this; + } + + /** + * Returns the topic partitions whose offsets are to be listed for a consumer group. + * {@code null} indicates that offsets of all partitions of the group are to be listed. + */ + public Collection topicPartitions() { + return topicPartitions; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof ListConsumerGroupOffsetsSpec)) { + return false; + } + ListConsumerGroupOffsetsSpec that = (ListConsumerGroupOffsetsSpec) o; + return Objects.equals(topicPartitions, that.topicPartitions); + } + + @Override + public int hashCode() { + return Objects.hash(topicPartitions); + } + + @Override + public String toString() { + return "ListConsumerGroupOffsetsSpec(" + + "topicPartitions=" + topicPartitions + + ')'; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java index d00db4b18c694..0e1b03d964fea 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.requests.AbstractResponse; import org.apache.kafka.common.requests.FindCoordinatorRequest.NoBatchedFindCoordinatorsException; +import org.apache.kafka.common.requests.OffsetFetchRequest.NoBatchedOffsetFetchRequestException; import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; @@ -253,7 +254,7 @@ public void onFailure( .collect(Collectors.toSet()); retryLookup(keysToUnmap); - } else if (t instanceof NoBatchedFindCoordinatorsException) { + } else if (t instanceof NoBatchedFindCoordinatorsException || t instanceof NoBatchedOffsetFetchRequestException) { ((CoordinatorStrategy) handler.lookupStrategy()).disableBatch(); Set keysToUnmap = spec.keys.stream() .filter(future.lookupKeys()::contains) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategy.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategy.java index e6fc0d624a0cd..02b68527c3c7e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategy.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategy.java @@ -120,6 +120,10 @@ public void disableBatch() { batch = false; } + public boolean batch() { + return batch; + } + private CoordinatorKey requireSingletonAndType(Set keys) { if (keys.size() != 1) { throw new IllegalArgumentException("Unexpected size of key set: expected 1, but got " + keys.size()); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java index 08648821f7c9f..21c7d8d488f3d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java @@ -17,14 +17,16 @@ package org.apache.kafka.clients.admin.internals; import java.util.ArrayList; +import java.util.Collection; 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.stream.Collectors; +import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsSpec; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; @@ -36,39 +38,26 @@ import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; -public class ListConsumerGroupOffsetsHandler extends AdminApiHandler.Batched> { +public class ListConsumerGroupOffsetsHandler implements AdminApiHandler> { - private final CoordinatorKey groupId; - private final List partitions; private final boolean requireStable; + private final Map groupSpecs; private final Logger log; - private final AdminApiLookupStrategy lookupStrategy; + private final CoordinatorStrategy lookupStrategy; public ListConsumerGroupOffsetsHandler( - String groupId, - List partitions, - LogContext logContext - ) { - this(groupId, partitions, false, logContext); - } - - public ListConsumerGroupOffsetsHandler( - String groupId, - List partitions, + Map groupSpecs, boolean requireStable, LogContext logContext ) { - this.groupId = CoordinatorKey.byGroupId(groupId); - this.partitions = partitions; - this.requireStable = requireStable; this.log = logContext.logger(ListConsumerGroupOffsetsHandler.class); this.lookupStrategy = new CoordinatorStrategy(CoordinatorType.GROUP, logContext); + this.groupSpecs = groupSpecs; + this.requireStable = requireStable; } - public static AdminApiFuture.SimpleAdminApiFuture> newFuture( - String groupId - ) { - return AdminApiFuture.forKeys(Collections.singleton(CoordinatorKey.byGroupId(groupId))); + public static AdminApiFuture.SimpleAdminApiFuture> newFuture(Collection groupIds) { + return AdminApiFuture.forKeys(coordinatorKeys(groupIds)); } @Override @@ -82,16 +71,45 @@ public AdminApiLookupStrategy lookupStrategy() { } private void validateKeys(Set groupIds) { - if (!groupIds.equals(Collections.singleton(groupId))) { + Set keys = coordinatorKeys(groupSpecs.keySet()); + if (!keys.containsAll(groupIds)) { throw new IllegalArgumentException("Received unexpected group ids " + groupIds + - " (expected only " + Collections.singleton(groupId) + ")"); + " (expected one of " + keys + ")"); } } + private static Set coordinatorKeys(Collection groupIds) { + return groupIds.stream() + .map(CoordinatorKey::byGroupId) + .collect(Collectors.toSet()); + } + + public OffsetFetchRequest.Builder buildBatchedRequest(Set groupIds) { + // Create a map that only contains the consumer groups owned by the coordinator. + Map> coordinatorGroupIdToTopicPartitions = new HashMap<>(groupIds.size()); + groupIds.forEach(g -> { + ListConsumerGroupOffsetsSpec spec = groupSpecs.get(g.idValue); + List partitions = spec.topicPartitions() != null ? new ArrayList<>(spec.topicPartitions()) : null; + coordinatorGroupIdToTopicPartitions.put(g.idValue, partitions); + }); + + return new OffsetFetchRequest.Builder(coordinatorGroupIdToTopicPartitions, requireStable, false); + } + @Override - public OffsetFetchRequest.Builder buildBatchedRequest(int coordinatorId, Set groupIds) { + public Collection> buildRequest(int brokerId, Set groupIds) { validateKeys(groupIds); - return new OffsetFetchRequest.Builder(groupId.idValue, requireStable, partitions, false); + + // When the OffsetFetchRequest fails with NoBatchedOffsetFetchRequestException, we completely disable + // the batching end-to-end, including the FindCoordinatorRequest. + if (lookupStrategy.batch()) { + return Collections.singletonList(new RequestAndKeys<>(buildBatchedRequest(groupIds), groupIds)); + } else { + return groupIds.stream().map(groupId -> { + Set keys = Collections.singleton(groupId); + return new RequestAndKeys<>(buildBatchedRequest(keys), keys); + }).collect(Collectors.toList()); + } } @Override @@ -104,44 +122,46 @@ public ApiResult> handleR final OffsetFetchResponse response = (OffsetFetchResponse) abstractResponse; - // the groupError will contain the group level error for v0-v8 OffsetFetchResponse - Errors groupError = response.groupLevelError(groupId.idValue); - if (groupError != Errors.NONE) { - final Map failed = new HashMap<>(); - final Set groupsToUnmap = new HashSet<>(); - - handleGroupError(groupId, groupError, failed, groupsToUnmap); - - return new ApiResult<>(Collections.emptyMap(), failed, new ArrayList<>(groupsToUnmap)); - } else { - final Map groupOffsetsListing = new HashMap<>(); - - response.partitionDataMap(groupId.idValue).forEach((topicPartition, partitionData) -> { - final Errors error = partitionData.error; - if (error == Errors.NONE) { - final long offset = partitionData.offset; - final String metadata = partitionData.metadata; - final Optional leaderEpoch = partitionData.leaderEpoch; - // Negative offset indicates that the group has no committed offset for this partition - if (offset < 0) { - groupOffsetsListing.put(topicPartition, null); + Map> completed = new HashMap<>(); + Map failed = new HashMap<>(); + List unmapped = new ArrayList<>(); + for (CoordinatorKey coordinatorKey : groupIds) { + String group = coordinatorKey.idValue; + if (response.groupHasError(group)) { + handleGroupError(CoordinatorKey.byGroupId(group), response.groupLevelError(group), failed, unmapped); + } else { + final Map groupOffsetsListing = new HashMap<>(); + Map responseData = response.partitionDataMap(group); + for (Map.Entry partitionEntry : responseData.entrySet()) { + final TopicPartition topicPartition = partitionEntry.getKey(); + OffsetFetchResponse.PartitionData partitionData = partitionEntry.getValue(); + final Errors error = partitionData.error; + + if (error == Errors.NONE) { + final long offset = partitionData.offset; + final String metadata = partitionData.metadata; + final Optional leaderEpoch = partitionData.leaderEpoch; + // Negative offset indicates that the group has no committed offset for this partition + if (offset < 0) { + groupOffsetsListing.put(topicPartition, null); + } else { + groupOffsetsListing.put(topicPartition, new OffsetAndMetadata(offset, leaderEpoch, metadata)); + } } else { - groupOffsetsListing.put(topicPartition, new OffsetAndMetadata(offset, leaderEpoch, metadata)); + log.warn("Skipping return offset for {} due to error {}.", topicPartition, error); } - } else { - log.warn("Skipping return offset for {} due to error {}.", topicPartition, error); } - }); - - return ApiResult.completed(groupId, groupOffsetsListing); + completed.put(CoordinatorKey.byGroupId(group), groupOffsetsListing); + } } + return new ApiResult<>(completed, failed, unmapped); } private void handleGroupError( CoordinatorKey groupId, Errors error, Map failed, - Set groupsToUnmap + List groupsToUnmap ) { switch (error) { case GROUP_AUTHORIZATION_FAILED: diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java index 213182ec8c4a5..4e25984668da5 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java @@ -173,8 +173,8 @@ public OffsetFetchResponse(int throttleTimeMs, Errors error, Map errors, Map> responseData) { + Map errors, + Map> responseData) { super(ApiKeys.OFFSET_FETCH); List groupList = new ArrayList<>(); for (Entry> entry : responseData.entrySet()) { @@ -250,7 +250,11 @@ public boolean hasError() { } public boolean groupHasError(String groupId) { - return groupLevelErrors.get(groupId) != Errors.NONE; + Errors error = groupLevelErrors.get(groupId); + if (error == null) { + return this.error != null && this.error != Errors.NONE; + } + return error != Errors.NONE; } public Errors error() { diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientTestUtils.java b/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientTestUtils.java index 6f98a166b17ca..d8b9f427d6b24 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientTestUtils.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientTestUtils.java @@ -24,6 +24,7 @@ import org.apache.kafka.clients.HostResolver; import org.apache.kafka.clients.admin.CreateTopicsResult.TopicMetadataAndConfig; import org.apache.kafka.clients.admin.internals.MetadataOperationContext; +import org.apache.kafka.clients.admin.internals.CoordinatorKey; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaFuture; @@ -104,14 +105,17 @@ public static DescribeTopicsResult describeTopicsResult(Map KafkaFuture.completedFuture(e.getValue())))); } - public static ListConsumerGroupOffsetsResult listConsumerGroupOffsetsResult(Map offsets) { - return new ListConsumerGroupOffsetsResult(KafkaFuture.completedFuture(offsets)); + public static ListConsumerGroupOffsetsResult listConsumerGroupOffsetsResult(Map> offsets) { + Map>> resultMap = offsets.entrySet().stream() + .collect(Collectors.toMap(e -> CoordinatorKey.byGroupId(e.getKey()), + e -> KafkaFutureImpl.completedFuture(e.getValue()))); + return new ListConsumerGroupOffsetsResult(resultMap); } - public static ListConsumerGroupOffsetsResult listConsumerGroupOffsetsResult(KafkaException exception) { + public static ListConsumerGroupOffsetsResult listConsumerGroupOffsetsResult(String group, KafkaException exception) { final KafkaFutureImpl> future = new KafkaFutureImpl<>(); future.completeExceptionally(exception); - return new ListConsumerGroupOffsetsResult(future); + return new ListConsumerGroupOffsetsResult(Collections.singletonMap(CoordinatorKey.byGroupId(group), future)); } /** 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 61a2aaa00b232..3d285a45f7b8f 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 @@ -108,6 +108,7 @@ import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData.CredentialInfo; import org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult; import org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult; +import org.apache.kafka.common.message.FindCoordinatorRequestData; import org.apache.kafka.common.message.FindCoordinatorResponseData; import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData; import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData.AlterConfigsResourceResponse; @@ -192,6 +193,7 @@ import org.apache.kafka.common.requests.OffsetDeleteResponse; import org.apache.kafka.common.requests.OffsetFetchRequest; import org.apache.kafka.common.requests.OffsetFetchResponse; +import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData; import org.apache.kafka.common.requests.RequestTestUtils; import org.apache.kafka.common.requests.UnregisterBrokerResponse; import org.apache.kafka.common.requests.UpdateFeaturesRequest; @@ -224,6 +226,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Optional; import java.util.OptionalInt; import java.util.OptionalLong; @@ -266,6 +269,7 @@ public class KafkaAdminClientTest { private static final Logger log = LoggerFactory.getLogger(KafkaAdminClientTest.class); private static final String GROUP_ID = "group-0"; + private static final int THROTTLE = 10; @Test public void testDefaultApiTimeoutAndRequestTimeoutConflicts() { @@ -501,6 +505,21 @@ private static FindCoordinatorResponse prepareOldFindCoordinatorResponse(Errors return FindCoordinatorResponse.prepareOldResponse(error, node); } + private static FindCoordinatorResponse prepareBatchedFindCoordinatorResponse(Errors error, Node node, Collection groups) { + FindCoordinatorResponseData data = new FindCoordinatorResponseData(); + List coordinators = groups.stream() + .map(group -> new FindCoordinatorResponseData.Coordinator() + .setErrorCode(error.code()) + .setErrorMessage(error.message()) + .setKey(group) + .setHost(node.host()) + .setPort(node.port()) + .setNodeId(node.id())) + .collect(Collectors.toList()); + data.setCoordinators(coordinators); + return new FindCoordinatorResponse(data); + } + private static MetadataResponse prepareMetadataResponse(Cluster cluster, Errors error) { List metadata = new ArrayList<>(); for (String topic : cluster.topics()) { @@ -3067,9 +3086,11 @@ public void testListConsumerGroupOffsetsOptions() throws Exception { env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); final TopicPartition tp1 = new TopicPartition("A", 0); - final ListConsumerGroupOffsetsOptions options = new ListConsumerGroupOffsetsOptions(); - options.topicPartitions(Collections.singletonList(tp1)).requireStable(true); - final ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets(GROUP_ID, options); + final ListConsumerGroupOffsetsOptions options = new ListConsumerGroupOffsetsOptions() + .requireStable(true); + final ListConsumerGroupOffsetsSpec groupSpec = new ListConsumerGroupOffsetsSpec() + .topicPartitions(Collections.singletonList(tp1)); + env.adminClient().listConsumerGroupOffsets(Collections.singletonMap(GROUP_ID, groupSpec), options); final MockClient mockClient = env.kafkaClient(); TestUtils.waitForCondition(() -> { @@ -3077,11 +3098,11 @@ public void testListConsumerGroupOffsetsOptions() throws Exception { if (clientRequest != null) { OffsetFetchRequestData data = ((OffsetFetchRequest.Builder) clientRequest.requestBuilder()).data; return data.requireStable() && - data.topics().get(0).name().equals("A") && - data.topics().get(0).partitionIndexes().equals(Collections.singletonList(0)); + data.groups().get(0).topics().get(0).name().equals("A") && + data.groups().get(0).topics().get(0).partitionIndexes().equals(Collections.singletonList(0)); } return false; - }, "Failed awaiting ListConsumerGroupOffsets request"); + }, "Failed awaiting ListConsumerGroupOfsets request"); } } @@ -3095,12 +3116,11 @@ public void testListConsumerGroupOffsetsNumRetries() throws Exception { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); - env.kafkaClient().prepareResponse(new OffsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); + env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); final ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets(GROUP_ID); - TestUtils.assertFutureError(result.partitionsToOffsetAndMetadata(), TimeoutException.class); } } @@ -3124,16 +3144,16 @@ public void testListConsumerGroupOffsetsRetryBackoff() throws Exception { mockClient.prepareResponse(body -> { firstAttemptTime.set(time.milliseconds()); return true; - }, new OffsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); + }, offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); mockClient.prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); mockClient.prepareResponse(body -> { secondAttemptTime.set(time.milliseconds()); return true; - }, new OffsetFetchResponse(Errors.NONE, Collections.emptyMap())); + }, offsetFetchResponse(Errors.NONE, Collections.emptyMap())); - final KafkaFuture> future = env.adminClient().listConsumerGroupOffsets("group-0").partitionsToOffsetAndMetadata(); + final KafkaFuture> future = env.adminClient().listConsumerGroupOffsets(GROUP_ID).partitionsToOffsetAndMetadata(); TestUtils.waitForCondition(() -> mockClient.numAwaitingResponses() == 1, "Failed awaiting ListConsumerGroupOffsets first request failure"); TestUtils.waitForCondition(() -> ((KafkaAdminClient) env.adminClient()).numPendingCalls() == 1, "Failed to add retry ListConsumerGroupOffsets call on first failure"); @@ -3157,7 +3177,8 @@ public void testListConsumerGroupOffsetsRetriableErrors() throws Exception { prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); env.kafkaClient().prepareResponse( - new OffsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Collections.emptyMap())); + offsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Collections.emptyMap())); + /* * We need to return two responses here, one for NOT_COORDINATOR call when calling list consumer offsets * api using coordinator that has moved. This will retry whole operation. So we need to again respond with a @@ -3166,19 +3187,19 @@ public void testListConsumerGroupOffsetsRetriableErrors() throws Exception { * And the same reason for the following COORDINATOR_NOT_AVAILABLE error response */ env.kafkaClient().prepareResponse( - new OffsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); + offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); env.kafkaClient().prepareResponse( prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); env.kafkaClient().prepareResponse( - new OffsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE, Collections.emptyMap())); + offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE, Collections.emptyMap())); env.kafkaClient().prepareResponse( prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); env.kafkaClient().prepareResponse( - new OffsetFetchResponse(Errors.NONE, Collections.emptyMap())); + offsetFetchResponse(Errors.NONE, Collections.emptyMap())); final ListConsumerGroupOffsetsResult errorResult1 = env.adminClient().listConsumerGroupOffsets(GROUP_ID); @@ -3199,8 +3220,7 @@ public void testListConsumerGroupOffsetsNonRetriableErrors() throws Exception { env.kafkaClient().prepareResponse( prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); - env.kafkaClient().prepareResponse( - new OffsetFetchResponse(error, Collections.emptyMap())); + env.kafkaClient().prepareResponse(offsetFetchResponse(error, Collections.emptyMap())); ListConsumerGroupOffsetsResult errorResult = env.adminClient().listConsumerGroupOffsets(GROUP_ID); @@ -3220,7 +3240,7 @@ public void testListConsumerGroupOffsets() throws Exception { env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); // Retriable errors should be retried - env.kafkaClient().prepareResponse(new OffsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Collections.emptyMap())); + env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Collections.emptyMap())); /* * We need to return two responses here, one for NOT_COORDINATOR error when calling list consumer group offsets @@ -3229,10 +3249,10 @@ public void testListConsumerGroupOffsets() throws Exception { * * And the same reason for the following COORDINATOR_NOT_AVAILABLE error response */ - env.kafkaClient().prepareResponse(new OffsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); + env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); - env.kafkaClient().prepareResponse(new OffsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE, Collections.emptyMap())); + env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE, Collections.emptyMap())); env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); TopicPartition myTopicPartition0 = new TopicPartition("my_topic", 0); @@ -3249,7 +3269,7 @@ public void testListConsumerGroupOffsets() throws Exception { Optional.empty(), "", Errors.NONE)); responseData.put(myTopicPartition3, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, Optional.empty(), "", Errors.NONE)); - env.kafkaClient().prepareResponse(new OffsetFetchResponse(Errors.NONE, responseData)); + env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.NONE, responseData)); final ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets(GROUP_ID); final Map partitionToOffsetAndMetadata = result.partitionsToOffsetAndMetadata().get(); @@ -3263,6 +3283,144 @@ public void testListConsumerGroupOffsets() throws Exception { } } + @Test + public void testBatchedListConsumerGroupOffsets() throws Exception { + Cluster cluster = mockCluster(1, 0); + Time time = new MockTime(); + Map groupSpecs = batchedListConsumerGroupOffsetsSpec(); + + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(time, cluster, AdminClientConfig.RETRIES_CONFIG, "0")) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + env.kafkaClient().prepareResponse(prepareBatchedFindCoordinatorResponse(Errors.NONE, env.cluster().controller(), groupSpecs.keySet())); + + ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets(groupSpecs, new ListConsumerGroupOffsetsOptions()); + sendOffsetFetchResponse(env.kafkaClient(), groupSpecs, true, Errors.NONE); + + verifyListOffsetsForMultipleGroups(groupSpecs, result); + } + } + + @Test + public void testBatchedListConsumerGroupOffsetsWithNoFindCoordinatorBatching() throws Exception { + Cluster cluster = mockCluster(1, 0); + Time time = new MockTime(); + Map groupSpecs = batchedListConsumerGroupOffsetsSpec(); + + ApiVersion findCoordinatorV3 = new ApiVersion() + .setApiKey(ApiKeys.FIND_COORDINATOR.id) + .setMinVersion((short) 0) + .setMaxVersion((short) 3); + ApiVersion offsetFetchV7 = new ApiVersion() + .setApiKey(ApiKeys.OFFSET_FETCH.id) + .setMinVersion((short) 0) + .setMaxVersion((short) 7); + + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(time, cluster, AdminClientConfig.RETRY_BACKOFF_MS_CONFIG, "0")) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create(Arrays.asList(findCoordinatorV3, offsetFetchV7))); + env.kafkaClient().prepareResponse(prepareOldFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode())); + env.kafkaClient().prepareResponse(prepareOldFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); + env.kafkaClient().prepareResponse(prepareOldFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); + + ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets(groupSpecs); + + // Fail the first request in order to ensure that the group is not batched when retried. + sendOffsetFetchResponse(env.kafkaClient(), groupSpecs, false, Errors.COORDINATOR_LOAD_IN_PROGRESS); + + sendOffsetFetchResponse(env.kafkaClient(), groupSpecs, false, Errors.NONE); + sendOffsetFetchResponse(env.kafkaClient(), groupSpecs, false, Errors.NONE); + + verifyListOffsetsForMultipleGroups(groupSpecs, result); + } + } + + @Test + public void testBatchedListConsumerGroupOffsetsWithNoOffsetFetchBatching() throws Exception { + Cluster cluster = mockCluster(1, 0); + Time time = new MockTime(); + Map groupSpecs = batchedListConsumerGroupOffsetsSpec(); + + ApiVersion offsetFetchV7 = new ApiVersion() + .setApiKey(ApiKeys.OFFSET_FETCH.id) + .setMinVersion((short) 0) + .setMaxVersion((short) 7); + + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(time, cluster, AdminClientConfig.RETRY_BACKOFF_MS_CONFIG, "0")) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create(Collections.singleton(offsetFetchV7))); + env.kafkaClient().prepareResponse(prepareBatchedFindCoordinatorResponse(Errors.NONE, env.cluster().controller(), groupSpecs.keySet())); + // Prepare a response to force client to attempt batched request creation that throws + // NoBatchedOffsetFetchRequestException. This triggers creation of non-batched requests. + env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE, Collections.emptyMap())); + + ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets(groupSpecs); + + // The request handler attempts both FindCoordinator and OffsetFetch requests. This seems + // ok since since we expect this scenario only during upgrades from versions < 3.0.0 where + // some upgraded brokers could handle batched FindCoordinator while non-upgraded coordinators + // rejected batched OffsetFetch requests. + sendFindCoordinatorResponse(env.kafkaClient(), env.cluster().controller()); + sendFindCoordinatorResponse(env.kafkaClient(), env.cluster().controller()); + sendOffsetFetchResponse(env.kafkaClient(), groupSpecs, false, Errors.NONE); + sendOffsetFetchResponse(env.kafkaClient(), groupSpecs, false, Errors.NONE); + + verifyListOffsetsForMultipleGroups(groupSpecs, result); + } + } + + private Map batchedListConsumerGroupOffsetsSpec() { + Set groupAPartitions = Collections.singleton(new TopicPartition("A", 1)); + Set groupBPartitions = Collections.singleton(new TopicPartition("B", 2)); + + ListConsumerGroupOffsetsSpec groupASpec = new ListConsumerGroupOffsetsSpec().topicPartitions(groupAPartitions); + ListConsumerGroupOffsetsSpec groupBSpec = new ListConsumerGroupOffsetsSpec().topicPartitions(groupBPartitions); + return Utils.mkMap(Utils.mkEntry("groupA", groupASpec), Utils.mkEntry("groupB", groupBSpec)); + } + + private void waitForRequest(MockClient mockClient, ApiKeys apiKeys) throws Exception { + TestUtils.waitForCondition(() -> { + ClientRequest clientRequest = mockClient.requests().peek(); + return clientRequest != null && clientRequest.apiKey() == apiKeys; + }, "Failed awaiting " + apiKeys + " request"); + } + + private void sendFindCoordinatorResponse(MockClient mockClient, Node coordinator) throws Exception { + waitForRequest(mockClient, ApiKeys.FIND_COORDINATOR); + + ClientRequest clientRequest = mockClient.requests().peek(); + FindCoordinatorRequestData data = ((FindCoordinatorRequest.Builder) clientRequest.requestBuilder()).data(); + mockClient.respond(prepareFindCoordinatorResponse(Errors.NONE, data.key(), coordinator)); + } + + private void sendOffsetFetchResponse(MockClient mockClient, Map groupSpecs, boolean batched, Errors error) throws Exception { + waitForRequest(mockClient, ApiKeys.OFFSET_FETCH); + + ClientRequest clientRequest = mockClient.requests().peek(); + OffsetFetchRequestData data = ((OffsetFetchRequest.Builder) clientRequest.requestBuilder()).data; + Map> results = new HashMap<>(); + Map errors = new HashMap<>(); + data.groups().forEach(group -> { + Map partitionResults = new HashMap<>(); + for (TopicPartition tp : groupSpecs.get(group.groupId()).topicPartitions()) { + partitionResults.put(tp, new PartitionData(10, Optional.empty(), "", Errors.NONE)); + } + results.put(group.groupId(), partitionResults); + errors.put(group.groupId(), error); + }); + if (!batched) { + assertEquals(1, data.groups().size()); + mockClient.respond(new OffsetFetchResponse(THROTTLE, error, results.values().iterator().next())); + } else + mockClient.respond(new OffsetFetchResponse(THROTTLE, errors, results)); + } + + private void verifyListOffsetsForMultipleGroups(Map groupSpecs, + ListConsumerGroupOffsetsResult result) throws Exception { + assertEquals(groupSpecs.size(), result.all().get(10, TimeUnit.SECONDS).size()); + for (Map.Entry entry : groupSpecs.entrySet()) { + assertEquals(entry.getValue().topicPartitions(), + result.partitionsToOffsetAndMetadata(entry.getKey()).get().keySet()); + } + } + @Test public void testDeleteConsumerGroupsNumRetries() throws Exception { final Cluster cluster = mockCluster(3, 0); @@ -6544,6 +6702,12 @@ private DescribeLogDirsResponse prepareDescribeLogDirsResponse(Errors error, Str .setLogDir(logDir)))); } + private OffsetFetchResponse offsetFetchResponse(Errors error, Map responseData) { + return new OffsetFetchResponse(THROTTLE, + Collections.singletonMap(GROUP_ID, error), + Collections.singletonMap(GROUP_ID, responseData)); + } + private static MemberDescription convertToMemberDescriptions(DescribedGroupMember member, MemberAssignment assignment) { return new MemberDescription(member.memberId(), diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java index ef858c5003d42..8c31c7cf691b5 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.admin; import org.apache.kafka.clients.admin.DescribeReplicaLogDirsResult.ReplicaLogDirInfo; +import org.apache.kafka.clients.admin.internals.CoordinatorKey; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.ElectionType; import org.apache.kafka.common.KafkaException; @@ -583,24 +584,29 @@ synchronized public ListConsumerGroupsResult listConsumerGroups(ListConsumerGrou } @Override - synchronized public ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId, ListConsumerGroupOffsetsOptions options) { - // ignoring the groupId and assume one test would only work on one group only + synchronized public ListConsumerGroupOffsetsResult listConsumerGroupOffsets(Map groupSpecs, ListConsumerGroupOffsetsOptions options) { + // ignoring the groups and assume one test would only work on one group only + if (groupSpecs.size() != 1) + throw new UnsupportedOperationException("Not implemented yet"); + + String group = groupSpecs.keySet().iterator().next(); + Collection topicPartitions = groupSpecs.get(group).topicPartitions(); final KafkaFutureImpl> future = new KafkaFutureImpl<>(); if (listConsumerGroupOffsetsException != null) { future.completeExceptionally(listConsumerGroupOffsetsException); } else { - if (options.topicPartitions().isEmpty()) { + if (topicPartitions.isEmpty()) { future.complete(committedOffsets.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getKey, entry -> new OffsetAndMetadata(entry.getValue())))); } else { future.complete(committedOffsets.entrySet().stream() - .filter(entry -> options.topicPartitions().contains(entry.getKey())) + .filter(entry -> topicPartitions.contains(entry.getKey())) .collect(Collectors.toMap(Map.Entry::getKey, entry -> new OffsetAndMetadata(entry.getValue())))); } } - return new ListConsumerGroupOffsetsResult(future); + return new ListConsumerGroupOffsetsResult(Collections.singletonMap(CoordinatorKey.byGroupId(group), future)); } @Override diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandlerTest.java index 27597ce035b00..95fabb3fc2a2f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandlerTest.java @@ -24,52 +24,140 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.Arrays; +import java.util.Collection; 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.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsSpec; +import org.apache.kafka.clients.admin.internals.AdminApiHandler.RequestAndKeys; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.GroupIdNotFoundException; import org.apache.kafka.common.errors.InvalidGroupIdException; +import org.apache.kafka.common.message.OffsetFetchRequestData.OffsetFetchRequestGroup; +import org.apache.kafka.common.message.OffsetFetchRequestData.OffsetFetchRequestTopics; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.OffsetFetchRequest; import org.apache.kafka.common.requests.OffsetFetchResponse; import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData; import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Utils; import org.junit.jupiter.api.Test; public class ListConsumerGroupOffsetsHandlerTest { private final LogContext logContext = new LogContext(); - private final String groupId = "group-id"; + private final int throttleMs = 10; + private final String groupZero = "group0"; + private final String groupOne = "group1"; + private final String groupTwo = "group2"; + private final List groups = Arrays.asList(groupZero, groupOne, groupTwo); private final TopicPartition t0p0 = new TopicPartition("t0", 0); private final TopicPartition t0p1 = new TopicPartition("t0", 1); private final TopicPartition t1p0 = new TopicPartition("t1", 0); private final TopicPartition t1p1 = new TopicPartition("t1", 1); - private final List tps = Arrays.asList(t0p0, t0p1, t1p0, t1p1); + private final TopicPartition t2p0 = new TopicPartition("t2", 0); + private final TopicPartition t2p1 = new TopicPartition("t2", 1); + private final TopicPartition t2p2 = new TopicPartition("t2", 2); + private final Map singleRequestMap = Collections.singletonMap(groupZero, + new ListConsumerGroupOffsetsSpec().topicPartitions(Arrays.asList(t0p0, t0p1, t1p0, t1p1))); + private final Map batchedRequestMap = + new HashMap() {{ + put(groupZero, new ListConsumerGroupOffsetsSpec().topicPartitions(singletonList(t0p0))); + put(groupOne, new ListConsumerGroupOffsetsSpec().topicPartitions(Arrays.asList(t0p0, t1p0, t1p1))); + put(groupTwo, new ListConsumerGroupOffsetsSpec().topicPartitions(Arrays.asList(t0p0, t1p0, t1p1, t2p0, t2p1, t2p2))); + }}; @Test public void testBuildRequest() { - ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(groupId, tps, logContext); - OffsetFetchRequest request = handler.buildBatchedRequest(1, singleton(CoordinatorKey.byGroupId(groupId))).build(); - assertEquals(groupId, request.data().groups().get(0).groupId()); + ListConsumerGroupOffsetsHandler handler = + new ListConsumerGroupOffsetsHandler(singleRequestMap, false, logContext); + OffsetFetchRequest request = handler.buildBatchedRequest(coordinatorKeys(groupZero)).build(); + assertEquals(groupZero, request.data().groups().get(0).groupId()); assertEquals(2, request.data().groups().get(0).topics().size()); assertEquals(2, request.data().groups().get(0).topics().get(0).partitionIndexes().size()); assertEquals(2, request.data().groups().get(0).topics().get(1).partitionIndexes().size()); } + @Test + public void testBuildRequestWithMultipleGroups() { + Map requestMap = new HashMap<>(this.batchedRequestMap); + String groupThree = "group3"; + requestMap.put(groupThree, new ListConsumerGroupOffsetsSpec() + .topicPartitions(Arrays.asList(new TopicPartition("t3", 0), new TopicPartition("t3", 1)))); + + ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(requestMap, false, logContext); + OffsetFetchRequest request1 = handler.buildBatchedRequest(coordinatorKeys(groupZero, groupOne, groupTwo)).build(); + assertEquals(Utils.mkSet(groupZero, groupOne, groupTwo), requestGroups(request1)); + + OffsetFetchRequest request2 = handler.buildBatchedRequest(coordinatorKeys(groupThree)).build(); + assertEquals(Utils.mkSet(groupThree), requestGroups(request2)); + + Map builtRequests = new HashMap<>(); + request1.groupIdsToPartitions().forEach((group, partitions) -> + builtRequests.put(group, new ListConsumerGroupOffsetsSpec().topicPartitions(partitions))); + request2.groupIdsToPartitions().forEach((group, partitions) -> + builtRequests.put(group, new ListConsumerGroupOffsetsSpec().topicPartitions(partitions))); + + assertEquals(requestMap, builtRequests); + Map> groupIdsToTopics = request1.groupIdsToTopics(); + + assertEquals(3, groupIdsToTopics.size()); + assertEquals(1, groupIdsToTopics.get(groupZero).size()); + assertEquals(2, groupIdsToTopics.get(groupOne).size()); + assertEquals(3, groupIdsToTopics.get(groupTwo).size()); + + assertEquals(1, groupIdsToTopics.get(groupZero).get(0).partitionIndexes().size()); + assertEquals(1, groupIdsToTopics.get(groupOne).get(0).partitionIndexes().size()); + assertEquals(2, groupIdsToTopics.get(groupOne).get(1).partitionIndexes().size()); + assertEquals(1, groupIdsToTopics.get(groupTwo).get(0).partitionIndexes().size()); + assertEquals(2, groupIdsToTopics.get(groupTwo).get(1).partitionIndexes().size()); + assertEquals(3, groupIdsToTopics.get(groupTwo).get(2).partitionIndexes().size()); + + groupIdsToTopics = request2.groupIdsToTopics(); + assertEquals(1, groupIdsToTopics.size()); + assertEquals(1, groupIdsToTopics.get(groupThree).size()); + assertEquals(2, groupIdsToTopics.get(groupThree).get(0).partitionIndexes().size()); + } + + @Test + public void testBuildRequestBatchGroups() { + ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(batchedRequestMap, false, logContext); + Collection> requests = handler.buildRequest(1, coordinatorKeys(groupZero, groupOne, groupTwo)); + assertEquals(1, requests.size()); + assertEquals(Utils.mkSet(groupZero, groupOne, groupTwo), requestGroups((OffsetFetchRequest) requests.iterator().next().request.build())); + } + + @Test + public void testBuildRequestDoesNotBatchGroup() { + ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(batchedRequestMap, false, logContext); + // Disable batching. + ((CoordinatorStrategy) handler.lookupStrategy()).disableBatch(); + Collection> requests = handler.buildRequest(1, coordinatorKeys(groupZero, groupOne, groupTwo)); + assertEquals(3, requests.size()); + assertEquals( + Utils.mkSet(Utils.mkSet(groupZero), Utils.mkSet(groupOne), Utils.mkSet(groupTwo)), + requests.stream().map(requestAndKey -> requestGroups((OffsetFetchRequest) requestAndKey.request.build())).collect(Collectors.toSet()) + ); + } + @Test public void testSuccessfulHandleResponse() { Map expected = new HashMap<>(); assertCompleted(handleWithError(Errors.NONE), expected); } - @Test public void testSuccessfulHandleResponseWithOnePartitionError() { Map expectedResult = Collections.singletonMap(t0p0, new OffsetAndMetadata(10L)); @@ -80,17 +168,62 @@ public void testSuccessfulHandleResponseWithOnePartitionError() { assertCompleted(handleWithPartitionError(Errors.UNSTABLE_OFFSET_COMMIT), expectedResult); } + @Test + public void testSuccessfulHandleResponseWithOnePartitionErrorWithMultipleGroups() { + Map offsetAndMetadataMapZero = + Collections.singletonMap(t0p0, new OffsetAndMetadata(10L)); + Map offsetAndMetadataMapOne = + Collections.singletonMap(t1p1, new OffsetAndMetadata(10L)); + Map offsetAndMetadataMapTwo = + Collections.singletonMap(t2p2, new OffsetAndMetadata(10L)); + Map> expectedResult = + new HashMap>() {{ + put(groupZero, offsetAndMetadataMapZero); + put(groupOne, offsetAndMetadataMapOne); + put(groupTwo, offsetAndMetadataMapTwo); + }}; + + assertCompletedForMultipleGroups( + handleWithPartitionErrorMultipleGroups(Errors.UNKNOWN_TOPIC_OR_PARTITION), expectedResult); + assertCompletedForMultipleGroups( + handleWithPartitionErrorMultipleGroups(Errors.TOPIC_AUTHORIZATION_FAILED), expectedResult); + assertCompletedForMultipleGroups( + handleWithPartitionErrorMultipleGroups(Errors.UNSTABLE_OFFSET_COMMIT), expectedResult); + } + + @Test + public void testSuccessfulHandleResponseWithMultipleGroups() { + Map> expected = new HashMap<>(); + Map errorMap = errorMap(groups, Errors.NONE); + assertCompletedForMultipleGroups(handleWithErrorWithMultipleGroups(errorMap, batchedRequestMap), expected); + } + @Test public void testUnmappedHandleResponse() { assertUnmapped(handleWithError(Errors.COORDINATOR_NOT_AVAILABLE)); assertUnmapped(handleWithError(Errors.NOT_COORDINATOR)); } + @Test + public void testUnmappedHandleResponseWithMultipleGroups() { + Map errorMap = new HashMap<>(); + errorMap.put(groupZero, Errors.NOT_COORDINATOR); + errorMap.put(groupOne, Errors.COORDINATOR_NOT_AVAILABLE); + errorMap.put(groupTwo, Errors.NOT_COORDINATOR); + assertUnmappedWithMultipleGroups(handleWithErrorWithMultipleGroups(errorMap, batchedRequestMap)); + } + @Test public void testRetriableHandleResponse() { assertRetriable(handleWithError(Errors.COORDINATOR_LOAD_IN_PROGRESS)); } + @Test + public void testRetriableHandleResponseWithMultipleGroups() { + Map errorMap = errorMap(groups, Errors.COORDINATOR_LOAD_IN_PROGRESS); + assertRetriable(handleWithErrorWithMultipleGroups(errorMap, batchedRequestMap)); + } + @Test public void testFailedHandleResponse() { assertFailed(GroupAuthorizationException.class, handleWithError(Errors.GROUP_AUTHORIZATION_FAILED)); @@ -98,10 +231,50 @@ public void testFailedHandleResponse() { assertFailed(InvalidGroupIdException.class, handleWithError(Errors.INVALID_GROUP_ID)); } + @Test + public void testFailedHandleResponseWithMultipleGroups() { + Map errorMap = new HashMap<>(); + errorMap.put(groupZero, Errors.GROUP_AUTHORIZATION_FAILED); + errorMap.put(groupOne, Errors.GROUP_ID_NOT_FOUND); + errorMap.put(groupTwo, Errors.INVALID_GROUP_ID); + Map> groupToExceptionMap = new HashMap<>(); + groupToExceptionMap.put(groupZero, GroupAuthorizationException.class); + groupToExceptionMap.put(groupOne, GroupIdNotFoundException.class); + groupToExceptionMap.put(groupTwo, InvalidGroupIdException.class); + assertFailedForMultipleGroups(groupToExceptionMap, + handleWithErrorWithMultipleGroups(errorMap, batchedRequestMap)); + } + private OffsetFetchResponse buildResponse(Errors error) { - Map responseData = new HashMap<>(); - OffsetFetchResponse response = new OffsetFetchResponse(error, responseData); - return response; + return new OffsetFetchResponse( + throttleMs, + Collections.singletonMap(groupZero, error), + Collections.singletonMap(groupZero, new HashMap<>())); + } + + private OffsetFetchResponse buildResponseWithMultipleGroups( + Map errorMap, + Map> responseData + ) { + return new OffsetFetchResponse(throttleMs, errorMap, responseData); + } + + private AdminApiHandler.ApiResult> handleWithErrorWithMultipleGroups( + Map errorMap, + Map groupSpecs + ) { + ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(groupSpecs, false, logContext); + Map> responseData = new HashMap<>(); + for (String group : errorMap.keySet()) { + responseData.put(group, new HashMap<>()); + } + OffsetFetchResponse response = buildResponseWithMultipleGroups(errorMap, responseData); + return handler.handleResponse(new Node(1, "host", 1234), + errorMap.keySet() + .stream() + .map(CoordinatorKey::byGroupId) + .collect(Collectors.toSet()), + response); } private OffsetFetchResponse buildResponseWithPartitionError(Errors error) { @@ -110,24 +283,68 @@ private OffsetFetchResponse buildResponseWithPartitionError(Errors error) { responseData.put(t0p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", Errors.NONE)); responseData.put(t0p1, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); - OffsetFetchResponse response = new OffsetFetchResponse(Errors.NONE, responseData); - return response; + return new OffsetFetchResponse(Errors.NONE, responseData); + } + + private OffsetFetchResponse buildResponseWithPartitionErrorWithMultipleGroups(Errors error) { + Map responseDataZero = new HashMap<>(); + responseDataZero.put(t0p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", Errors.NONE)); + + Map responseDataOne = new HashMap<>(); + responseDataOne.put(t0p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); + responseDataOne.put(t1p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); + responseDataOne.put(t1p1, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", Errors.NONE)); + + Map responseDataTwo = new HashMap<>(); + responseDataTwo.put(t0p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); + responseDataTwo.put(t1p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); + responseDataTwo.put(t1p1, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); + responseDataTwo.put(t2p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); + responseDataTwo.put(t2p1, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); + responseDataTwo.put(t2p2, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", Errors.NONE)); + + Map> responseData = + new HashMap>() {{ + put(groupZero, responseDataZero); + put(groupOne, responseDataOne); + put(groupTwo, responseDataTwo); + }}; + + Map errorMap = errorMap(groups, Errors.NONE); + return new OffsetFetchResponse(0, errorMap, responseData); } private AdminApiHandler.ApiResult> handleWithPartitionError( Errors error ) { - ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(groupId, tps, logContext); + ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(singleRequestMap, + false, logContext); OffsetFetchResponse response = buildResponseWithPartitionError(error); - return handler.handleResponse(new Node(1, "host", 1234), singleton(CoordinatorKey.byGroupId(groupId)), response); + return handler.handleResponse(new Node(1, "host", 1234), + singleton(CoordinatorKey.byGroupId(groupZero)), response); + } + + private AdminApiHandler.ApiResult> handleWithPartitionErrorMultipleGroups( + Errors error + ) { + ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler( + batchedRequestMap, false, logContext); + OffsetFetchResponse response = buildResponseWithPartitionErrorWithMultipleGroups(error); + return handler.handleResponse( + new Node(1, "host", 1234), + coordinatorKeys(groupZero, groupOne, groupTwo), + response); } private AdminApiHandler.ApiResult> handleWithError( Errors error ) { - ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(groupId, tps, logContext); + ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler( + singleRequestMap, false, logContext); OffsetFetchResponse response = buildResponse(error); - return handler.handleResponse(new Node(1, "host", 1234), singleton(CoordinatorKey.byGroupId(groupId)), response); + return handler.handleResponse(new Node(1, "host", 1234), + singleton(CoordinatorKey.byGroupId(groupZero)), + response); } private void assertUnmapped( @@ -135,11 +352,19 @@ private void assertUnmapped( ) { assertEquals(emptySet(), result.completedKeys.keySet()); assertEquals(emptySet(), result.failedKeys.keySet()); - assertEquals(singletonList(CoordinatorKey.byGroupId(groupId)), result.unmappedKeys); + assertEquals(singletonList(CoordinatorKey.byGroupId(groupZero)), result.unmappedKeys); + } + + private void assertUnmappedWithMultipleGroups( + AdminApiHandler.ApiResult> result + ) { + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(coordinatorKeys(groupZero, groupOne, groupTwo), new HashSet<>(result.unmappedKeys)); } private void assertRetriable( - AdminApiHandler.ApiResult> result + AdminApiHandler.ApiResult> result ) { assertEquals(emptySet(), result.completedKeys.keySet()); assertEquals(emptySet(), result.failedKeys.keySet()); @@ -150,21 +375,64 @@ private void assertCompleted( AdminApiHandler.ApiResult> result, Map expected ) { - CoordinatorKey key = CoordinatorKey.byGroupId(groupId); + CoordinatorKey key = CoordinatorKey.byGroupId(groupZero); assertEquals(emptySet(), result.failedKeys.keySet()); assertEquals(emptyList(), result.unmappedKeys); assertEquals(singleton(key), result.completedKeys.keySet()); - assertEquals(expected, result.completedKeys.get(CoordinatorKey.byGroupId(groupId))); + assertEquals(expected, result.completedKeys.get(key)); + } + + private void assertCompletedForMultipleGroups( + AdminApiHandler.ApiResult> result, + Map> expected + ) { + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + for (String g : expected.keySet()) { + CoordinatorKey key = CoordinatorKey.byGroupId(g); + assertTrue(result.completedKeys.containsKey(key)); + assertEquals(expected.get(g), result.completedKeys.get(key)); + } } private void assertFailed( Class expectedExceptionType, AdminApiHandler.ApiResult> result ) { - CoordinatorKey key = CoordinatorKey.byGroupId(groupId); + CoordinatorKey key = CoordinatorKey.byGroupId(groupZero); assertEquals(emptySet(), result.completedKeys.keySet()); assertEquals(emptyList(), result.unmappedKeys); assertEquals(singleton(key), result.failedKeys.keySet()); assertTrue(expectedExceptionType.isInstance(result.failedKeys.get(key))); } + + private void assertFailedForMultipleGroups( + Map> groupToExceptionMap, + AdminApiHandler.ApiResult> result + ) { + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + for (String g : groupToExceptionMap.keySet()) { + CoordinatorKey key = CoordinatorKey.byGroupId(g); + assertTrue(result.failedKeys.containsKey(key)); + assertTrue(groupToExceptionMap.get(g).isInstance(result.failedKeys.get(key))); + } + } + + private Set coordinatorKeys(String... groups) { + return Stream.of(groups) + .map(CoordinatorKey::byGroupId) + .collect(Collectors.toSet()); + } + + private Set requestGroups(OffsetFetchRequest request) { + return request.data().groups() + .stream() + .map(OffsetFetchRequestGroup::groupId) + .collect(Collectors.toSet()); + } + + private Map errorMap(Collection groups, Errors error) { + return groups.stream().collect(Collectors.toMap(Function.identity(), unused -> error)); + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index da3acf4983471..e7f25345c6da0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -172,6 +172,7 @@ public class KafkaConsumerTest { // Set auto commit interval lower than heartbeat so we don't need to deal with // a concurrent heartbeat request private final int autoCommitIntervalMs = 500; + private final int throttleMs = 10; private final String groupId = "mock-group"; private final String memberId = "memberId"; @@ -2434,7 +2435,10 @@ private OffsetFetchResponse offsetResponse(Map offsets, Er partitionData.put(entry.getKey(), new OffsetFetchResponse.PartitionData(entry.getValue(), Optional.empty(), "", error)); } - return new OffsetFetchResponse(Errors.NONE, partitionData); + return new OffsetFetchResponse( + throttleMs, + Collections.singletonMap(groupId, Errors.NONE), + Collections.singletonMap(groupId, partitionData)); } private ListOffsetsResponse listOffsetsResponse(Map offsets) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index c65d33176fcd5..db483c6c0f9c0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -71,6 +71,7 @@ import org.apache.kafka.common.requests.OffsetCommitRequest; import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.OffsetFetchResponse; +import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData; import org.apache.kafka.common.requests.RequestTestUtils; import org.apache.kafka.common.requests.SyncGroupRequest; import org.apache.kafka.common.requests.SyncGroupResponse; @@ -140,6 +141,7 @@ public abstract class ConsumerCoordinatorTest { private final long retryBackoffMs = 100; private final int autoCommitIntervalMs = 2000; private final int requestTimeoutMs = 30000; + private final int throttleMs = 10; private final MockTime time = new MockTime(); private GroupRebalanceConfig rebalanceConfig; @@ -2872,7 +2874,7 @@ public void testFetchCommittedOffsets() { OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(offset, leaderEpoch, metadata, Errors.NONE); - client.prepareResponse(new OffsetFetchResponse(Errors.NONE, singletonMap(t1p, data))); + client.prepareResponse(offsetFetchResponse(Errors.NONE, singletonMap(t1p, data))); Map fetchedOffsets = coordinator.fetchCommittedOffsets(singleton(t1p), time.timer(Long.MAX_VALUE)); @@ -2888,7 +2890,7 @@ public void testTopicAuthorizationFailedInOffsetFetch() { OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(-1, Optional.empty(), "", Errors.TOPIC_AUTHORIZATION_FAILED); - client.prepareResponse(new OffsetFetchResponse(Errors.NONE, singletonMap(t1p, data))); + client.prepareResponse(offsetFetchResponse(Errors.NONE, singletonMap(t1p, data))); TopicAuthorizationException exception = assertThrows(TopicAuthorizationException.class, () -> coordinator.fetchCommittedOffsets(singleton(t1p), time.timer(Long.MAX_VALUE))); @@ -2901,7 +2903,7 @@ public void testRefreshOffsetLoadInProgress() { coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); subscriptions.assignFromUser(singleton(t1p)); - client.prepareResponse(offsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS)); + client.prepareResponse(offsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Collections.emptyMap())); client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L)); coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); @@ -2916,7 +2918,7 @@ public void testRefreshOffsetsGroupNotAuthorized() { coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); subscriptions.assignFromUser(singleton(t1p)); - client.prepareResponse(offsetFetchResponse(Errors.GROUP_AUTHORIZATION_FAILED)); + client.prepareResponse(offsetFetchResponse(Errors.GROUP_AUTHORIZATION_FAILED, Collections.emptyMap())); try { coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); fail("Expected group authorization error"); @@ -2959,7 +2961,7 @@ public void testRefreshOffsetNotCoordinatorForConsumer() { coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); subscriptions.assignFromUser(singleton(t1p)); - client.prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR)); + client.prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L)); coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); @@ -3435,7 +3437,11 @@ private void supportStableFlag(final short upperVersion, final boolean expectThr OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(offset, leaderEpoch, metadata, Errors.NONE); - client.prepareResponse(new OffsetFetchResponse(Errors.NONE, singletonMap(t1p, data))); + if (upperVersion < 8) { + client.prepareResponse(new OffsetFetchResponse(Errors.NONE, singletonMap(t1p, data))); + } else { + client.prepareResponse(offsetFetchResponse(Errors.NONE, singletonMap(t1p, data))); + } if (expectThrows) { assertThrows(UnsupportedVersionException.class, () -> coordinator.fetchCommittedOffsets(singleton(t1p), time.timer(Long.MAX_VALUE))); @@ -3690,8 +3696,10 @@ private OffsetCommitResponse offsetCommitResponse(Map re return new OffsetCommitResponse(responseData); } - private OffsetFetchResponse offsetFetchResponse(Errors topLevelError) { - return new OffsetFetchResponse(topLevelError, Collections.emptyMap()); + private OffsetFetchResponse offsetFetchResponse(Errors error, Map responseData) { + return new OffsetFetchResponse(throttleMs, + singletonMap(groupId, error), + singletonMap(groupId, responseData)); } private OffsetFetchResponse offsetFetchResponse(TopicPartition tp, Errors partitionLevelError, String metadata, long offset) { @@ -3701,7 +3709,7 @@ private OffsetFetchResponse offsetFetchResponse(TopicPartition tp, Errors partit private OffsetFetchResponse offsetFetchResponse(TopicPartition tp, Errors partitionLevelError, String metadata, long offset, Optional epoch) { OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(offset, epoch, metadata, partitionLevelError); - return new OffsetFetchResponse(Errors.NONE, singletonMap(tp, data)); + return offsetFetchResponse(Errors.NONE, singletonMap(tp, data)); } private OffsetCommitCallback callback(final AtomicBoolean success) { diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala index 47c1d173b306b..d5aee881c9a0c 100755 --- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala +++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala @@ -18,7 +18,7 @@ package kafka.admin import java.time.{Duration, Instant} -import java.util.Properties +import java.util.{Collections, Properties} import com.fasterxml.jackson.dataformat.csv.CsvMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule import kafka.utils._ @@ -753,9 +753,9 @@ object ConsumerGroupCommand extends Logging { private def getCommittedOffsets(groupId: String): Map[TopicPartition, OffsetAndMetadata] = { adminClient.listConsumerGroupOffsets( - groupId, - withTimeoutMs(new ListConsumerGroupOffsetsOptions) - ).partitionsToOffsetAndMetadata.get.asScala + Collections.singletonMap(groupId, new ListConsumerGroupOffsetsSpec), + withTimeoutMs(new ListConsumerGroupOffsetsOptions()) + ).partitionsToOffsetAndMetadata(groupId).get().asScala } type GroupMetadata = immutable.Map[String, immutable.Map[TopicPartition, OffsetAndMetadata]] diff --git a/core/src/test/scala/unit/kafka/admin/ConsumerGroupServiceTest.scala b/core/src/test/scala/unit/kafka/admin/ConsumerGroupServiceTest.scala index 76a3855a87274..44b241a7ed3c6 100644 --- a/core/src/test/scala/unit/kafka/admin/ConsumerGroupServiceTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ConsumerGroupServiceTest.scala @@ -49,8 +49,8 @@ class ConsumerGroupServiceTest { when(admin.describeConsumerGroups(ArgumentMatchers.eq(Collections.singletonList(group)), any())) .thenReturn(describeGroupsResult(ConsumerGroupState.STABLE)) - when(admin.listConsumerGroupOffsets(ArgumentMatchers.eq(group), any())) - .thenReturn(listGroupOffsetsResult) + when(admin.listConsumerGroupOffsets(ArgumentMatchers.eq(listConsumerGroupOffsetsSpec), any())) + .thenReturn(listGroupOffsetsResult(group)) when(admin.listOffsets(offsetsArgMatcher, any())) .thenReturn(listOffsetsResult) @@ -60,7 +60,7 @@ class ConsumerGroupServiceTest { assertEquals(topicPartitions.size, assignments.get.size) verify(admin, times(1)).describeConsumerGroups(ArgumentMatchers.eq(Collections.singletonList(group)), any()) - verify(admin, times(1)).listConsumerGroupOffsets(ArgumentMatchers.eq(group), any()) + verify(admin, times(1)).listConsumerGroupOffsets(ArgumentMatchers.eq(listConsumerGroupOffsetsSpec), any()) verify(admin, times(1)).listOffsets(offsetsArgMatcher, any()) } @@ -112,8 +112,10 @@ class ConsumerGroupServiceTest { future.complete(consumerGroupDescription) when(admin.describeConsumerGroups(ArgumentMatchers.eq(Collections.singletonList(group)), any())) .thenReturn(new DescribeConsumerGroupsResult(Collections.singletonMap(group, future))) - when(admin.listConsumerGroupOffsets(ArgumentMatchers.eq(group), any())) - .thenReturn(AdminClientTestUtils.listConsumerGroupOffsetsResult(commitedOffsets)) + when(admin.listConsumerGroupOffsets(ArgumentMatchers.eq(listConsumerGroupOffsetsSpec), any())) + .thenReturn( + AdminClientTestUtils.listConsumerGroupOffsetsResult( + Collections.singletonMap(group, commitedOffsets))) when(admin.listOffsets( ArgumentMatchers.argThat(offsetsArgMatcher(assignedTopicPartitions)), any() @@ -142,7 +144,7 @@ class ConsumerGroupServiceTest { assertEquals(expectedOffsets, returnedOffsets) verify(admin, times(1)).describeConsumerGroups(ArgumentMatchers.eq(Collections.singletonList(group)), any()) - verify(admin, times(1)).listConsumerGroupOffsets(ArgumentMatchers.eq(group), any()) + verify(admin, times(1)).listConsumerGroupOffsets(ArgumentMatchers.eq(listConsumerGroupOffsetsSpec), any()) verify(admin, times(1)).listOffsets(ArgumentMatchers.argThat(offsetsArgMatcher(assignedTopicPartitions)), any()) verify(admin, times(1)).listOffsets(ArgumentMatchers.argThat(offsetsArgMatcher(unassignedTopicPartitions)), any()) } @@ -192,9 +194,9 @@ class ConsumerGroupServiceTest { new DescribeConsumerGroupsResult(Collections.singletonMap(group, future)) } - private def listGroupOffsetsResult: ListConsumerGroupOffsetsResult = { + private def listGroupOffsetsResult(groupId: String): ListConsumerGroupOffsetsResult = { val offsets = topicPartitions.map(_ -> new OffsetAndMetadata(100)).toMap.asJava - AdminClientTestUtils.listConsumerGroupOffsetsResult(offsets) + AdminClientTestUtils.listConsumerGroupOffsetsResult(Map(groupId -> offsets).asJava) } private def offsetsArgMatcher: util.Map[TopicPartition, OffsetSpec] = { @@ -217,4 +219,8 @@ class ConsumerGroupServiceTest { }.toMap AdminClientTestUtils.describeTopicsResult(topicDescriptions.asJava) } + + private def listConsumerGroupOffsetsSpec: util.Map[String, ListConsumerGroupOffsetsSpec] = { + Collections.singletonMap(group, new ListConsumerGroupOffsetsSpec()) + } } diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 6d17e937824d5..82c19949e3bcf 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -320,7 +320,7 @@ class RequestQuotaTest extends BaseRequestTest { ) ) case ApiKeys.OFFSET_FETCH => - new OffsetFetchRequest.Builder("test-group", false, List(tp).asJava, false) + new OffsetFetchRequest.Builder(Map("test-group"-> List(tp).asJava).asJava, false, false) case ApiKeys.FIND_COORDINATOR => new FindCoordinatorRequest.Builder( diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java index 02cfb0b49c9f1..5240534ce7929 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsOptions; +import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsSpec; import org.apache.kafka.clients.admin.ListOffsetsOptions; import org.apache.kafka.clients.admin.OffsetSpec; import org.apache.kafka.clients.consumer.Consumer; @@ -695,11 +696,12 @@ private Map committedOffsetForChangelogs(final Map(partitions)); - options.requireStable(true); - final Map committedOffsets = adminClient.listConsumerGroupOffsets(groupId, options) - .partitionsToOffsetAndMetadata().get().entrySet() + final ListConsumerGroupOffsetsOptions options = new ListConsumerGroupOffsetsOptions() + .requireStable(true); + final ListConsumerGroupOffsetsSpec spec = new ListConsumerGroupOffsetsSpec() + .topicPartitions(new ArrayList<>(partitions)); + final Map committedOffsets = adminClient.listConsumerGroupOffsets(Collections.singletonMap(groupId, spec)) + .partitionsToOffsetAndMetadata(groupId).get().entrySet() .stream() .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue() == null ? 0L : e.getValue().offset())); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java index 1961736620dbe..fbc8d4232613a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.clients.admin.AdminClientTestUtils; import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsOptions; import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsResult; +import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsSpec; import org.apache.kafka.clients.admin.ListOffsetsOptions; import org.apache.kafka.clients.admin.ListOffsetsResult; import org.apache.kafka.clients.admin.MockAdminClient; @@ -648,12 +649,12 @@ public void shouldRequestCommittedOffsetsAndHandleTimeoutException() { final AtomicBoolean functionCalled = new AtomicBoolean(false); final MockAdminClient adminClient = new MockAdminClient() { @Override - public synchronized ListConsumerGroupOffsetsResult listConsumerGroupOffsets(final String groupId, final ListConsumerGroupOffsetsOptions options) { + public synchronized ListConsumerGroupOffsetsResult listConsumerGroupOffsets(final Map groupSpecs, final ListConsumerGroupOffsetsOptions options) { if (functionCalled.get()) { - return super.listConsumerGroupOffsets(groupId, options); + return super.listConsumerGroupOffsets(groupSpecs, options); } else { functionCalled.set(true); - return AdminClientTestUtils.listConsumerGroupOffsetsResult(new TimeoutException("KABOOM!")); + return AdminClientTestUtils.listConsumerGroupOffsetsResult(groupSpecs.keySet().iterator().next(), new TimeoutException("KABOOM!")); } } }; @@ -708,7 +709,7 @@ public void shouldThrowIfCommittedOffsetsFail() { final MockAdminClient adminClient = new MockAdminClient() { @Override - public synchronized ListConsumerGroupOffsetsResult listConsumerGroupOffsets(final String groupId, final ListConsumerGroupOffsetsOptions options) { + public synchronized ListConsumerGroupOffsetsResult listConsumerGroupOffsets(final Map groupSpecs, final ListConsumerGroupOffsetsOptions options) { throw kaboom; } }; @@ -790,7 +791,7 @@ public void shouldNotUpdateLimitForNonSourceStandbyChangelog() { final MockAdminClient adminClient = new MockAdminClient() { @Override - public synchronized ListConsumerGroupOffsetsResult listConsumerGroupOffsets(final String groupId, final ListConsumerGroupOffsetsOptions options) { + public synchronized ListConsumerGroupOffsetsResult listConsumerGroupOffsets(final Map groupSpecs, final ListConsumerGroupOffsetsOptions options) { throw new AssertionError("Should not try to fetch committed offsets"); } }; From 610780668efa7c1e8d1be193985eb6e4d971fa0a Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Fri, 15 Jul 2022 09:21:35 +0100 Subject: [PATCH 007/175] MINOR: Fix options for old-style Admin.listConsumerGroupOffsets (#12406) Reviewers: David Jacot --- .../org/apache/kafka/clients/admin/Admin.java | 7 +-- .../clients/admin/KafkaAdminClientTest.java | 52 +++++++++++++------ 2 files changed, 40 insertions(+), 19 deletions(-) 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 0698d297029af..1d469a66436a6 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 @@ -921,12 +921,13 @@ default ListConsumerGroupsResult listConsumerGroups() { * @return The ListGroupOffsetsResult */ default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId, ListConsumerGroupOffsetsOptions options) { - ListConsumerGroupOffsetsOptions listOptions = new ListConsumerGroupOffsetsOptions() - .requireStable(options.requireStable()); @SuppressWarnings("deprecation") ListConsumerGroupOffsetsSpec groupSpec = new ListConsumerGroupOffsetsSpec() .topicPartitions(options.topicPartitions()); - return listConsumerGroupOffsets(Collections.singletonMap(groupId, groupSpec), listOptions); + + // We can use the provided options with the batched API, which uses topic partitions from + // the group spec and ignores any topic partitions set in the options. + return listConsumerGroupOffsets(Collections.singletonMap(groupId, groupSpec), options); } /** 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 3d285a45f7b8f..de57813679b99 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 @@ -131,6 +131,8 @@ import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopic; import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection; import org.apache.kafka.common.message.OffsetFetchRequestData; +import org.apache.kafka.common.message.OffsetFetchRequestData.OffsetFetchRequestGroup; +import org.apache.kafka.common.message.OffsetFetchRequestData.OffsetFetchRequestTopics; import org.apache.kafka.common.message.UnregisterBrokerResponseData; import org.apache.kafka.common.message.WriteTxnMarkersResponseData; import org.apache.kafka.common.protocol.ApiKeys; @@ -3075,7 +3077,17 @@ public void testDescribeNonConsumerGroups() throws Exception { } @Test - public void testListConsumerGroupOffsetsOptions() throws Exception { + public void testListConsumerGroupOffsetsOptionsWithUnbatchedApi() throws Exception { + verifyListConsumerGroupOffsetsOptions(false); + } + + @Test + public void testListConsumerGroupOffsetsOptionsWithBatchedApi() throws Exception { + verifyListConsumerGroupOffsetsOptions(true); + } + + @SuppressWarnings("deprecation") + private void verifyListConsumerGroupOffsetsOptions(boolean batchedApi) throws Exception { final Cluster cluster = mockCluster(3, 0); final Time time = new MockTime(); @@ -3085,24 +3097,32 @@ public void testListConsumerGroupOffsetsOptions() throws Exception { env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); - final TopicPartition tp1 = new TopicPartition("A", 0); + final List partitions = Collections.singletonList(new TopicPartition("A", 0)); final ListConsumerGroupOffsetsOptions options = new ListConsumerGroupOffsetsOptions() - .requireStable(true); - final ListConsumerGroupOffsetsSpec groupSpec = new ListConsumerGroupOffsetsSpec() - .topicPartitions(Collections.singletonList(tp1)); - env.adminClient().listConsumerGroupOffsets(Collections.singletonMap(GROUP_ID, groupSpec), options); + .requireStable(true) + .timeoutMs(300); + if (batchedApi) { + final ListConsumerGroupOffsetsSpec groupSpec = new ListConsumerGroupOffsetsSpec() + .topicPartitions(partitions); + env.adminClient().listConsumerGroupOffsets(Collections.singletonMap(GROUP_ID, groupSpec), options); + } else { + env.adminClient().listConsumerGroupOffsets(GROUP_ID, options.topicPartitions(partitions)); + } final MockClient mockClient = env.kafkaClient(); - TestUtils.waitForCondition(() -> { - final ClientRequest clientRequest = mockClient.requests().peek(); - if (clientRequest != null) { - OffsetFetchRequestData data = ((OffsetFetchRequest.Builder) clientRequest.requestBuilder()).data; - return data.requireStable() && - data.groups().get(0).topics().get(0).name().equals("A") && - data.groups().get(0).topics().get(0).partitionIndexes().equals(Collections.singletonList(0)); - } - return false; - }, "Failed awaiting ListConsumerGroupOfsets request"); + waitForRequest(mockClient, ApiKeys.OFFSET_FETCH); + + ClientRequest clientRequest = mockClient.requests().peek(); + assertNotNull(clientRequest); + assertEquals(300, clientRequest.requestTimeoutMs()); + OffsetFetchRequestData data = ((OffsetFetchRequest.Builder) clientRequest.requestBuilder()).data; + assertTrue(data.requireStable()); + assertEquals(Collections.singletonList(GROUP_ID), + data.groups().stream().map(OffsetFetchRequestGroup::groupId).collect(Collectors.toList())); + assertEquals(Collections.singletonList("A"), + data.groups().get(0).topics().stream().map(OffsetFetchRequestTopics::name).collect(Collectors.toList())); + assertEquals(Collections.singletonList(0), + data.groups().get(0).topics().get(0).partitionIndexes()); } } From a0a5290ea648e3e123b567a674d7b280616aa234 Mon Sep 17 00:00:00 2001 From: Okada Haruki Date: Mon, 18 Jul 2022 15:19:01 +0900 Subject: [PATCH 008/175] KAFKA-13572 Fix negative preferred replica imbalanced count metric (#12405) Currently, preferredReplicaImbalanceCount calculation has a race that becomes negative when topic deletion is initiated simultaneously. This PR addresses the problem by fixing cleanPreferredReplicaImbalanceMetric to be called only once per topic-deletion procedure Reviewers: Luke Chen --- .../kafka/controller/ControllerContext.scala | 13 +++++++++--- .../controller/ControllerContextTest.scala | 21 +++++++++++++++++++ 2 files changed, 31 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/controller/ControllerContext.scala b/core/src/main/scala/kafka/controller/ControllerContext.scala index 379196aa1d42c..7065d87c4c606 100644 --- a/core/src/main/scala/kafka/controller/ControllerContext.scala +++ b/core/src/main/scala/kafka/controller/ControllerContext.scala @@ -327,9 +327,16 @@ class ControllerContext { } } - def queueTopicDeletion(topics: Set[String]): Unit = { - topicsToBeDeleted ++= topics - topics.foreach(cleanPreferredReplicaImbalanceMetric) + def queueTopicDeletion(topicToBeAddedIntoDeletionList: Set[String]): Unit = { + // queueTopicDeletion could be called multiple times for same topic. + // e.g. 1) delete topic-A => 2) delete topic-B before A's deletion completes. + // In this case, at 2), queueTopicDeletion will be called with Set(topic-A, topic-B). + // However we should call cleanPreferredReplicaImbalanceMetric only once for same topic + // because otherwise, preferredReplicaImbalanceCount could be decremented wrongly at 2nd call. + // So we need to take a diff with already queued topics here. + val newlyDeletedTopics = topicToBeAddedIntoDeletionList.diff(topicsToBeDeleted) + topicsToBeDeleted ++= newlyDeletedTopics + newlyDeletedTopics.foreach(cleanPreferredReplicaImbalanceMetric) } def beginTopicDeletion(topics: Set[String]): Unit = { diff --git a/core/src/test/scala/unit/kafka/controller/ControllerContextTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerContextTest.scala index e8efa5af79356..e88bb321ad9b3 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerContextTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerContextTest.scala @@ -203,4 +203,25 @@ class ControllerContextTest { context.removeTopic(tp3.topic) assertEquals(0, context.preferredReplicaImbalanceCount) } + + @Test + def testPreferredReplicaImbalanceMetricOnConcurrentTopicDeletion(): Unit = { + val topicA = "A" + val topicB = "B" + val tpA = new TopicPartition(topicA, 0) + val tpB = new TopicPartition(topicB, 0) + context.updatePartitionFullReplicaAssignment(tpA, ReplicaAssignment(Seq(1, 2, 3))) + context.updatePartitionFullReplicaAssignment(tpB, ReplicaAssignment(Seq(1, 2, 3))) + assertEquals(0, context.preferredReplicaImbalanceCount) + + context.queueTopicDeletion(Set(topicA)) + // All partitions in topic will be marked as Offline during deletion procedure + context.putPartitionLeadershipInfo(tpA, LeaderIsrAndControllerEpoch(LeaderAndIsr(LeaderAndIsr.NoLeader, List(1, 2, 3)), 0)) + assertEquals(0, context.preferredReplicaImbalanceCount) + + // Initiate topicB's topic deletion before topicA's deletion completes. + // Since topicA's delete-topic ZK node still exists, context.queueTopicDeletion will be called with Set(topicA, topicB) + context.queueTopicDeletion(Set(topicA, topicB)) + assertEquals(0, context.preferredReplicaImbalanceCount) + } } From 9dd25ecd9ce17e608c6aba98e0422b26ed133c12 Mon Sep 17 00:00:00 2001 From: Alex Sorokoumov <918393+Gerrrr@users.noreply.github.com> Date: Mon, 18 Jul 2022 23:10:02 +0200 Subject: [PATCH 009/175] KAFKA-13769 Fix version check in SubscriptionJoinForeignProcessorSupplier (#12420) This commit changes the version check from != to > as the process method works correctly on both version 1 and 2. != incorrectly throws on v1 records. Reviewers: Matthias J. Sax --- .../SubscriptionJoinForeignProcessorSupplier.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java index fea8e73867c35..56d6a13321ffb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java @@ -70,7 +70,7 @@ public void process(final Record, Change value = valueAndTimestamp.value(); - if (value.getVersion() != SubscriptionWrapper.CURRENT_VERSION) { + if (value.getVersion() > SubscriptionWrapper.CURRENT_VERSION) { //Guard against modifications to SubscriptionWrapper. Need to ensure that there is compatibility //with previous versions to enable rolling upgrades. Must develop a strategy for upgrading //from older SubscriptionWrapper versions to newer versions. From 45009ef382145fb6e33c5ebec03600b37a1474c0 Mon Sep 17 00:00:00 2001 From: Walker Carlson <18128741+wcarlson5@users.noreply.github.com> Date: Tue, 19 Jul 2022 11:17:46 -0500 Subject: [PATCH 010/175] Revert "KAFKA-12887 Skip some RuntimeExceptions from exception handler (#11228)" (#12421) This reverts commit 4835c64f Reviewers: Matthias J. Sax --- .../apache/kafka/streams/KafkaStreams.java | 22 +------ .../EmitOnChangeIntegrationTest.java | 2 +- ...caughtExceptionHandlerIntegrationTest.java | 61 ++----------------- .../processor/internals/StreamThreadTest.java | 4 +- 4 files changed, 11 insertions(+), 78 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index a923c8e983ff3..3a61f05de11a1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -155,9 +155,6 @@ public class KafkaStreams implements AutoCloseable { private static final String JMX_PREFIX = "kafka.streams"; - private static final Set> EXCEPTIONS_NOT_TO_BE_HANDLED_BY_USERS = - new HashSet<>(Arrays.asList(IllegalStateException.class, IllegalArgumentException.class)); - // processId is expected to be unique across JVMs and to be used // in userData of the subscription request to allow assignor be aware // of the co-location of stream thread's consumers. It is for internal @@ -515,25 +512,10 @@ private void replaceStreamThread(final Throwable throwable) { } } - private boolean wrappedExceptionIsIn(final Throwable throwable, final Set> exceptionsOfInterest) { - return throwable.getCause() != null && exceptionsOfInterest.contains(throwable.getCause().getClass()); - } - - private StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse getActionForThrowable(final Throwable throwable, - final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) { - final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse action; - if (wrappedExceptionIsIn(throwable, EXCEPTIONS_NOT_TO_BE_HANDLED_BY_USERS)) { - action = SHUTDOWN_CLIENT; - } else { - action = streamsUncaughtExceptionHandler.handle(throwable); - } - return action; - } - private void handleStreamsUncaughtException(final Throwable throwable, final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler, final boolean skipThreadReplacement) { - final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse action = getActionForThrowable(throwable, streamsUncaughtExceptionHandler); + final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse action = streamsUncaughtExceptionHandler.handle(throwable); if (oldHandler) { log.warn("Stream's new uncaught exception handler is set as well as the deprecated old handler." + "The old handler will be ignored as long as a new handler is set."); @@ -549,7 +531,7 @@ private void handleStreamsUncaughtException(final Throwable throwable, break; case SHUTDOWN_CLIENT: log.error("Encountered the following exception during processing " + - "and Kafka Streams opted to " + action + "." + + "and the registered exception handler opted to " + action + "." + " The streams client is going to shut down now. ", throwable); closeToError(); break; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/EmitOnChangeIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/EmitOnChangeIntegrationTest.java index 27958730cfe21..f41c95a6bb763 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/EmitOnChangeIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/EmitOnChangeIntegrationTest.java @@ -111,7 +111,7 @@ public void shouldEmitSameRecordAfterFailover() throws Exception { .toStream() .map((key, value) -> { if (shouldThrow.compareAndSet(true, false)) { - throw new RuntimeException("Kaboom"); + throw new IllegalStateException("Kaboom"); } else { return new KeyValue<>(key, value); } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java index c81ddcfa74a5e..be98e8d9fcfa0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java @@ -93,9 +93,6 @@ public static void closeCluster() { } public static final Duration DEFAULT_DURATION = Duration.ofSeconds(30); - private static final AtomicBoolean THROW_ERROR = new AtomicBoolean(true); - private static final AtomicBoolean THROW_ILLEGAL_STATE_EXCEPTION = new AtomicBoolean(false); - private static final AtomicBoolean THROW_ILLEGAL_ARGUMENT_EXCEPTION = new AtomicBoolean(false); @Rule public final TestName testName = new TestName(); @@ -108,6 +105,7 @@ public static void closeCluster() { private final String outputTopic2 = "output2" + testId; private final StreamsBuilder builder = new StreamsBuilder(); private final List processorValueCollector = new ArrayList<>(); + private static AtomicBoolean throwError = new AtomicBoolean(true); private final Properties properties = basicProps(); @@ -173,47 +171,6 @@ public void shouldShutdownClient() throws InterruptedException { } } - - @Test - public void shouldShutdownClientWhenIllegalStateException() throws InterruptedException { - THROW_ILLEGAL_STATE_EXCEPTION.compareAndSet(false, true); - try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), properties)) { - kafkaStreams.setUncaughtExceptionHandler((t, e) -> fail("should not hit old handler")); - - kafkaStreams.setUncaughtExceptionHandler(exception -> REPLACE_THREAD); // if the user defined uncaught exception handler would be hit we would be replacing the thread - - StreamsTestUtils.startKafkaStreamsAndWaitForRunningState(kafkaStreams); - - produceMessages(0L, inputTopic, "A"); - waitForApplicationState(Collections.singletonList(kafkaStreams), KafkaStreams.State.ERROR, DEFAULT_DURATION); - - assertThat(processorValueCollector.size(), equalTo(1)); - } finally { - THROW_ILLEGAL_STATE_EXCEPTION.compareAndSet(true, false); - } - - } - - @Test - public void shouldShutdownClientWhenIllegalArgumentException() throws InterruptedException { - THROW_ILLEGAL_ARGUMENT_EXCEPTION.compareAndSet(false, true); - try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), properties)) { - kafkaStreams.setUncaughtExceptionHandler((t, e) -> fail("should not hit old handler")); - - kafkaStreams.setUncaughtExceptionHandler(exception -> REPLACE_THREAD); // if the user defined uncaught exception handler would be hit we would be replacing the thread - - StreamsTestUtils.startKafkaStreamsAndWaitForRunningState(kafkaStreams); - - produceMessages(0L, inputTopic, "A"); - waitForApplicationState(Collections.singletonList(kafkaStreams), KafkaStreams.State.ERROR, DEFAULT_DURATION); - - assertThat(processorValueCollector.size(), equalTo(1)); - } finally { - THROW_ILLEGAL_ARGUMENT_EXCEPTION.compareAndSet(true, false); - } - - } - @Test public void shouldReplaceThreads() throws InterruptedException { testReplaceThreads(2); @@ -365,16 +322,10 @@ private static class ShutdownProcessor extends org.apache.kafka.streams.processo @Override public void process(final String key, final String value) { valueList.add(value + " " + context.taskId()); - if (THROW_ERROR.get()) { - if (THROW_ILLEGAL_STATE_EXCEPTION.get()) { - throw new IllegalStateException("Something unexpected happened in " + Thread.currentThread().getName()); - } else if (THROW_ILLEGAL_ARGUMENT_EXCEPTION.get()) { - throw new IllegalArgumentException("Something unexpected happened in " + Thread.currentThread().getName()); - } else { - throw new StreamsException(Thread.currentThread().getName()); - } + if (throwError.get()) { + throw new StreamsException(Thread.currentThread().getName()); } - THROW_ERROR.set(true); + throwError.set(true); } } @@ -408,7 +359,7 @@ private void testReplaceThreads(final int numThreads) throws InterruptedExceptio final AtomicInteger count = new AtomicInteger(); kafkaStreams.setUncaughtExceptionHandler(exception -> { if (count.incrementAndGet() == numThreads) { - THROW_ERROR.set(false); + throwError.set(false); } return REPLACE_THREAD; }); @@ -416,7 +367,7 @@ private void testReplaceThreads(final int numThreads) throws InterruptedExceptio produceMessages(0L, inputTopic, "A"); TestUtils.waitForCondition(() -> count.get() == numThreads, "finished replacing threads"); - TestUtils.waitForCondition(() -> THROW_ERROR.get(), "finished replacing threads"); + TestUtils.waitForCondition(() -> throwError.get(), "finished replacing threads"); kafkaStreams.close(); waitForApplicationState(Collections.singletonList(kafkaStreams), KafkaStreams.State.NOT_RUNNING, DEFAULT_DURATION); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index 5d898c345bf28..a43b0793a2af0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -2362,9 +2362,9 @@ public void shouldCatchTimeoutExceptionFromHandleCorruptionAndInvokeExceptionHan expect(consumer.groupMetadata()).andStubReturn(consumerGroupMetadata); expect(consumerGroupMetadata.groupInstanceId()).andReturn(Optional.empty()); consumer.subscribe((Collection) anyObject(), anyObject()); - EasyMock.expectLastCall().atLeastOnce(); + EasyMock.expectLastCall().anyTimes(); consumer.unsubscribe(); - EasyMock.expectLastCall().atLeastOnce(); + EasyMock.expectLastCall().anyTimes(); EasyMock.replay(consumerGroupMetadata); final Task task1 = mock(Task.class); final Task task2 = mock(Task.class); From 9aabf498824907816f50a914d98113dcbd73e4d1 Mon Sep 17 00:00:00 2001 From: Shawn Date: Wed, 20 Jul 2022 10:03:43 +0800 Subject: [PATCH 011/175] KAFKA-14024: Consumer keeps Commit offset in onJoinPrepare in Cooperative rebalance (#12349) In KAFKA-13310, we tried to fix a issue that consumer#poll(duration) will be returned after the provided duration. It's because if rebalance needed, we'll try to commit current offset first before rebalance synchronously. And if the offset committing takes too long, the consumer#poll will spend more time than provided duration. To fix that, we change commit sync with commit async before rebalance (i.e. onPrepareJoin). However, in this ticket, we found the async commit will keep sending a new commit request during each Consumer#poll, because the offset commit never completes in time. The impact is that the existing consumer will be kicked out of the group after rebalance timeout without joining the group. That is, suppose we have consumer A in group G, and now consumer B joined the group, after the rebalance, only consumer B in the group. Besides, there's also another bug found during fixing this bug. Before KAFKA-13310, we commitOffset sync with rebalanceTimeout, which will retry when retriable error until timeout. After KAFKA-13310, we thought we have retry, but we'll retry after partitions revoking. That is, even though the retried offset commit successfully, it still causes some partitions offsets un-committed, and after rebalance, other consumers will consume overlapping records. Reviewers: RivenSun , Luke Chen --- .../internals/AbstractCoordinator.java | 5 +- .../internals/ConsumerCoordinator.java | 73 ++++++++--- .../internals/AbstractCoordinatorTest.java | 2 +- .../internals/ConsumerCoordinatorTest.java | 120 ++++++++++++++---- .../distributed/WorkerCoordinator.java | 2 +- .../kafka/api/AbstractConsumerTest.scala | 11 +- .../kafka/api/PlaintextConsumerTest.scala | 87 +++++++++++++ 7 files changed, 251 insertions(+), 49 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index c9ad797ebeb73..d2ece9efc587c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -187,11 +187,12 @@ public AbstractCoordinator(GroupRebalanceConfig rebalanceConfig, /** * Invoked prior to each group join or rejoin. This is typically used to perform any * cleanup from the previous generation (such as committing offsets for the consumer) + * @param timer Timer bounding how long this method can block * @param generation The previous generation or -1 if there was none * @param memberId The identifier of this member in the previous group or "" if there was none * @return true If onJoinPrepare async commit succeeded, false otherwise */ - protected abstract boolean onJoinPrepare(int generation, String memberId); + protected abstract boolean onJoinPrepare(Timer timer, int generation, String memberId); /** * Invoked when the leader is elected. This is used by the leader to perform the assignment @@ -426,7 +427,7 @@ boolean joinGroupIfNeeded(final Timer timer) { // exception, in which case upon retry we should not retry onJoinPrepare either. needsJoinPrepare = false; // return false when onJoinPrepare is waiting for committing offset - if (!onJoinPrepare(generation.generationId, generation.memberId)) { + if (!onJoinPrepare(timer, generation.generationId, generation.memberId)) { needsJoinPrepare = true; //should not initiateJoinGroup if needsJoinPrepare still is true return false; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index b853ff99e8231..9838e7dc8fe4b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -141,6 +141,12 @@ private boolean sameRequest(final Set currentRequest, final Gene } private final RebalanceProtocol protocol; + // pending commit offset request in onJoinPrepare + private RequestFuture autoCommitOffsetRequestFuture = null; + // a timer for join prepare to know when to stop. + // it'll set to rebalance timeout so that the member can join the group successfully + // even though offset commit failed. + private Timer joinPrepareTimer = null; /** * Initialize the coordination manager. @@ -740,24 +746,58 @@ private void validateCooperativeAssignment(final Map future = maybeAutoCommitOffsetsAsync(); - // return true when - // 1. future is null, which means no commit request sent, so it is still considered completed - // 2. offset commit completed - // 3. offset commit failed with non-retriable exception - if (future == null) - onJoinPrepareAsyncCommitCompleted = true; - else if (future.succeeded()) - onJoinPrepareAsyncCommitCompleted = true; - else if (future.failed() && !future.isRetriable()) { - log.error("Asynchronous auto-commit of offsets failed: {}", future.exception().getMessage()); - onJoinPrepareAsyncCommitCompleted = true; + // and there is no in-flight offset commit request + if (autoCommitEnabled && autoCommitOffsetRequestFuture == null) { + autoCommitOffsetRequestFuture = maybeAutoCommitOffsetsAsync(); } + // wait for commit offset response before timer expired + if (autoCommitOffsetRequestFuture != null) { + Timer pollTimer = timer.remainingMs() < joinPrepareTimer.remainingMs() ? + timer : joinPrepareTimer; + client.poll(autoCommitOffsetRequestFuture, pollTimer); + joinPrepareTimer.update(); + + // Keep retrying/waiting the offset commit when: + // 1. offset commit haven't done (and joinPrepareTimer not expired) + // 2. failed with retryable exception (and joinPrepareTimer not expired) + // Otherwise, continue to revoke partitions, ex: + // 1. if joinPrepareTime has expired + // 2. if offset commit failed with no-retryable exception + // 3. if offset commit success + boolean onJoinPrepareAsyncCommitCompleted = true; + if (joinPrepareTimer.isExpired()) { + log.error("Asynchronous auto-commit of offsets failed: joinPrepare timeout. Will continue to join group"); + } else if (!autoCommitOffsetRequestFuture.isDone()) { + onJoinPrepareAsyncCommitCompleted = false; + } else if (autoCommitOffsetRequestFuture.failed() && autoCommitOffsetRequestFuture.isRetriable()) { + log.debug("Asynchronous auto-commit of offsets failed with retryable error: {}. Will retry it.", + autoCommitOffsetRequestFuture.exception().getMessage()); + onJoinPrepareAsyncCommitCompleted = false; + } else if (autoCommitOffsetRequestFuture.failed() && !autoCommitOffsetRequestFuture.isRetriable()) { + log.error("Asynchronous auto-commit of offsets failed: {}. Will continue to join group.", + autoCommitOffsetRequestFuture.exception().getMessage()); + } + if (autoCommitOffsetRequestFuture.isDone()) { + autoCommitOffsetRequestFuture = null; + } + if (!onJoinPrepareAsyncCommitCompleted) { + pollTimer.sleep(Math.min(pollTimer.remainingMs(), rebalanceConfig.retryBackoffMs)); + timer.update(); + return false; + } + } // the generation / member-id can possibly be reset by the heartbeat thread // upon getting errors or heartbeat timeouts; in this case whatever is previously @@ -809,11 +849,14 @@ else if (future.failed() && !future.isRetriable()) { isLeader = false; subscriptions.resetGroupSubscription(); + joinPrepareTimer = null; + autoCommitOffsetRequestFuture = null; + timer.update(); if (exception != null) { throw new KafkaException("User rebalance callback throws an error", exception); } - return onJoinPrepareAsyncCommitCompleted; + return true; } @Override diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java index ddbebb6dde63e..cbc4e7495e161 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java @@ -1726,7 +1726,7 @@ protected Map onLeaderElected(String leaderId, } @Override - protected boolean onJoinPrepare(int generation, String memberId) { + protected boolean onJoinPrepare(Timer timer, int generation, String memberId) { onJoinPrepareInvokes++; return true; } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index db483c6c0f9c0..d948990d69b63 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -78,6 +78,7 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Timer; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; @@ -1301,9 +1302,71 @@ public void testForceMetadataDeleteForPatternSubscriptionDuringRebalance() { } } + @Test + public void testOnJoinPrepareWithOffsetCommitShouldSuccessAfterRetry() { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.empty(), false)) { + int generationId = 42; + String memberId = "consumer-42"; + + Timer pollTimer = time.timer(100L); + client.prepareResponse(offsetCommitResponse(singletonMap(t1p, Errors.UNKNOWN_TOPIC_OR_PARTITION))); + boolean res = coordinator.onJoinPrepare(pollTimer, generationId, memberId); + assertFalse(res); + + pollTimer = time.timer(100L); + client.prepareResponse(offsetCommitResponse(singletonMap(t1p, Errors.NONE))); + res = coordinator.onJoinPrepare(pollTimer, generationId, memberId); + assertTrue(res); + + assertFalse(client.hasPendingResponses()); + assertFalse(client.hasInFlightRequests()); + assertFalse(coordinator.coordinatorUnknown()); + } + } + + @Test + public void testOnJoinPrepareWithOffsetCommitShouldKeepJoinAfterNonRetryableException() { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.empty(), false)) { + int generationId = 42; + String memberId = "consumer-42"; + + Timer pollTimer = time.timer(100L); + client.prepareResponse(offsetCommitResponse(singletonMap(t1p, Errors.UNKNOWN_MEMBER_ID))); + boolean res = coordinator.onJoinPrepare(pollTimer, generationId, memberId); + assertTrue(res); + + assertFalse(client.hasPendingResponses()); + assertFalse(client.hasInFlightRequests()); + assertFalse(coordinator.coordinatorUnknown()); + } + } + + @Test + public void testOnJoinPrepareWithOffsetCommitShouldKeepJoinAfterRebalanceTimeout() { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.empty(), false)) { + int generationId = 42; + String memberId = "consumer-42"; + + Timer pollTimer = time.timer(100L); + time.sleep(150); + boolean res = coordinator.onJoinPrepare(pollTimer, generationId, memberId); + assertFalse(res); + + pollTimer = time.timer(100L); + time.sleep(rebalanceTimeoutMs); + client.respond(offsetCommitResponse(singletonMap(t1p, Errors.UNKNOWN_TOPIC_OR_PARTITION))); + res = coordinator.onJoinPrepare(pollTimer, generationId, memberId); + assertTrue(res); + + assertFalse(client.hasPendingResponses()); + assertFalse(client.hasInFlightRequests()); + assertFalse(coordinator.coordinatorUnknown()); + } + } + @Test public void testJoinPrepareWithDisableAutoCommit() { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.of("group-id"))) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.of("group-id"), true)) { coordinator.ensureActiveGroup(); prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.NONE); @@ -1311,7 +1374,7 @@ public void testJoinPrepareWithDisableAutoCommit() { int generationId = 42; String memberId = "consumer-42"; - boolean res = coordinator.onJoinPrepare(generationId, memberId); + boolean res = coordinator.onJoinPrepare(time.timer(0L), generationId, memberId); assertTrue(res); assertTrue(client.hasPendingResponses()); @@ -1322,14 +1385,14 @@ public void testJoinPrepareWithDisableAutoCommit() { @Test public void testJoinPrepareAndCommitCompleted() { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.of("group-id"))) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.of("group-id"), true)) { coordinator.ensureActiveGroup(); prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.NONE); int generationId = 42; String memberId = "consumer-42"; - boolean res = coordinator.onJoinPrepare(generationId, memberId); + boolean res = coordinator.onJoinPrepare(time.timer(0L), generationId, memberId); coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(res); @@ -1341,7 +1404,7 @@ public void testJoinPrepareAndCommitCompleted() { @Test public void testJoinPrepareAndCommitWithCoordinatorNotAvailable() { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.of("group-id"))) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.of("group-id"), true)) { coordinator.ensureActiveGroup(); prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.COORDINATOR_NOT_AVAILABLE); @@ -1349,7 +1412,7 @@ public void testJoinPrepareAndCommitWithCoordinatorNotAvailable() { int generationId = 42; String memberId = "consumer-42"; - boolean res = coordinator.onJoinPrepare(generationId, memberId); + boolean res = coordinator.onJoinPrepare(time.timer(0L), generationId, memberId); coordinator.invokeCompletedOffsetCommitCallbacks(); assertFalse(res); @@ -1361,7 +1424,7 @@ public void testJoinPrepareAndCommitWithCoordinatorNotAvailable() { @Test public void testJoinPrepareAndCommitWithUnknownMemberId() { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.of("group-id"))) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.of("group-id"), true)) { coordinator.ensureActiveGroup(); prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID); @@ -1369,7 +1432,7 @@ public void testJoinPrepareAndCommitWithUnknownMemberId() { int generationId = 42; String memberId = "consumer-42"; - boolean res = coordinator.onJoinPrepare(generationId, memberId); + boolean res = coordinator.onJoinPrepare(time.timer(0L), generationId, memberId); coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(res); @@ -3080,21 +3143,21 @@ public void run() { @Test public void testCloseDynamicAssignment() { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.empty())) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.empty(), true)) { gracefulCloseTest(coordinator, true); } } @Test public void testCloseManualAssignment() { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(false, true, Optional.empty())) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(false, true, Optional.empty(), true)) { gracefulCloseTest(coordinator, false); } } @Test public void testCloseCoordinatorNotKnownManualAssignment() throws Exception { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(false, true, Optional.empty())) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(false, true, Optional.empty(), true)) { makeCoordinatorUnknown(coordinator, Errors.NOT_COORDINATOR); time.sleep(autoCommitIntervalMs); closeVerifyTimeout(coordinator, 1000, 1000, 1000); @@ -3103,7 +3166,7 @@ public void testCloseCoordinatorNotKnownManualAssignment() throws Exception { @Test public void testCloseCoordinatorNotKnownNoCommits() throws Exception { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.empty())) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.empty(), true)) { makeCoordinatorUnknown(coordinator, Errors.NOT_COORDINATOR); closeVerifyTimeout(coordinator, 1000, 0, 0); } @@ -3111,7 +3174,7 @@ public void testCloseCoordinatorNotKnownNoCommits() throws Exception { @Test public void testCloseCoordinatorNotKnownWithCommits() throws Exception { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.empty())) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.empty(), true)) { makeCoordinatorUnknown(coordinator, Errors.NOT_COORDINATOR); time.sleep(autoCommitIntervalMs); closeVerifyTimeout(coordinator, 1000, 1000, 1000); @@ -3120,7 +3183,7 @@ public void testCloseCoordinatorNotKnownWithCommits() throws Exception { @Test public void testCloseCoordinatorUnavailableNoCommits() throws Exception { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.empty())) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.empty(), true)) { makeCoordinatorUnknown(coordinator, Errors.COORDINATOR_NOT_AVAILABLE); closeVerifyTimeout(coordinator, 1000, 0, 0); } @@ -3128,7 +3191,7 @@ public void testCloseCoordinatorUnavailableNoCommits() throws Exception { @Test public void testCloseTimeoutCoordinatorUnavailableForCommit() throws Exception { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId)) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId, true)) { makeCoordinatorUnknown(coordinator, Errors.COORDINATOR_NOT_AVAILABLE); time.sleep(autoCommitIntervalMs); closeVerifyTimeout(coordinator, 1000, 1000, 1000); @@ -3137,7 +3200,7 @@ public void testCloseTimeoutCoordinatorUnavailableForCommit() throws Exception { @Test public void testCloseMaxWaitCoordinatorUnavailableForCommit() throws Exception { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId)) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId, true)) { makeCoordinatorUnknown(coordinator, Errors.COORDINATOR_NOT_AVAILABLE); time.sleep(autoCommitIntervalMs); closeVerifyTimeout(coordinator, Long.MAX_VALUE, requestTimeoutMs, requestTimeoutMs); @@ -3146,7 +3209,7 @@ public void testCloseMaxWaitCoordinatorUnavailableForCommit() throws Exception { @Test public void testCloseNoResponseForCommit() throws Exception { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId)) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId, true)) { time.sleep(autoCommitIntervalMs); closeVerifyTimeout(coordinator, Long.MAX_VALUE, requestTimeoutMs, requestTimeoutMs); } @@ -3154,14 +3217,14 @@ public void testCloseNoResponseForCommit() throws Exception { @Test public void testCloseNoResponseForLeaveGroup() throws Exception { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.empty())) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.empty(), true)) { closeVerifyTimeout(coordinator, Long.MAX_VALUE, requestTimeoutMs, requestTimeoutMs); } } @Test public void testCloseNoWait() throws Exception { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId)) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId, true)) { time.sleep(autoCommitIntervalMs); closeVerifyTimeout(coordinator, 0, 0, 0); } @@ -3169,7 +3232,7 @@ public void testCloseNoWait() throws Exception { @Test public void testHeartbeatThreadClose() throws Exception { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId)) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId, true)) { coordinator.ensureActiveGroup(); time.sleep(heartbeatIntervalMs + 100); Thread.yield(); // Give heartbeat thread a chance to attempt heartbeat @@ -3236,7 +3299,7 @@ public void testGetGroupMetadata() { assertEquals(JoinGroupRequest.UNKNOWN_MEMBER_ID, groupMetadata.memberId()); assertFalse(groupMetadata.groupInstanceId().isPresent()); - try (final ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId)) { + try (final ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId, true)) { coordinator.ensureActiveGroup(); final ConsumerGroupMetadata joinedGroupMetadata = coordinator.groupMetadata(); @@ -3272,7 +3335,7 @@ public void onPartitionsRevoked(Collection partitions) { @Test public void testPrepareJoinAndRejoinAfterFailedRebalance() { final List partitions = singletonList(t1p); - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.of("group-id"))) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.of("group-id"), true)) { coordinator.ensureActiveGroup(); prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.REBALANCE_IN_PROGRESS); @@ -3292,7 +3355,7 @@ public void testPrepareJoinAndRejoinAfterFailedRebalance() { MockTime time = new MockTime(1); // onJoinPrepare will be executed and onJoinComplete will not. - boolean res = coordinator.joinGroupIfNeeded(time.timer(2)); + boolean res = coordinator.joinGroupIfNeeded(time.timer(100)); assertFalse(res); assertFalse(client.hasPendingResponses()); @@ -3337,7 +3400,7 @@ public void testPrepareJoinAndRejoinAfterFailedRebalance() { @Test public void shouldLoseAllOwnedPartitionsBeforeRejoiningAfterDroppingOutOfTheGroup() { final List partitions = singletonList(t1p); - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.of("group-id"))) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.of("group-id"), true)) { final SystemTime realTime = new SystemTime(); coordinator.ensureActiveGroup(); @@ -3370,7 +3433,7 @@ public void shouldLoseAllOwnedPartitionsBeforeRejoiningAfterDroppingOutOfTheGrou @Test public void shouldLoseAllOwnedPartitionsBeforeRejoiningAfterResettingGenerationId() { final List partitions = singletonList(t1p); - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.of("group-id"))) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.of("group-id"), true)) { final SystemTime realTime = new SystemTime(); coordinator.ensureActiveGroup(); @@ -3468,7 +3531,8 @@ private void receiveFencedInstanceIdException() { private ConsumerCoordinator prepareCoordinatorForCloseTest(final boolean useGroupManagement, final boolean autoCommit, - final Optional groupInstanceId) { + final Optional groupInstanceId, + final boolean shouldPoll) { rebalanceConfig = buildRebalanceConfig(groupInstanceId); ConsumerCoordinator coordinator = buildCoordinator(rebalanceConfig, new Metrics(), @@ -3487,7 +3551,9 @@ private ConsumerCoordinator prepareCoordinatorForCloseTest(final boolean useGrou } subscriptions.seek(t1p, 100); - coordinator.poll(time.timer(Long.MAX_VALUE)); + if (shouldPoll) { + coordinator.poll(time.timer(Long.MAX_VALUE)); + } return coordinator; } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java index 1ab3543361211..ced67427a3f13 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java @@ -225,7 +225,7 @@ protected Map onLeaderElected(String leaderId, } @Override - protected boolean onJoinPrepare(int generation, String memberId) { + protected boolean onJoinPrepare(Timer timer, int generation, String memberId) { log.info("Rebalance started"); leaderState(null); final ExtendedAssignment localAssignmentSnapshot = assignmentSnapshot; diff --git a/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala b/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala index 56bc47c79e956..23b56b8e91fd3 100644 --- a/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala @@ -342,15 +342,16 @@ abstract class AbstractConsumerTest extends BaseRequestTest { protected class ConsumerAssignmentPoller(consumer: Consumer[Array[Byte], Array[Byte]], topicsToSubscribe: List[String], - partitionsToAssign: Set[TopicPartition]) + partitionsToAssign: Set[TopicPartition], + userRebalanceListener: ConsumerRebalanceListener) extends ShutdownableThread("daemon-consumer-assignment", false) { def this(consumer: Consumer[Array[Byte], Array[Byte]], topicsToSubscribe: List[String]) = { - this(consumer, topicsToSubscribe, Set.empty[TopicPartition]) + this(consumer, topicsToSubscribe, Set.empty[TopicPartition], null) } def this(consumer: Consumer[Array[Byte], Array[Byte]], partitionsToAssign: Set[TopicPartition]) = { - this(consumer, List.empty[String], partitionsToAssign) + this(consumer, List.empty[String], partitionsToAssign, null) } @volatile var thrownException: Option[Throwable] = None @@ -363,10 +364,14 @@ abstract class AbstractConsumerTest extends BaseRequestTest { val rebalanceListener: ConsumerRebalanceListener = new ConsumerRebalanceListener { override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]) = { partitionAssignment ++= partitions.toArray(new Array[TopicPartition](0)) + if (userRebalanceListener != null) + userRebalanceListener.onPartitionsAssigned(partitions) } override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]) = { partitionAssignment --= partitions.toArray(new Array[TopicPartition](0)) + if (userRebalanceListener != null) + userRebalanceListener.onPartitionsRevoked(partitions) } } diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 4ede241b0c57a..5dc7c2ada1e0d 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -37,7 +37,11 @@ import kafka.server.QuotaType import kafka.server.KafkaServer import org.apache.kafka.clients.admin.NewPartitions import org.apache.kafka.clients.admin.NewTopic +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource +import java.util.concurrent.TimeUnit +import java.util.concurrent.locks.ReentrantLock import scala.collection.mutable /* We have some tests in this class instead of `BaseConsumerTest` in order to keep the build time under control. */ @@ -969,6 +973,89 @@ class PlaintextConsumerTest extends BaseConsumerTest { } } + @ParameterizedTest + @ValueSource(strings = Array( + "org.apache.kafka.clients.consumer.CooperativeStickyAssignor", + "org.apache.kafka.clients.consumer.RangeAssignor")) + def testRebalanceAndRejoin(assignmentStrategy: String): Unit = { + // create 2 consumers + this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "rebalance-and-rejoin-group") + this.consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, assignmentStrategy) + this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true") + val consumer1 = createConsumer() + val consumer2 = createConsumer() + + // create a new topic, have 2 partitions + val topic = "topic1" + val producer = createProducer() + val expectedAssignment = createTopicAndSendRecords(producer, topic, 2, 100) + + assertEquals(0, consumer1.assignment().size) + assertEquals(0, consumer2.assignment().size) + + val lock = new ReentrantLock() + var generationId1 = -1 + var memberId1 = "" + val customRebalanceListener = new ConsumerRebalanceListener { + override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]): Unit = { + } + override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]): Unit = { + if (!lock.tryLock(3000, TimeUnit.MILLISECONDS)) { + fail(s"Time out while awaiting for lock.") + } + try { + generationId1 = consumer1.groupMetadata().generationId() + memberId1 = consumer1.groupMetadata().memberId() + } finally { + lock.unlock() + } + } + } + val consumerPoller1 = new ConsumerAssignmentPoller(consumer1, List(topic), Set.empty, customRebalanceListener) + consumerPoller1.start() + TestUtils.waitUntilTrue(() => consumerPoller1.consumerAssignment() == expectedAssignment, + s"Timed out while awaiting expected assignment change to $expectedAssignment.") + + // Since the consumer1 already completed the rebalance, + // the `onPartitionsAssigned` rebalance listener will be invoked to set the generationId and memberId + var stableGeneration = -1 + var stableMemberId1 = "" + if (!lock.tryLock(3000, TimeUnit.MILLISECONDS)) { + fail(s"Time out while awaiting for lock.") + } + try { + stableGeneration = generationId1 + stableMemberId1 = memberId1 + } finally { + lock.unlock() + } + + val consumerPoller2 = subscribeConsumerAndStartPolling(consumer2, List(topic)) + TestUtils.waitUntilTrue(() => consumerPoller1.consumerAssignment().size == 1, + s"Timed out while awaiting expected assignment size change to 1.") + TestUtils.waitUntilTrue(() => consumerPoller2.consumerAssignment().size == 1, + s"Timed out while awaiting expected assignment size change to 1.") + + if (!lock.tryLock(3000, TimeUnit.MILLISECONDS)) { + fail(s"Time out while awaiting for lock.") + } + try { + if (assignmentStrategy.equals(classOf[CooperativeStickyAssignor].getName)) { + // cooperative rebalance should rebalance twice before finally stable + assertEquals(stableGeneration + 2, generationId1) + } else { + // eager rebalance should rebalance once before finally stable + assertEquals(stableGeneration + 1, generationId1) + } + assertEquals(stableMemberId1, memberId1) + } finally { + lock.unlock() + } + + consumerPoller1.shutdown() + consumerPoller2.shutdown() + } + /** * This test re-uses BaseConsumerTest's consumers. * As a result, it is testing the default assignment strategy set by BaseConsumerTest From 7a04de4b09d2f7706309bbe5dccdccf97077e681 Mon Sep 17 00:00:00 2001 From: Artem Livshits <84364232+artemlivshits@users.noreply.github.com> Date: Wed, 20 Jul 2022 08:19:31 -0700 Subject: [PATCH 012/175] KAFKA-14020: Performance regression in Producer (#12365) As part of KAFKA-10888 work, there were a couple regressions introduced: A call to time.milliseconds() got moved under the queue lock, moving it back outside the lock. The call may be expensive and cause lock contention. Now the call is moved back outside of the lock. The reference to ProducerRecord was held in the batch completion callback, so it was kept alive as long as the batch was alive, which may increase the amount of memory in certain scenario and cause excessive GC work. Now the reference is reset early, so the ProducerRecord lifetime isn't bound to the batch lifetime. Tested via manually crafted benchmark, lock profile shows ~15% lock contention on the ArrayQueue lock without the fix and ~5% lock contention with the fix (which is also consistent with pre-KAFKA-10888 profile). Alloc profile shows ~10% spent in ProducerBatch.completeFutureAndFireCallbacks without the fix vs. ~0.25% with the fix (which is also consistent with pre-KAFKA-10888 profile). Reviewers: Ismael Juma , Jun Rao --- .../kafka/clients/producer/KafkaProducer.java | 30 +++++++++++++------ .../producer/internals/RecordAccumulator.java | 13 +++++--- 2 files changed, 30 insertions(+), 13 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 74d408d9a5a49..2d5c8994b4df9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -1465,13 +1465,21 @@ public boolean isDone() { private class AppendCallbacks implements RecordAccumulator.AppendCallbacks { private final Callback userCallback; private final ProducerInterceptors interceptors; - private final ProducerRecord record; - protected int partition = RecordMetadata.UNKNOWN_PARTITION; + private final String topic; + private final Integer recordPartition; + private final String recordLogString; + private volatile int partition = RecordMetadata.UNKNOWN_PARTITION; + private volatile TopicPartition topicPartition; private AppendCallbacks(Callback userCallback, ProducerInterceptors interceptors, ProducerRecord record) { this.userCallback = userCallback; this.interceptors = interceptors; - this.record = record; + // Extract record info as we don't want to keep a reference to the record during + // whole lifetime of the batch. + // We don't want to have an NPE here, because the interceptors would not be notified (see .doSend). + topic = record != null ? record.topic() : null; + recordPartition = record != null ? record.partition() : null; + recordLogString = log.isTraceEnabled() && record != null ? record.toString() : ""; } @Override @@ -1491,7 +1499,7 @@ public void setPartition(int partition) { if (log.isTraceEnabled()) { // Log the message here, because we don't know the partition before that. - log.trace("Attempting to append record {} with callback {} to topic {} partition {}", record, userCallback, record.topic(), partition); + log.trace("Attempting to append record {} with callback {} to topic {} partition {}", recordLogString, userCallback, topic, partition); } } @@ -1500,11 +1508,15 @@ public int getPartition() { } public TopicPartition topicPartition() { - if (record == null) - return null; - return partition == RecordMetadata.UNKNOWN_PARTITION - ? ProducerInterceptors.extractTopicPartition(record) - : new TopicPartition(record.topic(), partition); + if (topicPartition == null && topic != null) { + if (partition != RecordMetadata.UNKNOWN_PARTITION) + topicPartition = new TopicPartition(topic, partition); + else if (recordPartition != null) + topicPartition = new TopicPartition(topic, recordPartition); + else + topicPartition = new TopicPartition(topic, RecordMetadata.UNKNOWN_PARTITION); + } + return topicPartition; } } } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index 4168ea68aa430..a1f684ac95cd4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -297,7 +297,12 @@ public RecordAppendResult append(String topic, byte maxUsableMagic = apiVersions.maxUsableProduceMagic(); int size = Math.max(this.batchSize, AbstractRecords.estimateSizeInBytesUpperBound(maxUsableMagic, compression, key, value, headers)); log.trace("Allocating a new {} byte message buffer for topic {} partition {} with remaining timeout {}ms", size, topic, partition, maxTimeToBlock); + // This call may block if we exhausted buffer space. buffer = free.allocate(size, maxTimeToBlock); + // Update the current time in case the buffer allocation blocked above. + // NOTE: getting time may be expensive, so calling it under a lock + // should be avoided. + nowMs = time.milliseconds(); } synchronized (dq) { @@ -307,7 +312,7 @@ public RecordAppendResult append(String topic, partitionInfo.partition(), topic); continue; } - RecordAppendResult appendResult = appendNewBatch(topic, effectivePartition, dq, timestamp, key, value, headers, callbacks, buffer); + RecordAppendResult appendResult = appendNewBatch(topic, effectivePartition, dq, timestamp, key, value, headers, callbacks, buffer, nowMs); // Set buffer to null, so that deallocate doesn't return it back to free pool, since it's used in the batch. if (appendResult.newBatchCreated) buffer = null; @@ -333,6 +338,7 @@ public RecordAppendResult append(String topic, * @param headers the Headers for the record * @param callbacks The callbacks to execute * @param buffer The buffer for the new batch + * @param nowMs The current time, in milliseconds */ private RecordAppendResult appendNewBatch(String topic, int partition, @@ -342,11 +348,10 @@ private RecordAppendResult appendNewBatch(String topic, byte[] value, Header[] headers, AppendCallbacks callbacks, - ByteBuffer buffer) { + ByteBuffer buffer, + long nowMs) { assert partition != RecordMetadata.UNKNOWN_PARTITION; - // Update the current time in case the buffer allocation blocked above. - long nowMs = time.milliseconds(); RecordAppendResult appendResult = tryAppend(timestamp, key, value, headers, callbacks, dq, nowMs); if (appendResult != null) { // Somebody else found us a batch, return the one we waited for! Hopefully this doesn't happen often... From 327439504be7dba899ca24d33b545801b5a36084 Mon Sep 17 00:00:00 2001 From: James Hughes Date: Thu, 21 Jul 2022 10:35:29 -0400 Subject: [PATCH 013/175] KAFKA-14076: Fix issues with KafkaStreams.CloseOptions (#12408) - used static memberId was incorrect - need to remove all threads/members from the group - need to use admit client correctly Add test to verify fixes. Reviewers: Matthias J. Sax --- .../apache/kafka/streams/KafkaStreams.java | 69 +++--- ...fkaStreamsCloseOptionsIntegrationTest.java | 198 ++++++++++++++++++ 2 files changed, 231 insertions(+), 36 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsCloseOptionsIntegrationTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index 3a61f05de11a1..37b82cda42cf0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -75,13 +75,11 @@ import org.apache.kafka.streams.state.internals.GlobalStateStoreProvider; import org.apache.kafka.streams.state.internals.QueryableStoreProvider; import org.apache.kafka.streams.state.internals.StreamThreadStateStoreProvider; - import org.slf4j.Logger; import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; -import java.util.function.BiConsumer; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -102,6 +100,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.stream.Collectors; @@ -1343,10 +1342,10 @@ public CloseOptions leaveGroup(final boolean leaveGroup) { * This will block until all threads have stopped. */ public void close() { - close(Long.MAX_VALUE); + close(Long.MAX_VALUE, false); } - private Thread shutdownHelper(final boolean error) { + private Thread shutdownHelper(final boolean error, final long timeoutMs, final boolean leaveGroup) { stateDirCleaner.shutdownNow(); if (rocksDBMetricsRecordingService != null) { rocksDBMetricsRecordingService.shutdownNow(); @@ -1378,6 +1377,10 @@ private Thread shutdownHelper(final boolean error) { } }); + if (leaveGroup) { + processStreamThread(streamThreadLeaveConsumerGroup(timeoutMs)); + } + log.info("Shutdown {} stream threads complete", numStreamThreads); if (globalStreamThread != null) { @@ -1412,7 +1415,7 @@ private Thread shutdownHelper(final boolean error) { }, clientId + "-CloseThread"); } - private boolean close(final long timeoutMs) { + private boolean close(final long timeoutMs, final boolean leaveGroup) { if (state.hasCompletedShutdown()) { log.info("Streams client is already in the terminal {} state, all resources are closed and the client has stopped.", state); return true; @@ -1437,7 +1440,8 @@ private boolean close(final long timeoutMs) { log.error("Failed to transition to PENDING_SHUTDOWN, current state is {}", state); throw new StreamsException("Failed to shut down while in state " + state); } else { - final Thread shutdownThread = shutdownHelper(false); + + final Thread shutdownThread = shutdownHelper(false, timeoutMs, leaveGroup); shutdownThread.setDaemon(true); shutdownThread.start(); @@ -1456,7 +1460,7 @@ private void closeToError() { if (!setState(State.PENDING_ERROR)) { log.info("Skipping shutdown since we are already in " + state()); } else { - final Thread shutdownThread = shutdownHelper(true); + final Thread shutdownThread = shutdownHelper(true, -1, false); shutdownThread.setDaemon(true); shutdownThread.start(); @@ -1484,7 +1488,7 @@ public synchronized boolean close(final Duration timeout) throws IllegalArgument log.debug("Stopping Streams client with timeoutMillis = {} ms.", timeoutMs); - return close(timeoutMs); + return close(timeoutMs, false); } /** @@ -1498,48 +1502,41 @@ public synchronized boolean close(final Duration timeout) throws IllegalArgument * @throws IllegalArgumentException if {@code timeout} can't be represented as {@code long milliseconds} */ public synchronized boolean close(final CloseOptions options) throws IllegalArgumentException { + Objects.requireNonNull(options, "options cannot be null"); final String msgPrefix = prepareMillisCheckFailMsgPrefix(options.timeout, "timeout"); final long timeoutMs = validateMillisecondDuration(options.timeout, msgPrefix); if (timeoutMs < 0) { throw new IllegalArgumentException("Timeout can't be negative."); } + log.debug("Stopping Streams client with timeoutMillis = {} ms.", timeoutMs); + return close(timeoutMs, options.leaveGroup); + } - final long startMs = time.milliseconds(); - - final boolean closeStatus = close(timeoutMs); - - final Optional groupInstanceId = clientSupplier - .getConsumer(applicationConfigs.getGlobalConsumerConfigs(clientId)) - .groupMetadata() - .groupInstanceId(); - - final long remainingTimeMs = Math.max(0, timeoutMs - (time.milliseconds() - startMs)); - - if (options.leaveGroup && groupInstanceId.isPresent()) { - log.debug("Sending leave group trigger to removing instance from consumer group"); - //removing instance from consumer group - - final MemberToRemove memberToRemove = new MemberToRemove(groupInstanceId.get()); - - final Collection membersToRemove = Collections.singletonList(memberToRemove); + private Consumer streamThreadLeaveConsumerGroup(final long remainingTimeMs) { + return thread -> { + final Optional groupInstanceId = thread.getGroupInstanceID(); + if (groupInstanceId.isPresent()) { + log.debug("Sending leave group trigger to removing instance from consumer group: {}.", + groupInstanceId.get()); + final MemberToRemove memberToRemove = new MemberToRemove(groupInstanceId.get()); + final Collection membersToRemove = Collections.singletonList(memberToRemove); - final RemoveMembersFromConsumerGroupResult removeMembersFromConsumerGroupResult = adminClient + final RemoveMembersFromConsumerGroupResult removeMembersFromConsumerGroupResult = adminClient .removeMembersFromConsumerGroup( applicationConfigs.getString(StreamsConfig.APPLICATION_ID_CONFIG), new RemoveMembersFromConsumerGroupOptions(membersToRemove) ); - try { - removeMembersFromConsumerGroupResult.memberResult(memberToRemove).get(remainingTimeMs, TimeUnit.MILLISECONDS); - } catch (final Exception e) { - log.error("Could not remove static member {} from consumer group {} due to a: {}", groupInstanceId.get(), + try { + removeMembersFromConsumerGroupResult.memberResult(memberToRemove) + .get(remainingTimeMs, TimeUnit.MILLISECONDS); + } catch (final Exception e) { + log.error("Could not remove static member {} from consumer group {} due to a: {}", + groupInstanceId.get(), applicationConfigs.getString(StreamsConfig.APPLICATION_ID_CONFIG), e); + } } - } - - log.debug("Stopping Streams client with timeoutMillis = {} ms.", timeoutMs); - - return closeStatus; + }; } /** diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsCloseOptionsIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsCloseOptionsIntegrationTest.java new file mode 100644 index 0000000000000..8d3cb8e87959a --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsCloseOptionsIntegrationTest.java @@ -0,0 +1,198 @@ +/* + * 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.integration; + +import kafka.server.KafkaConfig; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.LongDeserializer; +import org.apache.kafka.common.serialization.LongSerializer; +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.common.utils.MockTime; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KafkaStreams.CloseOptions; +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.integration.utils.EmbeddedKafkaCluster; +import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.test.IntegrationTest; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TemporaryFolder; +import org.junit.rules.TestName; +import org.junit.rules.Timeout; + +import java.io.IOException; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Properties; + +import static java.util.Collections.singletonList; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForEmptyConsumerGroup; + +@Category({IntegrationTest.class}) +public class KafkaStreamsCloseOptionsIntegrationTest { + @Rule + public Timeout globalTimeout = Timeout.seconds(600); + @Rule + public final TestName testName = new TestName(); + private static MockTime mockTime; + + @Rule + public final TemporaryFolder testFolder = new TemporaryFolder(TestUtils.tempDirectory()); + + protected static final String INPUT_TOPIC = "inputTopic"; + protected static final String OUTPUT_TOPIC = "outputTopic"; + + protected Properties streamsConfig; + protected static KafkaStreams streams; + protected static Admin adminClient; + protected Properties commonClientConfig; + private Properties producerConfig; + protected Properties resultConsumerConfig; + + public static final EmbeddedKafkaCluster CLUSTER; + + static { + final Properties brokerProps = new Properties(); + brokerProps.setProperty(KafkaConfig.GroupMaxSessionTimeoutMsProp(), Integer.toString(Integer.MAX_VALUE)); + CLUSTER = new EmbeddedKafkaCluster(1, brokerProps); + } + + @BeforeClass + public static void startCluster() throws IOException { + CLUSTER.start(); + } + + @AfterClass + public static void closeCluster() { + CLUSTER.stop(); + } + + @Before + public void before() throws Exception { + mockTime = CLUSTER.time; + + final String appID = IntegrationTestUtils.safeUniqueTestName(getClass(), testName); + + commonClientConfig = new Properties(); + commonClientConfig.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); + + streamsConfig = new Properties(); + streamsConfig.put(StreamsConfig.STATE_DIR_CONFIG, testFolder.getRoot().getPath()); + streamsConfig.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Long().getClass()); + streamsConfig.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + streamsConfig.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + streamsConfig.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); + streamsConfig.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 100); + streamsConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + // In this test, we set the SESSION_TIMEOUT_MS_CONFIG high in order to show that the call to + // `close(CloseOptions)` can remove the application from the Consumder Groups successfully. + streamsConfig.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, Integer.MAX_VALUE); + streamsConfig.putAll(commonClientConfig); + + producerConfig = new Properties(); + producerConfig.put(ProducerConfig.ACKS_CONFIG, "all"); + producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, LongSerializer.class); + producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class); + producerConfig.putAll(commonClientConfig); + + resultConsumerConfig = new Properties(); + resultConsumerConfig.put(ConsumerConfig.GROUP_ID_CONFIG, appID + "-result-consumer"); + resultConsumerConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + resultConsumerConfig.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, LongDeserializer.class); + resultConsumerConfig.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + resultConsumerConfig.putAll(commonClientConfig); + + if (adminClient == null) { + adminClient = Admin.create(commonClientConfig); + } + + CLUSTER.deleteAllTopicsAndWait(120_000L); + CLUSTER.createTopic(INPUT_TOPIC, 2, 1); + CLUSTER.createTopic(OUTPUT_TOPIC, 2, 1); + + add10InputElements(); + } + + @After + public void after() throws Exception { + if (streams != null) { + streams.close(Duration.ofSeconds(30)); + } + } + + @Test + public void testCloseOptions() throws Exception { + final String appID = IntegrationTestUtils.safeUniqueTestName(getClass(), testName); + streamsConfig.put(StreamsConfig.APPLICATION_ID_CONFIG, appID); + streamsConfig.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, "someGroupInstance"); + // Test with two threads to show that each of the threads is being called to remove clients from the CG. + streamsConfig.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 2); + + // RUN + streams = new KafkaStreams(setupTopologyWithoutIntermediateUserTopic(), streamsConfig); + IntegrationTestUtils.startApplicationAndWaitUntilRunning(singletonList(streams), Duration.ofSeconds(30)); + IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived(resultConsumerConfig, OUTPUT_TOPIC, 10); + + streams.close(new CloseOptions().leaveGroup(true).timeout(Duration.ofSeconds(30))); + waitForEmptyConsumerGroup(adminClient, appID, 0); + } + + protected Topology setupTopologyWithoutIntermediateUserTopic() { + final StreamsBuilder builder = new StreamsBuilder(); + + final KStream input = builder.stream(INPUT_TOPIC); + + input.to(OUTPUT_TOPIC, Produced.with(Serdes.Long(), Serdes.String())); + return builder.build(); + } + + private void add10InputElements() { + final List> records = Arrays.asList(KeyValue.pair(0L, "aaa"), + KeyValue.pair(1L, "bbb"), + KeyValue.pair(0L, "ccc"), + KeyValue.pair(1L, "ddd"), + KeyValue.pair(0L, "eee"), + KeyValue.pair(1L, "fff"), + KeyValue.pair(0L, "ggg"), + KeyValue.pair(1L, "hhh"), + KeyValue.pair(0L, "iii"), + KeyValue.pair(1L, "jjj")); + + for (final KeyValue record : records) { + mockTime.sleep(10); + IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(INPUT_TOPIC, Collections.singleton(record), producerConfig, mockTime.milliseconds()); + } + } +} From 158fc5f0269670d80d6c8f49186c51930bd600a1 Mon Sep 17 00:00:00 2001 From: Luke Chen Date: Fri, 22 Jul 2022 11:00:15 +0800 Subject: [PATCH 014/175] KAFKA-13919: expose log recovery metrics (#12347) Implementation for KIP-831. 1. add remainingLogsToRecover metric for the number of remaining logs for each log.dir to be recovered 2. add remainingSegmentsToRecover metric for the number of remaining segments for the current log assigned to the recovery thread. 3. remove these metrics after log loaded completely 4. add tests Reviewers: Jun Rao , Tom Bentley --- core/src/main/scala/kafka/log/LogLoader.scala | 26 ++- .../src/main/scala/kafka/log/LogManager.scala | 85 +++++-- .../src/main/scala/kafka/log/UnifiedLog.scala | 10 +- .../scala/unit/kafka/log/LogLoaderTest.scala | 3 +- .../scala/unit/kafka/log/LogManagerTest.scala | 220 +++++++++++++++++- .../scala/unit/kafka/log/LogTestUtils.scala | 7 +- .../scala/unit/kafka/utils/TestUtils.scala | 5 +- .../kafka/jmh/server/CheckpointBench.java | 2 +- 8 files changed, 319 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogLoader.scala b/core/src/main/scala/kafka/log/LogLoader.scala index 581d016e5e0e9..25ee89c72bbde 100644 --- a/core/src/main/scala/kafka/log/LogLoader.scala +++ b/core/src/main/scala/kafka/log/LogLoader.scala @@ -29,6 +29,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.InvalidOffsetException import org.apache.kafka.common.utils.Time +import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} import scala.collection.{Set, mutable} case class LoadedLogOffsets(logStartOffset: Long, @@ -64,6 +65,7 @@ object LogLoader extends Logging { * @param recoveryPointCheckpoint The checkpoint of the offset at which to begin the recovery * @param leaderEpochCache An optional LeaderEpochFileCache instance to be updated during recovery * @param producerStateManager The ProducerStateManager instance to be updated during recovery + * @param numRemainingSegments The remaining segments to be recovered in this log keyed by recovery thread name */ class LogLoader( dir: File, @@ -77,7 +79,8 @@ class LogLoader( logStartOffsetCheckpoint: Long, recoveryPointCheckpoint: Long, leaderEpochCache: Option[LeaderEpochFileCache], - producerStateManager: ProducerStateManager + producerStateManager: ProducerStateManager, + numRemainingSegments: ConcurrentMap[String, Int] = new ConcurrentHashMap[String, Int] ) extends Logging { logIdent = s"[LogLoader partition=$topicPartition, dir=${dir.getParent}] " @@ -404,12 +407,18 @@ class LogLoader( // If we have the clean shutdown marker, skip recovery. if (!hadCleanShutdown) { - val unflushed = segments.values(recoveryPointCheckpoint, Long.MaxValue).iterator + val unflushed = segments.values(recoveryPointCheckpoint, Long.MaxValue) + val numUnflushed = unflushed.size + val unflushedIter = unflushed.iterator var truncated = false + var numFlushed = 0 + val threadName = Thread.currentThread().getName + numRemainingSegments.put(threadName, numUnflushed) + + while (unflushedIter.hasNext && !truncated) { + val segment = unflushedIter.next() + info(s"Recovering unflushed segment ${segment.baseOffset}. $numFlushed/$numUnflushed recovered for $topicPartition.") - while (unflushed.hasNext && !truncated) { - val segment = unflushed.next() - info(s"Recovering unflushed segment ${segment.baseOffset}") val truncatedBytes = try { recoverSegment(segment) @@ -424,8 +433,13 @@ class LogLoader( // we had an invalid message, delete all remaining log warn(s"Corruption found in segment ${segment.baseOffset}," + s" truncating to offset ${segment.readNextOffset}") - removeAndDeleteSegmentsAsync(unflushed.toList) + removeAndDeleteSegmentsAsync(unflushedIter.toList) truncated = true + // segment is truncated, so set remaining segments to 0 + numRemainingSegments.put(threadName, 0) + } else { + numFlushed += 1 + numRemainingSegments.put(threadName, numUnflushed - numFlushed) } } } diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index bdc7ffd74dbea..886f56c63cfa9 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -262,7 +262,8 @@ class LogManager(logDirs: Seq[File], recoveryPoints: Map[TopicPartition, Long], logStartOffsets: Map[TopicPartition, Long], defaultConfig: LogConfig, - topicConfigOverrides: Map[String, LogConfig]): UnifiedLog = { + topicConfigOverrides: Map[String, LogConfig], + numRemainingSegments: ConcurrentMap[String, Int]): UnifiedLog = { val topicPartition = UnifiedLog.parseTopicPartitionName(logDir) val config = topicConfigOverrides.getOrElse(topicPartition.topic, defaultConfig) val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L) @@ -282,7 +283,8 @@ class LogManager(logDirs: Seq[File], logDirFailureChannel = logDirFailureChannel, lastShutdownClean = hadCleanShutdown, topicId = None, - keepPartitionMetadataFile = keepPartitionMetadataFile) + keepPartitionMetadataFile = keepPartitionMetadataFile, + numRemainingSegments = numRemainingSegments) if (logDir.getName.endsWith(UnifiedLog.DeleteDirSuffix)) { addLogToBeDeleted(log) @@ -307,6 +309,27 @@ class LogManager(logDirs: Seq[File], log } + // factory class for naming the log recovery threads used in metrics + class LogRecoveryThreadFactory(val dirPath: String) extends ThreadFactory { + val threadNum = new AtomicInteger(0) + + override def newThread(runnable: Runnable): Thread = { + KafkaThread.nonDaemon(logRecoveryThreadName(dirPath, threadNum.getAndIncrement()), runnable) + } + } + + // create a unique log recovery thread name for each log dir as the format: prefix-dirPath-threadNum, ex: "log-recovery-/tmp/kafkaLogs-0" + private def logRecoveryThreadName(dirPath: String, threadNum: Int, prefix: String = "log-recovery"): String = s"$prefix-$dirPath-$threadNum" + + /* + * decrement the number of remaining logs + * @return the number of remaining logs after decremented 1 + */ + private[log] def decNumRemainingLogs(numRemainingLogs: ConcurrentMap[String, Int], path: String): Int = { + require(path != null, "path cannot be null to update remaining logs metric.") + numRemainingLogs.compute(path, (_, oldVal) => oldVal - 1) + } + /** * Recover and load all logs in the given data directories */ @@ -317,6 +340,10 @@ class LogManager(logDirs: Seq[File], val offlineDirs = mutable.Set.empty[(String, IOException)] val jobs = ArrayBuffer.empty[Seq[Future[_]]] var numTotalLogs = 0 + // log dir path -> number of Remaining logs map for remainingLogsToRecover metric + val numRemainingLogs: ConcurrentMap[String, Int] = new ConcurrentHashMap[String, Int] + // log recovery thread name -> number of remaining segments map for remainingSegmentsToRecover metric + val numRemainingSegments: ConcurrentMap[String, Int] = new ConcurrentHashMap[String, Int] def handleIOException(logDirAbsolutePath: String, e: IOException): Unit = { offlineDirs.add((logDirAbsolutePath, e)) @@ -328,7 +355,7 @@ class LogManager(logDirs: Seq[File], var hadCleanShutdown: Boolean = false try { val pool = Executors.newFixedThreadPool(numRecoveryThreadsPerDataDir, - KafkaThread.nonDaemon(s"log-recovery-$logDirAbsolutePath", _)) + new LogRecoveryThreadFactory(logDirAbsolutePath)) threadPools.append(pool) val cleanShutdownFile = new File(dir, LogLoader.CleanShutdownFile) @@ -363,28 +390,32 @@ class LogManager(logDirs: Seq[File], val logsToLoad = Option(dir.listFiles).getOrElse(Array.empty).filter(logDir => logDir.isDirectory && UnifiedLog.parseTopicPartitionName(logDir).topic != KafkaRaftServer.MetadataTopic) - val numLogsLoaded = new AtomicInteger(0) numTotalLogs += logsToLoad.length + numRemainingLogs.put(dir.getAbsolutePath, logsToLoad.length) val jobsForDir = logsToLoad.map { logDir => val runnable: Runnable = () => { + debug(s"Loading log $logDir") + var log = None: Option[UnifiedLog] + val logLoadStartMs = time.hiResClockMs() try { - debug(s"Loading log $logDir") - - val logLoadStartMs = time.hiResClockMs() - val log = loadLog(logDir, hadCleanShutdown, recoveryPoints, logStartOffsets, - defaultConfig, topicConfigOverrides) - val logLoadDurationMs = time.hiResClockMs() - logLoadStartMs - val currentNumLoaded = numLogsLoaded.incrementAndGet() - - info(s"Completed load of $log with ${log.numberOfSegments} segments in ${logLoadDurationMs}ms " + - s"($currentNumLoaded/${logsToLoad.length} loaded in $logDirAbsolutePath)") + log = Some(loadLog(logDir, hadCleanShutdown, recoveryPoints, logStartOffsets, + defaultConfig, topicConfigOverrides, numRemainingSegments)) } catch { case e: IOException => handleIOException(logDirAbsolutePath, e) case e: KafkaStorageException if e.getCause.isInstanceOf[IOException] => // KafkaStorageException might be thrown, ex: during writing LeaderEpochFileCache // And while converting IOException to KafkaStorageException, we've already handled the exception. So we can ignore it here. + } finally { + val logLoadDurationMs = time.hiResClockMs() - logLoadStartMs + val remainingLogs = decNumRemainingLogs(numRemainingLogs, dir.getAbsolutePath) + val currentNumLoaded = logsToLoad.length - remainingLogs + log match { + case Some(loadedLog) => info(s"Completed load of $loadedLog with ${loadedLog.numberOfSegments} segments in ${logLoadDurationMs}ms " + + s"($currentNumLoaded/${logsToLoad.length} completed in $logDirAbsolutePath)") + case None => info(s"Error while loading logs in $logDir in ${logLoadDurationMs}ms ($currentNumLoaded/${logsToLoad.length} completed in $logDirAbsolutePath)") + } } } runnable @@ -398,6 +429,7 @@ class LogManager(logDirs: Seq[File], } try { + addLogRecoveryMetrics(numRemainingLogs, numRemainingSegments) for (dirJobs <- jobs) { dirJobs.foreach(_.get) } @@ -410,12 +442,37 @@ class LogManager(logDirs: Seq[File], error(s"There was an error in one of the threads during logs loading: ${e.getCause}") throw e.getCause } finally { + removeLogRecoveryMetrics() threadPools.foreach(_.shutdown()) } info(s"Loaded $numTotalLogs logs in ${time.hiResClockMs() - startMs}ms.") } + private[log] def addLogRecoveryMetrics(numRemainingLogs: ConcurrentMap[String, Int], + numRemainingSegments: ConcurrentMap[String, Int]): Unit = { + debug("Adding log recovery metrics") + for (dir <- logDirs) { + newGauge("remainingLogsToRecover", () => numRemainingLogs.get(dir.getAbsolutePath), + Map("dir" -> dir.getAbsolutePath)) + for (i <- 0 until numRecoveryThreadsPerDataDir) { + val threadName = logRecoveryThreadName(dir.getAbsolutePath, i) + newGauge("remainingSegmentsToRecover", () => numRemainingSegments.get(threadName), + Map("dir" -> dir.getAbsolutePath, "threadNum" -> i.toString)) + } + } + } + + private[log] def removeLogRecoveryMetrics(): Unit = { + debug("Removing log recovery metrics") + for (dir <- logDirs) { + removeMetric("remainingLogsToRecover", Map("dir" -> dir.getAbsolutePath)) + for (i <- 0 until numRecoveryThreadsPerDataDir) { + removeMetric("remainingSegmentsToRecover", Map("dir" -> dir.getAbsolutePath, "threadNum" -> i.toString)) + } + } + } + /** * Start the background threads to flush logs and do log cleanup */ diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index ddd66eb160fcb..c4a2300237cd8 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -18,11 +18,11 @@ package kafka.log import com.yammer.metrics.core.MetricName + import java.io.{File, IOException} import java.nio.file.Files import java.util.Optional -import java.util.concurrent.TimeUnit - +import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, TimeUnit} import kafka.common.{LongRef, OffsetsOutOfOrderException, UnexpectedAppendOffsetException} import kafka.log.AppendOrigin.RaftLeader import kafka.message.{BrokerCompressionCodec, CompressionCodec, NoCompressionCodec} @@ -1803,7 +1803,8 @@ object UnifiedLog extends Logging { logDirFailureChannel: LogDirFailureChannel, lastShutdownClean: Boolean = true, topicId: Option[Uuid], - keepPartitionMetadataFile: Boolean): UnifiedLog = { + keepPartitionMetadataFile: Boolean, + numRemainingSegments: ConcurrentMap[String, Int] = new ConcurrentHashMap[String, Int]): UnifiedLog = { // create the log directory if it doesn't exist Files.createDirectories(dir.toPath) val topicPartition = UnifiedLog.parseTopicPartitionName(dir) @@ -1828,7 +1829,8 @@ object UnifiedLog extends Logging { logStartOffset, recoveryPoint, leaderEpochCache, - producerStateManager + producerStateManager, + numRemainingSegments ).load() val localLog = new LocalLog(dir, config, segments, offsets.recoveryPoint, offsets.nextOffsetMetadata, scheduler, time, topicPartition, logDirFailureChannel) diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index 0d41a5073bf02..c6379ff3f3341 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -38,6 +38,7 @@ import org.mockito.ArgumentMatchers import org.mockito.ArgumentMatchers.{any, anyLong} import org.mockito.Mockito.{mock, reset, times, verify, when} +import java.util.concurrent.ConcurrentMap import scala.annotation.nowarn import scala.collection.mutable.ListBuffer import scala.collection.{Iterable, Map, mutable} @@ -117,7 +118,7 @@ class LogLoaderTest { override def loadLog(logDir: File, hadCleanShutdown: Boolean, recoveryPoints: Map[TopicPartition, Long], logStartOffsets: Map[TopicPartition, Long], defaultConfig: LogConfig, - topicConfigs: Map[String, LogConfig]): UnifiedLog = { + topicConfigs: Map[String, LogConfig], numRemainingSegments: ConcurrentMap[String, Int]): UnifiedLog = { if (simulateError.hasError) { simulateError.errorType match { case ErrorTypes.KafkaStorageExceptionWithIOExceptionCause => diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 5353df6db3855..1b2dd7809f3fb 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -17,10 +17,10 @@ package kafka.log -import com.yammer.metrics.core.MetricName +import com.yammer.metrics.core.{Gauge, MetricName} import kafka.server.checkpoints.OffsetCheckpointFile import kafka.server.metadata.{ConfigRepository, MockConfigRepository} -import kafka.server.{FetchDataInfo, FetchLogEnd} +import kafka.server.{BrokerTopicStats, FetchDataInfo, FetchLogEnd, LogDirFailureChannel} import kafka.utils._ import org.apache.directory.api.util.FileUtils import org.apache.kafka.common.errors.OffsetOutOfRangeException @@ -29,16 +29,17 @@ import org.apache.kafka.common.{KafkaException, TopicPartition} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.ArgumentMatchers.any -import org.mockito.{ArgumentMatchers, Mockito} -import org.mockito.Mockito.{doAnswer, mock, never, spy, times, verify} +import org.mockito.{ArgumentCaptor, ArgumentMatchers, Mockito} +import org.mockito.Mockito.{doAnswer, doNothing, mock, never, spy, times, verify} + import java.io._ import java.nio.file.Files -import java.util.concurrent.Future +import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, Future} import java.util.{Collections, Properties} - import org.apache.kafka.server.metrics.KafkaYammerMetrics -import scala.collection.mutable +import scala.collection.{Map, mutable} +import scala.collection.mutable.ArrayBuffer import scala.jdk.CollectionConverters._ import scala.util.{Failure, Try} @@ -421,12 +422,14 @@ class LogManagerTest { } private def createLogManager(logDirs: Seq[File] = Seq(this.logDir), - configRepository: ConfigRepository = new MockConfigRepository): LogManager = { + configRepository: ConfigRepository = new MockConfigRepository, + recoveryThreadsPerDataDir: Int = 1): LogManager = { TestUtils.createLogManager( defaultConfig = logConfig, configRepository = configRepository, logDirs = logDirs, - time = this.time) + time = this.time, + recoveryThreadsPerDataDir = recoveryThreadsPerDataDir) } @Test @@ -638,6 +641,205 @@ class LogManagerTest { assertTrue(logManager.partitionsInitializing.isEmpty) } + private def appendRecordsToLog(time: MockTime, parentLogDir: File, partitionId: Int, brokerTopicStats: BrokerTopicStats, expectedSegmentsPerLog: Int): Unit = { + def createRecord = TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds) + val tpFile = new File(parentLogDir, s"$name-$partitionId") + val segmentBytes = 1024 + + val log = LogTestUtils.createLog(tpFile, logConfig, brokerTopicStats, time.scheduler, time, 0, 0, + 5 * 60 * 1000, 60 * 60 * 1000, LogManager.ProducerIdExpirationCheckIntervalMs) + + assertTrue(expectedSegmentsPerLog > 0) + // calculate numMessages to append to logs. It'll create "expectedSegmentsPerLog" log segments with segment.bytes=1024 + val numMessages = Math.floor(segmentBytes * expectedSegmentsPerLog / createRecord.sizeInBytes).asInstanceOf[Int] + try { + for (_ <- 0 until numMessages) { + log.appendAsLeader(createRecord, leaderEpoch = 0) + } + + assertEquals(expectedSegmentsPerLog, log.numberOfSegments) + } finally { + log.close() + } + } + + private def verifyRemainingLogsToRecoverMetric(spyLogManager: LogManager, expectedParams: Map[String, Int]): Unit = { + val spyLogManagerClassName = spyLogManager.getClass().getSimpleName + // get all `remainingLogsToRecover` metrics + val logMetrics: ArrayBuffer[Gauge[Int]] = KafkaYammerMetrics.defaultRegistry.allMetrics.asScala + .filter { case (metric, _) => metric.getType == s"$spyLogManagerClassName" && metric.getName == "remainingLogsToRecover" } + .map { case (_, gauge) => gauge } + .asInstanceOf[ArrayBuffer[Gauge[Int]]] + + assertEquals(expectedParams.size, logMetrics.size) + + val capturedPath: ArgumentCaptor[String] = ArgumentCaptor.forClass(classOf[String]) + + val expectedCallTimes = expectedParams.values.sum + verify(spyLogManager, times(expectedCallTimes)).decNumRemainingLogs(any[ConcurrentMap[String, Int]], capturedPath.capture()); + + val paths = capturedPath.getAllValues + expectedParams.foreach { + case (path, totalLogs) => + // make sure each path is called "totalLogs" times, which means it is decremented to 0 in the end + assertEquals(totalLogs, Collections.frequency(paths, path)) + } + + // expected the end value is 0 + logMetrics.foreach { gauge => assertEquals(0, gauge.value()) } + } + + private def verifyRemainingSegmentsToRecoverMetric(spyLogManager: LogManager, + logDirs: Seq[File], + recoveryThreadsPerDataDir: Int, + mockMap: ConcurrentHashMap[String, Int], + expectedParams: Map[String, Int]): Unit = { + val spyLogManagerClassName = spyLogManager.getClass().getSimpleName + // get all `remainingSegmentsToRecover` metrics + val logSegmentMetrics: ArrayBuffer[Gauge[Int]] = KafkaYammerMetrics.defaultRegistry.allMetrics.asScala + .filter { case (metric, _) => metric.getType == s"$spyLogManagerClassName" && metric.getName == "remainingSegmentsToRecover" } + .map { case (_, gauge) => gauge } + .asInstanceOf[ArrayBuffer[Gauge[Int]]] + + // expected each log dir has 1 metrics for each thread + assertEquals(recoveryThreadsPerDataDir * logDirs.size, logSegmentMetrics.size) + + val capturedThreadName: ArgumentCaptor[String] = ArgumentCaptor.forClass(classOf[String]) + val capturedNumRemainingSegments: ArgumentCaptor[Int] = ArgumentCaptor.forClass(classOf[Int]) + + // Since we'll update numRemainingSegments from totalSegments to 0 for each thread, so we need to add 1 here + val expectedCallTimes = expectedParams.values.map( num => num + 1 ).sum + verify(mockMap, times(expectedCallTimes)).put(capturedThreadName.capture(), capturedNumRemainingSegments.capture()); + + // expected the end value is 0 + logSegmentMetrics.foreach { gauge => assertEquals(0, gauge.value()) } + + val threadNames = capturedThreadName.getAllValues + val numRemainingSegments = capturedNumRemainingSegments.getAllValues + + expectedParams.foreach { + case (threadName, totalSegments) => + // make sure we update the numRemainingSegments from totalSegments to 0 in order for each thread + var expectedCurRemainingSegments = totalSegments + 1 + for (i <- 0 until threadNames.size) { + if (threadNames.get(i).contains(threadName)) { + expectedCurRemainingSegments -= 1 + assertEquals(expectedCurRemainingSegments, numRemainingSegments.get(i)) + } + } + assertEquals(0, expectedCurRemainingSegments) + } + } + + private def verifyLogRecoverMetricsRemoved(spyLogManager: LogManager): Unit = { + val spyLogManagerClassName = spyLogManager.getClass().getSimpleName + // get all `remainingLogsToRecover` metrics + def logMetrics: mutable.Set[MetricName] = KafkaYammerMetrics.defaultRegistry.allMetrics.keySet.asScala + .filter { metric => metric.getType == s"$spyLogManagerClassName" && metric.getName == "remainingLogsToRecover" } + + assertTrue(logMetrics.isEmpty) + + // get all `remainingSegmentsToRecover` metrics + val logSegmentMetrics: mutable.Set[MetricName] = KafkaYammerMetrics.defaultRegistry.allMetrics.keySet.asScala + .filter { metric => metric.getType == s"$spyLogManagerClassName" && metric.getName == "remainingSegmentsToRecover" } + + assertTrue(logSegmentMetrics.isEmpty) + } + + @Test + def testLogRecoveryMetrics(): Unit = { + logManager.shutdown() + val logDir1 = TestUtils.tempDir() + val logDir2 = TestUtils.tempDir() + val logDirs = Seq(logDir1, logDir2) + val recoveryThreadsPerDataDir = 2 + // create logManager with expected recovery thread number + logManager = createLogManager(logDirs, recoveryThreadsPerDataDir = recoveryThreadsPerDataDir) + val spyLogManager = spy(logManager) + + assertEquals(2, spyLogManager.liveLogDirs.size) + + val mockTime = new MockTime() + val mockMap = mock(classOf[ConcurrentHashMap[String, Int]]) + val mockBrokerTopicStats = mock(classOf[BrokerTopicStats]) + val expectedSegmentsPerLog = 2 + + // create log segments for log recovery in each log dir + appendRecordsToLog(mockTime, logDir1, 0, mockBrokerTopicStats, expectedSegmentsPerLog) + appendRecordsToLog(mockTime, logDir2, 1, mockBrokerTopicStats, expectedSegmentsPerLog) + + // intercept loadLog method to pass expected parameter to do log recovery + doAnswer { invocation => + val dir: File = invocation.getArgument(0) + val topicConfigOverrides: mutable.Map[String, LogConfig] = invocation.getArgument(5) + + val topicPartition = UnifiedLog.parseTopicPartitionName(dir) + val config = topicConfigOverrides.getOrElse(topicPartition.topic, logConfig) + + UnifiedLog( + dir = dir, + config = config, + logStartOffset = 0, + recoveryPoint = 0, + maxTransactionTimeoutMs = 5 * 60 * 1000, + maxProducerIdExpirationMs = 5 * 60 * 1000, + producerIdExpirationCheckIntervalMs = LogManager.ProducerIdExpirationCheckIntervalMs, + scheduler = mockTime.scheduler, + time = mockTime, + brokerTopicStats = mockBrokerTopicStats, + logDirFailureChannel = mock(classOf[LogDirFailureChannel]), + // not clean shutdown + lastShutdownClean = false, + topicId = None, + keepPartitionMetadataFile = false, + // pass mock map for verification later + numRemainingSegments = mockMap) + + }.when(spyLogManager).loadLog(any[File], any[Boolean], any[Map[TopicPartition, Long]], any[Map[TopicPartition, Long]], + any[LogConfig], any[Map[String, LogConfig]], any[ConcurrentMap[String, Int]]) + + // do nothing for removeLogRecoveryMetrics for metrics verification + doNothing().when(spyLogManager).removeLogRecoveryMetrics() + + // start the logManager to do log recovery + spyLogManager.startup(Set.empty) + + // make sure log recovery metrics are added and removed + verify(spyLogManager, times(1)).addLogRecoveryMetrics(any[ConcurrentMap[String, Int]], any[ConcurrentMap[String, Int]]) + verify(spyLogManager, times(1)).removeLogRecoveryMetrics() + + // expected 1 log in each log dir since we created 2 partitions with 2 log dirs + val expectedRemainingLogsParams = Map[String, Int](logDir1.getAbsolutePath -> 1, logDir2.getAbsolutePath -> 1) + verifyRemainingLogsToRecoverMetric(spyLogManager, expectedRemainingLogsParams) + + val expectedRemainingSegmentsParams = Map[String, Int]( + logDir1.getAbsolutePath -> expectedSegmentsPerLog, logDir2.getAbsolutePath -> expectedSegmentsPerLog) + verifyRemainingSegmentsToRecoverMetric(spyLogManager, logDirs, recoveryThreadsPerDataDir, mockMap, expectedRemainingSegmentsParams) + } + + @Test + def testLogRecoveryMetricsShouldBeRemovedAfterLogRecovered(): Unit = { + logManager.shutdown() + val logDir1 = TestUtils.tempDir() + val logDir2 = TestUtils.tempDir() + val logDirs = Seq(logDir1, logDir2) + val recoveryThreadsPerDataDir = 2 + // create logManager with expected recovery thread number + logManager = createLogManager(logDirs, recoveryThreadsPerDataDir = recoveryThreadsPerDataDir) + val spyLogManager = spy(logManager) + + assertEquals(2, spyLogManager.liveLogDirs.size) + + // start the logManager to do log recovery + spyLogManager.startup(Set.empty) + + // make sure log recovery metrics are added and removed once + verify(spyLogManager, times(1)).addLogRecoveryMetrics(any[ConcurrentMap[String, Int]], any[ConcurrentMap[String, Int]]) + verify(spyLogManager, times(1)).removeLogRecoveryMetrics() + + verifyLogRecoverMetricsRemoved(spyLogManager) + } + @Test def testMetricsExistWhenLogIsRecreatedBeforeDeletion(): Unit = { val topicName = "metric-test" diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index f6b58d78ce8cc..50af76f556ca2 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -28,6 +28,7 @@ import org.apache.kafka.common.utils.{Time, Utils} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse} import java.nio.file.Files +import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} import scala.collection.Iterable import scala.jdk.CollectionConverters._ @@ -83,7 +84,8 @@ object LogTestUtils { producerIdExpirationCheckIntervalMs: Int = LogManager.ProducerIdExpirationCheckIntervalMs, lastShutdownClean: Boolean = true, topicId: Option[Uuid] = None, - keepPartitionMetadataFile: Boolean = true): UnifiedLog = { + keepPartitionMetadataFile: Boolean = true, + numRemainingSegments: ConcurrentMap[String, Int] = new ConcurrentHashMap[String, Int]): UnifiedLog = { UnifiedLog( dir = dir, config = config, @@ -98,7 +100,8 @@ object LogTestUtils { logDirFailureChannel = new LogDirFailureChannel(10), lastShutdownClean = lastShutdownClean, topicId = topicId, - keepPartitionMetadataFile = keepPartitionMetadataFile + keepPartitionMetadataFile = keepPartitionMetadataFile, + numRemainingSegments = numRemainingSegments ) } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 5a8d43795ae8a..c49a7bdde09be 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -1281,13 +1281,14 @@ object TestUtils extends Logging { configRepository: ConfigRepository = new MockConfigRepository, cleanerConfig: CleanerConfig = CleanerConfig(enableCleaner = false), time: MockTime = new MockTime(), - interBrokerProtocolVersion: MetadataVersion = MetadataVersion.latest): LogManager = { + interBrokerProtocolVersion: MetadataVersion = MetadataVersion.latest, + recoveryThreadsPerDataDir: Int = 4): LogManager = { new LogManager(logDirs = logDirs.map(_.getAbsoluteFile), initialOfflineDirs = Array.empty[File], configRepository = configRepository, initialDefaultConfig = defaultConfig, cleanerConfig = cleanerConfig, - recoveryThreadsPerDataDir = 4, + recoveryThreadsPerDataDir = recoveryThreadsPerDataDir, flushCheckMs = 1000L, flushRecoveryOffsetCheckpointMs = 10000L, flushStartOffsetCheckpointMs = 10000L, diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java index 3bf65afc22224..99fb8143274eb 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java @@ -108,7 +108,7 @@ public void setup() { this.logManager = TestUtils.createLogManager(JavaConverters.asScalaBuffer(files), LogConfig.apply(), new MockConfigRepository(), CleanerConfig.apply(1, 4 * 1024 * 1024L, 0.9d, 1024 * 1024, 32 * 1024 * 1024, - Double.MAX_VALUE, 15 * 1000, true, "MD5"), time, MetadataVersion.latest()); + Double.MAX_VALUE, 15 * 1000, true, "MD5"), time, MetadataVersion.latest(), 4); scheduler.startup(); final BrokerTopicStats brokerTopicStats = new BrokerTopicStats(); final MetadataCache metadataCache = From 1f34bde3d4c9d75a711bd014e3920711a27ab4aa Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Mon, 25 Jul 2022 09:30:38 -0400 Subject: [PATCH 015/175] KAFKA-14093: Use single-worker Connect cluster when testing fenced leader recovery (#12433) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Reviewers: Mickael Maison 

, Tom Bentley --- .../ExactlyOnceSourceIntegrationTest.java | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java index 25a419ed8f8f3..90fcaa8a21447 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java @@ -76,6 +76,7 @@ import java.util.stream.LongStream; import static org.apache.kafka.clients.producer.ProducerConfig.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.kafka.clients.producer.ProducerConfig.CLIENT_ID_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.TRANSACTIONAL_ID_CONFIG; import static org.apache.kafka.connect.integration.MonitorableSourceConnector.CUSTOM_EXACTLY_ONCE_SUPPORT_CONFIG; @@ -466,6 +467,7 @@ public void testConnectorBoundary() throws Exception { */ @Test public void testFencedLeaderRecovery() throws Exception { + connectBuilder.numWorkers(1); // Much slower offset commit interval; should never be triggered during this test workerProps.put(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG, "600000"); startConnect(); @@ -494,7 +496,10 @@ public void testFencedLeaderRecovery() throws Exception { assertEquals(404, connect.requestGet(connect.endpointForResource("connectors/nonexistent")).getStatus()); // fence out the leader of the cluster - Producer zombieLeader = transactionalProducer(DistributedConfig.transactionalProducerId(CLUSTER_GROUP_ID)); + Producer zombieLeader = transactionalProducer( + "simulated-zombie-leader", + DistributedConfig.transactionalProducerId(CLUSTER_GROUP_ID) + ); zombieLeader.initTransactions(); zombieLeader.close(); @@ -1030,9 +1035,10 @@ private void assertProducersAreFencedOnReconfiguration( // create a collection of producers that simulate the producers used for the existing tasks List> producers = IntStream.range(0, currentNumTasks) - .mapToObj(i -> Worker.taskTransactionalId(CLUSTER_GROUP_ID, CONNECTOR_NAME, i)) - .map(this::transactionalProducer) - .collect(Collectors.toList()); + .mapToObj(i -> transactionalProducer( + "simulated-task-producer-" + CONNECTOR_NAME + "-" + i, + Worker.taskTransactionalId(CLUSTER_GROUP_ID, CONNECTOR_NAME, i) + )).collect(Collectors.toList()); producers.forEach(KafkaProducer::initTransactions); @@ -1047,8 +1053,9 @@ private void assertProducersAreFencedOnReconfiguration( producers.forEach(producer -> assertTransactionalProducerIsFenced(producer, topic)); } - private KafkaProducer transactionalProducer(String transactionalId) { + private KafkaProducer transactionalProducer(String clientId, String transactionalId) { Map transactionalProducerProps = new HashMap<>(); + transactionalProducerProps.put(CLIENT_ID_CONFIG, clientId); transactionalProducerProps.put(ENABLE_IDEMPOTENCE_CONFIG, true); transactionalProducerProps.put(TRANSACTIONAL_ID_CONFIG, transactionalId); return connect.kafka().createProducer(transactionalProducerProps); From ebd63b54bddae886f8125d904e1676333e8f4e58 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Mon, 25 Jul 2022 13:24:40 -0700 Subject: [PATCH 016/175] KAFKA-14078; Do leader/epoch validation in Fetch before checking for valid replica (#12411) After the fix for https://github.com/apache/kafka/pull/12150, if a follower receives a request from another replica, it will return UNKNOWN_LEADER_EPOCH even if the leader epoch matches. We need to do epoch leader/epoch validation first before we check whether we have a valid replica. Reviewers: David Jacot --- .../main/scala/kafka/cluster/Partition.scala | 36 +++++++++++----- .../unit/kafka/cluster/PartitionTest.scala | 41 +++++++++++++++++++ 2 files changed, 66 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 319025226c7e1..538c51f9035c7 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -1205,22 +1205,32 @@ class Partition(val topicPartition: TopicPartition, minOneMessage: Boolean, updateFetchState: Boolean ): LogReadInfo = { - def readFromLocalLog(): LogReadInfo = { + def readFromLocalLog(log: UnifiedLog): LogReadInfo = { readRecords( + log, fetchPartitionData.lastFetchedEpoch, fetchPartitionData.fetchOffset, fetchPartitionData.currentLeaderEpoch, maxBytes, fetchParams.isolation, - minOneMessage, - fetchParams.fetchOnlyLeader + minOneMessage ) } if (fetchParams.isFromFollower) { // Check that the request is from a valid replica before doing the read - val replica = followerReplicaOrThrow(fetchParams.replicaId, fetchPartitionData) - val logReadInfo = readFromLocalLog() + val (replica, logReadInfo) = inReadLock(leaderIsrUpdateLock) { + val localLog = localLogWithEpochOrThrow( + fetchPartitionData.currentLeaderEpoch, + fetchParams.fetchOnlyLeader + ) + val replica = followerReplicaOrThrow( + fetchParams.replicaId, + fetchPartitionData + ) + val logReadInfo = readFromLocalLog(localLog) + (replica, logReadInfo) + } if (updateFetchState && logReadInfo.divergingEpoch.isEmpty) { updateFollowerFetchState( @@ -1234,7 +1244,13 @@ class Partition(val topicPartition: TopicPartition, logReadInfo } else { - readFromLocalLog() + inReadLock(leaderIsrUpdateLock) { + val localLog = localLogWithEpochOrThrow( + fetchPartitionData.currentLeaderEpoch, + fetchParams.fetchOnlyLeader + ) + readFromLocalLog(localLog) + } } } @@ -1270,16 +1286,14 @@ class Partition(val topicPartition: TopicPartition, } private def readRecords( + localLog: UnifiedLog, lastFetchedEpoch: Optional[Integer], fetchOffset: Long, currentLeaderEpoch: Optional[Integer], maxBytes: Int, fetchIsolation: FetchIsolation, - minOneMessage: Boolean, - fetchOnlyFromLeader: Boolean - ): LogReadInfo = inReadLock(leaderIsrUpdateLock) { - val localLog = localLogWithEpochOrThrow(currentLeaderEpoch, fetchOnlyFromLeader) - + minOneMessage: Boolean + ): LogReadInfo = { // Note we use the log end offset prior to the read. This ensures that any appends following // the fetch do not prevent a follower from coming into sync. val initialHighWatermark = localLog.highWatermark diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index 65a6cdadf433a..5038219579440 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -238,6 +238,47 @@ class PartitionTest extends AbstractPartitionTest { assertEquals(None, partition.futureLog) } + @Test + def testReplicaFetchToFollower(): Unit = { + val controllerEpoch = 3 + val followerId = brokerId + 1 + val leaderId = brokerId + 2 + val replicas = List[Integer](brokerId, followerId, leaderId).asJava + val isr = List[Integer](brokerId, followerId, leaderId).asJava + val leaderEpoch = 8 + val partitionEpoch = 1 + + assertTrue(partition.makeFollower(new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(leaderId) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setPartitionEpoch(partitionEpoch) + .setReplicas(replicas) + .setIsNew(true), + offsetCheckpoints, None + )) + + def assertFetchFromReplicaFails[T <: ApiException]( + expectedExceptionClass: Class[T], + leaderEpoch: Option[Int] + ): Unit = { + assertThrows(expectedExceptionClass, () => { + fetchFollower( + partition, + replicaId = followerId, + fetchOffset = 0L, + leaderEpoch = leaderEpoch + ) + }) + } + + assertFetchFromReplicaFails(classOf[NotLeaderOrFollowerException], None) + assertFetchFromReplicaFails(classOf[NotLeaderOrFollowerException], Some(leaderEpoch)) + assertFetchFromReplicaFails(classOf[UnknownLeaderEpochException], Some(leaderEpoch + 1)) + assertFetchFromReplicaFails(classOf[FencedLeaderEpochException], Some(leaderEpoch - 1)) + } + @Test def testFetchFromUnrecognizedFollower(): Unit = { val controllerEpoch = 3 From 8ffd1b4ddb10f4bd4bb837efa1a2e17d77b3a616 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Fri, 15 Jul 2022 15:48:35 -0400 Subject: [PATCH 017/175] KAFKA-14039 Fix AlterConfigPolicy usage in KRaft (#12374) Only pass configs from the request to the AlterConfigPolicy. This changes the KRaft usage of the AlterConfigPolicy to match the usage in ZK mode. Reviewers: Jason Gustafson --- ...minClientWithPoliciesIntegrationTest.scala | 40 +++++++++++++++++-- .../ConfigurationControlManager.java | 14 ++++--- .../ConfigurationControlManagerTest.java | 26 +++++++++--- 3 files changed, 64 insertions(+), 16 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala index 4d48bf5a8656b..5b2213a65e962 100644 --- a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala @@ -20,17 +20,19 @@ import kafka.log.LogConfig import kafka.server.{Defaults, KafkaConfig} import kafka.utils.TestUtils.assertFutureExceptionTypeEquals import kafka.utils.{Logging, TestInfoUtils, TestUtils} -import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterConfigsOptions, Config, ConfigEntry} +import org.apache.kafka.clients.admin.AlterConfigOp.OpType +import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterConfigOp, AlterConfigsOptions, Config, ConfigEntry} import org.apache.kafka.common.config.{ConfigResource, TopicConfig} import org.apache.kafka.common.errors.{InvalidConfigurationException, InvalidRequestException, PolicyViolationException} import org.apache.kafka.common.utils.Utils import org.apache.kafka.server.policy.AlterConfigPolicy -import org.junit.jupiter.api.Assertions.{assertEquals, assertNull} +import org.junit.jupiter.api.Assertions.{assertEquals, assertNull, assertTrue} import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo, Timeout} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource import scala.annotation.nowarn +import scala.collection.mutable import scala.jdk.CollectionConverters._ /** @@ -121,6 +123,14 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with val topicResource3 = new ConfigResource(ConfigResource.Type.TOPIC, topic3) createTopic(topic3, 1, 1) + // Set a mutable broker config + val brokerResource = new ConfigResource(ConfigResource.Type.BROKER, brokers.head.config.brokerId.toString) + val brokerConfigs = Seq(new ConfigEntry(KafkaConfig.MessageMaxBytesProp, "50000")).asJava + val alterResult1 = client.alterConfigs(Map(brokerResource -> new Config(brokerConfigs)).asJava) + alterResult1.all.get + assertEquals(Set(KafkaConfig.MessageMaxBytesProp), validationsForResource(brokerResource).head.configs().keySet().asScala) + validations.clear() + val topicConfigEntries1 = Seq( new ConfigEntry(LogConfig.MinCleanableDirtyRatioProp, "0.9"), new ConfigEntry(LogConfig.MinInSyncReplicasProp, "2") // policy doesn't allow this @@ -130,7 +140,6 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with val topicConfigEntries3 = Seq(new ConfigEntry(LogConfig.MinInSyncReplicasProp, "-1")).asJava - val brokerResource = new ConfigResource(ConfigResource.Type.BROKER, brokers.head.config.brokerId.toString) val brokerConfigEntries = Seq(new ConfigEntry(KafkaConfig.SslTruststorePasswordProp, "12313")).asJava // Alter configs: second is valid, the others are invalid @@ -146,6 +155,9 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with alterResult.values.get(topicResource2).get assertFutureExceptionTypeEquals(alterResult.values.get(topicResource3), classOf[InvalidConfigurationException]) assertFutureExceptionTypeEquals(alterResult.values.get(brokerResource), classOf[InvalidRequestException]) + assertTrue(validationsForResource(brokerResource).isEmpty, + "Should not see the broker resource in the AlterConfig policy when the broker configs are not being updated.") + validations.clear() // Verify that the second resource was updated and the others were not ensureConsistentKRaftMetadata() @@ -175,6 +187,9 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with alterResult.values.get(topicResource2).get assertFutureExceptionTypeEquals(alterResult.values.get(topicResource3), classOf[InvalidConfigurationException]) assertFutureExceptionTypeEquals(alterResult.values.get(brokerResource), classOf[InvalidRequestException]) + assertTrue(validationsForResource(brokerResource).isEmpty, + "Should not see the broker resource in the AlterConfig policy when the broker configs are not being updated.") + validations.clear() // Verify that no resources are updated since validate_only = true ensureConsistentKRaftMetadata() @@ -188,27 +203,44 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with assertEquals("0.8", configs.get(topicResource2).get(LogConfig.MinCleanableDirtyRatioProp).value) assertNull(configs.get(brokerResource).get(KafkaConfig.SslTruststorePasswordProp).value) + + // Do an incremental alter config on the broker, ensure we don't see the broker config we set earlier in the policy + alterResult = client.incrementalAlterConfigs(Map( + brokerResource -> + Seq(new AlterConfigOp( + new ConfigEntry(KafkaConfig.MaxConnectionsProp, "9999"), OpType.SET) + ).asJavaCollection + ).asJava) + alterResult.all.get + assertEquals(Set(KafkaConfig.MaxConnectionsProp), validationsForResource(brokerResource).head.configs().keySet().asScala) } } object AdminClientWithPoliciesIntegrationTest { + val validations = new mutable.ListBuffer[AlterConfigPolicy.RequestMetadata]() + + def validationsForResource(resource: ConfigResource): Seq[AlterConfigPolicy.RequestMetadata] = { + validations.filter { req => req.resource().equals(resource) }.toSeq + } + class Policy extends AlterConfigPolicy { var configs: Map[String, _] = _ var closed = false def configure(configs: util.Map[String, _]): Unit = { + validations.clear() this.configs = configs.asScala.toMap } def validate(requestMetadata: AlterConfigPolicy.RequestMetadata): Unit = { + validations.append(requestMetadata) require(!closed, "Policy should not be closed") require(!configs.isEmpty, "configure should have been called with non empty configs") require(!requestMetadata.configs.isEmpty, "request configs should not be empty") require(requestMetadata.resource.name.nonEmpty, "resource name should not be empty") - require(requestMetadata.resource.name.contains("topic")) if (requestMetadata.configs.containsKey(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG)) throw new PolicyViolationException("Min in sync replicas cannot be updated") } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java index cde9d39569232..4b8561a4d9089 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java @@ -248,24 +248,26 @@ private void incrementalAlterConfigResource(ConfigResource configResource, private ApiError validateAlterConfig(ConfigResource configResource, List newRecords, boolean newlyCreatedResource) { - Map newConfigs = new HashMap<>(); + Map allConfigs = new HashMap<>(); + Map alteredConfigs = new HashMap<>(); TimelineHashMap existingConfigs = configData.get(configResource); - if (existingConfigs != null) newConfigs.putAll(existingConfigs); + if (existingConfigs != null) allConfigs.putAll(existingConfigs); for (ApiMessageAndVersion newRecord : newRecords) { ConfigRecord configRecord = (ConfigRecord) newRecord.message(); if (configRecord.value() == null) { - newConfigs.remove(configRecord.name()); + allConfigs.remove(configRecord.name()); } else { - newConfigs.put(configRecord.name(), configRecord.value()); + allConfigs.put(configRecord.name(), configRecord.value()); } + alteredConfigs.put(configRecord.name(), configRecord.value()); } try { - validator.validate(configResource, newConfigs); + validator.validate(configResource, allConfigs); if (!newlyCreatedResource) { existenceChecker.accept(configResource); } if (alterConfigPolicy.isPresent()) { - alterConfigPolicy.get().validate(new RequestMetadata(configResource, newConfigs)); + alterConfigPolicy.get().validate(new RequestMetadata(configResource, alteredConfigs)); } } catch (ConfigException e) { return new ApiError(INVALID_CONFIG, e.getMessage()); diff --git a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java index 007e84ffc0db8..1c59892444844 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java @@ -281,18 +281,30 @@ public void configure(Map configs) { public void testIncrementalAlterConfigsWithPolicy() { MockAlterConfigsPolicy policy = new MockAlterConfigsPolicy(asList( new RequestMetadata(MYTOPIC, Collections.emptyMap()), - new RequestMetadata(BROKER0, toMap(entry("foo.bar", "123"), - entry("quux", "456"))))); + new RequestMetadata(BROKER0, toMap( + entry("foo.bar", "123"), + entry("quux", "456"), + entry("broker.config.to.remove", null))))); ConfigurationControlManager manager = new ConfigurationControlManager.Builder(). setKafkaConfigSchema(SCHEMA). setAlterConfigPolicy(Optional.of(policy)). build(); + // Existing configs should not be passed to the policy + manager.replay(new ConfigRecord().setResourceType(BROKER.id()).setResourceName("0"). + setName("broker.config").setValue("123")); + manager.replay(new ConfigRecord().setResourceType(TOPIC.id()).setResourceName(MYTOPIC.name()). + setName("topic.config").setValue("123")); + manager.replay(new ConfigRecord().setResourceType(BROKER.id()).setResourceName("0"). + setName("broker.config.to.remove").setValue("123")); assertEquals(ControllerResult.atomicOf(asList(new ApiMessageAndVersion( new ConfigRecord().setResourceType(BROKER.id()).setResourceName("0"). setName("foo.bar").setValue("123"), CONFIG_RECORD.highestSupportedVersion()), new ApiMessageAndVersion( new ConfigRecord().setResourceType(BROKER.id()).setResourceName("0"). - setName("quux").setValue("456"), CONFIG_RECORD.highestSupportedVersion())), - toMap(entry(MYTOPIC, new ApiError(Errors.POLICY_VIOLATION, + setName("quux").setValue("456"), CONFIG_RECORD.highestSupportedVersion()), new ApiMessageAndVersion( + new ConfigRecord().setResourceType(BROKER.id()).setResourceName("0"). + setName("broker.config.to.remove").setValue(null), CONFIG_RECORD.highestSupportedVersion()) + ), + toMap(entry(MYTOPIC, new ApiError(Errors.POLICY_VIOLATION, "Expected: AlterConfigPolicy.RequestMetadata(resource=ConfigResource(" + "type=TOPIC, name='mytopic'), configs={}). Got: " + "AlterConfigPolicy.RequestMetadata(resource=ConfigResource(" + @@ -301,8 +313,10 @@ public void testIncrementalAlterConfigsWithPolicy() { manager.incrementalAlterConfigs(toMap(entry(MYTOPIC, toMap( entry("foo.bar", entry(SET, "123")))), entry(BROKER0, toMap( - entry("foo.bar", entry(SET, "123")), - entry("quux", entry(SET, "456"))))), + entry("foo.bar", entry(SET, "123")), + entry("quux", entry(SET, "456")), + entry("broker.config.to.remove", entry(DELETE, null)) + ))), true)); } From 64ebc76df96cdc9cea419c0ab09cfa82d0ca743c Mon Sep 17 00:00:00 2001 From: "Christopher L. Shannon" Date: Mon, 18 Jul 2022 18:07:20 -0400 Subject: [PATCH 018/175] KAFKA-14079 - Ack failed records in WorkerSourceTask when error tolerance is ALL (#12415) Make sure to ack all records where produce failed, when a connector's `errors.tolerance` config property is set to `all`. Acking is essential so that the task will continue to commit future record offsets properly and remove the records from internal tracking, preventing a memory leak. (cherry picked and slightly modified from commit 63e06aafd0cf37f8488c3830946051b3a30db2a0) Reviewers: Chris Egerton , Randall Hauch --- .../runtime/AbstractWorkerSourceTask.java | 5 +++++ .../connect/runtime/WorkerSourceTaskTest.java | 21 ++++++++++++++++--- 2 files changed, 23 insertions(+), 3 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java index d89f577688f35..407f5fd828b84 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java @@ -38,6 +38,7 @@ import org.apache.kafka.connect.header.Headers; import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator; import org.apache.kafka.connect.runtime.errors.Stage; +import org.apache.kafka.connect.runtime.errors.ToleranceType; import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.source.SourceTask; import org.apache.kafka.connect.source.SourceTaskContext; @@ -406,6 +407,10 @@ boolean sendRecords() { } log.trace("{} Failed record: {}", AbstractWorkerSourceTask.this, preTransformRecord); producerSendFailed(false, producerRecord, preTransformRecord, e); + if (retryWithToleranceOperator.getErrorToleranceType() == ToleranceType.ALL) { + counter.skipRecord(); + submittedRecord.ifPresent(SubmittedRecords.SubmittedRecord::ack); + } } else { counter.completeRecord(); log.trace("{} Wrote record successfully: topic {} partition {} offset {}", diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java index 5ce0e44f3ea90..2d2cd00cf5348 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java @@ -696,24 +696,39 @@ public void testSourceTaskIgnoresProducerException() throws Exception { createWorkerTaskWithErrorToleration(); expectTopicCreation(TOPIC); + //Use different offsets for each record so we can verify all were committed + final Map offset2 = Collections.singletonMap("key", 13); + // send two records // record 1 will succeed // record 2 will invoke the producer's failure callback, but ignore the exception via retryOperator // and no ConnectException will be thrown SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - + SourceRecord record2 = new SourceRecord(PARTITION, offset2, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + expectOffsetFlush(true); expectSendRecordOnce(); expectSendRecordProducerCallbackFail(); sourceTask.commitRecord(EasyMock.anyObject(SourceRecord.class), EasyMock.isNull()); - EasyMock.expectLastCall(); + + //As of KAFKA-14079 all offsets should be committed, even for failed records (if ignored) + //Only the last offset will be passed to the method as everything up to that point is committed + //Before KAFKA-14079 offset 12 would have been passed and not 13 as it would have been unacked + offsetWriter.offset(PARTITION, offset2); + PowerMock.expectLastCall(); PowerMock.replayAll(); + //Send records and then commit offsets and verify both were committed and no exception Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); Whitebox.invokeMethod(workerTask, "sendRecords"); + Whitebox.invokeMethod(workerTask, "updateCommittableOffsets"); + workerTask.commitOffsets(); PowerMock.verifyAll(); + + //Double check to make sure all submitted records were cleared + assertEquals(0, ((SubmittedRecords) Whitebox.getInternalState(workerTask, + "submittedRecords")).records.size()); } @Test From dc0866557c5051b8711df67d68621a3306521b10 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 28 Jul 2022 11:18:09 -0400 Subject: [PATCH 019/175] KAFKA-14089: Only check for committed seqnos after disabling exactly-once support in Connect integration test (#12429) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Reviewers: Mickael Maison 

, Tom Bentley --- .../runtime/ExactlyOnceWorkerSourceTask.java | 3 + .../apache/kafka/connect/runtime/Worker.java | 13 ++ .../ExactlyOnceSourceIntegrationTest.java | 192 +++++++++++++----- .../MonitorableSourceConnector.java | 2 +- 4 files changed, 155 insertions(+), 55 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java index de78e592aa2a1..931917b9e15ce 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java @@ -217,6 +217,9 @@ protected void finalOffsetCommit(boolean failed) { if (failed) { log.debug("Skipping final offset commit as task has failed"); return; + } else if (isCancelled()) { + log.debug("Skipping final offset commit as task has been cancelled"); + return; } // It should be safe to commit here even if we were in the middle of retrying on RetriableExceptions in the diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java index 16e48d8f17edf..5bc67693d0a66 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java @@ -756,6 +756,19 @@ static Map exactlyOnceSourceTaskProducerConfigs(ConnectorTaskId ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy, String clusterId) { Map result = baseProducerConfigs(id.connector(), "connector-producer-" + id, config, connConfig, connectorClass, connectorClientConfigOverridePolicy, clusterId); + // The base producer properties forcibly disable idempotence; remove it from those properties + // if not explicitly requested by the user + boolean connectorProducerIdempotenceConfigured = connConfig.originals().containsKey( + ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX + ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG + ); + if (!connectorProducerIdempotenceConfigured) { + boolean workerProducerIdempotenceConfigured = config.originals().containsKey( + "producer." + ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG + ); + if (!workerProducerIdempotenceConfigured) { + result.remove(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG); + } + } ConnectUtils.ensureProperty( result, ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true", "for connectors when exactly-once source support is enabled", diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java index 90fcaa8a21447..bd9bceba064fd 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java @@ -407,12 +407,12 @@ public void testConnectorBoundary() throws Exception { Map consumerProps = new HashMap<>(); consumerProps.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); // consume all records from the source topic or fail, to ensure that they were correctly produced - ConsumerRecords sourceRecords = connect.kafka() - .consume( - recordsProduced, - TimeUnit.MINUTES.toMillis(1), - consumerProps, - "test-topic"); + ConsumerRecords sourceRecords = connect.kafka().consumeAll( + CONSUME_RECORDS_TIMEOUT_MS, + Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"), + null, + topic + ); assertTrue("Not enough records produced by source connector. Expected at least: " + recordsProduced + " + but got " + sourceRecords.count(), sourceRecords.count() >= recordsProduced); @@ -434,8 +434,7 @@ public void testConnectorBoundary() throws Exception { offsetsTopic ); - List actualOffsetSeqnos = new ArrayList<>(); - offsetRecords.forEach(record -> actualOffsetSeqnos.add(parseAndAssertOffsetForSingleTask(record))); + List actualOffsetSeqnos = parseAndAssertOffsetsForSingleTask(offsetRecords); assertEquals("Committed offsets should match connector-defined transaction boundaries", expectedOffsetSeqnos, actualOffsetSeqnos.subList(0, expectedOffsetSeqnos.size())); @@ -716,6 +715,20 @@ public void testTasksFailOnInabilityToFence() throws Exception { connect.assertions().assertConnectorAndExactlyNumTasksAreRunning(CONNECTOR_NAME, tasksMax, "Connector and task should have restarted successfully"); } + /** + * This test focuses extensively on the per-connector offsets feature. + *

+ * First, a connector is brought up whose producer is configured to write to a different Kafka cluster + * than the one the Connect cluster users for its internal topics, then the contents of the connector's + * dedicated offsets topic and the worker's internal offsets topic are inspected to ensure that offsets + * have been backed up from the dedicated topic to the global topic. + *

+ * Then, a "soft downgrade" is simulated: the Connect cluster is shut down and reconfigured to disable + * exactly-once support. The cluster is brought up again, the connector is allowed to produce some data, + * the connector is shut down, and this time, the records the connector has produced are inspected for + * accuracy. Because of the downgrade, exactly-once guarantees are lost, but we check to make sure that + * the task has maintained exactly-once delivery up to the last-committed record. + */ @Test public void testSeparateOffsetsTopic() throws Exception { final String globalOffsetsTopic = "connect-worker-offsets-topic"; @@ -761,7 +774,7 @@ public void testSeparateOffsetsTopic() throws Exception { // wait for the connector tasks to commit enough records connectorHandle.awaitCommits(TimeUnit.MINUTES.toMillis(1)); - // consume all records from the source topic or fail, to ensure that they were correctly produced + // consume at least the expected number of records from the source topic or fail, to ensure that they were correctly produced int recordNum = connectorTargetedCluster .consume( recordsProduced, @@ -772,28 +785,33 @@ public void testSeparateOffsetsTopic() throws Exception { assertTrue("Not enough records produced by source connector. Expected at least: " + recordsProduced + " + but got " + recordNum, recordNum >= recordsProduced); - // also consume from the connector's dedicated offsets topic; just need to read one offset record - ConsumerRecord offsetRecord = connectorTargetedCluster - .consume( - 1, + // also consume from the connector's dedicated offsets topic + ConsumerRecords offsetRecords = connectorTargetedCluster + .consumeAll( TimeUnit.MINUTES.toMillis(1), Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"), + null, offsetsTopic - ).iterator().next(); - long seqno = parseAndAssertOffsetForSingleTask(offsetRecord); - assertEquals("Offset commits should occur on connector-defined poll boundaries, which happen every " + recordsProduced + " records", - 0, seqno % recordsProduced); + ); + List seqnos = parseAndAssertOffsetsForSingleTask(offsetRecords); + seqnos.forEach(seqno -> + assertEquals("Offset commits should occur on connector-defined poll boundaries, which happen every " + recordsProduced + " records", + 0, seqno % recordsProduced) + ); - // also consume from the cluster's global offsets topic; again, just need to read one offset record - offsetRecord = connect.kafka() - .consume( - 1, + // also consume from the cluster's global offsets topic + offsetRecords = connect.kafka() + .consumeAll( TimeUnit.MINUTES.toMillis(1), + null, + null, globalOffsetsTopic - ).iterator().next(); - seqno = parseAndAssertOffsetForSingleTask(offsetRecord); - assertEquals("Offset commits should occur on connector-defined poll boundaries, which happen every " + recordsProduced + " records", - 0, seqno % recordsProduced); + ); + seqnos = parseAndAssertOffsetsForSingleTask(offsetRecords); + seqnos.forEach(seqno -> + assertEquals("Offset commits should occur on connector-defined poll boundaries, which happen every " + recordsProduced + " records", + 0, seqno % recordsProduced) + ); // Shut down the whole cluster connect.workers().forEach(connect::removeWorker); @@ -831,15 +849,22 @@ public void testSeparateOffsetsTopic() throws Exception { assertConnectorStopped(connectorStop); // consume all records from the source topic or fail, to ensure that they were correctly produced - ConsumerRecords records = connectorTargetedCluster.consumeAll( + ConsumerRecords sourceRecords = connectorTargetedCluster.consumeAll( CONSUME_RECORDS_TIMEOUT_MS, Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"), null, topic ); - assertTrue("Not enough records produced by source connector. Expected at least: " + recordsProduced + " + but got " + records.count(), - records.count() >= recordsProduced); - assertExactlyOnceSeqnos(records, numTasks); + assertTrue("Not enough records produced by source connector. Expected at least: " + recordsProduced + " + but got " + sourceRecords.count(), + sourceRecords.count() >= recordsProduced); + // also have to check which offsets have actually been committed, since we no longer have exactly-once guarantees + offsetRecords = connectorTargetedCluster.consumeAll( + CONSUME_RECORDS_TIMEOUT_MS, + Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"), + null, + offsetsTopic + ); + assertAtLeastOnceSeqnos(sourceRecords, offsetRecords, numTasks); } } @@ -896,27 +921,10 @@ private ConfigInfo findConfigInfo(String property, ConfigInfos validationResult) .orElseThrow(() -> new AssertionError("Failed to find configuration validation result for property '" + property + "'")); } - @SuppressWarnings("unchecked") - private long parseAndAssertOffsetForSingleTask(ConsumerRecord offsetRecord) { - JsonConverter offsetsConverter = new JsonConverter(); - // The JSON converter behaves identically for keys and values. If that ever changes, we may need to update this test to use - // separate converter instances. - - offsetsConverter.configure(Collections.singletonMap(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "false"), false); - Object keyObject = offsetsConverter.toConnectData("topic name is not used by converter", offsetRecord.key()).value(); - Object valueObject = offsetsConverter.toConnectData("topic name is not used by converter", offsetRecord.value()).value(); - - assertNotNull("Offset value should not be null", valueObject); - - assertEquals("Serialized source partition should match expected format", - Arrays.asList(CONNECTOR_NAME, MonitorableSourceConnector.sourcePartition(MonitorableSourceConnector.taskId(CONNECTOR_NAME, 0))), - keyObject); - - Map value = assertAndCast(valueObject, Map.class, "Value"); - - Object seqnoObject = value.get("saved"); - assertNotNull("Serialized source offset should contain 'seqno' field from MonitorableSourceConnector", seqnoObject); - return assertAndCast(seqnoObject, Long.class, "Seqno offset field"); + private List parseAndAssertOffsetsForSingleTask(ConsumerRecords offsetRecords) { + Map> parsedOffsets = parseOffsetForTasks(offsetRecords); + assertEquals("Expected records to only be produced from a single task", Collections.singleton(0), parsedOffsets.keySet()); + return parsedOffsets.get(0); } private List parseAndAssertValuesForSingleTask(ConsumerRecords sourceRecords) { @@ -927,6 +935,25 @@ private List parseAndAssertValuesForSingleTask(ConsumerRecords sourceRecords, int numTasks) { Map> parsedValues = parseValuesForTasks(sourceRecords); + assertSeqnos(parsedValues, numTasks); + } + + private void assertAtLeastOnceSeqnos(ConsumerRecords sourceRecords, ConsumerRecords offsetRecords, int numTasks) { + Map> parsedValues = parseValuesForTasks(sourceRecords); + Map lastCommittedValues = parseOffsetForTasks(offsetRecords) + .entrySet().stream().collect(Collectors.toMap( + Map.Entry::getKey, + e -> Collections.max(e.getValue()) + )); + parsedValues.replaceAll((task, values) -> { + Long committedValue = lastCommittedValues.get(task); + assertNotNull("No committed offset found for task " + task, committedValue); + return values.stream().filter(v -> v <= committedValue).collect(Collectors.toList()); + }); + assertSeqnos(parsedValues, numTasks); + } + + private void assertSeqnos(Map> parsedValues, int numTasks) { Set expectedKeys = IntStream.range(0, numTasks).boxed().collect(Collectors.toSet()); assertEquals("Expected records to be produced by each task", expectedKeys, parsedValues.keySet()); @@ -935,10 +962,19 @@ private void assertExactlyOnceSeqnos(ConsumerRecords sourceRecor // which makes in-order consumption impossible Set expectedSeqnos = LongStream.range(1, seqnos.size() + 1).boxed().collect(Collectors.toSet()); Set actualSeqnos = new HashSet<>(seqnos); - assertEquals( - "Seqnos for task " + taskId + " should start at 1 and increase strictly by 1 with each record", - expectedSeqnos, - actualSeqnos + + Set missingSeqnos = new HashSet<>(expectedSeqnos); + missingSeqnos.removeAll(actualSeqnos); + Set extraSeqnos = new HashSet<>(actualSeqnos); + extraSeqnos.removeAll(expectedSeqnos); + + // Try to provide the most friendly error message possible if this test fails + assertTrue( + "Seqnos for task " + taskId + " should start at 1 and increase strictly by 1 with each record, " + + "but the actual seqnos did not.\n" + + "Seqnos that should have been emitted but were not: " + missingSeqnos + "\n" + + "seqnos that should not have been emitted but were: " + extraSeqnos, + missingSeqnos.isEmpty() && extraSeqnos.isEmpty() ); }); } @@ -986,6 +1022,54 @@ private Map> parseValuesForTasks(ConsumerRecords> parseOffsetForTasks(ConsumerRecords offsetRecords) { + JsonConverter offsetsConverter = new JsonConverter(); + // The JSON converter behaves identically for keys and values. If that ever changes, we may need to update this test to use + // separate converter instances. + offsetsConverter.configure(Collections.singletonMap(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "false"), false); + + Map> result = new HashMap<>(); + for (ConsumerRecord offsetRecord : offsetRecords) { + Object keyObject = offsetsConverter.toConnectData("topic name is not used by converter", offsetRecord.key()).value(); + Object valueObject = offsetsConverter.toConnectData("topic name is not used by converter", offsetRecord.value()).value(); + + assertNotNull("Offset key should not be null", keyObject); + assertNotNull("Offset value should not be null", valueObject); + + @SuppressWarnings("unchecked") + List key = assertAndCast(keyObject, List.class, "Key"); + assertEquals( + "Offset topic key should be a list containing two elements: the name of the connector, and the connector-provided source partition", + 2, + key.size() + ); + assertEquals(CONNECTOR_NAME, key.get(0)); + @SuppressWarnings("unchecked") + Map partition = assertAndCast(key.get(1), Map.class, "Key[1]"); + Object taskIdObject = partition.get("task.id"); + assertNotNull("Serialized source partition should contain 'task.id' field from MonitorableSourceConnector", taskIdObject); + String taskId = assertAndCast(taskIdObject, String.class, "task ID"); + assertTrue("task ID should match pattern '-", taskId.startsWith(CONNECTOR_NAME + "-")); + String taskIdRemainder = taskId.substring(CONNECTOR_NAME.length() + 1); + int taskNum; + try { + taskNum = Integer.parseInt(taskIdRemainder); + } catch (NumberFormatException e) { + throw new AssertionError("task ID should match pattern '-', where is an integer", e); + } + + @SuppressWarnings("unchecked") + Map value = assertAndCast(valueObject, Map.class, "Value"); + + Object seqnoObject = value.get("saved"); + assertNotNull("Serialized source offset should contain 'seqno' field from MonitorableSourceConnector", seqnoObject); + long seqno = assertAndCast(seqnoObject, Long.class, "Seqno offset field"); + + result.computeIfAbsent(taskNum, t -> new ArrayList<>()).add(seqno); + } + return result; + } + @SuppressWarnings("unchecked") private static T assertAndCast(Object o, Class klass, String objectDescription) { String className = o == null ? "null" : o.getClass().getName(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSourceConnector.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSourceConnector.java index c2820315d6b82..33ba1588a7d04 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSourceConnector.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSourceConnector.java @@ -172,7 +172,7 @@ public void start(Map props) { batchSize = Integer.parseInt(props.getOrDefault(MESSAGES_PER_POLL_CONFIG, "1")); taskHandle = RuntimeHandles.get().connectorHandle(connectorName).taskHandle(taskId); Map offset = Optional.ofNullable( - context.offsetStorageReader().offset(Collections.singletonMap("task.id", taskId))) + context.offsetStorageReader().offset(sourcePartition(taskId))) .orElse(Collections.emptyMap()); startingSeqno = Optional.ofNullable((Long) offset.get("saved")).orElse(0L); seqno = startingSeqno; From e607f10e97ad603c0024bc91d7ffc02051588f26 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Mon, 1 Aug 2022 10:32:27 -0700 Subject: [PATCH 020/175] MINOR; Synchronize access to snapshots' TreeMap (#12464) Read and write access to the TreeMap in snapshots needs to be synchronized. Reviewers: David Arthur --- core/src/main/scala/kafka/raft/KafkaMetadataLog.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index dba8975d437ed..83b8bee444795 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -257,7 +257,11 @@ final class KafkaMetadataLog private ( } override def storeSnapshot(snapshotId: OffsetAndEpoch): Optional[RawSnapshotWriter] = { - if (snapshots.contains(snapshotId)) { + val containsSnapshotId = snapshots synchronized { + snapshots.contains(snapshotId) + } + + if (containsSnapshotId) { Optional.empty() } else { Optional.of(FileRawSnapshotWriter.create(log.dir.toPath, snapshotId, Optional.of(this))) From a687d4d3f6874f7821996d644da6df9491bf9232 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Wed, 3 Aug 2022 13:28:06 -0400 Subject: [PATCH 021/175] KAFKA-14111 Fix sensitive dynamic broker configs in KRaft (#12455) Enable some of the dynamic broker reconfiguration tests in KRaft mode --- .../scala/kafka/admin/ConfigCommand.scala | 2 +- .../kafka/server/DynamicBrokerConfig.scala | 8 +- .../main/scala/kafka/server/KafkaBroker.scala | 1 + .../main/scala/kafka/server/KafkaConfig.scala | 2 +- .../scala/kafka/utils/PasswordEncoder.scala | 45 ++++-- .../DynamicBrokerReconfigurationTest.scala | 128 +++++++++++------- .../kafka/utils/PasswordEncoderTest.scala | 10 +- .../scala/unit/kafka/utils/TestUtils.scala | 25 ++-- 8 files changed, 145 insertions(+), 76 deletions(-) diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index 4676bfd1013a7..9a42f9b874dc3 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -211,7 +211,7 @@ object ConfigCommand extends Logging { encoderConfigs.get(KafkaConfig.PasswordEncoderSecretProp) val encoderSecret = encoderConfigs.getOrElse(KafkaConfig.PasswordEncoderSecretProp, throw new IllegalArgumentException("Password encoder secret not specified")) - new PasswordEncoder(new Password(encoderSecret), + PasswordEncoder.encrypting(new Password(encoderSecret), None, encoderConfigs.get(KafkaConfig.PasswordEncoderCipherAlgorithmProp).getOrElse(Defaults.PasswordEncoderCipherAlgorithm), encoderConfigs.get(KafkaConfig.PasswordEncoderKeyLengthProp).map(_.toInt).getOrElse(Defaults.PasswordEncoderKeyLength), diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 33511147e6b4c..76a42b74fa509 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -211,7 +211,11 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging private val brokerReconfigurables = new CopyOnWriteArrayList[BrokerReconfigurable]() private val lock = new ReentrantReadWriteLock private var currentConfig: KafkaConfig = null - private val dynamicConfigPasswordEncoder = maybeCreatePasswordEncoder(kafkaConfig.passwordEncoderSecret) + private val dynamicConfigPasswordEncoder = if (kafkaConfig.processRoles.isEmpty) { + maybeCreatePasswordEncoder(kafkaConfig.passwordEncoderSecret) + } else { + Some(PasswordEncoder.noop()) + } private[server] def initialize(zkClientOpt: Option[KafkaZkClient]): Unit = { currentConfig = new KafkaConfig(kafkaConfig.props, false, None) @@ -340,7 +344,7 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging private def maybeCreatePasswordEncoder(secret: Option[Password]): Option[PasswordEncoder] = { secret.map { secret => - new PasswordEncoder(secret, + PasswordEncoder.encrypting(secret, kafkaConfig.passwordEncoderKeyFactoryAlgorithm, kafkaConfig.passwordEncoderCipherAlgorithm, kafkaConfig.passwordEncoderKeyLength, diff --git a/core/src/main/scala/kafka/server/KafkaBroker.scala b/core/src/main/scala/kafka/server/KafkaBroker.scala index 46f2e7e8b19c7..b02b1167c50be 100644 --- a/core/src/main/scala/kafka/server/KafkaBroker.scala +++ b/core/src/main/scala/kafka/server/KafkaBroker.scala @@ -89,6 +89,7 @@ trait KafkaBroker extends KafkaMetricsGroup { def shutdown(): Unit def brokerTopicStats: BrokerTopicStats def credentialProvider: CredentialProvider + def clientToControllerChannelManager: BrokerToControllerChannelManager // For backwards compatibility, we need to keep older metrics tied // to their original name when this class was named `KafkaServer` diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index a9fbda6c21079..4e253047ee604 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -1493,6 +1493,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami // Cache the current config to avoid acquiring read lock to access from dynamicConfig @volatile private var currentConfig = this + val processRoles: Set[ProcessRole] = parseProcessRoles() private[server] val dynamicConfig = dynamicConfigOverride.getOrElse(new DynamicBrokerConfig(this)) private[server] def updateCurrentConfig(newConfig: KafkaConfig): Unit = { @@ -1612,7 +1613,6 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val maxReservedBrokerId: Int = getInt(KafkaConfig.MaxReservedBrokerIdProp) var brokerId: Int = getInt(KafkaConfig.BrokerIdProp) val nodeId: Int = getInt(KafkaConfig.NodeIdProp) - val processRoles: Set[ProcessRole] = parseProcessRoles() val initialRegistrationTimeoutMs: Int = getInt(KafkaConfig.InitialBrokerRegistrationTimeoutMsProp) val brokerHeartbeatIntervalMs: Int = getInt(KafkaConfig.BrokerHeartbeatIntervalMsProp) val brokerSessionTimeoutMs: Int = getInt(KafkaConfig.BrokerSessionTimeoutMsProp) diff --git a/core/src/main/scala/kafka/utils/PasswordEncoder.scala b/core/src/main/scala/kafka/utils/PasswordEncoder.scala index f748a455c62bf..3373223e36f1c 100644 --- a/core/src/main/scala/kafka/utils/PasswordEncoder.scala +++ b/core/src/main/scala/kafka/utils/PasswordEncoder.scala @@ -38,6 +38,33 @@ object PasswordEncoder { val IterationsProp = "iterations" val EncyrptedPasswordProp = "encryptedPassword" val PasswordLengthProp = "passwordLength" + + def encrypting(secret: Password, + keyFactoryAlgorithm: Option[String], + cipherAlgorithm: String, + keyLength: Int, + iterations: Int): EncryptingPasswordEncoder = { + new EncryptingPasswordEncoder(secret, keyFactoryAlgorithm, cipherAlgorithm, keyLength, iterations) + } + + def noop(): NoOpPasswordEncoder = { + new NoOpPasswordEncoder() + } +} + +trait PasswordEncoder { + def encode(password: Password): String + def decode(encodedPassword: String): Password + + private[utils] def base64Decode(encoded: String): Array[Byte] = Base64.getDecoder.decode(encoded) +} + +/** + * A password encoder that does not modify the given password. This is used in KRaft mode only. + */ +class NoOpPasswordEncoder extends PasswordEncoder { + override def encode(password: Password): String = password.value() + override def decode(encodedPassword: String): Password = new Password(encodedPassword) } /** @@ -55,16 +82,18 @@ object PasswordEncoder { * The values used for encoding are stored along with the encoded password and the stored values are used for decoding. * */ -class PasswordEncoder(secret: Password, - keyFactoryAlgorithm: Option[String], - cipherAlgorithm: String, - keyLength: Int, - iterations: Int) extends Logging { +class EncryptingPasswordEncoder( + secret: Password, + keyFactoryAlgorithm: Option[String], + cipherAlgorithm: String, + keyLength: Int, + iterations: Int +) extends PasswordEncoder with Logging { private val secureRandom = new SecureRandom private val cipherParamsEncoder = cipherParamsInstance(cipherAlgorithm) - def encode(password: Password): String = { + override def encode(password: Password): String = { val salt = new Array[Byte](256) secureRandom.nextBytes(salt) val cipher = Cipher.getInstance(cipherAlgorithm) @@ -84,7 +113,7 @@ class PasswordEncoder(secret: Password, encryptedMap.map { case (k, v) => s"$k:$v" }.mkString(",") } - def decode(encodedPassword: String): Password = { + override def decode(encodedPassword: String): Password = { val params = CoreUtils.parseCsvMap(encodedPassword) val keyFactoryAlg = params(KeyFactoryAlgorithmProp) val cipherAlg = params(CipherAlgorithmProp) @@ -131,8 +160,6 @@ class PasswordEncoder(secret: Password, private def base64Encode(bytes: Array[Byte]): String = Base64.getEncoder.encodeToString(bytes) - private[utils] def base64Decode(encoded: String): Array[Byte] = Base64.getDecoder.decode(encoded) - private def cipherParamsInstance(cipherAlgorithm: String): CipherParamsEncoder = { val aesPattern = "AES/(.*)/.*".r cipherAlgorithm match { diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index ccfe63e7b566d..c7be8ce831da8 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -31,7 +31,7 @@ import com.yammer.metrics.core.MetricName import kafka.admin.ConfigCommand import kafka.api.{KafkaSasl, SaslSetup} import kafka.controller.{ControllerBrokerStateInfo, ControllerChannelManager} -import kafka.log.{CleanerConfig, LogConfig} +import kafka.log.{CleanerConfig, LogConfig, UnifiedLog} import kafka.message.ProducerCompressionCodec import kafka.network.{Processor, RequestChannel} import kafka.utils._ @@ -64,6 +64,8 @@ import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, Test, TestInfo} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import scala.annotation.nowarn import scala.collection._ @@ -80,7 +82,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup import DynamicBrokerReconfigurationTest._ - private val servers = new ArrayBuffer[KafkaServer] + private val servers = new ArrayBuffer[KafkaBroker] private val numServers = 3 private val numPartitions = 10 private val producers = new ArrayBuffer[KafkaProducer[String, String]] @@ -111,15 +113,22 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup (0 until numServers).foreach { brokerId => - val props = TestUtils.createBrokerConfig(brokerId, zkConnect) + val props = if (isKRaftTest()) { + val properties = TestUtils.createBrokerConfig(brokerId, null) + properties.put(KafkaConfig.AdvertisedListenersProp, s"$SecureInternal://localhost:0, $SecureExternal://localhost:0") + properties + } else { + val properties = TestUtils.createBrokerConfig(brokerId, zkConnect) + properties.put(KafkaConfig.ZkEnableSecureAclsProp, "true") + properties + } props ++= securityProps(sslProperties1, TRUSTSTORE_PROPS) // Ensure that we can support multiple listeners per security protocol and multiple security protocols props.put(KafkaConfig.ListenersProp, s"$SecureInternal://localhost:0, $SecureExternal://localhost:0") - props.put(KafkaConfig.ListenerSecurityProtocolMapProp, s"$SecureInternal:SSL, $SecureExternal:SASL_SSL") + props.put(KafkaConfig.ListenerSecurityProtocolMapProp, s"$SecureInternal:SSL, $SecureExternal:SASL_SSL, CONTROLLER:$controllerListenerSecurityProtocol") props.put(KafkaConfig.InterBrokerListenerNameProp, SecureInternal) props.put(KafkaConfig.SslClientAuthProp, "requested") props.put(KafkaConfig.SaslMechanismInterBrokerProtocolProp, "PLAIN") - props.put(KafkaConfig.ZkEnableSecureAclsProp, "true") props.put(KafkaConfig.SaslEnabledMechanismsProp, kafkaServerSaslMechanisms.mkString(",")) props.put(KafkaConfig.LogSegmentBytesProp, "2000") // low value to test log rolling on config update props.put(KafkaConfig.NumReplicaFetchersProp, "2") // greater than one to test reducing threads @@ -138,17 +147,21 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup props ++= securityProps(sslProperties1, KEYSTORE_PROPS, listenerPrefix(SecureExternal)) val kafkaConfig = KafkaConfig.fromProps(props) - configureDynamicKeystoreInZooKeeper(kafkaConfig, sslProperties1) + if (!isKRaftTest()) { + configureDynamicKeystoreInZooKeeper(kafkaConfig, sslProperties1) + } - servers += TestUtils.createServer(kafkaConfig) + servers += createBroker(kafkaConfig) } - TestUtils.createTopic(zkClient, topic, numPartitions, replicationFactor = numServers, servers) - TestUtils.createTopic(zkClient, Topic.GROUP_METADATA_TOPIC_NAME, servers.head.config.offsetsTopicPartitions, - replicationFactor = numServers, servers, servers.head.groupCoordinator.offsetsTopicConfigs) - createAdminClient(SecurityProtocol.SSL, SecureInternal) + TestUtils.createTopicWithAdmin(adminClients.head, topic, servers, numPartitions, replicationFactor = numServers) + TestUtils.createTopicWithAdmin(adminClients.head, Topic.GROUP_METADATA_TOPIC_NAME, servers, + numPartitions = servers.head.config.offsetsTopicPartitions, + replicationFactor = numServers, + topicConfig = servers.head.groupCoordinator.offsetsTopicConfigs) + TestMetricsReporter.testReporters.clear() } @@ -166,8 +179,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup closeSasl() } - @Test - def testConfigDescribeUsingAdminClient(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testConfigDescribeUsingAdminClient(quorum: String): Unit = { def verifyConfig(configName: String, configEntry: ConfigEntry, isSensitive: Boolean, isReadOnly: Boolean, expectedProps: Properties): Unit = { @@ -226,9 +240,12 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup val adminClient = adminClients.head alterSslKeystoreUsingConfigCommand(sslProperties1, SecureExternal) - val configDesc = describeConfig(adminClient) - verifySslConfig("listener.name.external.", sslProperties1, configDesc) - verifySslConfig("", invalidSslProperties, configDesc) + val configDesc = TestUtils.tryUntilNoAssertionError() { + val describeConfigsResult = describeConfig(adminClient) + verifySslConfig("listener.name.external.", sslProperties1, describeConfigsResult) + verifySslConfig("", invalidSslProperties, describeConfigsResult) + describeConfigsResult + } // Verify a few log configs with and without synonyms val expectedProps = new Properties @@ -262,8 +279,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup assertEquals(List((KafkaConfig.LogCleanerThreadsProp, ConfigSource.DEFAULT_CONFIG)), synonymsList(logCleanerThreads)) } - @Test - def testUpdatesUsingConfigProvider(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testUpdatesUsingConfigProvider(quorum: String): Unit = { val PollingIntervalVal = f"$${file:polling.interval:interval}" val PollingIntervalUpdateVal = f"$${file:polling.interval:updinterval}" val SslTruststoreTypeVal = f"$${file:ssl.truststore.type:storetype}" @@ -309,11 +327,13 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup assertFalse(reporter.kafkaMetrics.isEmpty, "No metrics found") } - // fetch from ZK, values should be unresolved - val props = fetchBrokerConfigsFromZooKeeper(servers.head) - assertTrue(props.getProperty(TestMetricsReporter.PollingIntervalProp) == PollingIntervalVal, "polling interval is not updated in ZK") - assertTrue(props.getProperty(configPrefix+KafkaConfig.SslTruststoreTypeProp) == SslTruststoreTypeVal, "store type is not updated in ZK") - assertTrue(props.getProperty(configPrefix+KafkaConfig.SslKeystorePasswordProp) == SslKeystorePasswordVal, "keystore password is not updated in ZK") + if (!isKRaftTest()) { + // fetch from ZK, values should be unresolved + val props = fetchBrokerConfigsFromZooKeeper(servers.head) + assertTrue(props.getProperty(TestMetricsReporter.PollingIntervalProp) == PollingIntervalVal, "polling interval is not updated in ZK") + assertTrue(props.getProperty(configPrefix + KafkaConfig.SslTruststoreTypeProp) == SslTruststoreTypeVal, "store type is not updated in ZK") + assertTrue(props.getProperty(configPrefix + KafkaConfig.SslKeystorePasswordProp) == SslKeystorePasswordVal, "keystore password is not updated in ZK") + } // verify the update // 1. verify update not occurring if the value of property is same. @@ -332,10 +352,10 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } } - @Test + @Test // TODO KAFKA-14126 add KRaft support def testKeyStoreAlter(): Unit = { val topic2 = "testtopic2" - TestUtils.createTopic(zkClient, topic2, numPartitions, replicationFactor = numServers, servers) + TestUtils.createTopicWithAdmin(adminClients.head, topic2, servers, numPartitions, replicationFactor = numServers) // Start a producer and consumer that work with the current broker keystore. // This should continue working while changes are made @@ -399,7 +419,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup stopAndVerifyProduceConsume(producerThread, consumerThread) } - @Test + @Test // TODO KAFKA-14126 add KRaft support def testTrustStoreAlter(): Unit = { val producerBuilder = ProducerBuilder().listenerName(SecureInternal).securityProtocol(SecurityProtocol.SSL) @@ -481,7 +501,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup verifySslProduceConsume(sslProperties2, "alter-truststore-7") waitForAuthenticationFailure(producerBuilder.keyStoreProps(sslProperties1)) - val controller = servers.find(_.config.brokerId == TestUtils.waitUntilControllerElected(zkClient)).get + val controller = servers.find(_.config.brokerId == TestUtils.waitUntilControllerElected(zkClient)).get.asInstanceOf[KafkaServer] val controllerChannelManager = controller.kafkaController.controllerChannelManager val brokerStateInfo: mutable.HashMap[Int, ControllerBrokerStateInfo] = JTestUtils.fieldValue(controllerChannelManager, classOf[ControllerChannelManager], "brokerStateInfo") @@ -492,8 +512,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup verifyBrokerToControllerCall(controller) } - @Test - def testLogCleanerConfig(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testLogCleanerConfig(quorum: String): Unit = { val (producerThread, consumerThread) = startProduceConsume(retries = 0) verifyThreads("kafka-log-cleaner-thread-", countPerBroker = 1) @@ -537,13 +558,23 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup stopAndVerifyProduceConsume(producerThread, consumerThread) } - @Test - def testConsecutiveConfigChange(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testConsecutiveConfigChange(quorum: String): Unit = { val topic2 = "testtopic2" val topicProps = new Properties topicProps.put(KafkaConfig.MinInSyncReplicasProp, "2") - TestUtils.createTopic(zkClient, topic2, 1, replicationFactor = numServers, servers, topicProps) - var log = servers.head.logManager.getLog(new TopicPartition(topic2, 0)).getOrElse(throw new IllegalStateException("Log not found")) + TestUtils.createTopicWithAdmin(adminClients.head, topic2, servers, numPartitions = 1, replicationFactor = numServers, topicConfig = topicProps) + + def getLogOrThrow(tp: TopicPartition): UnifiedLog = { + var (logOpt, found) = TestUtils.computeUntilTrue { + servers.head.logManager.getLog(tp) + }(_.isDefined) + assertTrue(found, "Log not found") + logOpt.get + } + + var log = getLogOrThrow(new TopicPartition(topic2, 0)) assertTrue(log.config.overriddenConfigs.contains(KafkaConfig.MinInSyncReplicasProp)) assertEquals("2", log.config.originals().get(KafkaConfig.MinInSyncReplicasProp).toString) @@ -558,7 +589,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } } - log = servers.head.logManager.getLog(new TopicPartition(topic2, 0)).getOrElse(throw new IllegalStateException("Log not found")) + log = getLogOrThrow(new TopicPartition(topic2, 0)) assertTrue(log.config.overriddenConfigs.contains(KafkaConfig.MinInSyncReplicasProp)) assertEquals("2", log.config.originals().get(KafkaConfig.MinInSyncReplicasProp).toString) // Verify topic-level config survives @@ -566,7 +597,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup props.clear() props.put(KafkaConfig.LogRetentionTimeMillisProp, "604800000") reconfigureServers(props, perBrokerConfig = false, (KafkaConfig.LogRetentionTimeMillisProp, "604800000")) - log = servers.head.logManager.getLog(new TopicPartition(topic2, 0)).getOrElse(throw new IllegalStateException("Log not found")) + log = getLogOrThrow(new TopicPartition(topic2, 0)) assertTrue(log.config.overriddenConfigs.contains(KafkaConfig.MinInSyncReplicasProp)) assertEquals("2", log.config.originals().get(KafkaConfig.MinInSyncReplicasProp).toString) // Verify topic-level config still survives } @@ -974,6 +1005,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } @Test + // Modifying advertised listeners is not supported in KRaft def testAdvertisedListenerUpdate(): Unit = { val adminClient = adminClients.head val externalAdminClient = createAdminClient(SecurityProtocol.SASL_SSL, SecureExternal) @@ -994,11 +1026,13 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } // Verify that endpoints have been updated in ZK for all brokers - servers.foreach(validateEndpointsInZooKeeper(_, endpoints => endpoints.contains(invalidHost))) + servers.foreach { server => + validateEndpointsInZooKeeper(server.asInstanceOf[KafkaServer], endpoints => endpoints.contains(invalidHost)) + } // Trigger session expiry and ensure that controller registers new advertised listener after expiry val controllerEpoch = zkClient.getControllerEpoch - val controllerServer = servers(zkClient.getControllerId.getOrElse(throw new IllegalStateException("No controller"))) + val controllerServer = servers(zkClient.getControllerId.getOrElse(throw new IllegalStateException("No controller"))).asInstanceOf[KafkaServer] val controllerZkClient = controllerServer.zkClient val sessionExpiringClient = createZooKeeperClientToTriggerSessionExpiry(controllerZkClient.currentZooKeeper) sessionExpiringClient.close() @@ -1022,7 +1056,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup .getCause.isInstanceOf[org.apache.kafka.common.errors.TimeoutException]) alterAdvertisedListener(adminClient, externalAdminClient, invalidHost, "localhost") - servers.foreach(validateEndpointsInZooKeeper(_, endpoints => !endpoints.contains(invalidHost))) + servers.foreach { server => + validateEndpointsInZooKeeper(server.asInstanceOf[KafkaServer], endpoints => !endpoints.contains(invalidHost)) + } // Verify that produce/consume work now val topic2 = "testtopic2" @@ -1119,7 +1155,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup assertTrue(partitions.exists(_.leader == null), "Did not find partitions with no leader") } - private def addListener(servers: Seq[KafkaServer], listenerName: String, securityProtocol: SecurityProtocol, + private def addListener(servers: Seq[KafkaBroker], listenerName: String, securityProtocol: SecurityProtocol, saslMechanisms: Seq[String]): Unit = { val config = servers.head.config val existingListenerCount = config.listeners.size @@ -1264,11 +1300,11 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup verifyProduceConsume(producer, consumer, numRecords = 10, topic) } - private def hasListenerMetric(server: KafkaServer, listenerName: String): Boolean = { + private def hasListenerMetric(server: KafkaBroker, listenerName: String): Boolean = { server.socketServer.metrics.metrics.keySet.asScala.exists(_.tags.get("listener") == listenerName) } - private def fetchBrokerConfigsFromZooKeeper(server: KafkaServer): Properties = { + private def fetchBrokerConfigsFromZooKeeper(server: KafkaBroker): Properties = { val props = adminZkClient.fetchEntityConfig(ConfigType.Broker, server.config.brokerId.toString) server.config.dynamicConfig.fromPersistentProps(props, perBrokerConfig = true) } @@ -1322,7 +1358,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup }, "Did not fail authentication with invalid config") } - private def describeConfig(adminClient: Admin, servers: Seq[KafkaServer] = this.servers): Config = { + private def describeConfig(adminClient: Admin, servers: Seq[KafkaBroker] = this.servers): Config = { val configResources = servers.map { server => new ConfigResource(ConfigResource.Type.BROKER, server.config.brokerId.toString) } @@ -1419,7 +1455,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } @nowarn("cat=deprecation") - private def alterConfigsOnServer(server: KafkaServer, props: Properties): Unit = { + private def alterConfigsOnServer(server: KafkaBroker, props: Properties): Unit = { val configEntries = props.asScala.map { case (k, v) => new ConfigEntry(k, v) }.toList.asJava val newConfig = new Config(configEntries) val configs = Map(new ConfigResource(ConfigResource.Type.BROKER, server.config.brokerId.toString) -> newConfig).asJava @@ -1428,7 +1464,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } @nowarn("cat=deprecation") - private def alterConfigs(servers: Seq[KafkaServer], adminClient: Admin, props: Properties, + private def alterConfigs(servers: Seq[KafkaBroker], adminClient: Admin, props: Properties, perBrokerConfig: Boolean): AlterConfigsResult = { val configEntries = props.asScala.map { case (k, v) => new ConfigEntry(k, v) }.toList.asJava val newConfig = new Config(configEntries) @@ -1507,7 +1543,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup private def createPasswordEncoder(config: KafkaConfig, secret: Option[Password]): PasswordEncoder = { val encoderSecret = secret.getOrElse(throw new IllegalStateException("Password encoder secret not configured")) - new PasswordEncoder(encoderSecret, + PasswordEncoder.encrypting(encoderSecret, config.passwordEncoderKeyFactoryAlgorithm, config.passwordEncoderCipherAlgorithm, config.passwordEncoderKeyLength, @@ -1518,7 +1554,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup servers.foreach { server => waitForConfigOnServer(server, propName, propValue, maxWaitMs) } } - private def waitForConfigOnServer(server: KafkaServer, propName: String, propValue: String, maxWaitMs: Long = 10000): Unit = { + private def waitForConfigOnServer(server: KafkaBroker, propName: String, propValue: String, maxWaitMs: Long = 10000): Unit = { TestUtils.retry(maxWaitMs) { assertEquals(propValue, server.config.originals.get(propName)) } diff --git a/core/src/test/scala/unit/kafka/utils/PasswordEncoderTest.scala b/core/src/test/scala/unit/kafka/utils/PasswordEncoderTest.scala index 0a5d5ac029814..50cdceabbca66 100755 --- a/core/src/test/scala/unit/kafka/utils/PasswordEncoderTest.scala +++ b/core/src/test/scala/unit/kafka/utils/PasswordEncoderTest.scala @@ -30,7 +30,7 @@ class PasswordEncoderTest { @Test def testEncodeDecode(): Unit = { - val encoder = new PasswordEncoder(new Password("password-encoder-secret"), + val encoder = PasswordEncoder.encrypting(new Password("password-encoder-secret"), None, Defaults.PasswordEncoderCipherAlgorithm, Defaults.PasswordEncoderKeyLength, @@ -54,7 +54,7 @@ class PasswordEncoderTest { @Test def testEncoderConfigChange(): Unit = { - val encoder = new PasswordEncoder(new Password("password-encoder-secret"), + val encoder = PasswordEncoder.encrypting(new Password("password-encoder-secret"), Some("PBKDF2WithHmacSHA1"), "DES/CBC/PKCS5Padding", 64, @@ -68,7 +68,7 @@ class PasswordEncoderTest { assertEquals("DES/CBC/PKCS5Padding", encodedMap(PasswordEncoder.CipherAlgorithmProp)) // Test that decoding works even if PasswordEncoder algorithm, iterations etc. are altered - val decoder = new PasswordEncoder(new Password("password-encoder-secret"), + val decoder = PasswordEncoder.encrypting(new Password("password-encoder-secret"), Some("PBKDF2WithHmacSHA1"), "AES/CBC/PKCS5Padding", 128, @@ -76,7 +76,7 @@ class PasswordEncoderTest { assertEquals(password, decoder.decode(encoded).value) // Test that decoding fails if secret is altered - val decoder2 = new PasswordEncoder(new Password("secret-2"), + val decoder2 = PasswordEncoder.encrypting(new Password("secret-2"), Some("PBKDF2WithHmacSHA1"), "AES/CBC/PKCS5Padding", 128, @@ -92,7 +92,7 @@ class PasswordEncoderTest { def testEncodeDecodeAlgorithms(): Unit = { def verifyEncodeDecode(keyFactoryAlg: Option[String], cipherAlg: String, keyLength: Int): Unit = { - val encoder = new PasswordEncoder(new Password("password-encoder-secret"), + val encoder = PasswordEncoder.encrypting(new Password("password-encoder-secret"), keyFactoryAlg, cipherAlg, keyLength, diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index c49a7bdde09be..d0266bdee9d03 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -511,7 +511,7 @@ object TestUtils extends Logging { topic: String, numPartitions: Int = 1, replicationFactor: Int = 1, - servers: Seq[KafkaServer], + servers: Seq[KafkaBroker], topicConfig: Properties = new Properties): scala.collection.immutable.Map[Int, Int] = { val adminZkClient = new AdminZkClient(zkClient) // create topic @@ -543,7 +543,7 @@ object TestUtils extends Logging { def createTopic(zkClient: KafkaZkClient, topic: String, partitionReplicaAssignment: collection.Map[Int, Seq[Int]], - servers: Seq[KafkaServer]): scala.collection.immutable.Map[Int, Int] = { + servers: Seq[KafkaBroker]): scala.collection.immutable.Map[Int, Int] = { createTopic(zkClient, topic, partitionReplicaAssignment, servers, new Properties()) } @@ -555,7 +555,7 @@ object TestUtils extends Logging { def createTopic(zkClient: KafkaZkClient, topic: String, partitionReplicaAssignment: collection.Map[Int, Seq[Int]], - servers: Seq[KafkaServer], + servers: Seq[KafkaBroker], topicConfig: Properties): scala.collection.immutable.Map[Int, Int] = { val adminZkClient = new AdminZkClient(zkClient) // create topic @@ -583,7 +583,7 @@ object TestUtils extends Logging { * Create the consumer offsets/group metadata topic and wait until the leader is elected and metadata is propagated * to all brokers. */ - def createOffsetsTopic(zkClient: KafkaZkClient, servers: Seq[KafkaServer]): Unit = { + def createOffsetsTopic(zkClient: KafkaZkClient, servers: Seq[KafkaBroker]): Unit = { val server = servers.head createTopic(zkClient, Topic.GROUP_METADATA_TOPIC_NAME, server.config.getInt(KafkaConfig.OffsetsTopicPartitionsProp), @@ -1043,18 +1043,19 @@ object TestUtils extends Logging { * otherwise difficult to poll for. `computeUntilTrue` and `waitUntilTrue` should be preferred in cases where we can * easily wait on a condition before evaluating the assertions. */ - def tryUntilNoAssertionError(waitTime: Long = JTestUtils.DEFAULT_MAX_WAIT_MS, pause: Long = 100L)(assertions: => Unit) = { - val (error, success) = TestUtils.computeUntilTrue({ + def tryUntilNoAssertionError[T](waitTime: Long = JTestUtils.DEFAULT_MAX_WAIT_MS, pause: Long = 100L)(assertions: => T): T = { + val (either, success) = TestUtils.computeUntilTrue({ try { - assertions - None + val res = assertions + Left(res) } catch { - case ae: AssertionError => Some(ae) + case ae: AssertionError => Right(ae) } - }, waitTime = waitTime, pause = pause)(_.isEmpty) + }, waitTime = waitTime, pause = pause)(_.isLeft) - if (!success) { - throw error.get + either match { + case Left(res) => res + case Right(err) => throw err } } From 98643608b0e16c0d9b7b1e800b65267dcb72d40d Mon Sep 17 00:00:00 2001 From: dengziming Date: Tue, 2 Aug 2022 20:38:52 +0800 Subject: [PATCH 022/175] MINOR; Use right enum value for broker registration change (#12236) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The code used BrokerRegistrationFencingChange.FENCE when unfencing a broker and used BrokerRegistrationFencingChange.UNFENCE when fencing a broker, this is confusing. This commit flips the values of the two enums and changes their usage at all of the call sites. Reviewers: José Armando García Sancio --- .../controller/ClusterControlManager.java | 4 +- .../controller/ReplicationControlManager.java | 4 +- .../org/apache/kafka/image/ClusterDelta.java | 4 +- .../BrokerRegistrationFencingChange.java | 4 +- .../controller/ClusterControlManagerTest.java | 6 +-- .../BrokerRegistrationFencingChangeTest.java | 8 ++-- ...trationInControlledShutdownChangeTest.java | 48 +++++++++++++++++++ 7 files changed, 63 insertions(+), 15 deletions(-) create mode 100644 metadata/src/test/java/org/apache/kafka/metadata/BrokerRegistrationInControlledShutdownChangeTest.java diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java index 41b9728cda869..235f077cfff7a 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java @@ -422,7 +422,7 @@ public void replay(FenceBrokerRecord record) { record, record.id(), record.epoch(), - BrokerRegistrationFencingChange.UNFENCE.asBoolean(), + BrokerRegistrationFencingChange.FENCE.asBoolean(), BrokerRegistrationInControlledShutdownChange.NONE.asBoolean() ); } @@ -432,7 +432,7 @@ public void replay(UnfenceBrokerRecord record) { record, record.id(), record.epoch(), - BrokerRegistrationFencingChange.FENCE.asBoolean(), + BrokerRegistrationFencingChange.UNFENCE.asBoolean(), BrokerRegistrationInControlledShutdownChange.NONE.asBoolean() ); } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java index a5e0667bf3bcb..3a3788c41a595 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java @@ -1172,7 +1172,7 @@ void handleBrokerFenced(int brokerId, List records) { if (featureControl.metadataVersion().isBrokerRegistrationChangeRecordSupported()) { records.add(new ApiMessageAndVersion(new BrokerRegistrationChangeRecord(). setBrokerId(brokerId).setBrokerEpoch(brokerRegistration.epoch()). - setFenced(BrokerRegistrationFencingChange.UNFENCE.value()), + setFenced(BrokerRegistrationFencingChange.FENCE.value()), (short) 0)); } else { records.add(new ApiMessageAndVersion(new FenceBrokerRecord(). @@ -1215,7 +1215,7 @@ void handleBrokerUnfenced(int brokerId, long brokerEpoch, List Date: Fri, 29 Jul 2022 02:48:35 +0530 Subject: [PATCH 023/175] MINOR: Update comment on verifyTaskGenerationAndOwnership method in DistributedHerder Reviewers: Chris Egerton --- .../connect/runtime/distributed/DistributedHerder.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java index ded833da59914..388bfa4218a73 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java @@ -1732,9 +1732,7 @@ private boolean startTask(ConnectorTaskId taskId) { throw ConnectUtils.maybeWrap(cause, "Failed to perform round of zombie fencing"); } }, - () -> { - verifyTaskGenerationAndOwnership(taskId, taskGeneration); - } + () -> verifyTaskGenerationAndOwnership(taskId, taskGeneration) ); } else { return worker.startSourceTask( @@ -1941,8 +1939,8 @@ private void reconfigureConnector(final String connName, final Callback cb } } - // Currently unused, but will be invoked by exactly-once source tasks after they have successfully - // initialized their transactional producer + // Invoked by exactly-once worker source tasks after they have successfully initialized their transactional + // producer to ensure that it is still safe to bring up the task private void verifyTaskGenerationAndOwnership(ConnectorTaskId id, int initialTaskGen) { log.debug("Reading to end of config topic to ensure it is still safe to bring up source task {} with exactly-once support", id); if (!refreshConfigSnapshot(Long.MAX_VALUE)) { From 3fa327203658e458f23aa9f7f7ca3e7d3d9b3e43 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Thu, 4 Aug 2022 15:09:08 -0400 Subject: [PATCH 024/175] KAFKA-14136 Generate ConfigRecord for brokers even if the value is unchanged (#12483) --- .../DynamicBrokerReconfigurationTest.scala | 54 ++++++++++++------- .../ConfigurationControlManager.java | 13 +++-- 2 files changed, 44 insertions(+), 23 deletions(-) diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index c7be8ce831da8..bd6308c0b8896 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -67,6 +67,7 @@ import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, Test, TestInfo} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource +import java.util.concurrent.atomic.AtomicInteger import scala.annotation.nowarn import scala.collection._ import scala.collection.mutable.ArrayBuffer @@ -352,8 +353,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } } - @Test // TODO KAFKA-14126 add KRaft support - def testKeyStoreAlter(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testKeyStoreAlter(quorum: String): Unit = { val topic2 = "testtopic2" TestUtils.createTopicWithAdmin(adminClients.head, topic2, servers, numPartitions, replicationFactor = numServers) @@ -419,8 +421,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup stopAndVerifyProduceConsume(producerThread, consumerThread) } - @Test // TODO KAFKA-14126 add KRaft support - def testTrustStoreAlter(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testTrustStoreAlter(quorum: String): Unit = { val producerBuilder = ProducerBuilder().listenerName(SecureInternal).securityProtocol(SecurityProtocol.SSL) // Producer with new keystore should fail to connect before truststore update @@ -467,9 +470,12 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup assertFalse(response.wasDisconnected(), "Request failed because broker is not available") } + val group_id = new AtomicInteger(1) + def next_group_name(): String = s"alter-truststore-${group_id.getAndIncrement()}" + // Produce/consume should work with old as well as new client keystore - verifySslProduceConsume(sslProperties1, "alter-truststore-1") - verifySslProduceConsume(sslProperties2, "alter-truststore-2") + verifySslProduceConsume(sslProperties1, next_group_name()) + verifySslProduceConsume(sslProperties2, next_group_name()) // Revert to old truststore with only one certificate and update. Clients should connect only with old keystore. val oldTruststoreProps = new Properties @@ -478,7 +484,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup reconfigureServers(oldTruststoreProps, perBrokerConfig = true, (s"$prefix$SSL_TRUSTSTORE_LOCATION_CONFIG", sslProperties1.getProperty(SSL_TRUSTSTORE_LOCATION_CONFIG))) verifyAuthenticationFailure(producerBuilder.keyStoreProps(sslProperties2).build()) - verifySslProduceConsume(sslProperties1, "alter-truststore-3") + verifySslProduceConsume(sslProperties1, next_group_name()) // Update same truststore file to contain both certificates without changing any configs. // Clients should connect successfully with either keystore after admin client AlterConfigsRequest completes. @@ -486,8 +492,14 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup Paths.get(sslProperties1.getProperty(SSL_TRUSTSTORE_LOCATION_CONFIG)), StandardCopyOption.REPLACE_EXISTING) TestUtils.incrementalAlterConfigs(servers, adminClients.head, oldTruststoreProps, perBrokerConfig = true).all.get() - verifySslProduceConsume(sslProperties1, "alter-truststore-4") - verifySslProduceConsume(sslProperties2, "alter-truststore-5") + TestUtils.retry(30000) { + try { + verifySslProduceConsume(sslProperties1, next_group_name()) + verifySslProduceConsume(sslProperties2, next_group_name()) + } catch { + case t: Throwable => throw new AssertionError(t) + } + } // Update internal keystore/truststore and validate new client connections from broker (e.g. controller). // Alter internal keystore from `sslProperties1` to `sslProperties2`, force disconnect of a controller connection @@ -495,21 +507,23 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup val props2 = securityProps(sslProperties2, KEYSTORE_PROPS, prefix) props2 ++= securityProps(combinedStoreProps, TRUSTSTORE_PROPS, prefix) TestUtils.incrementalAlterConfigs(servers, adminClients.head, props2, perBrokerConfig = true).all.get(15, TimeUnit.SECONDS) - verifySslProduceConsume(sslProperties2, "alter-truststore-6") + verifySslProduceConsume(sslProperties2, next_group_name()) props2 ++= securityProps(sslProperties2, TRUSTSTORE_PROPS, prefix) TestUtils.incrementalAlterConfigs(servers, adminClients.head, props2, perBrokerConfig = true).all.get(15, TimeUnit.SECONDS) - verifySslProduceConsume(sslProperties2, "alter-truststore-7") + verifySslProduceConsume(sslProperties2, next_group_name()) waitForAuthenticationFailure(producerBuilder.keyStoreProps(sslProperties1)) - val controller = servers.find(_.config.brokerId == TestUtils.waitUntilControllerElected(zkClient)).get.asInstanceOf[KafkaServer] - val controllerChannelManager = controller.kafkaController.controllerChannelManager - val brokerStateInfo: mutable.HashMap[Int, ControllerBrokerStateInfo] = - JTestUtils.fieldValue(controllerChannelManager, classOf[ControllerChannelManager], "brokerStateInfo") - brokerStateInfo(0).networkClient.disconnect("0") - TestUtils.createTopic(zkClient, "testtopic2", numPartitions, replicationFactor = numServers, servers) - - // validate that the brokerToController request works fine - verifyBrokerToControllerCall(controller) + if (!isKRaftTest()) { + val controller = servers.find(_.config.brokerId == TestUtils.waitUntilControllerElected(zkClient)).get.asInstanceOf[KafkaServer] + val controllerChannelManager = controller.kafkaController.controllerChannelManager + val brokerStateInfo: mutable.HashMap[Int, ControllerBrokerStateInfo] = + JTestUtils.fieldValue(controllerChannelManager, classOf[ControllerChannelManager], "brokerStateInfo") + brokerStateInfo(0).networkClient.disconnect("0") + TestUtils.createTopic(zkClient, "testtopic2", numPartitions, replicationFactor = numServers, servers) + + // validate that the brokerToController request works fine + verifyBrokerToControllerCall(controller) + } } @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) diff --git a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java index 4b8561a4d9089..4d6736b878d5b 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.config.ConfigResource.Type; import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.common.config.types.Password; import org.apache.kafka.common.metadata.ConfigRecord; import org.apache.kafka.common.requests.ApiError; import org.apache.kafka.common.utils.LogContext; @@ -228,7 +229,8 @@ private void incrementalAlterConfigResource(ConfigResource configResource, newValue = String.join(",", oldValueList); break; } - if (!Objects.equals(currentValue, newValue)) { + if (!Objects.equals(currentValue, newValue) || configResource.type().equals(Type.BROKER)) { + // KAFKA-14136 We need to generate records even if the value is unchanged to trigger reloads on the brokers newRecords.add(new ApiMessageAndVersion(new ConfigRecord(). setResourceType(configResource.type().id()). setResourceName(configResource.name()). @@ -317,7 +319,8 @@ private void legacyAlterConfigResource(ConfigResource configResource, String key = entry.getKey(); String newValue = entry.getValue(); String currentValue = currentConfigs.get(key); - if (!Objects.equals(newValue, currentValue)) { + if (!Objects.equals(currentValue, newValue) || configResource.type().equals(Type.BROKER)) { + // KAFKA-14136 We need to generate records even if the value is unchanged to trigger reloads on the brokers newRecords.add(new ApiMessageAndVersion(new ConfigRecord(). setResourceType(configResource.type().id()). setResourceName(configResource.name()). @@ -381,7 +384,11 @@ public void replay(ConfigRecord record) { if (configs.isEmpty()) { configData.remove(configResource); } - log.info("{}: set configuration {} to {}", configResource, record.name(), record.value()); + if (configSchema.isSensitive(record)) { + log.info("{}: set configuration {} to {}", configResource, record.name(), Password.HIDDEN); + } else { + log.info("{}: set configuration {} to {}", configResource, record.name(), record.value()); + } } // VisibleForTesting From eed9192e8ad0230d58184939a64afee90dc0dae8 Mon Sep 17 00:00:00 2001 From: Andrew Borley <848217+ajborley@users.noreply.github.com> Date: Fri, 5 Aug 2022 22:29:04 +0100 Subject: [PATCH 025/175] KAFKA-14107: Upgrade Jetty version for CVE fixes (#12440) Reviewers: Mickael Maison , Aaron Freeland --- gradle/dependencies.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 92c098851c618..f73531ce560c3 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -70,7 +70,7 @@ versions += [ jacksonDatabind: "2.13.3", jacoco: "0.8.7", javassist: "3.27.0-GA", - jetty: "9.4.44.v20210927", + jetty: "9.4.48.v20220622", jersey: "2.34", jline: "3.21.0", jmh: "1.35", From 2dafe45f15eb730a7a6bad258defd298ec173a08 Mon Sep 17 00:00:00 2001 From: Niket Date: Mon, 8 Aug 2022 15:03:04 -0700 Subject: [PATCH 026/175] KAFKA-14104; Add CRC validation when iterating over Metadata Log Records (#12457) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This commit adds a check to ensure the RecordBatch CRC is valid when iterating over a Batch of Records using the RecordsIterator. The RecordsIterator is used by both Snapshot reads and Log Records reads in Kraft. The check can be turned off by a class parameter and is on by default. Reviewers: José Armando García Sancio --- .../common/record/DefaultRecordBatch.java | 2 +- .../server/RaftClusterSnapshotTest.scala | 3 +- .../controller/QuorumControllerTest.java | 3 +- .../apache/kafka/metalog/LocalLogManager.java | 3 +- .../apache/kafka/raft/KafkaRaftClient.java | 10 ++- .../raft/internals/RecordsBatchReader.java | 5 +- .../kafka/raft/internals/RecordsIterator.java | 13 +++- .../kafka/snapshot/RecordsSnapshotReader.java | 5 +- .../kafka/raft/RaftEventSimulationTest.java | 2 +- .../internals/RecordsBatchReaderTest.java | 3 +- .../raft/internals/RecordsIteratorTest.java | 61 ++++++++++++++++--- .../snapshot/SnapshotWriterReaderTest.java | 5 +- 12 files changed, 92 insertions(+), 23 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java index bd80981d84bda..bc8f32491c0a8 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java @@ -107,7 +107,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe static final int PARTITION_LEADER_EPOCH_LENGTH = 4; static final int MAGIC_OFFSET = PARTITION_LEADER_EPOCH_OFFSET + PARTITION_LEADER_EPOCH_LENGTH; static final int MAGIC_LENGTH = 1; - static final int CRC_OFFSET = MAGIC_OFFSET + MAGIC_LENGTH; + public static final int CRC_OFFSET = MAGIC_OFFSET + MAGIC_LENGTH; static final int CRC_LENGTH = 4; static final int ATTRIBUTES_OFFSET = CRC_OFFSET + CRC_LENGTH; static final int ATTRIBUTE_LENGTH = 2; diff --git a/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala b/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala index 503ce7d2beefd..f8dccd17d0d8a 100644 --- a/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala +++ b/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala @@ -78,7 +78,8 @@ class RaftClusterSnapshotTest { raftManager.replicatedLog.latestSnapshot.get(), new MetadataRecordSerde(), BufferSupplier.create(), - 1 + 1, + true ) ) { snapshot => // Check that the snapshot is non-empty diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java index a62b1f682f067..2cdec699da230 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -738,7 +738,8 @@ private SnapshotReader createSnapshotReader(RawSnapshotRea reader, new MetadataRecordSerde(), BufferSupplier.create(), - Integer.MAX_VALUE + Integer.MAX_VALUE, + true ); } diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java index c8e39ae32896e..e24d86bd873ff 100644 --- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java +++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java @@ -496,7 +496,8 @@ private void scheduleLogCheck() { snapshot.get(), new MetadataRecordSerde(), BufferSupplier.create(), - Integer.MAX_VALUE + Integer.MAX_VALUE, + true ) ); } diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 53372728aab46..cac7a8a3cb998 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -333,7 +333,12 @@ private void updateListenersProgress(long highWatermark) { private Optional> latestSnapshot() { return log.latestSnapshot().map(reader -> - RecordsSnapshotReader.of(reader, serde, BufferSupplier.create(), MAX_BATCH_SIZE_BYTES) + RecordsSnapshotReader.of(reader, + serde, + BufferSupplier.create(), + MAX_BATCH_SIZE_BYTES, + true /* Validate batch CRC*/ + ) ); } @@ -2519,7 +2524,8 @@ private void fireHandleCommit(long baseOffset, Records records) { serde, BufferSupplier.create(), MAX_BATCH_SIZE_BYTES, - this + this, + true /* Validate batch CRC*/ ) ); } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java index e95206100a305..61819a9dccaea 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java @@ -100,11 +100,12 @@ public static RecordsBatchReader of( RecordSerde serde, BufferSupplier bufferSupplier, int maxBatchSize, - CloseListener> closeListener + CloseListener> closeListener, + boolean doCrcValidation ) { return new RecordsBatchReader<>( baseOffset, - new RecordsIterator<>(records, serde, bufferSupplier, maxBatchSize), + new RecordsIterator<>(records, serde, bufferSupplier, maxBatchSize, doCrcValidation), closeListener ); } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java index 866f541fb2484..ff415aa72ada0 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java @@ -41,6 +41,9 @@ public final class RecordsIterator implements Iterator>, AutoCloseab private final RecordSerde serde; private final BufferSupplier bufferSupplier; private final int batchSize; + // Setting to true will make the RecordsIterator perform a CRC Validation + // on the batch header when iterating over them + private final boolean doCrcValidation; private Iterator nextBatches = Collections.emptyIterator(); private Optional> nextBatch = Optional.empty(); @@ -54,12 +57,14 @@ public RecordsIterator( Records records, RecordSerde serde, BufferSupplier bufferSupplier, - int batchSize + int batchSize, + boolean doCrcValidation ) { this.records = records; this.serde = serde; this.bufferSupplier = bufferSupplier; this.batchSize = Math.max(batchSize, Records.HEADER_SIZE_UP_TO_MAGIC); + this.doCrcValidation = doCrcValidation; } @Override @@ -163,7 +168,6 @@ private Optional> nextBatch() { if (nextBatches.hasNext()) { MutableRecordBatch nextBatch = nextBatches.next(); - // Update the buffer position to reflect the read batch allocatedBuffer.ifPresent(buffer -> buffer.position(buffer.position() + nextBatch.sizeInBytes())); @@ -180,6 +184,11 @@ private Optional> nextBatch() { } private Batch readBatch(DefaultRecordBatch batch) { + if (doCrcValidation) { + // Perform a CRC validity check on this batch + batch.ensureValid(); + } + final Batch result; if (batch.isControlBatch()) { result = Batch.control( diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java index 89ad26322299b..92b695146c39f 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java @@ -104,11 +104,12 @@ public static RecordsSnapshotReader of( RawSnapshotReader snapshot, RecordSerde serde, BufferSupplier bufferSupplier, - int maxBatchSize + int maxBatchSize, + boolean doCrcValidation ) { return new RecordsSnapshotReader<>( snapshot.snapshotId(), - new RecordsIterator<>(snapshot.records(), serde, bufferSupplier, maxBatchSize) + new RecordsIterator<>(snapshot.records(), serde, bufferSupplier, maxBatchSize, doCrcValidation) ); } diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java index 4f79dc18cc6ee..a6117a33ca0e5 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java @@ -1112,7 +1112,7 @@ private void assertCommittedData(RaftNode node) { startOffset.set(snapshotId.offset); try (SnapshotReader snapshot = - RecordsSnapshotReader.of(log.readSnapshot(snapshotId).get(), node.intSerde, BufferSupplier.create(), Integer.MAX_VALUE)) { + RecordsSnapshotReader.of(log.readSnapshot(snapshotId).get(), node.intSerde, BufferSupplier.create(), Integer.MAX_VALUE, true)) { // Expect only one batch with only one record assertTrue(snapshot.hasNext()); Batch batch = snapshot.next(); diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java index 6fe540711c268..ae8b1dfb8e2c0 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java @@ -100,7 +100,8 @@ private void testBatchReader( serde, bufferSupplier, MAX_BATCH_BYTES, - closeListener + closeListener, + true ); for (TestBatch batch : expectedBatches) { diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java index 7d98489312059..9dfbfd62fbfed 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java @@ -30,7 +30,9 @@ import java.util.stream.Stream; import net.jqwik.api.ForAll; import net.jqwik.api.Property; +import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.DefaultRecordBatch; import org.apache.kafka.common.record.FileRecords; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Records; @@ -42,6 +44,7 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; import org.mockito.Mockito; +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.assertThrows; @@ -60,7 +63,7 @@ private static Stream emptyRecords() throws IOException { @ParameterizedTest @MethodSource("emptyRecords") void testEmptyRecords(Records records) { - testIterator(Collections.emptyList(), records); + testIterator(Collections.emptyList(), records, true); } @Property @@ -71,7 +74,7 @@ public void testMemoryRecords( List> batches = createBatches(seed); MemoryRecords memRecords = buildRecords(compressionType, batches); - testIterator(batches, memRecords); + testIterator(batches, memRecords, true); } @Property @@ -85,18 +88,58 @@ public void testFileRecords( FileRecords fileRecords = FileRecords.open(TestUtils.tempFile()); fileRecords.append(memRecords); - testIterator(batches, fileRecords); + testIterator(batches, fileRecords, true); + fileRecords.close(); + } + + @Property + public void testCrcValidation( + @ForAll CompressionType compressionType, + @ForAll long seed + ) throws IOException { + List> batches = createBatches(seed); + MemoryRecords memRecords = buildRecords(compressionType, batches); + // Read the Batch CRC for the first batch from the buffer + ByteBuffer readBuf = memRecords.buffer(); + readBuf.position(DefaultRecordBatch.CRC_OFFSET); + int actualCrc = readBuf.getInt(); + // Corrupt the CRC on the first batch + memRecords.buffer().putInt(DefaultRecordBatch.CRC_OFFSET, actualCrc + 1); + + assertThrows(CorruptRecordException.class, () -> testIterator(batches, memRecords, true)); + + FileRecords fileRecords = FileRecords.open(TestUtils.tempFile()); + fileRecords.append(memRecords); + assertThrows(CorruptRecordException.class, () -> testIterator(batches, fileRecords, true)); + + // Verify check does not trigger when doCrcValidation is false + assertDoesNotThrow(() -> testIterator(batches, memRecords, false)); + assertDoesNotThrow(() -> testIterator(batches, fileRecords, false)); + + // Fix the corruption + memRecords.buffer().putInt(DefaultRecordBatch.CRC_OFFSET, actualCrc); + + // Verify check does not trigger when the corruption is fixed + assertDoesNotThrow(() -> testIterator(batches, memRecords, true)); + FileRecords moreFileRecords = FileRecords.open(TestUtils.tempFile()); + moreFileRecords.append(memRecords); + assertDoesNotThrow(() -> testIterator(batches, moreFileRecords, true)); + + fileRecords.close(); + moreFileRecords.close(); } private void testIterator( List> expectedBatches, - Records records + Records records, + boolean validateCrc ) { Set allocatedBuffers = Collections.newSetFromMap(new IdentityHashMap<>()); RecordsIterator iterator = createIterator( records, - mockBufferSupplier(allocatedBuffers) + mockBufferSupplier(allocatedBuffers), + validateCrc ); for (TestBatch batch : expectedBatches) { @@ -111,8 +154,12 @@ private void testIterator( assertEquals(Collections.emptySet(), allocatedBuffers); } - static RecordsIterator createIterator(Records records, BufferSupplier bufferSupplier) { - return new RecordsIterator<>(records, STRING_SERDE, bufferSupplier, Records.HEADER_SIZE_UP_TO_MAGIC); + static RecordsIterator createIterator( + Records records, + BufferSupplier bufferSupplier, + boolean validateCrc + ) { + return new RecordsIterator<>(records, STRING_SERDE, bufferSupplier, Records.HEADER_SIZE_UP_TO_MAGIC, validateCrc); } static BufferSupplier mockBufferSupplier(Set buffers) { diff --git a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java index 05d1929f27157..cd86c709ff980 100644 --- a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java +++ b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java @@ -192,7 +192,8 @@ private SnapshotReader readSnapshot( context.log.readSnapshot(snapshotId).get(), context.serde, BufferSupplier.create(), - maxBatchSize + maxBatchSize, + true ); } @@ -246,7 +247,7 @@ record = records.next(); public static void assertSnapshot(List> batches, RawSnapshotReader reader) { assertSnapshot( batches, - RecordsSnapshotReader.of(reader, new StringSerde(), BufferSupplier.create(), Integer.MAX_VALUE) + RecordsSnapshotReader.of(reader, new StringSerde(), BufferSupplier.create(), Integer.MAX_VALUE, true) ); } From 9fda5fcac6b3f7727af1c6363562da3aea415f77 Mon Sep 17 00:00:00 2001 From: David Mao <47232755+splett2@users.noreply.github.com> Date: Tue, 9 Aug 2022 08:55:38 -0700 Subject: [PATCH 027/175] KAFKA-14144:; Compare AlterPartition LeaderAndIsr before fencing partition epoch (#12489) This PR fixes an AlterPartition regression introduced in https://github.com/apache/kafka/pull/12032 When an AlterPartition request succeeds, the partition epoch gets bumped. In Zk controller mode the sender also relies on the AlterPartition response to be informed of the new partition epoch. If the sender times out the request before a response is sent, the sender will have a stale partition epoch compared to the ZK controller state and will be fenced on subsequent AlterPartition request attempts. The sender will not receive an updated partition epoch until it receives a LeaderAndIsr request for controller-initiated ISR changes. Reviewers: Jason Gustafson --- .../main/scala/kafka/api/LeaderAndIsr.scala | 5 +- .../kafka/controller/KafkaController.scala | 11 +-- .../ControllerIntegrationTest.scala | 82 +++++++++++-------- 3 files changed, 58 insertions(+), 40 deletions(-) diff --git a/core/src/main/scala/kafka/api/LeaderAndIsr.scala b/core/src/main/scala/kafka/api/LeaderAndIsr.scala index dd1d381a14455..da68cdb479c1c 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsr.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsr.scala @@ -62,7 +62,7 @@ case class LeaderAndIsr( if (leader == LeaderAndIsr.NoLeader) None else Some(leader) } - def equalsIgnorePartitionEpoch(other: LeaderAndIsr): Boolean = { + def equalsAllowStalePartitionEpoch(other: LeaderAndIsr): Boolean = { if (this == other) { true } else if (other == null) { @@ -71,7 +71,8 @@ case class LeaderAndIsr( leader == other.leader && leaderEpoch == other.leaderEpoch && isr.equals(other.isr) && - leaderRecoveryState == other.leaderRecoveryState + leaderRecoveryState == other.leaderRecoveryState && + partitionEpoch <= other.partitionEpoch } } diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 8d16eb7e1da8c..d179bcd6ca475 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -2339,14 +2339,15 @@ class KafkaController(val config: KafkaConfig, if (newLeaderAndIsr.leaderEpoch != currentLeaderAndIsr.leaderEpoch) { partitionResponses(tp) = Left(Errors.FENCED_LEADER_EPOCH) None - } else if (newLeaderAndIsr.partitionEpoch < currentLeaderAndIsr.partitionEpoch) { - partitionResponses(tp) = Left(Errors.INVALID_UPDATE_VERSION) - None - } else if (newLeaderAndIsr.equalsIgnorePartitionEpoch(currentLeaderAndIsr)) { + } else if (newLeaderAndIsr.equalsAllowStalePartitionEpoch(currentLeaderAndIsr)) { // If a partition is already in the desired state, just return it + // this check must be done before fencing based on partition epoch to maintain idempotency partitionResponses(tp) = Right(currentLeaderAndIsr) None - } else if (newLeaderAndIsr.leaderRecoveryState == LeaderRecoveryState.RECOVERING && newLeaderAndIsr.isr.length > 1) { + } else if (newLeaderAndIsr.partitionEpoch != currentLeaderAndIsr.partitionEpoch) { + partitionResponses(tp) = Left(Errors.INVALID_UPDATE_VERSION) + None + } else if (newLeaderAndIsr.leaderRecoveryState == LeaderRecoveryState.RECOVERING && newLeaderAndIsr.isr.length > 1) { partitionResponses(tp) = Left(Errors.INVALID_REQUEST) info( s"Rejecting AlterPartition from node $brokerId for $tp because leader is recovering and ISR is greater than 1: " + diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala index 57cbeafd4d033..7618889437139 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala @@ -1002,47 +1002,58 @@ class ControllerIntegrationTest extends QuorumTestHarness { val controller = getController().kafkaController val leaderIsrAndControllerEpochMap = zkClient.getTopicPartitionStates(Seq(tp)) - val newLeaderAndIsr = leaderIsrAndControllerEpochMap(tp).leaderAndIsr + val oldLeaderAndIsr = leaderIsrAndControllerEpochMap(tp).leaderAndIsr + val newIsr = List(oldLeaderAndIsr.leader) + val newPartitionEpoch = oldLeaderAndIsr.partitionEpoch + 1 val topicId = controller.controllerContext.topicIds(tp.topic) val brokerId = otherBroker.config.brokerId val brokerEpoch = controller.controllerContext.liveBrokerIdAndEpochs(otherBroker.config.brokerId) - // When re-sending the current ISR, we should not get and error or any ISR changes - val alterPartitionRequest = new AlterPartitionRequestData() - .setBrokerId(brokerId) - .setBrokerEpoch(brokerEpoch) - .setTopics(Seq(new AlterPartitionRequestData.TopicData() - .setTopicId(topicId) - .setPartitions(Seq(new AlterPartitionRequestData.PartitionData() - .setPartitionIndex(tp.partition) - .setLeaderEpoch(newLeaderAndIsr.leaderEpoch) - .setPartitionEpoch(newLeaderAndIsr.partitionEpoch) - .setNewIsr(newLeaderAndIsr.isr.map(Int.box).asJava) - .setLeaderRecoveryState(newLeaderAndIsr.leaderRecoveryState.value) + def sendAndVerifyAlterPartitionResponse(requestPartitionEpoch: Int): Unit = { + val alterPartitionRequest = new AlterPartitionRequestData() + .setBrokerId(brokerId) + .setBrokerEpoch(brokerEpoch) + .setTopics(Seq(new AlterPartitionRequestData.TopicData() + .setTopicId(topicId) + .setPartitions(Seq(new AlterPartitionRequestData.PartitionData() + .setPartitionIndex(tp.partition) + .setLeaderEpoch(oldLeaderAndIsr.leaderEpoch) + .setPartitionEpoch(requestPartitionEpoch) + .setNewIsr(newIsr.map(Int.box).asJava) + .setLeaderRecoveryState(oldLeaderAndIsr.leaderRecoveryState.value) + ).asJava) ).asJava) - ).asJava) - val future = new CompletableFuture[AlterPartitionResponseData]() - controller.eventManager.put(AlterPartitionReceived( - alterPartitionRequest, - AlterPartitionRequestData.HIGHEST_SUPPORTED_VERSION, - future.complete - )) + val future = new CompletableFuture[AlterPartitionResponseData]() + controller.eventManager.put(AlterPartitionReceived( + alterPartitionRequest, + AlterPartitionRequestData.HIGHEST_SUPPORTED_VERSION, + future.complete + )) - val expectedAlterPartitionResponse = new AlterPartitionResponseData() - .setTopics(Seq(new AlterPartitionResponseData.TopicData() - .setTopicId(topicId) - .setPartitions(Seq(new AlterPartitionResponseData.PartitionData() - .setPartitionIndex(tp.partition) - .setLeaderId(brokerId) - .setLeaderEpoch(newLeaderAndIsr.leaderEpoch) - .setPartitionEpoch(newLeaderAndIsr.partitionEpoch) - .setIsr(newLeaderAndIsr.isr.map(Int.box).asJava) - .setLeaderRecoveryState(newLeaderAndIsr.leaderRecoveryState.value) + // When re-sending an ISR update, we should not get and error or any ISR changes + val expectedAlterPartitionResponse = new AlterPartitionResponseData() + .setTopics(Seq(new AlterPartitionResponseData.TopicData() + .setTopicId(topicId) + .setPartitions(Seq(new AlterPartitionResponseData.PartitionData() + .setPartitionIndex(tp.partition) + .setLeaderId(brokerId) + .setLeaderEpoch(oldLeaderAndIsr.leaderEpoch) + .setPartitionEpoch(newPartitionEpoch) + .setIsr(newIsr.map(Int.box).asJava) + .setLeaderRecoveryState(oldLeaderAndIsr.leaderRecoveryState.value) + ).asJava) ).asJava) - ).asJava) + assertEquals(expectedAlterPartitionResponse, future.get(10, TimeUnit.SECONDS)) + } - assertEquals(expectedAlterPartitionResponse, future.get(10, TimeUnit.SECONDS)) + // send a request, expect the partition epoch to be incremented + sendAndVerifyAlterPartitionResponse(oldLeaderAndIsr.partitionEpoch) + + // re-send the same request with various partition epochs (less/equal/greater than the current + // epoch), expect it to succeed while the partition epoch remains the same + sendAndVerifyAlterPartitionResponse(oldLeaderAndIsr.partitionEpoch) + sendAndVerifyAlterPartitionResponse(newPartitionEpoch) } @Test @@ -1100,7 +1111,6 @@ class ControllerIntegrationTest extends QuorumTestHarness { assertAlterPartition( partitionError = Errors.UNKNOWN_TOPIC_ID, - topicPartition = tp, topicIdOpt = Some(Uuid.randomUuid()) ) @@ -1118,9 +1128,15 @@ class ControllerIntegrationTest extends QuorumTestHarness { assertAlterPartition( partitionError = Errors.INVALID_UPDATE_VERSION, + isr = Set(leaderId), partitionEpoch = partitionEpoch - 1 ) + assertAlterPartition( + partitionError = Errors.INVALID_UPDATE_VERSION, + partitionEpoch = partitionEpoch + 1 + ) + assertAlterPartition( partitionError = Errors.FENCED_LEADER_EPOCH, leaderEpoch = leaderEpoch - 1 From ce7d0f5fa51586c00148370c1fecd058b078c3e5 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Tue, 9 Aug 2022 13:24:24 -0700 Subject: [PATCH 028/175] MINOR: BrokerMetadataSnapshotter must avoid exceeding batch size (#12486) BrokerMetadataSnapshotter should split up record lists that exceed the batch size. Reviewers: David Arthur --- .../metadata/BrokerMetadataSnapshotter.scala | 36 ++++++++++++++++++- .../BrokerMetadataSnapshotterTest.scala | 30 ++++++++++++++++ .../controller/SnapshotGeneratorTest.java | 3 +- 3 files changed, 67 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataSnapshotter.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataSnapshotter.scala index dd77b277c8b61..2a236ca749706 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataSnapshotter.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataSnapshotter.scala @@ -24,18 +24,50 @@ import org.apache.kafka.queue.{EventQueue, KafkaEventQueue} import org.apache.kafka.server.common.ApiMessageAndVersion import org.apache.kafka.snapshot.SnapshotWriter +import java.util.function.Consumer + trait SnapshotWriterBuilder { def build(committedOffset: Long, committedEpoch: Int, lastContainedLogTime: Long): Option[SnapshotWriter[ApiMessageAndVersion]] } +/** + * The RecordListConsumer takes as input a potentially long list of records, and feeds the + * SnapshotWriter a series of smaller lists of records. + * + * Note: from the perspective of Kafka, the snapshot file is really just a list of records, + * and we don't care about batches. Batching is irrelevant to the meaning of the snapshot. + */ +class RecordListConsumer( + val maxRecordsInBatch: Int, + val writer: SnapshotWriter[ApiMessageAndVersion] +) extends Consumer[java.util.List[ApiMessageAndVersion]] { + override def accept(messages: java.util.List[ApiMessageAndVersion]): Unit = { + var i = 0 + while (i < messages.size()) { + writer.append(messages.subList(i, Math.min(i + maxRecordsInBatch, messages.size()))); + i += maxRecordsInBatch + } + } +} + class BrokerMetadataSnapshotter( brokerId: Int, val time: Time, threadNamePrefix: Option[String], writerBuilder: SnapshotWriterBuilder ) extends Logging with MetadataSnapshotter { + /** + * The maximum number of records we will put in each batch. + * + * From the perspective of the Raft layer, the limit on batch size is specified in terms of + * bytes, not number of records. @See {@link KafkaRaftClient#MAX_BATCH_SIZE_BYTES} for details. + * However, it's more convenient to limit the batch size here in terms of number of records. + * So we chose a low number that will not cause problems. + */ + private val maxRecordsInBatch = 1024 + private val logContext = new LogContext(s"[BrokerMetadataSnapshotter id=$brokerId] ") logIdent = logContext.logPrefix() @@ -77,9 +109,11 @@ class BrokerMetadataSnapshotter( class CreateSnapshotEvent(image: MetadataImage, writer: SnapshotWriter[ApiMessageAndVersion]) extends EventQueue.Event { + override def run(): Unit = { try { - image.write(writer.append(_)) + val consumer = new RecordListConsumer(maxRecordsInBatch, writer) + image.write(consumer) writer.freeze() } finally { try { diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataSnapshotterTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataSnapshotterTest.scala index e6702ee287fef..ff2326c92fa37 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataSnapshotterTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataSnapshotterTest.scala @@ -21,6 +21,7 @@ import java.nio.ByteBuffer import java.util.Optional import java.util.concurrent.{CompletableFuture, CountDownLatch} import org.apache.kafka.common.memory.MemoryPool +import org.apache.kafka.common.metadata.FenceBrokerRecord import org.apache.kafka.common.protocol.ByteBufferAccessor import org.apache.kafka.common.record.{CompressionType, MemoryRecords} import org.apache.kafka.common.utils.Time @@ -33,6 +34,8 @@ import org.apache.kafka.snapshot.{MockRawSnapshotWriter, RecordsSnapshotWriter, import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} import org.junit.jupiter.api.Test +import java.util +import java.util.Arrays.asList import scala.compat.java8.OptionConverters._ class BrokerMetadataSnapshotterTest { @@ -104,4 +107,31 @@ class BrokerMetadataSnapshotterTest { snapshotter.close() } } + + class MockSnapshotWriter extends SnapshotWriter[ApiMessageAndVersion] { + val batches = new util.ArrayList[util.List[ApiMessageAndVersion]] + override def snapshotId(): OffsetAndEpoch = new OffsetAndEpoch(0, 0) + override def lastContainedLogOffset(): Long = 0 + override def lastContainedLogEpoch(): Int = 0 + override def isFrozen: Boolean = false + override def append(batch: util.List[ApiMessageAndVersion]): Unit = batches.add(batch) + override def freeze(): Unit = {} + override def close(): Unit = {} + } + + @Test + def testRecordListConsumer(): Unit = { + val writer = new MockSnapshotWriter() + val consumer = new RecordListConsumer(3, writer) + val m = new ApiMessageAndVersion(new FenceBrokerRecord().setId(1).setEpoch(1), 0.toShort) + consumer.accept(asList(m, m)) + assertEquals(asList(asList(m, m)), writer.batches) + consumer.accept(asList(m)) + assertEquals(asList(asList(m, m), asList(m)), writer.batches) + consumer.accept(asList(m, m, m, m)) + assertEquals(asList(asList(m, m), asList(m), asList(m, m, m), asList(m)), writer.batches) + consumer.accept(asList(m, m, m, m, m, m, m, m)) + assertEquals(asList(asList(m, m), asList(m), asList(m, m, m), asList(m), asList(m, m, m), asList(m, m, m), asList(m, m)), + writer.batches) + } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java b/metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java index 2c61dbcdc74df..f7fa18f20a4cf 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java @@ -41,6 +41,7 @@ import java.util.OptionalLong; import java.util.Optional; +import static org.apache.kafka.raft.KafkaRaftClient.MAX_BATCH_SIZE_BYTES; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -94,7 +95,7 @@ private SnapshotWriter createSnapshotWriter( ) { return RecordsSnapshotWriter.createWithHeader( () -> createNewSnapshot(new OffsetAndEpoch(committedOffset + 1, 1)), - 1024, + MAX_BATCH_SIZE_BYTES, MemoryPool.NONE, new MockTime(), lastContainedLogTime, From 8db3f0998c61bd45a57ca60d17ec238f9864b7b1 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Thu, 4 Aug 2022 22:49:45 -0700 Subject: [PATCH 029/175] KAFKA-14124: improve quorum controller fault handling (#12447) Before trying to commit a batch of records to the __cluster_metadata log, the active controller should try to apply them to its current in-memory state. If this application process fails, the active controller process should exit, allowing another node to take leadership. This will prevent most bad metadata records from ending up in the log and help to surface errors during testing. Similarly, if the active controller attempts to renounce leadership, and the renunciation process itself fails, the process should exit. This will help avoid bugs where the active controller continues in an undefined state. In contrast, standby controllers that experience metadata application errors should continue on, in order to avoid a scenario where a bad record brings down the whole controller cluster. The intended effect of these changes is to make it harder to commit a bad record to the metadata log, but to continue to ride out the bad record as well as possible if such a record does get committed. This PR introduces the FaultHandler interface to implement these concepts. In junit tests, we use a FaultHandler implementation which does not exit the process. This allows us to avoid terminating the gradle test runner, which would be very disruptive. It also allows us to ensure that the test surfaces these exceptions, which we previously were not doing (the mock fault handler stores the exception). In addition to the above, this PR fixes a bug where RaftClient#resign was not being called from the renounce() function. This bug could have resulted in the raft layer not being informed of an active controller resigning. Reviewers: David Arthur --- build.gradle | 2 + checkstyle/import-control-core.xml | 1 + checkstyle/import-control.xml | 4 + checkstyle/suppressions.xml | 2 + .../scala/kafka/server/ControllerServer.scala | 10 +- .../scala/kafka/server/KafkaRaftServer.scala | 6 +- .../kafka/testkit/KafkaClusterTestKit.java | 36 +- .../kafka/server/QuorumTestHarness.scala | 6 + .../kafka/controller/QuorumController.java | 382 +++++++++++------- .../fault/MetadataFaultException.java | 32 ++ .../metadata/fault/MetadataFaultHandler.java | 36 ++ .../controller/QuorumControllerTest.java | 25 ++ .../controller/QuorumControllerTestEnv.java | 15 + .../kafka/server/fault/FaultHandler.java | 58 +++ .../fault/ProcessExitingFaultHandler.java | 37 ++ .../kafka/server/fault/MockFaultHandler.java | 65 +++ .../fault/MockFaultHandlerException.java | 38 ++ 17 files changed, 586 insertions(+), 169 deletions(-) create mode 100644 metadata/src/main/java/org/apache/kafka/metadata/fault/MetadataFaultException.java create mode 100644 metadata/src/main/java/org/apache/kafka/metadata/fault/MetadataFaultHandler.java create mode 100644 server-common/src/main/java/org/apache/kafka/server/fault/FaultHandler.java create mode 100644 server-common/src/main/java/org/apache/kafka/server/fault/ProcessExitingFaultHandler.java create mode 100644 server-common/src/test/java/org/apache/kafka/server/fault/MockFaultHandler.java create mode 100644 server-common/src/test/java/org/apache/kafka/server/fault/MockFaultHandlerException.java diff --git a/build.gradle b/build.gradle index 73c36c31170fa..aa930ce1bff00 100644 --- a/build.gradle +++ b/build.gradle @@ -875,6 +875,7 @@ project(':core') { testImplementation project(':clients').sourceSets.test.output testImplementation project(':metadata').sourceSets.test.output testImplementation project(':raft').sourceSets.test.output + testImplementation project(':server-common').sourceSets.test.output testImplementation libs.bcpkix testImplementation libs.mockitoCore testImplementation(libs.apacheda) { @@ -1157,6 +1158,7 @@ project(':metadata') { testImplementation libs.slf4jlog4j testImplementation project(':clients').sourceSets.test.output testImplementation project(':raft').sourceSets.test.output + testImplementation project(':server-common').sourceSets.test.output generator project(':generator') } diff --git a/checkstyle/import-control-core.xml b/checkstyle/import-control-core.xml index 28b325b093dae..4042cba402fdd 100644 --- a/checkstyle/import-control-core.xml +++ b/checkstyle/import-control-core.xml @@ -54,6 +54,7 @@ + diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 211d23ff60aeb..4b07a26cba5c9 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -232,6 +232,7 @@ + @@ -276,6 +277,9 @@ + + + diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index f6ca0d02fe316..bec3da1637a9d 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -39,6 +39,8 @@ + controllerBuilder.setAuthorizer(a) diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala index 07a3118372046..e7cf8f8f1fab5 100644 --- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala +++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala @@ -29,9 +29,11 @@ import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.utils.{AppInfoParser, Time} import org.apache.kafka.common.{KafkaException, Uuid} import org.apache.kafka.controller.BootstrapMetadata +import org.apache.kafka.metadata.fault.MetadataFaultHandler import org.apache.kafka.metadata.{KafkaConfigSchema, MetadataRecordSerde} import org.apache.kafka.raft.RaftConfig import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} +import org.apache.kafka.server.fault.ProcessExitingFaultHandler import org.apache.kafka.server.metrics.KafkaYammerMetrics import java.nio.file.Paths @@ -106,7 +108,9 @@ class KafkaRaftServer( controllerQuorumVotersFuture, KafkaRaftServer.configSchema, raftManager.apiVersions, - bootstrapMetadata + bootstrapMetadata, + new MetadataFaultHandler(), + new ProcessExitingFaultHandler(), )) } else { None diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java index c961d71bbe588..42120324f5f83 100644 --- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java +++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java @@ -40,6 +40,7 @@ import org.apache.kafka.raft.RaftConfig; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.fault.MockFaultHandler; import org.apache.kafka.test.TestUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -115,6 +116,8 @@ public void close() { public static class Builder { private TestKitNodes nodes; private Map configProps = new HashMap<>(); + private MockFaultHandler metadataFaultHandler = new MockFaultHandler("metadataFaultHandler"); + private MockFaultHandler fatalFaultHandler = new MockFaultHandler("fatalFaultHandler"); public Builder(TestKitNodes nodes) { this.nodes = nodes; @@ -190,7 +193,9 @@ metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftS connectFutureManager.future, KafkaRaftServer.configSchema(), raftManager.apiVersions(), - bootstrapMetadata + bootstrapMetadata, + metadataFaultHandler, + fatalFaultHandler ); controllers.put(node.id(), controller); controller.socketServerFirstBoundPortFuture().whenComplete((port, e) -> { @@ -273,7 +278,8 @@ metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftS throw e; } return new KafkaClusterTestKit(executorService, nodes, controllers, - brokers, raftManagers, connectFutureManager, baseDirectory); + brokers, raftManagers, connectFutureManager, baseDirectory, + metadataFaultHandler, fatalFaultHandler); } private String listeners(int node) { @@ -314,14 +320,20 @@ static private void setupNodeDirectories(File baseDirectory, private final Map> raftManagers; private final ControllerQuorumVotersFutureManager controllerQuorumVotersFutureManager; private final File baseDirectory; - - private KafkaClusterTestKit(ExecutorService executorService, - TestKitNodes nodes, - Map controllers, - Map brokers, - Map> raftManagers, - ControllerQuorumVotersFutureManager controllerQuorumVotersFutureManager, - File baseDirectory) { + private final MockFaultHandler metadataFaultHandler; + private final MockFaultHandler fatalFaultHandler; + + private KafkaClusterTestKit( + ExecutorService executorService, + TestKitNodes nodes, + Map controllers, + Map brokers, + Map> raftManagers, + ControllerQuorumVotersFutureManager controllerQuorumVotersFutureManager, + File baseDirectory, + MockFaultHandler metadataFaultHandler, + MockFaultHandler fatalFaultHandler + ) { this.executorService = executorService; this.nodes = nodes; this.controllers = controllers; @@ -329,6 +341,8 @@ private KafkaClusterTestKit(ExecutorService executorService, this.raftManagers = raftManagers; this.controllerQuorumVotersFutureManager = controllerQuorumVotersFutureManager; this.baseDirectory = baseDirectory; + this.metadataFaultHandler = metadataFaultHandler; + this.fatalFaultHandler = fatalFaultHandler; } public void format() throws Exception { @@ -520,6 +534,8 @@ public void close() throws Exception { executorService.shutdownNow(); executorService.awaitTermination(5, TimeUnit.MINUTES); } + metadataFaultHandler.maybeRethrowFirstException(); + fatalFaultHandler.maybeRethrowFirstException(); } private void waitForAllFutures(List>> futureEntries) diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala index b82f86a8cb335..766eb3a7e535a 100755 --- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala +++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala @@ -36,6 +36,7 @@ import org.apache.kafka.controller.BootstrapMetadata import org.apache.kafka.metadata.MetadataRecordSerde import org.apache.kafka.raft.RaftConfig.{AddressSpec, InetAddressSpec} import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} +import org.apache.kafka.server.fault.MockFaultHandler import org.apache.zookeeper.client.ZKClientConfig import org.apache.zookeeper.{WatchedEvent, Watcher, ZooKeeper} import org.junit.jupiter.api.Assertions._ @@ -179,6 +180,8 @@ abstract class QuorumTestHarness extends Logging { } } + val faultHandler = new MockFaultHandler("quorumTestHarnessFaultHandler") + // Note: according to the junit documentation: "JUnit Jupiter does not guarantee the execution // order of multiple @BeforeEach methods that are declared within a single test class or test // interface." Therefore, if you have things you would like to do before each test case runs, it @@ -296,6 +299,8 @@ abstract class QuorumTestHarness extends Logging { configSchema = KafkaRaftServer.configSchema, raftApiVersions = raftManager.apiVersions, bootstrapMetadata = BootstrapMetadata.create(metadataVersion, bootstrapRecords.asJava), + metadataFaultHandler = faultHandler, + fatalFaultHandler = faultHandler, ) controllerServer.socketServerFirstBoundPortFuture.whenComplete((port, e) => { if (e != null) { @@ -362,6 +367,7 @@ abstract class QuorumTestHarness extends Logging { } System.clearProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM) Configuration.setConfiguration(null) + faultHandler.maybeRethrowFirstException() } // Trigger session expiry by reusing the session id in another client diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index 0290e0040c2d2..a4cc1d92cba4b 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -91,6 +91,7 @@ import org.apache.kafka.server.authorizer.AclDeleteResult; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.fault.FaultHandler; import org.apache.kafka.server.policy.AlterConfigPolicy; import org.apache.kafka.server.policy.CreateTopicPolicy; import org.apache.kafka.snapshot.SnapshotReader; @@ -149,6 +150,8 @@ public final class QuorumController implements Controller { static public class Builder { private final int nodeId; private final String clusterId; + private FaultHandler fatalFaultHandler = null; + private FaultHandler metadataFaultHandler = null; private Time time = Time.SYSTEM; private String threadNamePrefix = null; private LogContext logContext = null; @@ -175,6 +178,16 @@ public Builder(int nodeId, String clusterId) { this.clusterId = clusterId; } + public Builder setFatalFaultHandler(FaultHandler fatalFaultHandler) { + this.fatalFaultHandler = fatalFaultHandler; + return this; + } + + public Builder setMetadataFaultHandler(FaultHandler metadataFaultHandler) { + this.metadataFaultHandler = metadataFaultHandler; + return this; + } + public int nodeId() { return nodeId; } @@ -287,6 +300,10 @@ public QuorumController build() throws Exception { throw new IllegalStateException("You must specify an initial metadata.version using the kafka-storage tool."); } else if (quorumFeatures == null) { throw new IllegalStateException("You must specify the quorum features"); + } else if (fatalFaultHandler == null) { + throw new IllegalStateException("You must specify a fatal fault handler."); + } else if (metadataFaultHandler == null) { + throw new IllegalStateException("You must specify a metadata fault handler."); } if (threadNamePrefix == null) { @@ -304,6 +321,8 @@ public QuorumController build() throws Exception { try { queue = new KafkaEventQueue(time, logContext, threadNamePrefix + "QuorumController"); return new QuorumController( + fatalFaultHandler, + metadataFaultHandler, logContext, nodeId, clusterId, @@ -426,12 +445,18 @@ private Throwable handleEventException(String name, exception.getClass().getSimpleName(), deltaUs); return exception; } - log.warn("{}: failed with unknown server exception {} at epoch {} in {} us. " + - "Renouncing leadership and reverting to the last committed offset {}.", - name, exception.getClass().getSimpleName(), curClaimEpoch, deltaUs, - lastCommittedOffset, exception); - raftClient.resign(curClaimEpoch); - renounce(); + if (isActiveController()) { + log.warn("{}: failed with unknown server exception {} at epoch {} in {} us. " + + "Renouncing leadership and reverting to the last committed offset {}.", + name, exception.getClass().getSimpleName(), curClaimEpoch, deltaUs, + lastCommittedOffset, exception); + renounce(); + } else { + log.warn("{}: failed with unknown server exception {} in {} us. " + + "The controller is already in standby mode.", + name, exception.getClass().getSimpleName(), deltaUs, + exception); + } return new UnknownServerException(exception); } @@ -702,7 +727,7 @@ public void run() throws Exception { long now = time.nanoseconds(); controllerMetrics.updateEventQueueTime(NANOSECONDS.toMillis(now - eventCreatedTimeNs)); int controllerEpoch = curClaimEpoch; - if (controllerEpoch == -1) { + if (!isActiveController()) { throw newNotControllerException(); } startProcessingTimeNs = OptionalLong.of(now); @@ -728,9 +753,26 @@ public void run() throws Exception { "reaches offset {}", this, resultAndOffset.offset()); } } else { - // If the operation returned a batch of records, those records need to be - // written before we can return our result to the user. Here, we hand off - // the batch of records to the raft client. They will be written out + // Start by trying to apply the record to our in-memory state. This should always + // succeed; if it does not, that's a fatal error. It is important to do this before + // scheduling the record for Raft replication. + int i = 1; + for (ApiMessageAndVersion message : result.records()) { + try { + replay(message.message(), Optional.empty()); + } catch (Throwable e) { + String failureMessage = String.format("Unable to apply %s record, which was " + + "%d of %d record(s) in the batch following last writeOffset %d.", + message.message().getClass().getSimpleName(), i, result.records().size(), + writeOffset); + fatalFaultHandler.handleFault(failureMessage, e); + } + i++; + } + + // If the operation returned a batch of records, and those records could be applied, + // they need to be written before we can return our result to the user. Here, we + // hand off the batch of records to the raft client. They will be written out // asynchronously. final long offset; if (result.isAtomic()) { @@ -741,9 +783,6 @@ public void run() throws Exception { op.processBatchEndOffset(offset); updateWriteOffset(offset); resultAndOffset = ControllerResultAndOffset.of(offset, result); - for (ApiMessageAndVersion message : result.records()) { - replay(message.message(), Optional.empty(), offset); - } snapshotRegistry.getOrCreateSnapshot(offset); log.debug("Read-write operation {} will be completed when the log " + @@ -789,9 +828,9 @@ private CompletableFuture prependWriteEvent(String name, return event.future(); } - private CompletableFuture appendWriteEvent(String name, - OptionalLong deadlineNs, - ControllerWriteOperation op) { + CompletableFuture appendWriteEvent(String name, + OptionalLong deadlineNs, + ControllerWriteOperation op) { ControllerWriteEvent event = new ControllerWriteEvent<>(name, op); if (deadlineNs.isPresent()) { queue.appendWithDeadline(deadlineNs.getAsLong(), event); @@ -841,11 +880,20 @@ public void handleCommit(BatchReader reader) { "offset {} and epoch {}.", offset, epoch); } } - for (ApiMessageAndVersion messageAndVersion : messages) { - replay(messageAndVersion.message(), Optional.empty(), offset); + int i = 1; + for (ApiMessageAndVersion message : messages) { + try { + replay(message.message(), Optional.empty()); + } catch (Throwable e) { + String failureMessage = String.format("Unable to apply %s record on standby " + + "controller, which was %d of %d record(s) in the batch with baseOffset %d.", + message.message().getClass().getSimpleName(), i, messages.size(), + batch.baseOffset()); + metadataFaultHandler.handleFault(failureMessage, e); + } + i++; } } - updateLastCommittedState(offset, epoch, batch.appendTimestamp()); processedRecordsSize += batch.sizeInBytes(); } @@ -862,13 +910,9 @@ public void handleSnapshot(SnapshotReader reader) { appendRaftEvent(String.format("handleSnapshot[snapshotId=%s]", reader.snapshotId()), () -> { try { if (isActiveController()) { - throw new IllegalStateException( - String.format( - "Asked to load snapshot (%s) when it is the active controller (%d)", - reader.snapshotId(), - curClaimEpoch - ) - ); + fatalFaultHandler.handleFault(String.format("Asked to load snapshot " + + "(%s) when it is the active controller (%d)", reader.snapshotId(), + curClaimEpoch)); } log.info("Starting to replay snapshot ({}), from last commit offset ({}) and epoch ({})", reader.snapshotId(), lastCommittedOffset, lastCommittedEpoch); @@ -882,26 +926,28 @@ public void handleSnapshot(SnapshotReader reader) { if (log.isDebugEnabled()) { if (log.isTraceEnabled()) { - log.trace( - "Replaying snapshot ({}) batch with last offset of {}: {}", - reader.snapshotId(), - offset, - messages - .stream() - .map(ApiMessageAndVersion::toString) - .collect(Collectors.joining(", ")) - ); + log.trace("Replaying snapshot ({}) batch with last offset of {}: {}", + reader.snapshotId(), offset, messages.stream().map(ApiMessageAndVersion::toString). + collect(Collectors.joining(", "))); } else { - log.debug( - "Replaying snapshot ({}) batch with last offset of {}", - reader.snapshotId(), - offset - ); + log.debug("Replaying snapshot ({}) batch with last offset of {}", + reader.snapshotId(), offset); } } - for (ApiMessageAndVersion messageAndVersion : messages) { - replay(messageAndVersion.message(), Optional.of(reader.snapshotId()), offset); + int i = 1; + for (ApiMessageAndVersion message : messages) { + try { + replay(message.message(), Optional.of(reader.snapshotId())); + } catch (Throwable e) { + String failureMessage = String.format("Unable to apply %s record " + + "from snapshot %s on standby controller, which was %d of " + + "%d record(s) in the batch with baseOffset %d.", + message.message().getClass().getSimpleName(), reader.snapshotId(), + i, messages.size(), batch.baseOffset()); + metadataFaultHandler.handleFault(failureMessage, e); + } + i++; } } updateLastCommittedState( @@ -968,10 +1014,14 @@ public void handleLeaderChange(LeaderAndEpoch newLeader) { if (exception != null) { log.error("Failed to bootstrap metadata.", exception); appendRaftEvent("bootstrapMetadata[" + curClaimEpoch + "]", () -> { - log.warn("Renouncing the leadership at oldEpoch {} since we could not bootstrap " + - "metadata. Reverting to last committed offset {}.", - curClaimEpoch, lastCommittedOffset); - renounce(); + if (isActiveController()) { + log.warn("Renouncing the leadership at oldEpoch {} since we could not bootstrap " + + "metadata. Reverting to last committed offset {}.", + curClaimEpoch, lastCommittedOffset); + renounce(); + } else { + log.warn("Unable to bootstrap metadata on standby controller."); + } }); } }); @@ -998,10 +1048,12 @@ public void handleLeaderChange(LeaderAndEpoch newLeader) { }); } else if (isActiveController()) { appendRaftEvent("handleRenounce[" + curClaimEpoch + "]", () -> { - log.warn("Renouncing the leadership at oldEpoch {} due to a metadata " + - "log event. Reverting to last committed offset {}.", curClaimEpoch, - lastCommittedOffset); - renounce(); + if (isActiveController()) { + log.warn("Renouncing the leadership at oldEpoch {} due to a metadata " + + "log event. Reverting to last committed offset {}.", curClaimEpoch, + lastCommittedOffset); + renounce(); + } }); } } @@ -1078,26 +1130,34 @@ private void updateLastCommittedState(long offset, int epoch, long timestamp) { } private void renounce() { - curClaimEpoch = -1; - controllerMetrics.setActive(false); - purgatory.failAll(newNotControllerException()); - - if (snapshotRegistry.hasSnapshot(lastCommittedOffset)) { - newBytesSinceLastSnapshot = 0; - snapshotRegistry.revertToSnapshot(lastCommittedOffset); - authorizer.ifPresent(a -> a.loadSnapshot(aclControlManager.idToAcl())); - } else { - resetState(); - raftClient.unregister(metaLogListener); - metaLogListener = new QuorumMetaLogListener(); - raftClient.register(metaLogListener); + try { + if (curClaimEpoch == -1) { + throw new RuntimeException("Cannot renounce leadership because we are not the " + + "current leader."); + } + raftClient.resign(curClaimEpoch); + curClaimEpoch = -1; + controllerMetrics.setActive(false); + purgatory.failAll(newNotControllerException()); + + if (snapshotRegistry.hasSnapshot(lastCommittedOffset)) { + newBytesSinceLastSnapshot = 0; + snapshotRegistry.revertToSnapshot(lastCommittedOffset); + authorizer.ifPresent(a -> a.loadSnapshot(aclControlManager.idToAcl())); + } else { + resetState(); + raftClient.unregister(metaLogListener); + metaLogListener = new QuorumMetaLogListener(); + raftClient.register(metaLogListener); + } + updateWriteOffset(-1); + clusterControl.deactivate(); + cancelMaybeFenceReplicas(); + cancelMaybeBalancePartitionLeaders(); + cancelNextWriteNoOpRecord(); + } catch (Throwable e) { + fatalFaultHandler.handleFault("exception while renouncing leadership", e); } - - updateWriteOffset(-1); - clusterControl.deactivate(); - cancelMaybeFenceReplicas(); - cancelMaybeBalancePartitionLeaders(); - cancelNextWriteNoOpRecord(); } private void scheduleDeferredWriteEvent(String name, long deadlineNs, @@ -1246,70 +1306,60 @@ private void handleFeatureControlChange() { } @SuppressWarnings("unchecked") - private void replay(ApiMessage message, Optional snapshotId, long offset) { - try { - MetadataRecordType type = MetadataRecordType.fromId(message.apiKey()); - switch (type) { - case REGISTER_BROKER_RECORD: - clusterControl.replay((RegisterBrokerRecord) message); - break; - case UNREGISTER_BROKER_RECORD: - clusterControl.replay((UnregisterBrokerRecord) message); - break; - case TOPIC_RECORD: - replicationControl.replay((TopicRecord) message); - break; - case PARTITION_RECORD: - replicationControl.replay((PartitionRecord) message); - break; - case CONFIG_RECORD: - configurationControl.replay((ConfigRecord) message); - break; - case PARTITION_CHANGE_RECORD: - replicationControl.replay((PartitionChangeRecord) message); - break; - case FENCE_BROKER_RECORD: - clusterControl.replay((FenceBrokerRecord) message); - break; - case UNFENCE_BROKER_RECORD: - clusterControl.replay((UnfenceBrokerRecord) message); - break; - case REMOVE_TOPIC_RECORD: - replicationControl.replay((RemoveTopicRecord) message); - break; - case FEATURE_LEVEL_RECORD: - featureControl.replay((FeatureLevelRecord) message); - handleFeatureControlChange(); - break; - case CLIENT_QUOTA_RECORD: - clientQuotaControlManager.replay((ClientQuotaRecord) message); - break; - case PRODUCER_IDS_RECORD: - producerIdControlManager.replay((ProducerIdsRecord) message); - break; - case BROKER_REGISTRATION_CHANGE_RECORD: - clusterControl.replay((BrokerRegistrationChangeRecord) message); - break; - case ACCESS_CONTROL_ENTRY_RECORD: - aclControlManager.replay((AccessControlEntryRecord) message, snapshotId); - break; - case REMOVE_ACCESS_CONTROL_ENTRY_RECORD: - aclControlManager.replay((RemoveAccessControlEntryRecord) message, snapshotId); - break; - case NO_OP_RECORD: - // NoOpRecord is an empty record and doesn't need to be replayed - break; - default: - throw new RuntimeException("Unhandled record type " + type); - } - } catch (Exception e) { - if (snapshotId.isPresent()) { - log.error("Error replaying record {} from snapshot {} at last offset {}.", - message.toString(), snapshotId.get(), offset, e); - } else { - log.error("Error replaying record {} at last offset {}.", - message.toString(), offset, e); - } + private void replay(ApiMessage message, Optional snapshotId) { + MetadataRecordType type = MetadataRecordType.fromId(message.apiKey()); + switch (type) { + case REGISTER_BROKER_RECORD: + clusterControl.replay((RegisterBrokerRecord) message); + break; + case UNREGISTER_BROKER_RECORD: + clusterControl.replay((UnregisterBrokerRecord) message); + break; + case TOPIC_RECORD: + replicationControl.replay((TopicRecord) message); + break; + case PARTITION_RECORD: + replicationControl.replay((PartitionRecord) message); + break; + case CONFIG_RECORD: + configurationControl.replay((ConfigRecord) message); + break; + case PARTITION_CHANGE_RECORD: + replicationControl.replay((PartitionChangeRecord) message); + break; + case FENCE_BROKER_RECORD: + clusterControl.replay((FenceBrokerRecord) message); + break; + case UNFENCE_BROKER_RECORD: + clusterControl.replay((UnfenceBrokerRecord) message); + break; + case REMOVE_TOPIC_RECORD: + replicationControl.replay((RemoveTopicRecord) message); + break; + case FEATURE_LEVEL_RECORD: + featureControl.replay((FeatureLevelRecord) message); + handleFeatureControlChange(); + break; + case CLIENT_QUOTA_RECORD: + clientQuotaControlManager.replay((ClientQuotaRecord) message); + break; + case PRODUCER_IDS_RECORD: + producerIdControlManager.replay((ProducerIdsRecord) message); + break; + case BROKER_REGISTRATION_CHANGE_RECORD: + clusterControl.replay((BrokerRegistrationChangeRecord) message); + break; + case ACCESS_CONTROL_ENTRY_RECORD: + aclControlManager.replay((AccessControlEntryRecord) message, snapshotId); + break; + case REMOVE_ACCESS_CONTROL_ENTRY_RECORD: + aclControlManager.replay((RemoveAccessControlEntryRecord) message, snapshotId); + break; + case NO_OP_RECORD: + // NoOpRecord is an empty record and doesn't need to be replayed + break; + default: + throw new RuntimeException("Unhandled record type " + type); } } @@ -1344,8 +1394,24 @@ private void resetState() { updateLastCommittedState(-1, -1, -1); } + /** + * Handles faults that should normally be fatal to the process. + */ + private final FaultHandler fatalFaultHandler; + + /** + * Handles faults in metadata handling that are normally not fatal. + */ + private final FaultHandler metadataFaultHandler; + + /** + * The slf4j log context, used to create new loggers. + */ private final LogContext logContext; + /** + * The slf4j logger. + */ private final Logger log; /** @@ -1530,28 +1596,34 @@ private enum ImbalanceSchedule { private final BootstrapMetadata bootstrapMetadata; - private QuorumController(LogContext logContext, - int nodeId, - String clusterId, - KafkaEventQueue queue, - Time time, - KafkaConfigSchema configSchema, - RaftClient raftClient, - QuorumFeatures quorumFeatures, - short defaultReplicationFactor, - int defaultNumPartitions, - ReplicaPlacer replicaPlacer, - long snapshotMaxNewRecordBytes, - OptionalLong leaderImbalanceCheckIntervalNs, - OptionalLong maxIdleIntervalNs, - long sessionTimeoutNs, - ControllerMetrics controllerMetrics, - Optional createTopicPolicy, - Optional alterConfigPolicy, - ConfigurationValidator configurationValidator, - Optional authorizer, - Map staticConfig, - BootstrapMetadata bootstrapMetadata) { + private QuorumController( + FaultHandler fatalFaultHandler, + FaultHandler metadataFaultHandler, + LogContext logContext, + int nodeId, + String clusterId, + KafkaEventQueue queue, + Time time, + KafkaConfigSchema configSchema, + RaftClient raftClient, + QuorumFeatures quorumFeatures, + short defaultReplicationFactor, + int defaultNumPartitions, + ReplicaPlacer replicaPlacer, + long snapshotMaxNewRecordBytes, + OptionalLong leaderImbalanceCheckIntervalNs, + OptionalLong maxIdleIntervalNs, + long sessionTimeoutNs, + ControllerMetrics controllerMetrics, + Optional createTopicPolicy, + Optional alterConfigPolicy, + ConfigurationValidator configurationValidator, + Optional authorizer, + Map staticConfig, + BootstrapMetadata bootstrapMetadata + ) { + this.fatalFaultHandler = fatalFaultHandler; + this.metadataFaultHandler = metadataFaultHandler; this.logContext = logContext; this.log = logContext.logger(QuorumController.class); this.nodeId = nodeId; diff --git a/metadata/src/main/java/org/apache/kafka/metadata/fault/MetadataFaultException.java b/metadata/src/main/java/org/apache/kafka/metadata/fault/MetadataFaultException.java new file mode 100644 index 0000000000000..c57ce46fb359f --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/metadata/fault/MetadataFaultException.java @@ -0,0 +1,32 @@ +/* + * 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.metadata.fault; + + +/** + * A fault that we encountered while we replayed cluster metadata. + */ +public class MetadataFaultException extends RuntimeException { + public MetadataFaultException(String message, Throwable cause) { + super(message, cause); + } + + public MetadataFaultException(String message) { + super(message); + } +} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/fault/MetadataFaultHandler.java b/metadata/src/main/java/org/apache/kafka/metadata/fault/MetadataFaultHandler.java new file mode 100644 index 0000000000000..e9f71b80e675e --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/metadata/fault/MetadataFaultHandler.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.metadata.fault; + +import org.apache.kafka.server.fault.FaultHandler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Handles faults in Kafka metadata management. + */ +public class MetadataFaultHandler implements FaultHandler { + private static final Logger log = LoggerFactory.getLogger(MetadataFaultHandler.class); + + @Override + public void handleFault(String failureMessage, Throwable cause) { + FaultHandler.logFailureMessage(log, failureMessage, cause); + throw new MetadataFaultException("Encountered metadata fault: " + failureMessage, cause); + } +} diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java index 2cdec699da230..e839289562682 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -45,6 +45,7 @@ import org.apache.kafka.common.errors.BrokerIdNotRegisteredException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.message.RequestHeaderData; +import org.apache.kafka.common.metadata.ConfigRecord; import org.apache.kafka.common.security.auth.KafkaPrincipal; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.config.ConfigResource; @@ -1182,6 +1183,30 @@ public void testQuorumControllerCompletesAuthorizerInitialLoad() throws Throwabl } } + @Test + public void testFatalMetadataReplayErrorOnActive() throws Throwable { + try (LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv(3, Optional.empty())) { + try (QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv(logEnv, b -> { + })) { + QuorumController active = controlEnv.activeController(); + CompletableFuture future = active.appendWriteEvent("errorEvent", + OptionalLong.empty(), () -> { + return ControllerResult.of(Collections.singletonList(new ApiMessageAndVersion( + new ConfigRecord(). + setName(null). + setResourceName(null). + setResourceType((byte) 255). + setValue(null), (short) 0)), null); + }); + assertThrows(ExecutionException.class, () -> future.get()); + assertEquals(NullPointerException.class, + controlEnv.fatalFaultHandler().firstException().getCause().getClass()); + controlEnv.fatalFaultHandler().setIgnore(true); + controlEnv.metadataFaultHandler().setIgnore(true); + } + } + } + private static void assertInitialLoadFuturesNotComplete(List authorizers) { for (int i = 0; i < authorizers.size(); i++) { assertFalse(authorizers.get(i).initialLoadFuture().isDone(), diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java index 4cc45a9774b34..40dd21c88d330 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java @@ -34,6 +34,7 @@ import org.apache.kafka.metalog.LocalLogManagerTestEnv; import org.apache.kafka.raft.LeaderAndEpoch; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.fault.MockFaultHandler; import org.apache.kafka.test.TestUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,6 +45,8 @@ public class QuorumControllerTestEnv implements AutoCloseable { private final List controllers; private final LocalLogManagerTestEnv logEnv; + private final MockFaultHandler fatalFaultHandler = new MockFaultHandler("fatalFaultHandler"); + private final MockFaultHandler metadataFaultHandler = new MockFaultHandler("metadataFaultHandler"); public QuorumControllerTestEnv( LocalLogManagerTestEnv logEnv, @@ -84,6 +87,8 @@ public QuorumControllerTestEnv( sessionTimeoutMillis.ifPresent(timeout -> { builder.setSessionTimeoutNs(NANOSECONDS.convert(timeout, TimeUnit.MILLISECONDS)); }); + builder.setFatalFaultHandler(fatalFaultHandler); + builder.setMetadataFaultHandler(metadataFaultHandler); builderConsumer.accept(builder); this.controllers.add(builder.build()); } @@ -117,6 +122,14 @@ public List controllers() { return controllers; } + public MockFaultHandler fatalFaultHandler() { + return fatalFaultHandler; + } + + public MockFaultHandler metadataFaultHandler() { + return metadataFaultHandler; + } + @Override public void close() throws InterruptedException { for (QuorumController controller : controllers) { @@ -125,5 +138,7 @@ public void close() throws InterruptedException { for (QuorumController controller : controllers) { controller.close(); } + fatalFaultHandler.maybeRethrowFirstException(); + metadataFaultHandler.maybeRethrowFirstException(); } } diff --git a/server-common/src/main/java/org/apache/kafka/server/fault/FaultHandler.java b/server-common/src/main/java/org/apache/kafka/server/fault/FaultHandler.java new file mode 100644 index 0000000000000..4c03eacc32f3e --- /dev/null +++ b/server-common/src/main/java/org/apache/kafka/server/fault/FaultHandler.java @@ -0,0 +1,58 @@ +/* + * 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.server.fault; + +import org.slf4j.Logger; + + +/** + * Handle a server fault. + */ +public interface FaultHandler { + /** + * Handle a fault. + * + * @param failureMessage The failure message to log. + */ + default void handleFault(String failureMessage) { + handleFault(failureMessage, null); + } + + /** + * Handle a fault. + * + * @param failureMessage The failure message to log. + * @param cause The exception that caused the problem, or null. + */ + void handleFault(String failureMessage, Throwable cause); + + /** + * Log a failure message about a fault. + * + * @param log The log4j logger. + * @param failureMessage The failure message. + * @param cause The exception which caused the failure, or null. + */ + static void logFailureMessage(Logger log, String failureMessage, Throwable cause) { + if (cause == null) { + log.error("Encountered fatal fault: {}", failureMessage); + } else { + log.error("Encountered fatal fault: {}", failureMessage, cause); + } + } +} diff --git a/server-common/src/main/java/org/apache/kafka/server/fault/ProcessExitingFaultHandler.java b/server-common/src/main/java/org/apache/kafka/server/fault/ProcessExitingFaultHandler.java new file mode 100644 index 0000000000000..e3b9f25a3bea8 --- /dev/null +++ b/server-common/src/main/java/org/apache/kafka/server/fault/ProcessExitingFaultHandler.java @@ -0,0 +1,37 @@ +/* + * 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.server.fault; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.kafka.common.utils.Exit; + + +/** + * This is a fault handler which exits the Java process. + */ +public class ProcessExitingFaultHandler implements FaultHandler { + private static final Logger log = LoggerFactory.getLogger(ProcessExitingFaultHandler.class); + + @Override + public void handleFault(String failureMessage, Throwable cause) { + FaultHandler.logFailureMessage(log, failureMessage, cause); + Exit.exit(1); + } +} diff --git a/server-common/src/test/java/org/apache/kafka/server/fault/MockFaultHandler.java b/server-common/src/test/java/org/apache/kafka/server/fault/MockFaultHandler.java new file mode 100644 index 0000000000000..39b3ed078471d --- /dev/null +++ b/server-common/src/test/java/org/apache/kafka/server/fault/MockFaultHandler.java @@ -0,0 +1,65 @@ +/* + * 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.server.fault; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * This is a fault handler suitable for use in JUnit tests. It will store the result of the first + * call to handleFault that was made. + */ +public class MockFaultHandler implements FaultHandler { + private static final Logger log = LoggerFactory.getLogger(MockFaultHandler.class); + + private final String name; + private MockFaultHandlerException firstException = null; + private boolean ignore = false; + + public MockFaultHandler(String name) { + this.name = name; + } + + @Override + public synchronized void handleFault(String failureMessage, Throwable cause) { + FaultHandler.logFailureMessage(log, failureMessage, cause); + MockFaultHandlerException e = (cause == null) ? + new MockFaultHandlerException(name + ": " + failureMessage) : + new MockFaultHandlerException(name + ": " + failureMessage + + ": " + cause.getMessage(), cause); + if (firstException == null) { + firstException = e; + } + throw e; + } + + public synchronized void maybeRethrowFirstException() { + if (firstException != null && !ignore) { + throw firstException; + } + } + + public synchronized MockFaultHandlerException firstException() { + return firstException; + } + + public synchronized void setIgnore(boolean ignore) { + this.ignore = ignore; + } +} diff --git a/server-common/src/test/java/org/apache/kafka/server/fault/MockFaultHandlerException.java b/server-common/src/test/java/org/apache/kafka/server/fault/MockFaultHandlerException.java new file mode 100644 index 0000000000000..ef9b11bdeb53e --- /dev/null +++ b/server-common/src/test/java/org/apache/kafka/server/fault/MockFaultHandlerException.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.server.fault; + + +/** + * An exception thrown by MockFaultHandler. + */ +public class MockFaultHandlerException extends RuntimeException { + public MockFaultHandlerException(String failureMessage, Throwable cause) { + super(failureMessage, cause); + // If a cause exception was provided, set our the stack trace its stack trace. This is + // useful in junit tests where a limited number of stack frames are printed, and usually + // the stack frames of cause exceptions get trimmed. + if (cause != null) { + setStackTrace(cause.getStackTrace()); + } + } + + public MockFaultHandlerException(String failureMessage) { + this(failureMessage, null); + } +} From 2ee58902d4cc23fd90459ed7fbc782be191f809a Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Fri, 5 Aug 2022 19:17:50 -0700 Subject: [PATCH 030/175] MINOR: add :server-common test dependency to :storage (#12488) Fix a bug in the KAFKA-14124 PR where a gradle test dependency was missing. This causes missing test class exceptions. Reviewers: Ismael Juma --- build.gradle | 2 ++ 1 file changed, 2 insertions(+) diff --git a/build.gradle b/build.gradle index aa930ce1bff00..c154ece691469 100644 --- a/build.gradle +++ b/build.gradle @@ -1597,6 +1597,8 @@ project(':storage') { testImplementation project(':clients').sourceSets.test.output testImplementation project(':core') testImplementation project(':core').sourceSets.test.output + testImplementation project(':server-common') + testImplementation project(':server-common').sourceSets.test.output testImplementation libs.junitJupiter testImplementation libs.mockitoCore testImplementation libs.bcpkix From bbd659325a60c8923afaa0bf099ad0536e297914 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Tue, 2 Aug 2022 15:39:47 -0700 Subject: [PATCH 031/175] KAFKA-14129: KRaft must check manual assignments for createTopics are contiguous (#12467) KRaft should validate that manual assignments given to createTopics are contiguous. In other words, they must start with partition 0, and progress through 1, 2, 3, etc. ZK mode does this, but KRaft mode previously did not. Also fix a null pointer exception when the placement for partition 0 was not specified. Convert over AddPartitionsTest to use KRaft. This PR converts all of the test except for some of the placement logic tests, which will need to be redone for KRaft mode in a future change. Fix null pointer exception in KRaftMetadataCache#getPartitionInfo. Specifically, we should not assume that the partition will be found in the hash map. This is another case where we had "Some(x)" but it should be "Option(x)." Fix a potential null pointer exception in BrokerServer#state. Reviewers: dengziming , Jason Gustafson --- .../scala/kafka/server/BrokerServer.scala | 5 +- .../server/metadata/KRaftMetadataCache.scala | 2 +- .../unit/kafka/admin/AddPartitionsTest.scala | 164 ++++++++++++------ .../controller/ReplicationControlManager.java | 8 +- 4 files changed, 123 insertions(+), 56 deletions(-) diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index eb21c1ed25e45..0bdd673497568 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -81,7 +81,8 @@ class BrokerServer( val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]] ) extends KafkaBroker { - override def brokerState: BrokerState = lifecycleManager.state + override def brokerState: BrokerState = Option(lifecycleManager). + flatMap(m => Some(m.state)).getOrElse(BrokerState.NOT_RUNNING) import kafka.server.Server._ @@ -89,7 +90,7 @@ class BrokerServer( this.logIdent = logContext.logPrefix - @volatile private var lifecycleManager: BrokerLifecycleManager = null + @volatile var lifecycleManager: BrokerLifecycleManager = null private val isShuttingDown = new AtomicBoolean(false) diff --git a/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala b/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala index ae2e65235739b..525772115037e 100644 --- a/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala +++ b/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala @@ -229,7 +229,7 @@ class KRaftMetadataCache(val brokerId: Int) extends MetadataCache with Logging w override def getPartitionInfo(topicName: String, partitionId: Int): Option[UpdateMetadataPartitionState] = { Option(_currentImage.topics().getTopic(topicName)). - flatMap(topic => Some(topic.partitions().get(partitionId))). + flatMap(topic => Option(topic.partitions().get(partitionId))). flatMap(partition => Some(new UpdateMetadataPartitionState(). setTopicName(topicName). setPartitionIndex(partitionId). diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala index ea4215d9c39f5..4e2bfee60ee54 100755 --- a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala @@ -17,18 +17,24 @@ package kafka.admin -import java.util.Optional +import java.util.{Collections, Optional} import kafka.controller.ReplicaAssignment -import kafka.server.BaseRequestTest -import kafka.utils.TestUtils +import kafka.server.{BaseRequestTest, BrokerServer} +import kafka.utils.{TestInfoUtils, TestUtils} import kafka.utils.TestUtils._ -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.clients.admin.{Admin, NewPartitions, NewTopic} import org.apache.kafka.common.errors.InvalidReplicaAssignmentException import org.apache.kafka.common.requests.MetadataResponse.TopicMetadata import org.apache.kafka.common.requests.{MetadataRequest, MetadataResponse} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} - +import org.junit.jupiter.api.{BeforeEach, TestInfo} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource + +import java.util +import java.util.Arrays.asList +import java.util.Collections.singletonList +import java.util.concurrent.ExecutionException import scala.jdk.CollectionConverters._ class AddPartitionsTest extends BaseRequestTest { @@ -47,44 +53,97 @@ class AddPartitionsTest extends BaseRequestTest { val topic4Assignment = Map(0 -> ReplicaAssignment(Seq(0,3), List(), List())) val topic5 = "new-topic5" val topic5Assignment = Map(1 -> ReplicaAssignment(Seq(0,1), List(), List())) + var admin: Admin = null @BeforeEach override def setUp(testInfo: TestInfo): Unit = { super.setUp(testInfo) + if (isKRaftTest()) { + brokers.foreach(broker => broker.asInstanceOf[BrokerServer].lifecycleManager.initialUnfenceFuture.get()) + } createTopicWithAssignment(topic1, partitionReplicaAssignment = topic1Assignment.map { case (k, v) => k -> v.replicas }) createTopicWithAssignment(topic2, partitionReplicaAssignment = topic2Assignment.map { case (k, v) => k -> v.replicas }) createTopicWithAssignment(topic3, partitionReplicaAssignment = topic3Assignment.map { case (k, v) => k -> v.replicas }) createTopicWithAssignment(topic4, partitionReplicaAssignment = topic4Assignment.map { case (k, v) => k -> v.replicas }) + admin = createAdminClient() } - @Test - def testWrongReplicaCount(): Unit = { - assertThrows(classOf[InvalidReplicaAssignmentException], () => adminZkClient.addPartitions(topic1, topic1Assignment, adminZkClient.getBrokerMetadatas(), 2, - Some(Map(0 -> Seq(0, 1), 1 -> Seq(0, 1, 2))))) + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testWrongReplicaCount(quorum: String): Unit = { + assertEquals(classOf[InvalidReplicaAssignmentException], assertThrows(classOf[ExecutionException], () => { + admin.createPartitions(Collections.singletonMap(topic1, + NewPartitions.increaseTo(2, singletonList(asList(0, 1, 2))))).all().get() + }).getCause.getClass) } - @Test - def testMissingPartition0(): Unit = { - val e = assertThrows(classOf[AdminOperationException], () => adminZkClient.addPartitions(topic5, topic5Assignment, adminZkClient.getBrokerMetadatas(), 2, - Some(Map(1 -> Seq(0, 1), 2 -> Seq(0, 1, 2))))) - assertTrue(e.getMessage.contains("Unexpected existing replica assignment for topic 'new-topic5', partition id 0 is missing")) + /** + * Test that when we supply a manual partition assignment to createTopics, it must be 0-based + * and consecutive. + */ + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testMissingPartitionsInCreateTopics(quorum: String): Unit = { + val topic6Placements = new util.HashMap[Integer, util.List[Integer]] + topic6Placements.put(1, asList(0, 1)) + topic6Placements.put(2, asList(1, 0)) + val topic7Placements = new util.HashMap[Integer, util.List[Integer]] + topic7Placements.put(2, asList(0, 1)) + topic7Placements.put(3, asList(1, 0)) + val futures = admin.createTopics(asList( + new NewTopic("new-topic6", topic6Placements), + new NewTopic("new-topic7", topic7Placements))).values() + val topic6Cause = assertThrows(classOf[ExecutionException], () => futures.get("new-topic6").get()).getCause + assertEquals(classOf[InvalidReplicaAssignmentException], topic6Cause.getClass) + assertTrue(topic6Cause.getMessage.contains("partitions should be a consecutive 0-based integer sequence"), + "Unexpected error message: " + topic6Cause.getMessage) + val topic7Cause = assertThrows(classOf[ExecutionException], () => futures.get("new-topic7").get()).getCause + assertEquals(classOf[InvalidReplicaAssignmentException], topic7Cause.getClass) + assertTrue(topic7Cause.getMessage.contains("partitions should be a consecutive 0-based integer sequence"), + "Unexpected error message: " + topic7Cause.getMessage) } - @Test - def testIncrementPartitions(): Unit = { - adminZkClient.addPartitions(topic1, topic1Assignment, adminZkClient.getBrokerMetadatas(), 3) + /** + * Test that when we supply a manual partition assignment to createPartitions, it must contain + * enough partitions. + */ + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testMissingPartitionsInCreatePartitions(quorum: String): Unit = { + val cause = assertThrows(classOf[ExecutionException], () => + admin.createPartitions(Collections.singletonMap(topic1, + NewPartitions.increaseTo(3, singletonList(asList(0, 1, 2))))).all().get()).getCause + assertEquals(classOf[InvalidReplicaAssignmentException], cause.getClass) + if (isKRaftTest()) { + assertTrue(cause.getMessage.contains("Attempted to add 2 additional partition(s), but only 1 assignment(s) " + + "were specified."), "Unexpected error message: " + cause.getMessage) + } else { + assertTrue(cause.getMessage.contains("Increasing the number of partitions by 2 but 1 assignments provided."), + "Unexpected error message: " + cause.getMessage) + } + if (!isKRaftTest()) { + // In ZK mode, test the raw AdminZkClient method as well. + val e = assertThrows(classOf[AdminOperationException], () => adminZkClient.addPartitions( + topic5, topic5Assignment, adminZkClient.getBrokerMetadatas(), 2, + Some(Map(1 -> Seq(0, 1), 2 -> Seq(0, 1, 2))))) + assertTrue(e.getMessage.contains("Unexpected existing replica assignment for topic 'new-topic5', partition " + + "id 0 is missing")) + } + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testIncrementPartitions(quorum: String): Unit = { + admin.createPartitions(Collections.singletonMap(topic1, NewPartitions.increaseTo(3))).all().get() + // wait until leader is elected - val leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 1) - val leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 2) - val leader1FromZk = zkClient.getLeaderForPartition(new TopicPartition(topic1, 1)).get - val leader2FromZk = zkClient.getLeaderForPartition(new TopicPartition(topic1, 2)).get - assertEquals(leader1, leader1FromZk) - assertEquals(leader2, leader2FromZk) + waitUntilLeaderIsElectedOrChangedWithAdmin(admin, topic1, 1) + waitUntilLeaderIsElectedOrChangedWithAdmin(admin, topic1, 2) // read metadata from a broker and verify the new topic partitions exist - TestUtils.waitForPartitionMetadata(servers, topic1, 1) - TestUtils.waitForPartitionMetadata(servers, topic1, 2) + TestUtils.waitForPartitionMetadata(brokers, topic1, 1) + TestUtils.waitForPartitionMetadata(brokers, topic1, 2) val response = connectAndReceive[MetadataResponse]( new MetadataRequest.Builder(Seq(topic1).asJava, false).build) assertEquals(1, response.topicMetadata.size) @@ -102,22 +161,21 @@ class AddPartitionsTest extends BaseRequestTest { } } - @Test - def testManualAssignmentOfReplicas(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testManualAssignmentOfReplicas(quorum: String): Unit = { // Add 2 partitions - adminZkClient.addPartitions(topic2, topic2Assignment, adminZkClient.getBrokerMetadatas(), 3, - Some(Map(0 -> Seq(1, 2), 1 -> Seq(0, 1), 2 -> Seq(2, 3)))) + admin.createPartitions(Collections.singletonMap(topic2, NewPartitions.increaseTo(3, + asList(asList(0, 1), asList(2, 3))))).all().get() // wait until leader is elected - val leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 1) - val leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 2) - val leader1FromZk = zkClient.getLeaderForPartition(new TopicPartition(topic2, 1)).get - val leader2FromZk = zkClient.getLeaderForPartition(new TopicPartition(topic2, 2)).get - assertEquals(leader1, leader1FromZk) - assertEquals(leader2, leader2FromZk) + val leader1 = waitUntilLeaderIsElectedOrChangedWithAdmin(admin, topic2, 1) + val leader2 = waitUntilLeaderIsElectedOrChangedWithAdmin(admin, topic2, 2) // read metadata from a broker and verify the new topic partitions exist - TestUtils.waitForPartitionMetadata(servers, topic2, 1) - TestUtils.waitForPartitionMetadata(servers, topic2, 2) + val partition1Metadata = TestUtils.waitForPartitionMetadata(brokers, topic2, 1) + assertEquals(leader1, partition1Metadata.leader()) + val partition2Metadata = TestUtils.waitForPartitionMetadata(brokers, topic2, 2) + assertEquals(leader2, partition2Metadata.leader()) val response = connectAndReceive[MetadataResponse]( new MetadataRequest.Builder(Seq(topic2).asJava, false).build) assertEquals(1, response.topicMetadata.size) @@ -132,17 +190,18 @@ class AddPartitionsTest extends BaseRequestTest { assertEquals(Set(0, 1), replicas.asScala.toSet) } - @Test - def testReplicaPlacementAllServers(): Unit = { - adminZkClient.addPartitions(topic3, topic3Assignment, adminZkClient.getBrokerMetadatas(), 7) + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk")) // TODO: add kraft support + def testReplicaPlacementAllServers(quorum: String): Unit = { + admin.createPartitions(Collections.singletonMap(topic3, NewPartitions.increaseTo(7))).all().get() // read metadata from a broker and verify the new topic partitions exist - TestUtils.waitForPartitionMetadata(servers, topic3, 1) - TestUtils.waitForPartitionMetadata(servers, topic3, 2) - TestUtils.waitForPartitionMetadata(servers, topic3, 3) - TestUtils.waitForPartitionMetadata(servers, topic3, 4) - TestUtils.waitForPartitionMetadata(servers, topic3, 5) - TestUtils.waitForPartitionMetadata(servers, topic3, 6) + TestUtils.waitForPartitionMetadata(brokers, topic3, 1) + TestUtils.waitForPartitionMetadata(brokers, topic3, 2) + TestUtils.waitForPartitionMetadata(brokers, topic3, 3) + TestUtils.waitForPartitionMetadata(brokers, topic3, 4) + TestUtils.waitForPartitionMetadata(brokers, topic3, 5) + TestUtils.waitForPartitionMetadata(brokers, topic3, 6) val response = connectAndReceive[MetadataResponse]( new MetadataRequest.Builder(Seq(topic3).asJava, false).build) @@ -157,13 +216,14 @@ class AddPartitionsTest extends BaseRequestTest { validateLeaderAndReplicas(topicMetadata, 6, 0, Set(0, 1, 2, 3)) } - @Test - def testReplicaPlacementPartialServers(): Unit = { - adminZkClient.addPartitions(topic2, topic2Assignment, adminZkClient.getBrokerMetadatas(), 3) + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk")) // TODO: add kraft support + def testReplicaPlacementPartialServers(quorum: String): Unit = { + admin.createPartitions(Collections.singletonMap(topic2, NewPartitions.increaseTo(3))).all().get() // read metadata from a broker and verify the new topic partitions exist - TestUtils.waitForPartitionMetadata(servers, topic2, 1) - TestUtils.waitForPartitionMetadata(servers, topic2, 2) + TestUtils.waitForPartitionMetadata(brokers, topic2, 1) + TestUtils.waitForPartitionMetadata(brokers, topic2, 2) val response = connectAndReceive[MetadataResponse]( new MetadataRequest.Builder(Seq(topic2).asJava, false).build) diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java index 3a3788c41a595..bf3a679d2cef0 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java @@ -666,6 +666,12 @@ private ApiError createTopic(CreatableTopic topic, Replicas.toArray(assignment.brokerIds()), Replicas.toArray(isr), Replicas.NONE, Replicas.NONE, isr.get(0), LeaderRecoveryState.RECOVERED, 0, 0)); } + for (int i = 0; i < newParts.size(); i++) { + if (!newParts.containsKey(i)) { + return new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT, + "partitions should be a consecutive 0-based integer sequence"); + } + } ApiError error = maybeCheckCreateTopicPolicy(() -> { Map> assignments = new HashMap<>(); newParts.entrySet().forEach(e -> assignments.put(e.getKey(), @@ -744,7 +750,7 @@ private ApiError createTopic(CreatableTopic topic, setIsSensitive(entry.isSensitive())); } result.setNumPartitions(newParts.size()); - result.setReplicationFactor((short) newParts.get(0).replicas.length); + result.setReplicationFactor((short) newParts.values().iterator().next().replicas.length); result.setTopicConfigErrorCode(NONE.code()); } else { result.setTopicConfigErrorCode(TOPIC_AUTHORIZATION_FAILED.code()); From 4a94d25b6d2de94f92c2795eb40e585940198bb7 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Fri, 29 Jul 2022 13:36:40 -0700 Subject: [PATCH 032/175] MINOR: convert some more junit tests to support KRaft (#12456) * MINOR: convert some more junit tests to support KRaft Introduce TestUtils#waitUntilLeaderIsElectedOrChangedWithAdmin, a ZK-free alternative to TestUtils#waitUntilLeaderIsElectedOrChanged. Convert PlaintextProducerSendTest, SslProducerSendTest, TransactionsWithMaxInFlightOneTest, AddPartitionsToTxnRequestServerTest and KafkaMetricsReporterTest to support KRaft Reviewers: dengziming , David Arthur --- .../kafka/api/BaseProducerSendTest.scala | 102 +++++++++++------- .../kafka/api/PlaintextProducerSendTest.scala | 51 +++++---- .../TransactionsWithMaxInFlightOneTest.scala | 24 +++-- .../kafka/server/QuorumTestHarness.scala | 40 ++++--- .../AddPartitionsToTxnRequestServerTest.scala | 16 +-- .../server/KafkaMetricsReporterTest.scala | 55 ++++++---- .../scala/unit/kafka/utils/TestUtils.scala | 60 ++++++++++- 7 files changed, 226 insertions(+), 122 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index 61870b073d845..ce3cd32afdea3 100644 --- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -19,22 +19,24 @@ package kafka.api import java.time.Duration import java.nio.charset.StandardCharsets -import java.util.Properties +import java.util.{Collections, Properties} import java.util.concurrent.TimeUnit - import kafka.integration.KafkaServerTestHarness import kafka.log.LogConfig import kafka.server.KafkaConfig -import kafka.utils.TestUtils +import kafka.utils.{TestInfoUtils, TestUtils} +import org.apache.kafka.clients.admin.{Admin, NewPartitions} import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer._ import org.apache.kafka.common.errors.TimeoutException -import org.apache.kafka.common.network.ListenerName +import org.apache.kafka.common.network.{ListenerName, Mode} import org.apache.kafka.common.record.TimestampType import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.{KafkaException, TopicPartition} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} +import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import scala.jdk.CollectionConverters._ import scala.collection.mutable.Buffer @@ -42,16 +44,17 @@ import scala.concurrent.ExecutionException abstract class BaseProducerSendTest extends KafkaServerTestHarness { - def generateConfigs = { + def generateConfigs: scala.collection.Seq[KafkaConfig] = { val overridingProps = new Properties() val numServers = 2 overridingProps.put(KafkaConfig.NumPartitionsProp, 4.toString) - TestUtils.createBrokerConfigs(numServers, zkConnect, false, interBrokerSecurityProtocol = Some(securityProtocol), + TestUtils.createBrokerConfigs(numServers, zkConnectOrNull, false, interBrokerSecurityProtocol = Some(securityProtocol), trustStoreFile = trustStoreFile, saslProperties = serverSaslProperties).map(KafkaConfig.fromProps(_, overridingProps)) } private var consumer: KafkaConsumer[Array[Byte], Array[Byte]] = _ private val producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() + protected var admin: Admin = null protected val topic = "topic" private val numRecords = 100 @@ -59,6 +62,15 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { super.setUp(testInfo) + + admin = TestUtils.createAdminClient(brokers, listenerName, + TestUtils.securityConfigs(Mode.CLIENT, + securityProtocol, + trustStoreFile, + "adminClient", + TestUtils.SslCertificateCn, + clientSaslProperties)) + consumer = TestUtils.createConsumer( bootstrapServers(listenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)), securityProtocol = SecurityProtocol.PLAINTEXT @@ -70,6 +82,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { consumer.close() // Ensure that all producers are closed since unclosed producers impact other tests when Kafka server ports are reused producers.foreach(_.close()) + admin.close() super.tearDown() } @@ -105,8 +118,9 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { * 1. Send with null key/value/partition-id should be accepted; send with null topic should be rejected. * 2. Last message of the non-blocking send should return the correct offset metadata */ - @Test - def testSendOffset(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testSendOffset(quorum: String): Unit = { val producer = createProducer() val partition = 0 @@ -134,7 +148,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { try { // create topic - createTopic(topic, 1, 2) + TestUtils.createTopicWithAdmin(admin, topic, brokers, 1, 2) // send a normal record val record0 = new ProducerRecord[Array[Byte], Array[Byte]](topic, partition, "key".getBytes(StandardCharsets.UTF_8), @@ -166,8 +180,9 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { } } - @Test - def testSendCompressedMessageWithCreateTime(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testSendCompressedMessageWithCreateTime(quorum: String): Unit = { val producer = createProducer( compressionType = "gzip", lingerMs = Int.MaxValue, @@ -175,8 +190,9 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { sendAndVerifyTimestamp(producer, TimestampType.CREATE_TIME) } - @Test - def testSendNonCompressedMessageWithCreateTime(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testSendNonCompressedMessageWithCreateTime(quorum: String): Unit = { val producer = createProducer(lingerMs = Int.MaxValue, deliveryTimeoutMs = Int.MaxValue) sendAndVerifyTimestamp(producer, TimestampType.CREATE_TIME) } @@ -186,7 +202,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { timeoutMs: Long = 20000L): Unit = { val partition = 0 try { - createTopic(topic, 1, 2) + TestUtils.createTopicWithAdmin(admin, topic, brokers, 1, 2) val futures = for (i <- 1 to numRecords) yield { val record = new ProducerRecord(topic, partition, s"key$i".getBytes(StandardCharsets.UTF_8), @@ -241,7 +257,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { topicProps.setProperty(LogConfig.MessageTimestampTypeProp, "LogAppendTime") else topicProps.setProperty(LogConfig.MessageTimestampTypeProp, "CreateTime") - createTopic(topic, 1, 2, topicProps) + TestUtils.createTopicWithAdmin(admin, topic, brokers, 1, 2, topicConfig = topicProps) val recordAndFutures = for (i <- 1 to numRecords) yield { val record = new ProducerRecord(topic, partition, baseTimestamp + i, s"key$i".getBytes(StandardCharsets.UTF_8), @@ -267,13 +283,14 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { * * After close() returns, all messages should be sent with correct returned offset metadata */ - @Test - def testClose(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testClose(quorum: String): Unit = { val producer = createProducer() try { // create topic - createTopic(topic, 1, 2) + TestUtils.createTopicWithAdmin(admin, topic, brokers, 1, 2) // non-blocking send a list of records val record0 = new ProducerRecord[Array[Byte], Array[Byte]](topic, null, "key".getBytes(StandardCharsets.UTF_8), @@ -300,12 +317,13 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { * * The specified partition-id should be respected */ - @Test - def testSendToPartition(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testSendToPartition(quorum: String): Unit = { val producer = createProducer() try { - createTopic(topic, 2, 2) + TestUtils.createTopicWithAdmin(admin, topic, brokers, 2, 2) val partition = 1 val now = System.currentTimeMillis() @@ -345,14 +363,15 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { * Producer will attempt to send messages to the partition specified in each record, and should * succeed as long as the partition is included in the metadata. */ - @Test - def testSendBeforeAndAfterPartitionExpansion(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testSendBeforeAndAfterPartitionExpansion(quorum: String): Unit = { val producer = createProducer(maxBlockMs = 5 * 1000L) // create topic - createTopic(topic, 1, 2) - val partition0 = 0 + TestUtils.createTopicWithAdmin(admin, topic, brokers, 1, 2) + val partition0 = 0 var futures0 = (1 to numRecords).map { i => producer.send(new ProducerRecord(topic, partition0, null, ("value" + i).getBytes(StandardCharsets.UTF_8))) }.map(_.get(30, TimeUnit.SECONDS)) @@ -369,13 +388,11 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { val e = assertThrows(classOf[ExecutionException], () => producer.send(new ProducerRecord(topic, partition1, null, "value".getBytes(StandardCharsets.UTF_8))).get()) assertEquals(classOf[TimeoutException], e.getCause.getClass) - val existingAssignment = zkClient.getFullReplicaAssignmentForTopics(Set(topic)).map { - case (topicPartition, assignment) => topicPartition.partition -> assignment - } - adminZkClient.addPartitions(topic, existingAssignment, adminZkClient.getBrokerMetadatas(), 2) + admin.createPartitions(Collections.singletonMap(topic, NewPartitions.increaseTo(2))).all().get() + // read metadata from a broker and verify the new topic partitions exist - TestUtils.waitForPartitionMetadata(servers, topic, 0) - TestUtils.waitForPartitionMetadata(servers, topic, 1) + TestUtils.waitForPartitionMetadata(brokers, topic, 0) + TestUtils.waitForPartitionMetadata(brokers, topic, 1) // send records to the newly added partition after confirming that metadata have been updated. val futures1 = (1 to numRecords).map { i => @@ -404,11 +421,12 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { /** * Test that flush immediately sends all accumulated requests. */ - @Test - def testFlush(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testFlush(quorum: String): Unit = { val producer = createProducer(lingerMs = Int.MaxValue, deliveryTimeoutMs = Int.MaxValue) try { - createTopic(topic, 2, 2) + TestUtils.createTopicWithAdmin(admin, topic, brokers, 2, 2) val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, "value".getBytes(StandardCharsets.UTF_8)) for (_ <- 0 until 50) { @@ -425,9 +443,10 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { /** * Test close with zero timeout from caller thread */ - @Test - def testCloseWithZeroTimeoutFromCallerThread(): Unit = { - createTopic(topic, 2, 2) + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCloseWithZeroTimeoutFromCallerThread(quorum: String): Unit = { + TestUtils.createTopicWithAdmin(admin, topic, brokers, 2, 2) val partition = 0 consumer.assign(List(new TopicPartition(topic, partition)).asJava) val record0 = new ProducerRecord[Array[Byte], Array[Byte]](topic, partition, null, @@ -450,9 +469,10 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { /** * Test close with zero and non-zero timeout from sender thread */ - @Test - def testCloseWithZeroTimeoutFromSenderThread(): Unit = { - createTopic(topic, 1, 2) + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCloseWithZeroTimeoutFromSenderThread(quorum: String): Unit = { + TestUtils.createTopicWithAdmin(admin, topic, brokers, 1, 2) val partition = 0 consumer.assign(List(new TopicPartition(topic, partition)).asJava) val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, partition, null, "value".getBytes(StandardCharsets.UTF_8)) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala index 06ff201e0b2e6..c25eb184b3e13 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala @@ -19,22 +19,23 @@ package kafka.api import java.util.Properties import java.util.concurrent.{ExecutionException, Future, TimeUnit} - import kafka.log.LogConfig import kafka.server.Defaults -import kafka.utils.TestUtils +import kafka.utils.{TestInfoUtils, TestUtils} import org.apache.kafka.clients.producer.{BufferExhaustedException, KafkaProducer, ProducerConfig, ProducerRecord, RecordMetadata} import org.apache.kafka.common.errors.{InvalidTimestampException, RecordTooLargeException, SerializationException, TimeoutException} import org.apache.kafka.common.record.{DefaultRecord, DefaultRecordBatch, Records, TimestampType} import org.apache.kafka.common.serialization.ByteArraySerializer import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.Test +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource class PlaintextProducerSendTest extends BaseProducerSendTest { - @Test - def testWrongSerializer(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testWrongSerializer(quorum: String): Unit = { val producerProps = new Properties() producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()) producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer") @@ -44,8 +45,9 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { assertThrows(classOf[SerializationException], () => producer.send(record)) } - @Test - def testBatchSizeZero(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testBatchSizeZero(quorum: String): Unit = { val producer = createProducer( lingerMs = Int.MaxValue, deliveryTimeoutMs = Int.MaxValue, @@ -53,8 +55,9 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { sendAndVerify(producer) } - @Test - def testSendCompressedMessageWithLogAppendTime(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testSendCompressedMessageWithLogAppendTime(quorum: String): Unit = { val producer = createProducer( compressionType = "gzip", lingerMs = Int.MaxValue, @@ -62,8 +65,9 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { sendAndVerifyTimestamp(producer, TimestampType.LOG_APPEND_TIME) } - @Test - def testSendNonCompressedMessageWithLogAppendTime(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testSendNonCompressedMessageWithLogAppendTime(quorum: String): Unit = { val producer = createProducer(lingerMs = Int.MaxValue, deliveryTimeoutMs = Int.MaxValue) sendAndVerifyTimestamp(producer, TimestampType.LOG_APPEND_TIME) } @@ -73,8 +77,9 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { * * The topic should be created upon sending the first message */ - @Test - def testAutoCreateTopic(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testAutoCreateTopic(quorum: String): Unit = { val producer = createProducer() try { // Send a message to auto-create the topic @@ -82,18 +87,18 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { assertEquals(0L, producer.send(record).get.offset, "Should have offset 0") // double check that the topic is created with leader elected - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) - + TestUtils.waitUntilLeaderIsElectedOrChangedWithAdmin(admin, topic, 0) } finally { producer.close() } } - @Test - def testSendWithInvalidCreateTime(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testSendWithInvalidCreateTime(quorum: String): Unit = { val topicProps = new Properties() topicProps.setProperty(LogConfig.MessageTimestampDifferenceMaxMsProp, "1000") - createTopic(topic, 1, 2, topicProps) + TestUtils.createTopicWithAdmin(admin, topic, brokers, 1, 2, topicConfig = topicProps) val producer = createProducer() try { @@ -118,8 +123,9 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { // Test that producer with max.block.ms=0 can be used to send in non-blocking mode // where requests are failed immediately without blocking if metadata is not available // or buffer is full. - @Test - def testNonBlockingProducer(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testNonBlockingProducer(quorum: String): Unit = { def send(producer: KafkaProducer[Array[Byte],Array[Byte]]): Future[RecordMetadata] = { producer.send(new ProducerRecord(topic, 0, "key".getBytes, new Array[Byte](1000))) @@ -173,8 +179,9 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { verifySendSuccess(future2) // previous batch should be completed and sent now } - @Test - def testSendRecordBatchWithMaxRequestSizeAndHigher(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testSendRecordBatchWithMaxRequestSizeAndHigher(quorum: String): Unit = { val producerProps = new Properties() producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()) val producer = registerProducer(new KafkaProducer(producerProps, new ByteArraySerializer, new ByteArraySerializer)) diff --git a/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala index eacc58e76cc59..5dd82b6b224bc 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala @@ -18,15 +18,16 @@ package kafka.api import java.util.Properties - import kafka.integration.KafkaServerTestHarness import kafka.server.KafkaConfig -import kafka.utils.TestUtils +import kafka.utils.{TestInfoUtils, TestUtils} import kafka.utils.TestUtils.consumeRecords import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer.KafkaProducer import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} +import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import scala.collection.Seq import scala.collection.mutable.Buffer @@ -37,7 +38,7 @@ import scala.jdk.CollectionConverters._ * A single broker is used to verify edge cases where different requests are queued on the same connection. */ class TransactionsWithMaxInFlightOneTest extends KafkaServerTestHarness { - val numServers = 1 + val numBrokers = 1 val topic1 = "topic1" val topic2 = "topic2" @@ -47,7 +48,7 @@ class TransactionsWithMaxInFlightOneTest extends KafkaServerTestHarness { val transactionalConsumers = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]() override def generateConfigs: Seq[KafkaConfig] = { - TestUtils.createBrokerConfigs(numServers, zkConnect).map(KafkaConfig.fromProps(_, serverProps())) + TestUtils.createBrokerConfigs(numBrokers, zkConnectOrNull).map(KafkaConfig.fromProps(_, serverProps())) } @BeforeEach @@ -55,8 +56,8 @@ class TransactionsWithMaxInFlightOneTest extends KafkaServerTestHarness { super.setUp(testInfo) val topicConfig = new Properties() topicConfig.put(KafkaConfig.MinInSyncReplicasProp, 1.toString) - createTopic(topic1, numPartitions, numServers, topicConfig) - createTopic(topic2, numPartitions, numServers, topicConfig) + createTopic(topic1, numPartitions, numBrokers, topicConfig) + createTopic(topic2, numPartitions, numBrokers, topicConfig) createTransactionalProducer("transactional-producer") createReadCommittedConsumer("transactional-group") @@ -69,10 +70,11 @@ class TransactionsWithMaxInFlightOneTest extends KafkaServerTestHarness { super.tearDown() } - @Test - def testTransactionalProducerSingleBrokerMaxInFlightOne(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testTransactionalProducerSingleBrokerMaxInFlightOne(quorum: String): Unit = { // We want to test with one broker to verify multiple requests queued on a connection - assertEquals(1, servers.size) + assertEquals(1, brokers.size) val producer = transactionalProducers.head val consumer = transactionalConsumers.head @@ -124,7 +126,7 @@ class TransactionsWithMaxInFlightOneTest extends KafkaServerTestHarness { } private def createTransactionalProducer(transactionalId: String): KafkaProducer[Array[Byte], Array[Byte]] = { - val producer = TestUtils.createTransactionalProducer(transactionalId, servers, maxInFlight = 1) + val producer = TestUtils.createTransactionalProducer(transactionalId, brokers, maxInFlight = 1) transactionalProducers += producer producer } diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala index 766eb3a7e535a..9894df9c5f747 100755 --- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala +++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala @@ -48,9 +48,12 @@ import scala.compat.java8.OptionConverters._ import scala.jdk.CollectionConverters._ trait QuorumImplementation { - def createBroker(config: KafkaConfig, - time: Time, - startup: Boolean): KafkaBroker + def createBroker( + config: KafkaConfig, + time: Time = Time.SYSTEM, + startup: Boolean = true, + threadNamePrefix: Option[String] = None, + ): KafkaBroker def shutdown(): Unit } @@ -62,10 +65,13 @@ class ZooKeeperQuorumImplementation( val adminZkClient: AdminZkClient, val log: Logging ) extends QuorumImplementation { - override def createBroker(config: KafkaConfig, - time: Time, - startup: Boolean): KafkaBroker = { - val server = new KafkaServer(config, time, None, false) + override def createBroker( + config: KafkaConfig, + time: Time, + startup: Boolean, + threadNamePrefix: Option[String], + ): KafkaBroker = { + val server = new KafkaServer(config, time, threadNamePrefix, false) if (startup) server.startup() server } @@ -82,9 +88,12 @@ class KRaftQuorumImplementation(val raftManager: KafkaRaftManager[ApiMessageAndV val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]], val clusterId: String, val log: Logging) extends QuorumImplementation { - override def createBroker(config: KafkaConfig, - time: Time, - startup: Boolean): KafkaBroker = { + override def createBroker( + config: KafkaConfig, + time: Time, + startup: Boolean, + threadNamePrefix: Option[String], + ): KafkaBroker = { val broker = new BrokerServer(config = config, metaProps = new MetaProperties(clusterId, config.nodeId), raftManager = raftManager, @@ -222,10 +231,13 @@ abstract class QuorumTestHarness extends Logging { } } - def createBroker(config: KafkaConfig, - time: Time = Time.SYSTEM, - startup: Boolean = true): KafkaBroker = { - implementation.createBroker(config, time, startup) + def createBroker( + config: KafkaConfig, + time: Time = Time.SYSTEM, + startup: Boolean = true, + threadNamePrefix: Option[String] = None + ): KafkaBroker = { + implementation.createBroker(config, time, startup, threadNamePrefix) } def shutdownZooKeeper(): Unit = asZk().shutdown() diff --git a/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala b/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala index 0a98d2626cd23..74320e62b49a1 100644 --- a/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala @@ -17,13 +17,16 @@ package kafka.server -import java.util.Properties +import kafka.utils.TestInfoUtils +import java.util.Properties import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AddPartitionsToTxnRequest, AddPartitionsToTxnResponse} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} +import org.junit.jupiter.api.{BeforeEach, TestInfo} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import scala.jdk.CollectionConverters._ @@ -37,11 +40,12 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { super.setUp(testInfo) - createTopic(topic1, numPartitions, servers.size, new Properties()) + createTopic(topic1, numPartitions, brokers.size, new Properties()) } - @Test - def shouldReceiveOperationNotAttemptedWhenOtherPartitionHasError(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def shouldReceiveOperationNotAttemptedWhenOtherPartitionHasError(quorum: String): Unit = { // The basic idea is that we have one unknown topic and one created topic. We should get the 'UNKNOWN_TOPIC_OR_PARTITION' // error for the unknown topic and the 'OPERATION_NOT_ATTEMPTED' error for the known and authorized topic. val nonExistentTopic = new TopicPartition("unknownTopic", 0) @@ -58,7 +62,7 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest { List(createdTopicPartition, nonExistentTopic).asJava) .build() - val leaderId = servers.head.config.brokerId + val leaderId = brokers.head.config.brokerId val response = connectAndReceive[AddPartitionsToTxnResponse](request, brokerSocketServer(leaderId)) assertEquals(2, response.errors.size) diff --git a/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala b/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala index 7e5d791db252b..1adf544819fa3 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala @@ -17,15 +17,13 @@ package kafka.server import java.util - import java.util.concurrent.atomic.AtomicReference - -import kafka.utils.{CoreUtils, TestUtils} -import kafka.server.QuorumTestHarness +import kafka.utils.{CoreUtils, TestInfoUtils, TestUtils} import org.apache.kafka.common.metrics.{KafkaMetric, MetricsContext, MetricsReporter} -import org.junit.jupiter.api.Assertions.{assertEquals} -import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} +import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource object KafkaMetricsReporterTest { @@ -43,52 +41,63 @@ object KafkaMetricsReporterTest { override def contextChange(metricsContext: MetricsContext): Unit = { //read jmxPrefix - MockMetricsReporter.JMXPREFIX.set(metricsContext.contextLabels().get("_namespace").toString) - MockMetricsReporter.CLUSTERID.set(metricsContext.contextLabels().get("kafka.cluster.id").toString) - MockMetricsReporter.BROKERID.set(metricsContext.contextLabels().get("kafka.broker.id").toString) + MockMetricsReporter.JMXPREFIX.set(contextLabelOrNull("_namespace", metricsContext)) + MockMetricsReporter.CLUSTERID.set(contextLabelOrNull("kafka.cluster.id", metricsContext)) + MockMetricsReporter.BROKERID.set(contextLabelOrNull("kafka.broker.id", metricsContext)) + MockMetricsReporter.NODEID.set(contextLabelOrNull("kafka.node.id", metricsContext)) } - override def configure(configs: util.Map[String, _]): Unit = {} + private def contextLabelOrNull(name: String, metricsContext: MetricsContext): String = { + Option(metricsContext.contextLabels().get(name)).flatMap(v => Option(v.toString())).getOrElse(null) + } + override def configure(configs: util.Map[String, _]): Unit = {} } object MockMetricsReporter { val JMXPREFIX: AtomicReference[String] = new AtomicReference[String] val BROKERID : AtomicReference[String] = new AtomicReference[String] + val NODEID : AtomicReference[String] = new AtomicReference[String] val CLUSTERID : AtomicReference[String] = new AtomicReference[String] } } class KafkaMetricsReporterTest extends QuorumTestHarness { - var server: KafkaServer = null + var broker: KafkaBroker = null var config: KafkaConfig = null @BeforeEach override def setUp(testInfo: TestInfo): Unit = { super.setUp(testInfo) - val props = TestUtils.createBrokerConfig(1, zkConnect) + val props = TestUtils.createBrokerConfig(1, zkConnectOrNull) props.setProperty(KafkaConfig.MetricReporterClassesProp, "kafka.server.KafkaMetricsReporterTest$MockMetricsReporter") props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "true") - props.setProperty(KafkaConfig.BrokerIdProp, "-1") + props.setProperty(KafkaConfig.BrokerIdProp, "1") config = KafkaConfig.fromProps(props) - server = new KafkaServer(config, threadNamePrefix = Option(this.getClass.getName)) - server.startup() + broker = createBroker(config, threadNamePrefix = Option(this.getClass.getName)) + broker.startup() } - @Test - def testMetricsContextNamespacePresent(): Unit = { - assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.CLUSTERID) - assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.BROKERID) - assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.JMXPREFIX) - assertEquals("kafka.server", KafkaMetricsReporterTest.MockMetricsReporter.JMXPREFIX.get()) + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testMetricsContextNamespacePresent(quorum: String): Unit = { + assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.CLUSTERID.get()) + if (isKRaftTest()) { + assertNull(KafkaMetricsReporterTest.MockMetricsReporter.BROKERID.get()) + assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.NODEID.get()) + } else { + assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.BROKERID.get()) + assertNull(KafkaMetricsReporterTest.MockMetricsReporter.NODEID.get()) + } + assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.JMXPREFIX.get()) - server.shutdown() + broker.shutdown() TestUtils.assertNoNonDaemonThreads(this.getClass.getName) } @AfterEach override def tearDown(): Unit = { - server.shutdown() + broker.shutdown() CoreUtils.delete(config.logDirs) super.tearDown() } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index d0266bdee9d03..1e0d5981daca4 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -47,7 +47,7 @@ import org.apache.kafka.clients.admin._ import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.consumer.internals.AbstractCoordinator import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} -import org.apache.kafka.common.TopicIdPartition +import org.apache.kafka.common.{KafkaFuture, Node, TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.acl.{AccessControlEntry, AccessControlEntryFilter, AclBinding, AclBindingFilter} import org.apache.kafka.common.config.{ConfigException, ConfigResource} import org.apache.kafka.common.config.ConfigResource.Type.TOPIC @@ -67,7 +67,6 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, KafkaPrincipalSerd import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer, Deserializer, IntegerSerializer, Serializer} import org.apache.kafka.common.utils.Utils._ import org.apache.kafka.common.utils.{Time, Utils} -import org.apache.kafka.common.{KafkaFuture, TopicPartition, Uuid} import org.apache.kafka.controller.QuorumController import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, Authorizer => JAuthorizer} import org.apache.kafka.server.common.MetadataVersion @@ -871,8 +870,59 @@ object TestUtils extends Logging { * LeaderDuringDelete). * @throws AssertionError if the expected condition is not true within the timeout. */ - def waitUntilLeaderIsElectedOrChanged(zkClient: KafkaZkClient, topic: String, partition: Int, timeoutMs: Long = 30000L, - oldLeaderOpt: Option[Int] = None, newLeaderOpt: Option[Int] = None): Int = { + def waitUntilLeaderIsElectedOrChanged( + zkClient: KafkaZkClient, + topic: String, + partition: Int, + timeoutMs: Long = 30000L, + oldLeaderOpt: Option[Int] = None, + newLeaderOpt: Option[Int] = None + ): Int = { + def getPartitionLeader(topic: String, partition: Int): Option[Int] = { + zkClient.getLeaderForPartition(new TopicPartition(topic, partition)) + } + doWaitUntilLeaderIsElectedOrChanged(getPartitionLeader, topic, partition, timeoutMs, oldLeaderOpt, newLeaderOpt) + } + + /** + * If neither oldLeaderOpt nor newLeaderOpt is defined, wait until the leader of a partition is elected. + * If oldLeaderOpt is defined, it waits until the new leader is different from the old leader. + * If newLeaderOpt is defined, it waits until the new leader becomes the expected new leader. + * + * @return The new leader (note that negative values are used to indicate conditions like NoLeader and + * LeaderDuringDelete). + * @throws AssertionError if the expected condition is not true within the timeout. + */ + def waitUntilLeaderIsElectedOrChangedWithAdmin( + admin: Admin, + topic: String, + partition: Int, + timeoutMs: Long = 30000L, + oldLeaderOpt: Option[Int] = None, + newLeaderOpt: Option[Int] = None + ): Int = { + def getPartitionLeader(topic: String, partition: Int): Option[Int] = { + admin.describeTopics(Collections.singletonList(topic)).allTopicNames().get().get(topic).partitions().asScala. + find(_.partition() == partition). + flatMap { p => + if (p.leader().id() == Node.noNode().id()) { + None + } else { + Some(p.leader().id()) + } + } + } + doWaitUntilLeaderIsElectedOrChanged(getPartitionLeader, topic, partition, timeoutMs, oldLeaderOpt, newLeaderOpt) + } + + private def doWaitUntilLeaderIsElectedOrChanged( + getPartitionLeader: (String, Int) => Option[Int], + topic: String, + partition: Int, + timeoutMs: Long, + oldLeaderOpt: Option[Int], + newLeaderOpt: Option[Int] + ): Int = { require(!(oldLeaderOpt.isDefined && newLeaderOpt.isDefined), "Can't define both the old and the new leader") val startTime = System.currentTimeMillis() val topicPartition = new TopicPartition(topic, partition) @@ -884,7 +934,7 @@ object TestUtils extends Logging { var electedOrChangedLeader: Option[Int] = None while (electedOrChangedLeader.isEmpty && System.currentTimeMillis() < startTime + timeoutMs) { // check if leader is elected - leader = zkClient.getLeaderForPartition(topicPartition) + leader = getPartitionLeader(topic, partition) leader match { case Some(l) => (newLeaderOpt, oldLeaderOpt) match { case (Some(newLeader), _) if newLeader == l => From 480e97914e1146ba79ae883c4e987f5de20702cb Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 26 Jul 2022 19:08:59 -0400 Subject: [PATCH 033/175] KAFKA-13166 Fix missing ControllerApis error handling (#12403) Makes all ControllerApis request handlers return a `CompletableFuture[Unit]`. Also adds an additional completion stage which ensures we capture errors thrown during response building. Reviewed-by: Colin P. McCabe --- .../src/main/scala/kafka/server/AclApis.scala | 32 +++-- .../scala/kafka/server/ControllerApis.scala | 121 ++++++++++-------- .../kafka/server/ControllerApisTest.scala | 30 +++++ 3 files changed, 124 insertions(+), 59 deletions(-) diff --git a/core/src/main/scala/kafka/server/AclApis.scala b/core/src/main/scala/kafka/server/AclApis.scala index 97b685bc0aae2..485cafeca2038 100644 --- a/core/src/main/scala/kafka/server/AclApis.scala +++ b/core/src/main/scala/kafka/server/AclApis.scala @@ -24,14 +24,16 @@ import org.apache.kafka.common.acl.AclOperation._ import org.apache.kafka.common.acl.AclBinding import org.apache.kafka.common.errors._ import org.apache.kafka.common.message.CreateAclsResponseData.AclCreationResult +import org.apache.kafka.common.message.DeleteAclsResponseData.DeleteAclsFilterResult import org.apache.kafka.common.message._ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests._ import org.apache.kafka.common.resource.Resource.CLUSTER_NAME import org.apache.kafka.common.resource.ResourceType import org.apache.kafka.server.authorizer._ -import java.util +import java.util +import java.util.concurrent.CompletableFuture import scala.collection.mutable.ArrayBuffer import scala.collection.mutable import scala.compat.java8.OptionConverters._ @@ -53,7 +55,7 @@ class AclApis(authHelper: AuthHelper, def close(): Unit = alterAclsPurgatory.shutdown() - def handleDescribeAcls(request: RequestChannel.Request): Unit = { + def handleDescribeAcls(request: RequestChannel.Request): CompletableFuture[Unit] = { authHelper.authorizeClusterOperation(request, DESCRIBE) val describeAclsRequest = request.body[DescribeAclsRequest] authorizer match { @@ -74,9 +76,10 @@ class AclApis(authHelper: AuthHelper, .setResources(DescribeAclsResponse.aclsResources(returnedAcls)), describeAclsRequest.version)) } + CompletableFuture.completedFuture[Unit](()) } - def handleCreateAcls(request: RequestChannel.Request): Unit = { + def handleCreateAcls(request: RequestChannel.Request): CompletableFuture[Unit] = { authHelper.authorizeClusterOperation(request, ALTER) val createAclsRequest = request.body[CreateAclsRequest] @@ -84,6 +87,7 @@ class AclApis(authHelper: AuthHelper, case None => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => createAclsRequest.getErrorResponse(requestThrottleMs, new SecurityDisabledException("No Authorizer is configured."))) + CompletableFuture.completedFuture[Unit](()) case Some(auth) => val allBindings = createAclsRequest.aclCreations.asScala.map(CreateAclsRequest.aclBinding) val errorResults = mutable.Map[AclBinding, AclCreateResult]() @@ -103,6 +107,7 @@ class AclApis(authHelper: AuthHelper, validBindings += acl } + val future = new CompletableFuture[util.List[AclCreationResult]]() val createResults = auth.createAcls(request.context, validBindings.asJava).asScala.map(_.toCompletableFuture) def sendResponseCallback(): Unit = { @@ -117,17 +122,20 @@ class AclApis(authHelper: AuthHelper, } creationResult } + future.complete(aclCreationResults.asJava) + } + alterAclsPurgatory.tryCompleteElseWatch(config.connectionsMaxIdleMs, createResults, sendResponseCallback) + + future.thenApply[Unit] { aclCreationResults => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => new CreateAclsResponse(new CreateAclsResponseData() .setThrottleTimeMs(requestThrottleMs) - .setResults(aclCreationResults.asJava))) + .setResults(aclCreationResults))) } - - alterAclsPurgatory.tryCompleteElseWatch(config.connectionsMaxIdleMs, createResults, sendResponseCallback) } } - def handleDeleteAcls(request: RequestChannel.Request): Unit = { + def handleDeleteAcls(request: RequestChannel.Request): CompletableFuture[Unit] = { authHelper.authorizeClusterOperation(request, ALTER) val deleteAclsRequest = request.body[DeleteAclsRequest] authorizer match { @@ -135,13 +143,20 @@ class AclApis(authHelper: AuthHelper, requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => deleteAclsRequest.getErrorResponse(requestThrottleMs, new SecurityDisabledException("No Authorizer is configured."))) + CompletableFuture.completedFuture[Unit](()) case Some(auth) => + val future = new CompletableFuture[util.List[DeleteAclsFilterResult]]() val deleteResults = auth.deleteAcls(request.context, deleteAclsRequest.filters) .asScala.map(_.toCompletableFuture).toList def sendResponseCallback(): Unit = { val filterResults = deleteResults.map(_.get).map(DeleteAclsResponse.filterResult).asJava + future.complete(filterResults) + } + + alterAclsPurgatory.tryCompleteElseWatch(config.connectionsMaxIdleMs, deleteResults, sendResponseCallback) + future.thenApply[Unit] { filterResults => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => new DeleteAclsResponse( new DeleteAclsResponseData() @@ -149,7 +164,6 @@ class AclApis(authHelper: AuthHelper, .setFilterResults(filterResults), deleteAclsRequest.version)) } - alterAclsPurgatory.tryCompleteElseWatch(config.connectionsMaxIdleMs, deleteResults, sendResponseCallback) } } -} + } diff --git a/core/src/main/scala/kafka/server/ControllerApis.scala b/core/src/main/scala/kafka/server/ControllerApis.scala index 74bc4dd406712..efb6a36c3dbce 100644 --- a/core/src/main/scala/kafka/server/ControllerApis.scala +++ b/core/src/main/scala/kafka/server/ControllerApis.scala @@ -20,7 +20,7 @@ package kafka.server import java.util import java.util.{Collections, OptionalLong} import java.util.Map.Entry -import java.util.concurrent.{CompletableFuture, ExecutionException} +import java.util.concurrent.{CompletableFuture, CompletionException} import kafka.network.RequestChannel import kafka.raft.RaftManager import kafka.server.QuotaFactory.QuotaManagers @@ -78,7 +78,7 @@ class ControllerApis(val requestChannel: RequestChannel, override def handle(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = { try { - request.header.apiKey match { + val handlerFuture: CompletableFuture[Unit] = request.header.apiKey match { case ApiKeys.FETCH => handleFetch(request) case ApiKeys.FETCH_SNAPSHOT => handleFetchSnapshot(request) case ApiKeys.CREATE_TOPICS => handleCreateTopics(request) @@ -109,10 +109,24 @@ class ControllerApis(val requestChannel: RequestChannel, case ApiKeys.UPDATE_FEATURES => handleUpdateFeatures(request) case _ => throw new ApiException(s"Unsupported ApiKey ${request.context.header.apiKey}") } + + // This catches exceptions in the future and subsequent completion stages returned by the request handlers. + handlerFuture.whenComplete { (_, exception) => + if (exception != null) { + // CompletionException does not include the stack frames in its "cause" exception, so we need to + // log the original exception here + error(s"Unexpected error handling request ${request.requestDesc(true)} " + + s"with context ${request.context}", exception) + + // For building the correct error request, we do need send the "cause" exception + val actualException = if (exception.isInstanceOf[CompletionException]) exception.getCause else exception + requestHelper.handleError(request, actualException) + } + } } catch { case e: FatalExitError => throw e - case e: Throwable => { - val t = if (e.isInstanceOf[ExecutionException]) e.getCause else e + case t: Throwable => { + // This catches exceptions in the blocking parts of the request handlers error(s"Unexpected error handling request ${request.requestDesc(true)} " + s"with context ${request.context}", t) requestHelper.handleError(request, t) @@ -125,38 +139,41 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleEnvelopeRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = { + def handleEnvelopeRequest(request: RequestChannel.Request, requestLocal: RequestLocal): CompletableFuture[Unit] = { if (!authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) { requestHelper.sendErrorResponseMaybeThrottle(request, new ClusterAuthorizationException( s"Principal ${request.context.principal} does not have required CLUSTER_ACTION for envelope")) } else { EnvelopeUtils.handleEnvelopeRequest(request, requestChannel.metrics, handle(_, requestLocal)) } + CompletableFuture.completedFuture[Unit](()) } - def handleSaslHandshakeRequest(request: RequestChannel.Request): Unit = { + def handleSaslHandshakeRequest(request: RequestChannel.Request): CompletableFuture[Unit] = { val responseData = new SaslHandshakeResponseData().setErrorCode(ILLEGAL_SASL_STATE.code) requestHelper.sendResponseMaybeThrottle(request, _ => new SaslHandshakeResponse(responseData)) + CompletableFuture.completedFuture[Unit](()) } - def handleSaslAuthenticateRequest(request: RequestChannel.Request): Unit = { + def handleSaslAuthenticateRequest(request: RequestChannel.Request): CompletableFuture[Unit] = { val responseData = new SaslAuthenticateResponseData() .setErrorCode(ILLEGAL_SASL_STATE.code) .setErrorMessage("SaslAuthenticate request received after successful authentication") requestHelper.sendResponseMaybeThrottle(request, _ => new SaslAuthenticateResponse(responseData)) + CompletableFuture.completedFuture[Unit](()) } - def handleFetch(request: RequestChannel.Request): Unit = { + def handleFetch(request: RequestChannel.Request): CompletableFuture[Unit] = { authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) handleRaftRequest(request, response => new FetchResponse(response.asInstanceOf[FetchResponseData])) } - def handleFetchSnapshot(request: RequestChannel.Request): Unit = { + def handleFetchSnapshot(request: RequestChannel.Request): CompletableFuture[Unit] = { authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) handleRaftRequest(request, response => new FetchSnapshotResponse(response.asInstanceOf[FetchSnapshotResponseData])) } - def handleDeleteTopics(request: RequestChannel.Request): Unit = { + def handleDeleteTopics(request: RequestChannel.Request): CompletableFuture[Unit] = { val deleteTopicsRequest = request.body[DeleteTopicsRequest] val context = new ControllerRequestContext(request.context.header.data, request.context.principal, requestTimeoutMsToDeadlineNs(time, deleteTopicsRequest.data.timeoutMs)) @@ -166,7 +183,7 @@ class ControllerApis(val requestChannel: RequestChannel, authHelper.authorize(request.context, DELETE, CLUSTER, CLUSTER_NAME, logIfDenied = false), names => authHelper.filterByAuthorized(request.context, DESCRIBE, TOPIC, names)(n => n), names => authHelper.filterByAuthorized(request.context, DELETE, TOPIC, names)(n => n)) - future.whenComplete { (results, exception) => + future.handle[Unit] { (results, exception) => requestHelper.sendResponseMaybeThrottle(request, throttleTimeMs => { if (exception != null) { deleteTopicsRequest.getErrorResponse(throttleTimeMs, exception) @@ -320,7 +337,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleCreateTopics(request: RequestChannel.Request): Unit = { + def handleCreateTopics(request: RequestChannel.Request): CompletableFuture[Unit] = { val createTopicsRequest = request.body[CreateTopicsRequest] val context = new ControllerRequestContext(request.context.header.data, request.context.principal, requestTimeoutMsToDeadlineNs(time, createTopicsRequest.data.timeoutMs)) @@ -330,7 +347,7 @@ class ControllerApis(val requestChannel: RequestChannel, names => authHelper.filterByAuthorized(request.context, CREATE, TOPIC, names)(identity), names => authHelper.filterByAuthorized(request.context, DESCRIBE_CONFIGS, TOPIC, names, logIfDenied = false)(identity)) - future.whenComplete { (result, exception) => + future.handle[Unit] { (result, exception) => requestHelper.sendResponseMaybeThrottle(request, throttleTimeMs => { if (exception != null) { createTopicsRequest.getErrorResponse(throttleTimeMs, exception) @@ -392,7 +409,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleApiVersionsRequest(request: RequestChannel.Request): Unit = { + def handleApiVersionsRequest(request: RequestChannel.Request): CompletableFuture[Unit] = { // Note that broker returns its full list of supported ApiKeys and versions regardless of current // authentication state (e.g., before SASL authentication on an SASL listener, do note that no // Kafka protocol requests may take place on an SSL listener before the SSL handshake is finished). @@ -410,6 +427,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } requestHelper.sendResponseMaybeThrottle(request, createResponseCallback) + CompletableFuture.completedFuture[Unit](()) } def authorizeAlterResource(requestContext: RequestContext, @@ -431,7 +449,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleLegacyAlterConfigs(request: RequestChannel.Request): Unit = { + def handleLegacyAlterConfigs(request: RequestChannel.Request): CompletableFuture[Unit] = { val response = new AlterConfigsResponseData() val alterConfigsRequest = request.body[AlterConfigsRequest] val context = new ControllerRequestContext(request.context.header.data, request.context.principal, OptionalLong.empty()) @@ -474,7 +492,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } controller.legacyAlterConfigs(context, configChanges, alterConfigsRequest.data.validateOnly) - .whenComplete { (controllerResults, exception) => + .handle[Unit] { (controllerResults, exception) => if (exception != null) { requestHelper.handleError(request, exception) } else { @@ -490,33 +508,33 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleVote(request: RequestChannel.Request): Unit = { + def handleVote(request: RequestChannel.Request): CompletableFuture[Unit] = { authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) handleRaftRequest(request, response => new VoteResponse(response.asInstanceOf[VoteResponseData])) } - def handleBeginQuorumEpoch(request: RequestChannel.Request): Unit = { + def handleBeginQuorumEpoch(request: RequestChannel.Request): CompletableFuture[Unit] = { authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) handleRaftRequest(request, response => new BeginQuorumEpochResponse(response.asInstanceOf[BeginQuorumEpochResponseData])) } - def handleEndQuorumEpoch(request: RequestChannel.Request): Unit = { + def handleEndQuorumEpoch(request: RequestChannel.Request): CompletableFuture[Unit] = { authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) handleRaftRequest(request, response => new EndQuorumEpochResponse(response.asInstanceOf[EndQuorumEpochResponseData])) } - def handleDescribeQuorum(request: RequestChannel.Request): Unit = { + def handleDescribeQuorum(request: RequestChannel.Request): CompletableFuture[Unit] = { authHelper.authorizeClusterOperation(request, DESCRIBE) handleRaftRequest(request, response => new DescribeQuorumResponse(response.asInstanceOf[DescribeQuorumResponseData])) } - def handleElectLeaders(request: RequestChannel.Request): Unit = { + def handleElectLeaders(request: RequestChannel.Request): CompletableFuture[Unit] = { authHelper.authorizeClusterOperation(request, ALTER) val electLeadersRequest = request.body[ElectLeadersRequest] val context = new ControllerRequestContext(request.context.header.data, request.context.principal, requestTimeoutMsToDeadlineNs(time, electLeadersRequest.data.timeoutMs)) val future = controller.electLeaders(context, electLeadersRequest.data) - future.whenComplete { (responseData, exception) => + future.handle[Unit] { (responseData, exception) => if (exception != null) { requestHelper.sendResponseMaybeThrottle(request, throttleMs => { electLeadersRequest.getErrorResponse(throttleMs, exception) @@ -529,13 +547,13 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleAlterPartitionRequest(request: RequestChannel.Request): Unit = { + def handleAlterPartitionRequest(request: RequestChannel.Request): CompletableFuture[Unit] = { val alterPartitionRequest = request.body[AlterPartitionRequest] val context = new ControllerRequestContext(request.context.header.data, request.context.principal, OptionalLong.empty()) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) val future = controller.alterPartition(context, alterPartitionRequest.data) - future.whenComplete { (result, exception) => + future.handle[Unit] { (result, exception) => val response = if (exception != null) { alterPartitionRequest.getErrorResponse(exception) } else { @@ -545,7 +563,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleBrokerHeartBeatRequest(request: RequestChannel.Request): Unit = { + def handleBrokerHeartBeatRequest(request: RequestChannel.Request): CompletableFuture[Unit] = { val heartbeatRequest = request.body[BrokerHeartbeatRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) val context = new ControllerRequestContext(request.context.header.data, request.context.principal, @@ -572,7 +590,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleUnregisterBroker(request: RequestChannel.Request): Unit = { + def handleUnregisterBroker(request: RequestChannel.Request): CompletableFuture[Unit] = { val decommissionRequest = request.body[UnregisterBrokerRequest] authHelper.authorizeClusterOperation(request, ALTER) val context = new ControllerRequestContext(request.context.header.data, request.context.principal, @@ -595,7 +613,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleBrokerRegistration(request: RequestChannel.Request): Unit = { + def handleBrokerRegistration(request: RequestChannel.Request): CompletableFuture[Unit] = { val registrationRequest = request.body[BrokerRegistrationRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) val context = new ControllerRequestContext(request.context.header.data, request.context.principal, @@ -622,11 +640,10 @@ class ControllerApis(val requestChannel: RequestChannel, } private def handleRaftRequest(request: RequestChannel.Request, - buildResponse: ApiMessage => AbstractResponse): Unit = { + buildResponse: ApiMessage => AbstractResponse): CompletableFuture[Unit] = { val requestBody = request.body[AbstractRequest] val future = raftManager.handleRequest(request.header, requestBody.data, time.milliseconds()) - - future.whenComplete { (responseData, exception) => + future.handle[Unit] { (responseData, exception) => val response = if (exception != null) { requestBody.getErrorResponse(exception) } else { @@ -636,13 +653,13 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleAlterClientQuotas(request: RequestChannel.Request): Unit = { + def handleAlterClientQuotas(request: RequestChannel.Request): CompletableFuture[Unit] = { val quotaRequest = request.body[AlterClientQuotasRequest] authHelper.authorizeClusterOperation(request, ALTER_CONFIGS) val context = new ControllerRequestContext(request.context.header.data, request.context.principal, OptionalLong.empty()) controller.alterClientQuotas(context, quotaRequest.entries, quotaRequest.validateOnly) - .whenComplete { (results, exception) => + .handle[Unit] { (results, exception) => if (exception != null) { requestHelper.handleError(request, exception) } else { @@ -652,7 +669,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleIncrementalAlterConfigs(request: RequestChannel.Request): Unit = { + def handleIncrementalAlterConfigs(request: RequestChannel.Request): CompletableFuture[Unit] = { val response = new IncrementalAlterConfigsResponseData() val alterConfigsRequest = request.body[IncrementalAlterConfigsRequest] val context = new ControllerRequestContext(request.context.header.data, request.context.principal, @@ -700,7 +717,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } controller.incrementalAlterConfigs(context, configChanges, alterConfigsRequest.data.validateOnly) - .whenComplete { (controllerResults, exception) => + .handle[Unit] { (controllerResults, exception) => if (exception != null) { requestHelper.handleError(request, exception) } else { @@ -716,7 +733,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleCreatePartitions(request: RequestChannel.Request): Unit = { + def handleCreatePartitions(request: RequestChannel.Request): CompletableFuture[Unit] = { def filterAlterAuthorizedTopics(topics: Iterable[String]): Set[String] = { authHelper.filterByAuthorized(request.context, ALTER, TOPIC, topics)(n => n) } @@ -726,7 +743,7 @@ class ControllerApis(val requestChannel: RequestChannel, val future = createPartitions(context, createPartitionsRequest.data(), filterAlterAuthorizedTopics) - future.whenComplete { (responses, exception) => + future.handle[Unit] { (responses, exception) => if (exception != null) { requestHelper.handleError(request, exception) } else { @@ -778,33 +795,37 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleAlterPartitionReassignments(request: RequestChannel.Request): Unit = { + def handleAlterPartitionReassignments(request: RequestChannel.Request): CompletableFuture[Unit] = { val alterRequest = request.body[AlterPartitionReassignmentsRequest] authHelper.authorizeClusterOperation(request, ALTER) val context = new ControllerRequestContext(request.context.header.data, request.context.principal, requestTimeoutMsToDeadlineNs(time, alterRequest.data.timeoutMs)) - val response = controller.alterPartitionReassignments(context, alterRequest.data).get() - requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => - new AlterPartitionReassignmentsResponse(response.setThrottleTimeMs(requestThrottleMs))) + controller.alterPartitionReassignments(context, alterRequest.data) + .thenApply[Unit] { response => + requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => + new AlterPartitionReassignmentsResponse(response.setThrottleTimeMs(requestThrottleMs))) + } } - def handleListPartitionReassignments(request: RequestChannel.Request): Unit = { + def handleListPartitionReassignments(request: RequestChannel.Request): CompletableFuture[Unit] = { val listRequest = request.body[ListPartitionReassignmentsRequest] authHelper.authorizeClusterOperation(request, DESCRIBE) val context = new ControllerRequestContext(request.context.header.data, request.context.principal, OptionalLong.empty()) - val response = controller.listPartitionReassignments(context, listRequest.data).get() - requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => - new ListPartitionReassignmentsResponse(response.setThrottleTimeMs(requestThrottleMs))) + controller.listPartitionReassignments(context, listRequest.data) + .thenApply[Unit] { response => + requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => + new ListPartitionReassignmentsResponse(response.setThrottleTimeMs(requestThrottleMs))) + } } - def handleAllocateProducerIdsRequest(request: RequestChannel.Request): Unit = { + def handleAllocateProducerIdsRequest(request: RequestChannel.Request): CompletableFuture[Unit] = { val allocatedProducerIdsRequest = request.body[AllocateProducerIdsRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) val context = new ControllerRequestContext(request.context.header.data, request.context.principal, OptionalLong.empty()) controller.allocateProducerIds(context, allocatedProducerIdsRequest.data) - .whenComplete((results, exception) => { + .handle[Unit] { (results, exception) => if (exception != null) { requestHelper.handleError(request, exception) } else { @@ -813,22 +834,22 @@ class ControllerApis(val requestChannel: RequestChannel, new AllocateProducerIdsResponse(results) }) } - }) + } } - def handleUpdateFeatures(request: RequestChannel.Request): Unit = { + def handleUpdateFeatures(request: RequestChannel.Request): CompletableFuture[Unit] = { val updateFeaturesRequest = request.body[UpdateFeaturesRequest] authHelper.authorizeClusterOperation(request, ALTER) val context = new ControllerRequestContext(request.context.header.data, request.context.principal, OptionalLong.empty()) controller.updateFeatures(context, updateFeaturesRequest.data) - .whenComplete((response, exception) => { + .handle[Unit] { (response, exception) => if (exception != null) { requestHelper.handleError(request, exception) } else { requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => new UpdateFeaturesResponse(response.setThrottleTimeMs(requestThrottleMs))) } - }) + } } } diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala index 86a0c8547057b..0fc961145273b 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala @@ -63,6 +63,7 @@ import java.net.InetAddress import java.util import java.util.Collections.singletonList import java.util.concurrent.{CompletableFuture, ExecutionException, TimeUnit} +import java.util.concurrent.atomic.AtomicReference import java.util.{Collections, Properties} import scala.annotation.nowarn import scala.jdk.CollectionConverters._ @@ -902,6 +903,35 @@ class ControllerApisTest { } } + @Test + def testCompletableFutureExceptions(): Unit = { + // This test simulates an error in a completable future as we return from the controller. We need to ensure + // that any exception throw in the completion phase is properly captured and translated to an error response. + val request = buildRequest(new FetchRequest(new FetchRequestData(), 12)) + val response = new FetchResponseData() + val responseFuture = new CompletableFuture[ApiMessage]() + val errorResponseFuture = new AtomicReference[AbstractResponse]() + when(raftManager.handleRequest(any(), any(), any())).thenReturn(responseFuture) + when(requestChannel.sendResponse(any(), any(), any())).thenAnswer { _ => + // Simulate an encoding failure in the initial fetch response + throw new UnsupportedVersionException("Something went wrong") + }.thenAnswer { invocation => + val resp = invocation.getArgument(1, classOf[AbstractResponse]) + errorResponseFuture.set(resp) + } + + // Calling handle does not block since we do not call get() in ControllerApis + createControllerApis(None, + new MockController.Builder().build()).handle(request, null) + + // When we complete this future, the completion stages will fire (including the error handler in ControllerApis#request) + responseFuture.complete(response) + + // Now we should get an error response with UNSUPPORTED_VERSION + val errorResponse = errorResponseFuture.get() + assertEquals(1, errorResponse.errorCounts().getOrDefault(Errors.UNSUPPORTED_VERSION, 0)) + } + @AfterEach def tearDown(): Unit = { quotas.shutdown() From 112294334f4a2e539c37f8ea2a6064af7443351f Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Wed, 27 Jul 2022 10:38:31 -0700 Subject: [PATCH 034/175] MINOR: Convert some junit tests to kraft (#12443) Convert ProducerCompressionTest, MirrorMakerIntegrationTest, EdgeCaseRequestTest to kraft. Make it explicit that ServerShutdownTest#testControllerShutdownDuringSend is ZK-only. Reviewers: David Arthur --- .../kafka/api/ProducerCompressionTest.scala | 49 ++++++++++--------- .../tools/MirrorMakerIntegrationTest.scala | 24 +++++---- .../kafka/server/EdgeCaseRequestTest.scala | 32 +++++++----- .../kafka/server/ServerShutdownTest.scala | 10 ++-- 4 files changed, 64 insertions(+), 51 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala index ccdfe7d3d3676..07d9ccb024f9a 100755 --- a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala @@ -17,19 +17,19 @@ package kafka.api.test -import kafka.server.{KafkaConfig, KafkaServer} +import kafka.server.{KafkaBroker, KafkaConfig, QuorumTestHarness} import kafka.utils.TestUtils -import kafka.server.QuorumTestHarness import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.network.ListenerName +import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.serialization.ByteArraySerializer import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.{Arguments, MethodSource} +import org.junit.jupiter.params.provider.CsvSource import java.util.{Collections, Properties} -import scala.jdk.CollectionConverters._ class ProducerCompressionTest extends QuorumTestHarness { @@ -37,18 +37,18 @@ class ProducerCompressionTest extends QuorumTestHarness { private val topic = "topic" private val numRecords = 2000 - private var server: KafkaServer = null + private var broker: KafkaBroker = null @BeforeEach override def setUp(testInfo: TestInfo): Unit = { super.setUp(testInfo) - val props = TestUtils.createBrokerConfig(brokerId, zkConnect) - server = TestUtils.createServer(KafkaConfig.fromProps(props)) + val props = TestUtils.createBrokerConfig(brokerId, zkConnectOrNull) + broker = createBroker(new KafkaConfig(props)) } @AfterEach override def tearDown(): Unit = { - TestUtils.shutdownServers(Seq(server)) + TestUtils.shutdownServers(Seq(broker)) super.tearDown() } @@ -58,11 +58,18 @@ class ProducerCompressionTest extends QuorumTestHarness { * Compressed messages should be able to sent and consumed correctly */ @ParameterizedTest - @MethodSource(Array("parameters")) - def testCompression(compression: String): Unit = { + @CsvSource(value = Array( + "kraft,none", + "kraft,gzip", + "kraft,snappy", + "kraft,lz4", + "kraft,zstd", + "zk,gzip" + )) + def testCompression(quorum: String, compression: String): Unit = { val producerProps = new Properties() - val bootstrapServers = TestUtils.plaintextBootstrapServers(Seq(server)) + val bootstrapServers = TestUtils.plaintextBootstrapServers(Seq(broker)) producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers) producerProps.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, compression) producerProps.put(ProducerConfig.BATCH_SIZE_CONFIG, "66000") @@ -72,7 +79,13 @@ class ProducerCompressionTest extends QuorumTestHarness { try { // create topic - TestUtils.createTopic(zkClient, topic, 1, 1, List(server)) + val admin = TestUtils.createAdminClient(Seq(broker), + ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)) + try { + TestUtils.createTopicWithAdmin(admin, topic, Seq(broker)) + } finally { + admin.close() + } val partition = 0 // prepare the messages @@ -103,15 +116,3 @@ class ProducerCompressionTest extends QuorumTestHarness { } } } - -object ProducerCompressionTest { - def parameters: java.util.stream.Stream[Arguments] = { - Seq( - Arguments.of("none"), - Arguments.of("gzip"), - Arguments.of("snappy"), - Arguments.of("lz4"), - Arguments.of("zstd") - ).asJava.stream() - } -} diff --git a/core/src/test/scala/integration/kafka/tools/MirrorMakerIntegrationTest.scala b/core/src/test/scala/integration/kafka/tools/MirrorMakerIntegrationTest.scala index 4f673cdd60ada..c64d25fe4e6a6 100644 --- a/core/src/test/scala/integration/kafka/tools/MirrorMakerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/tools/MirrorMakerIntegrationTest.scala @@ -18,26 +18,27 @@ package kafka.tools import java.util.Properties import java.util.concurrent.atomic.AtomicBoolean - import scala.collection.Seq import kafka.integration.KafkaServerTestHarness import kafka.server.KafkaConfig import kafka.tools.MirrorMaker.{ConsumerWrapper, MirrorMakerProducer, NoRecordsException} -import kafka.utils.TestUtils +import kafka.utils.{TestInfoUtils, TestUtils} import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.TimeoutException import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer} import org.apache.kafka.common.utils.Exit -import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} +import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource @deprecated(message = "Use the Connect-based MirrorMaker instead (aka MM2).", since = "3.0") class MirrorMakerIntegrationTest extends KafkaServerTestHarness { override def generateConfigs: Seq[KafkaConfig] = - TestUtils.createBrokerConfigs(1, zkConnect).map(KafkaConfig.fromProps(_, new Properties())) + TestUtils.createBrokerConfigs(1, zkConnectOrNull).map(KafkaConfig.fromProps(_, new Properties())) val exited = new AtomicBoolean(false) @@ -57,8 +58,9 @@ class MirrorMakerIntegrationTest extends KafkaServerTestHarness { } } - @Test - def testCommitOffsetsThrowTimeoutException(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCommitOffsetsThrowTimeoutException(quorum: String): Unit = { val consumerProps = new Properties consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, "test-group") consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") @@ -70,8 +72,9 @@ class MirrorMakerIntegrationTest extends KafkaServerTestHarness { assertThrows(classOf[TimeoutException], () => mirrorMakerConsumer.commit()) } - @Test - def testCommitOffsetsRemoveNonExistentTopics(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCommitOffsetsRemoveNonExistentTopics(quorum: String): Unit = { val consumerProps = new Properties consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, "test-group") consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") @@ -85,8 +88,9 @@ class MirrorMakerIntegrationTest extends KafkaServerTestHarness { assertTrue(mirrorMakerConsumer.offsets.isEmpty, "Offsets for non-existent topics should be removed") } - @Test - def testCommaSeparatedRegex(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCommaSeparatedRegex(quorum: String): Unit = { val topic = "new-topic" val msg = "a test message" diff --git a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala index 1a383a8fbcdf9..1bbde3ffb6b88 100755 --- a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala @@ -35,19 +35,20 @@ import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.ByteUtils import org.apache.kafka.common.{TopicPartition, requests} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.Test +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import scala.jdk.CollectionConverters._ class EdgeCaseRequestTest extends KafkaServerTestHarness { def generateConfigs = { - val props = TestUtils.createBrokerConfig(1, zkConnect) + val props = TestUtils.createBrokerConfig(1, zkConnectOrNull) props.setProperty(KafkaConfig.AutoCreateTopicsEnableProp, "false") List(KafkaConfig.fromProps(props)) } - private def socketServer = servers.head.socketServer + private def socketServer = brokers.head.socketServer private def connect(s: SocketServer = socketServer, protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): Socket = { new Socket("localhost", s.boundPort(ListenerName.forSecurityProtocol(protocol))) @@ -116,8 +117,9 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness { } } - @Test - def testProduceRequestWithNullClientId(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testProduceRequestWithNullClientId(quorum: String): Unit = { val topic = "topic" val topicPartition = new TopicPartition(topic, 0) val correlationId = -1 @@ -161,23 +163,27 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness { assertEquals(Errors.NONE, Errors.forCode(partitionProduceResponse.errorCode), "There should be no error") } - @Test - def testHeaderOnlyRequest(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testHeaderOnlyRequest(quorum: String): Unit = { verifyDisconnect(requestHeaderBytes(ApiKeys.PRODUCE.id, 1)) } - @Test - def testInvalidApiKeyRequest(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testInvalidApiKeyRequest(quorum: String): Unit = { verifyDisconnect(requestHeaderBytes(-1, 0)) } - @Test - def testInvalidApiVersionRequest(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testInvalidApiVersionRequest(quorum: String): Unit = { verifyDisconnect(requestHeaderBytes(ApiKeys.PRODUCE.id, -1)) } - @Test - def testMalformedHeaderRequest(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testMalformedHeaderRequest(quorum: String): Unit = { val serializedBytes = { // Only send apiKey and apiVersion val buffer = ByteBuffer.allocate( diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index 96aeac5fa61ec..70554d9427c2f 100644 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -38,7 +38,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.serialization.{IntegerDeserializer, IntegerSerializer, StringDeserializer, StringSerializer} import org.apache.kafka.common.utils.Time import org.apache.kafka.metadata.BrokerState -import org.junit.jupiter.api.{BeforeEach, Disabled, Test, TestInfo, Timeout} +import org.junit.jupiter.api.{BeforeEach, Disabled, TestInfo, Timeout} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.function.Executable import org.junit.jupiter.params.ParameterizedTest @@ -251,9 +251,11 @@ class ServerShutdownTest extends KafkaServerTestHarness { } // Verify that if controller is in the midst of processing a request, shutdown completes - // without waiting for request timeout. - @Test - def testControllerShutdownDuringSend(): Unit = { + // without waiting for request timeout. Since this involves LeaderAndIsr request, it is + // ZK-only for now. + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk")) + def testControllerShutdownDuringSend(quorum: String): Unit = { val securityProtocol = SecurityProtocol.PLAINTEXT val listenerName = ListenerName.forSecurityProtocol(securityProtocol) From 8c8cb111a4fb6f17d80fb55393ff4ffff03828a6 Mon Sep 17 00:00:00 2001 From: Ron Dagostino Date: Thu, 21 Jul 2022 19:59:05 -0400 Subject: [PATCH 035/175] KAFKA-14051: Create metrics reporters in KRaft remote controllers (#12396) KRaft remote controllers do not yet support dynamic reconfiguration (https://issues.apache.org/jira/browse/KAFKA-14057). Until we implement that, in the meantime we see that the instantiation of the configured metric reporters is actually performed as part of the wiring for dynamic reconfiguration. Since that wiring does not exist yet for KRaft remote controllers, this patch refactors out the instantiation of the metric reporters from the reconfiguration of them and adjusts the controller startup sequence to explicitly instantiate the reporters if the controller is a remote one. Reviewers: Colin P. McCabe --- .../scala/kafka/server/ControllerServer.scala | 15 ++++++ .../kafka/server/DynamicBrokerConfig.scala | 46 +++++++++++++------ 2 files changed, 46 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index 28c98643c3aff..212c092e1ab2e 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -27,6 +27,7 @@ import kafka.network.{DataPlaneAcceptor, SocketServer} import kafka.raft.RaftManager import kafka.security.CredentialProvider import kafka.server.KafkaConfig.{AlterConfigPolicyClassNameProp, CreateTopicPolicyClassNameProp} +import kafka.server.KafkaRaftServer.BrokerRole import kafka.server.QuotaFactory.QuotaManagers import kafka.utils.{CoreUtils, Logging} import org.apache.kafka.clients.ApiVersions @@ -70,6 +71,8 @@ class ControllerServer( ) extends Logging with KafkaMetricsGroup { import kafka.server.Server._ + config.dynamicConfig.initialize(zkClientOpt = None) + val lock = new ReentrantLock() val awaitShutdownCond = lock.newCondition() var status: ProcessStatus = SHUTDOWN @@ -99,6 +102,13 @@ class ControllerServer( true } + private def doRemoteKraftSetup(): Unit = { + // Explicitly configure metric reporters on this remote controller. + // We do not yet support dynamic reconfiguration on remote controllers in general; + // remove this once that is implemented. + new DynamicMetricReporterState(config.nodeId, config, metrics, clusterId) + } + def clusterId: String = metaProperties.clusterId def startup(): Unit = { @@ -206,6 +216,11 @@ class ControllerServer( } controller = controllerBuilder.build() + // Perform any setup that is done only when this node is a controller-only node. + if (!config.processRoles.contains(BrokerRole)) { + doRemoteKraftSetup() + } + quotaManagers = QuotaFactory.instantiate(config, metrics, time, threadNamePrefix.getOrElse("")) controllerApis = new ControllerApis(socketServer.dataPlaneRequestChannel, authorizer, diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 76a42b74fa509..a860938124e6a 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -30,7 +30,7 @@ import kafka.utils.Implicits._ import kafka.zk.{AdminZkClient, KafkaZkClient} import org.apache.kafka.common.Reconfigurable import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, SslConfigs} -import org.apache.kafka.common.metrics.MetricsReporter +import org.apache.kafka.common.metrics.{Metrics, MetricsReporter} import org.apache.kafka.common.config.types.Password import org.apache.kafka.common.network.{ListenerName, ListenerReconfigurable} import org.apache.kafka.common.security.authenticator.LoginManager @@ -258,7 +258,7 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging case _ => } addReconfigurable(kafkaServer.kafkaYammerMetrics) - addReconfigurable(new DynamicMetricsReporters(kafkaConfig.brokerId, kafkaServer)) + addReconfigurable(new DynamicMetricsReporters(kafkaConfig.brokerId, kafkaServer.config, kafkaServer.metrics, kafkaServer.clusterId)) addReconfigurable(new DynamicClientQuotaCallback(kafkaServer)) addBrokerReconfigurable(new DynamicThreadPool(kafkaServer)) @@ -748,17 +748,18 @@ class DynamicThreadPool(server: KafkaBroker) extends BrokerReconfigurable { } } -class DynamicMetricsReporters(brokerId: Int, server: KafkaBroker) extends Reconfigurable { +class DynamicMetricsReporters(brokerId: Int, config: KafkaConfig, metrics: Metrics, clusterId: String) extends Reconfigurable { + private val reporterState = new DynamicMetricReporterState(brokerId, config, metrics, clusterId) + private val currentReporters = reporterState.currentReporters + private val dynamicConfig = reporterState.dynamicConfig - private val dynamicConfig = server.config.dynamicConfig - private val metrics = server.metrics - private val propsOverride = Map[String, AnyRef](KafkaConfig.BrokerIdProp -> brokerId.toString) - private val currentReporters = mutable.Map[String, MetricsReporter]() + private def metricsReporterClasses(configs: util.Map[String, _]): mutable.Buffer[String] = + reporterState.metricsReporterClasses(configs) - createReporters(dynamicConfig.currentKafkaConfig.getList(KafkaConfig.MetricReporterClassesProp), - Collections.emptyMap[String, Object]) + private def createReporters(reporterClasses: util.List[String], updatedConfigs: util.Map[String, _]): Unit = + reporterState.createReporters(reporterClasses, updatedConfigs) - private[server] def currentMetricsReporters: List[MetricsReporter] = currentReporters.values.toList + private def removeReporter(className: String): Unit = reporterState.removeReporter(className) override def configure(configs: util.Map[String, _]): Unit = {} @@ -801,8 +802,23 @@ class DynamicMetricsReporters(brokerId: Int, server: KafkaBroker) extends Reconf val added = updatedMetricsReporters.filterNot(currentReporters.keySet) createReporters(added.asJava, configs) } +} + +class DynamicMetricReporterState(brokerId: Int, config: KafkaConfig, metrics: Metrics, clusterId: String) { + private[server] val dynamicConfig = config.dynamicConfig + private val propsOverride = Map[String, AnyRef](KafkaConfig.BrokerIdProp -> brokerId.toString) + private[server] val currentReporters = mutable.Map[String, MetricsReporter]() + createReporters(config, clusterId, metricsReporterClasses(dynamicConfig.currentKafkaConfig.values()).asJava, + Collections.emptyMap[String, Object]) + + private[server] def createReporters(reporterClasses: util.List[String], + updatedConfigs: util.Map[String, _]): Unit = { + createReporters(config, clusterId, reporterClasses, updatedConfigs) + } - private def createReporters(reporterClasses: util.List[String], + private def createReporters(config: KafkaConfig, + clusterId: String, + reporterClasses: util.List[String], updatedConfigs: util.Map[String, _]): Unit = { val props = new util.HashMap[String, AnyRef] updatedConfigs.forEach((k, v) => props.put(k, v.asInstanceOf[AnyRef])) @@ -811,19 +827,19 @@ class DynamicMetricsReporters(brokerId: Int, server: KafkaBroker) extends Reconf // Call notifyMetricsReporters first to satisfy the contract for MetricsReporter.contextChange, // which provides that MetricsReporter.contextChange must be called before the first call to MetricsReporter.init. // The first call to MetricsReporter.init is done when we call metrics.addReporter below. - KafkaBroker.notifyMetricsReporters(server.clusterId, server.config, reporters.asScala) + KafkaBroker.notifyMetricsReporters(clusterId, config, reporters.asScala) reporters.forEach { reporter => metrics.addReporter(reporter) currentReporters += reporter.getClass.getName -> reporter } - KafkaBroker.notifyClusterListeners(server.clusterId, reporters.asScala) + KafkaBroker.notifyClusterListeners(clusterId, reporters.asScala) } - private def removeReporter(className: String): Unit = { + private[server] def removeReporter(className: String): Unit = { currentReporters.remove(className).foreach(metrics.removeReporter) } - private def metricsReporterClasses(configs: util.Map[String, _]): mutable.Buffer[String] = { + private[server] def metricsReporterClasses(configs: util.Map[String, _]): mutable.Buffer[String] = { configs.get(KafkaConfig.MetricReporterClassesProp).asInstanceOf[util.List[String]].asScala } } From f3cf6db3a53a032fb0480258b29ecf416e8385d3 Mon Sep 17 00:00:00 2001 From: Niket Goel Date: Wed, 27 Jul 2022 18:52:33 -0700 Subject: [PATCH 036/175] KAFKA-14114: Add Metadata Error Related Metrics This PR adds in 3 metrics as described in KIP-859: kafka.server:type=broker-metadata-metrics,name=metadata-load-error-count kafka.server:type=broker-metadata-metrics,name=metadata-apply-error-count kafka.controller:type=KafkaController,name=MetadataErrorCount These metrics are incremented by fault handlers when the appropriate fault happens. Broker-side load errors happen in BrokerMetadataListener. Broker-side apply errors happen in the BrokerMetadataPublisher. The metric on the controller is incremented when the standby controller (not active) encounters a metadata error. In BrokerMetadataPublisher, try to limit the damage caused by an exception by introducing more catch blocks. The only fatal failures here are those that happen during initialization, when we initialize the manager objects (these would also be fatal in ZK mode). In BrokerMetadataListener, try to improve the logging of faults, especially ones that happen when replaying a snapshot. Try to limit the damage caused by an exception. Replace MetadataFaultHandler with LoggingFaultHandler, which is more flexible and takes a Runnable argument. Add LoggingFaultHandlerTest. Make QuorumControllerMetricsTest stricter. Fix a bug where we weren't cleaning up some metrics from the yammer registry on close in QuorumControllerMetrics. Co-author: Colin P. McCabe --- .../scala/kafka/server/BrokerServer.scala | 15 +- .../scala/kafka/server/ControllerServer.scala | 5 +- .../scala/kafka/server/KafkaRaftServer.scala | 27 +- .../metadata/BrokerMetadataListener.scala | 56 +++- .../metadata/BrokerMetadataPublisher.scala | 300 +++++++++++------- .../server/metadata/BrokerServerMetrics.scala | 26 +- .../kafka/testkit/KafkaClusterTestKit.java | 17 +- .../kafka/server/QuorumTestHarness.scala | 34 +- .../metadata/BrokerServerMetricsTest.scala | 38 ++- .../unit/kafka/metrics/MetricsTest.scala | 25 +- .../metadata/BrokerMetadataListenerTest.scala | 17 +- .../BrokerMetadataPublisherTest.scala | 77 ++++- .../kafka/controller/ControllerMetrics.java | 4 + .../kafka/controller/QuorumController.java | 8 +- .../controller/QuorumControllerMetrics.java | 24 ++ .../fault/MetadataFaultException.java | 32 -- .../metadata/fault/MetadataFaultHandler.java | 36 --- .../controller/MockControllerMetrics.java | 13 + .../QuorumControllerMetricsTest.java | 30 ++ .../kafka/server/fault/FaultHandler.java | 25 +- .../server/fault/FaultHandlerException.java} | 8 +- .../server/fault/LoggingFaultHandler.java | 54 ++++ .../fault/ProcessExitingFaultHandler.java | 9 +- .../server/fault/LoggingFaultHandlerTest.java | 57 ++++ .../kafka/server/fault/MockFaultHandler.java | 20 +- 25 files changed, 669 insertions(+), 288 deletions(-) delete mode 100644 metadata/src/main/java/org/apache/kafka/metadata/fault/MetadataFaultException.java delete mode 100644 metadata/src/main/java/org/apache/kafka/metadata/fault/MetadataFaultHandler.java rename server-common/src/{test/java/org/apache/kafka/server/fault/MockFaultHandlerException.java => main/java/org/apache/kafka/server/fault/FaultHandlerException.java} (83%) create mode 100644 server-common/src/main/java/org/apache/kafka/server/fault/LoggingFaultHandler.java create mode 100644 server-common/src/test/java/org/apache/kafka/server/fault/LoggingFaultHandlerTest.java diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 0bdd673497568..1008decadb11e 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -50,6 +50,7 @@ import org.apache.kafka.raft.RaftConfig.AddressSpec import org.apache.kafka.raft.{RaftClient, RaftConfig} import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.common.ApiMessageAndVersion +import org.apache.kafka.server.fault.FaultHandler import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.snapshot.SnapshotWriter @@ -76,9 +77,13 @@ class BrokerServer( val raftManager: RaftManager[ApiMessageAndVersion], val time: Time, val metrics: Metrics, + val brokerMetrics: BrokerServerMetrics, val threadNamePrefix: Option[String], val initialOfflineDirs: Seq[String], - val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]] + val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]], + val fatalFaultHandler: FaultHandler, + val metadataLoadingFaultHandler: FaultHandler, + val metadataPublishingFaultHandler: FaultHandler ) extends KafkaBroker { override def brokerState: BrokerState = Option(lifecycleManager). @@ -315,8 +320,8 @@ class BrokerServer( threadNamePrefix, config.metadataSnapshotMaxNewRecordBytes, metadataSnapshotter, - BrokerServerMetrics(metrics) - ) + brokerMetrics, + metadataLoadingFaultHandler) val networkListeners = new ListenerCollection() config.effectiveAdvertisedListeners.foreach { ep => @@ -432,7 +437,9 @@ class BrokerServer( transactionCoordinator, clientQuotaMetadataManager, dynamicConfigHandlers.toMap, - authorizer) + authorizer, + fatalFaultHandler, + metadataPublishingFaultHandler) // Tell the metadata listener to start publishing its output, and wait for the first // publish operation to complete. This first operation will initialize logManager, diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index 212c092e1ab2e..19a6e307d62be 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -37,7 +37,7 @@ import org.apache.kafka.common.security.scram.internals.ScramMechanism import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache import org.apache.kafka.common.utils.{LogContext, Time} import org.apache.kafka.common.{ClusterResource, Endpoint} -import org.apache.kafka.controller.{BootstrapMetadata, Controller, QuorumController, QuorumControllerMetrics, QuorumFeatures} +import org.apache.kafka.controller.{BootstrapMetadata, Controller, ControllerMetrics, QuorumController, QuorumFeatures} import org.apache.kafka.metadata.KafkaConfigSchema import org.apache.kafka.raft.RaftConfig import org.apache.kafka.raft.RaftConfig.AddressSpec @@ -61,6 +61,7 @@ class ControllerServer( val raftManager: RaftManager[ApiMessageAndVersion], val time: Time, val metrics: Metrics, + val controllerMetrics: ControllerMetrics, val threadNamePrefix: Option[String], val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]], val configSchema: KafkaConfigSchema, @@ -201,7 +202,7 @@ class ControllerServer( setSnapshotMaxNewRecordBytes(config.metadataSnapshotMaxNewRecordBytes). setLeaderImbalanceCheckIntervalNs(leaderImbalanceCheckIntervalNs). setMaxIdleIntervalNs(maxIdleIntervalNs). - setMetrics(new QuorumControllerMetrics(KafkaYammerMetrics.defaultRegistry(), time)). + setMetrics(controllerMetrics). setCreateTopicPolicy(createTopicPolicy.asJava). setAlterConfigPolicy(alterConfigPolicy.asJava). setConfigurationValidator(new ControllerConfigurationValidator()). diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala index e7cf8f8f1fab5..2338ef5e7c413 100644 --- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala +++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala @@ -23,17 +23,17 @@ import kafka.log.{LogConfig, UnifiedLog} import kafka.metrics.KafkaMetricsReporter import kafka.raft.KafkaRaftManager import kafka.server.KafkaRaftServer.{BrokerRole, ControllerRole} +import kafka.server.metadata.BrokerServerMetrics import kafka.utils.{CoreUtils, Logging, Mx4jLoader, VerifiableProperties} import org.apache.kafka.common.config.{ConfigDef, ConfigResource} import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.utils.{AppInfoParser, Time} import org.apache.kafka.common.{KafkaException, Uuid} -import org.apache.kafka.controller.BootstrapMetadata -import org.apache.kafka.metadata.fault.MetadataFaultHandler +import org.apache.kafka.controller.{BootstrapMetadata, QuorumControllerMetrics} import org.apache.kafka.metadata.{KafkaConfigSchema, MetadataRecordSerde} import org.apache.kafka.raft.RaftConfig import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} -import org.apache.kafka.server.fault.ProcessExitingFaultHandler +import org.apache.kafka.server.fault.{LoggingFaultHandler, ProcessExitingFaultHandler} import org.apache.kafka.server.metrics.KafkaYammerMetrics import java.nio.file.Paths @@ -83,34 +83,49 @@ class KafkaRaftServer( ) private val broker: Option[BrokerServer] = if (config.processRoles.contains(BrokerRole)) { + val brokerMetrics = BrokerServerMetrics(metrics) + val fatalFaultHandler = new ProcessExitingFaultHandler() + val metadataLoadingFaultHandler = new LoggingFaultHandler("metadata loading", + () => brokerMetrics.metadataLoadErrorCount.getAndIncrement()) + val metadataApplyingFaultHandler = new LoggingFaultHandler("metadata application", + () => brokerMetrics.metadataApplyErrorCount.getAndIncrement()) Some(new BrokerServer( config, metaProps, raftManager, time, metrics, + brokerMetrics, threadNamePrefix, offlineDirs, - controllerQuorumVotersFuture + controllerQuorumVotersFuture, + fatalFaultHandler, + metadataLoadingFaultHandler, + metadataApplyingFaultHandler )) } else { None } private val controller: Option[ControllerServer] = if (config.processRoles.contains(ControllerRole)) { + val controllerMetrics = new QuorumControllerMetrics(KafkaYammerMetrics.defaultRegistry(), time) + val metadataFaultHandler = new LoggingFaultHandler("controller metadata", + () => controllerMetrics.incrementMetadataErrorCount()) + val fatalFaultHandler = new ProcessExitingFaultHandler() Some(new ControllerServer( metaProps, config, raftManager, time, metrics, + controllerMetrics, threadNamePrefix, controllerQuorumVotersFuture, KafkaRaftServer.configSchema, raftManager.apiVersions, bootstrapMetadata, - new MetadataFaultHandler(), - new ProcessExitingFaultHandler(), + metadataFaultHandler, + fatalFaultHandler )) } else { None diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala index 3b79526a954e9..3984f467eddab 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala @@ -19,13 +19,13 @@ package kafka.server.metadata import java.util import java.util.concurrent.{CompletableFuture, TimeUnit} import java.util.function.Consumer - import kafka.metrics.KafkaMetricsGroup import org.apache.kafka.image.{MetadataDelta, MetadataImage} import org.apache.kafka.common.utils.{LogContext, Time} import org.apache.kafka.queue.{EventQueue, KafkaEventQueue} import org.apache.kafka.raft.{Batch, BatchReader, LeaderAndEpoch, RaftClient} import org.apache.kafka.server.common.ApiMessageAndVersion +import org.apache.kafka.server.fault.FaultHandler import org.apache.kafka.snapshot.SnapshotReader @@ -40,7 +40,8 @@ class BrokerMetadataListener( threadNamePrefix: Option[String], val maxBytesBetweenSnapshots: Long, val snapshotter: Option[MetadataSnapshotter], - brokerMetrics: BrokerServerMetrics + brokerMetrics: BrokerServerMetrics, + metadataLoadingFaultHandler: FaultHandler ) extends RaftClient.Listener[ApiMessageAndVersion] with KafkaMetricsGroup { private val logContext = new LogContext(s"[BrokerMetadataListener id=$brokerId] ") private val log = logContext.logger(classOf[BrokerMetadataListener]) @@ -109,11 +110,16 @@ class BrokerMetadataListener( extends EventQueue.FailureLoggingEvent(log) { override def run(): Unit = { val results = try { - val loadResults = loadBatches(_delta, reader, None, None, None) + val loadResults = loadBatches(_delta, reader, None, None, None, None) if (isDebugEnabled) { debug(s"Loaded new commits: $loadResults") } loadResults + } catch { + case e: Throwable => + metadataLoadingFaultHandler.handleFault(s"Unable to load metadata commits " + + s"from the BatchReader starting at base offset ${reader.baseOffset()}", e) + return } finally { reader.close() } @@ -156,19 +162,26 @@ class BrokerMetadataListener( class HandleSnapshotEvent(reader: SnapshotReader[ApiMessageAndVersion]) extends EventQueue.FailureLoggingEvent(log) { override def run(): Unit = { + val snapshotName = s"${reader.snapshotId().offset}-${reader.snapshotId().epoch}" try { - info(s"Loading snapshot ${reader.snapshotId().offset}-${reader.snapshotId().epoch}.") + info(s"Loading snapshot ${snapshotName}") _delta = new MetadataDelta(_image) // Discard any previous deltas. - val loadResults = loadBatches( - _delta, + val loadResults = loadBatches(_delta, reader, Some(reader.lastContainedLogTimestamp), Some(reader.lastContainedLogOffset), - Some(reader.lastContainedLogEpoch) - ) - _delta.finishSnapshot() - info(s"Loaded snapshot ${reader.snapshotId().offset}-${reader.snapshotId().epoch}: " + - s"$loadResults") + Some(reader.lastContainedLogEpoch), + Some(snapshotName)) + try { + _delta.finishSnapshot() + } catch { + case e: Throwable => metadataLoadingFaultHandler.handleFault( + s"Error finishing snapshot ${snapshotName}", e) + } + info(s"Loaded snapshot ${snapshotName}: ${loadResults}") + } catch { + case t: Throwable => metadataLoadingFaultHandler.handleFault("Uncaught exception while " + + s"loading broker metadata from Metadata snapshot ${snapshotName}", t) } finally { reader.close() } @@ -201,7 +214,8 @@ class BrokerMetadataListener( iterator: util.Iterator[Batch[ApiMessageAndVersion]], lastAppendTimestamp: Option[Long], lastCommittedOffset: Option[Long], - lastCommittedEpoch: Option[Int] + lastCommittedEpoch: Option[Int], + snapshotName: Option[String] ): BatchLoadResults = { val startTimeNs = time.nanoseconds() var numBatches = 0 @@ -220,12 +234,20 @@ class BrokerMetadataListener( trace(s"Metadata batch ${batch.lastOffset}: processing [${index + 1}/${batch.records.size}]:" + s" ${messageAndVersion.message}") } - _highestOffset = lastCommittedOffset.getOrElse(batch.baseOffset() + index) - - delta.replay(highestMetadataOffset, epoch, messageAndVersion.message()) - numRecords += 1 - index += 1 + try { + delta.replay(highestMetadataOffset, epoch, messageAndVersion.message()) + } catch { + case e: Throwable => snapshotName match { + case None => metadataLoadingFaultHandler.handleFault( + s"Error replaying metadata log record at offset ${_highestOffset}", e) + case Some(name) => metadataLoadingFaultHandler.handleFault( + s"Error replaying record ${index} from snapshot ${name} at offset ${_highestOffset}", e) + } + } finally { + numRecords += 1 + index += 1 + } } numBytes = numBytes + batch.sizeInBytes() metadataBatchSizeHist.update(batch.records().size()) diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala index 212f188504e92..0192bb4afcfd7 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala @@ -31,6 +31,7 @@ import org.apache.kafka.common.internals.Topic import org.apache.kafka.image.{MetadataDelta, MetadataImage, TopicDelta, TopicsImage} import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer import org.apache.kafka.server.authorizer.Authorizer +import org.apache.kafka.server.fault.FaultHandler import scala.collection.mutable @@ -94,15 +95,19 @@ object BrokerMetadataPublisher extends Logging { } } -class BrokerMetadataPublisher(conf: KafkaConfig, - metadataCache: KRaftMetadataCache, - logManager: LogManager, - replicaManager: ReplicaManager, - groupCoordinator: GroupCoordinator, - txnCoordinator: TransactionCoordinator, - clientQuotaMetadataManager: ClientQuotaMetadataManager, - dynamicConfigHandlers: Map[String, ConfigHandler], - private val _authorizer: Option[Authorizer]) extends MetadataPublisher with Logging { +class BrokerMetadataPublisher( + conf: KafkaConfig, + metadataCache: KRaftMetadataCache, + logManager: LogManager, + replicaManager: ReplicaManager, + groupCoordinator: GroupCoordinator, + txnCoordinator: TransactionCoordinator, + clientQuotaMetadataManager: ClientQuotaMetadataManager, + dynamicConfigHandlers: Map[String, ConfigHandler], + private val _authorizer: Option[Authorizer], + fatalFaultHandler: FaultHandler, + metadataPublishingFaultHandler: FaultHandler +) extends MetadataPublisher with Logging { logIdent = s"[BrokerMetadataPublisher id=${conf.nodeId}] " import BrokerMetadataPublisher._ @@ -125,8 +130,15 @@ class BrokerMetadataPublisher(conf: KafkaConfig, override def publish(delta: MetadataDelta, newImage: MetadataImage): Unit = { val highestOffsetAndEpoch = newImage.highestOffsetAndEpoch() + val deltaName = if (_firstPublish) { + s"initial MetadataDelta up to ${highestOffsetAndEpoch.offset}" + } else { + s"MetadataDelta up to ${highestOffsetAndEpoch.offset}" + } try { - trace(s"Publishing delta $delta with highest offset $highestOffsetAndEpoch") + if (isTraceEnabled) { + trace(s"Publishing delta $delta with highest offset $highestOffsetAndEpoch") + } // Publish the new metadata image to the metadata cache. metadataCache.setImage(newImage) @@ -151,37 +163,50 @@ class BrokerMetadataPublisher(conf: KafkaConfig, // Apply topic deltas. Option(delta.topicsDelta()).foreach { topicsDelta => - // Notify the replica manager about changes to topics. - replicaManager.applyDelta(topicsDelta, newImage) - - // Update the group coordinator of local changes - updateCoordinator( - newImage, - delta, - Topic.GROUP_METADATA_TOPIC_NAME, - groupCoordinator.onElection, - groupCoordinator.onResignation - ) - - // Update the transaction coordinator of local changes - updateCoordinator( - newImage, - delta, - Topic.TRANSACTION_STATE_TOPIC_NAME, - txnCoordinator.onElection, - txnCoordinator.onResignation - ) - - // Notify the group coordinator about deleted topics. - val deletedTopicPartitions = new mutable.ArrayBuffer[TopicPartition]() - topicsDelta.deletedTopicIds().forEach { id => - val topicImage = topicsDelta.image().getTopic(id) - topicImage.partitions().keySet().forEach { - id => deletedTopicPartitions += new TopicPartition(topicImage.name(), id) - } + try { + // Notify the replica manager about changes to topics. + replicaManager.applyDelta(topicsDelta, newImage) + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error applying topics " + + s"delta in ${deltaName}", t) + } + try { + // Update the group coordinator of local changes + updateCoordinator(newImage, + delta, + Topic.GROUP_METADATA_TOPIC_NAME, + groupCoordinator.onElection, + groupCoordinator.onResignation) + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating group " + + s"coordinator with local changes in ${deltaName}", t) + } + try { + // Update the transaction coordinator of local changes + updateCoordinator(newImage, + delta, + Topic.TRANSACTION_STATE_TOPIC_NAME, + txnCoordinator.onElection, + txnCoordinator.onResignation) + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating txn " + + s"coordinator with local changes in ${deltaName}", t) } - if (deletedTopicPartitions.nonEmpty) { - groupCoordinator.handleDeletedPartitions(deletedTopicPartitions, RequestLocal.NoCaching) + try { + // Notify the group coordinator about deleted topics. + val deletedTopicPartitions = new mutable.ArrayBuffer[TopicPartition]() + topicsDelta.deletedTopicIds().forEach { id => + val topicImage = topicsDelta.image().getTopic(id) + topicImage.partitions().keySet().forEach { + id => deletedTopicPartitions += new TopicPartition(topicImage.name(), id) + } + } + if (deletedTopicPartitions.nonEmpty) { + groupCoordinator.handleDeletedPartitions(deletedTopicPartitions, RequestLocal.NoCaching) + } + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating group " + + s"coordinator with deleted partitions in ${deltaName}", t) } } @@ -191,39 +216,62 @@ class BrokerMetadataPublisher(conf: KafkaConfig, val props = newImage.configs().configProperties(resource) resource.`type`() match { case TOPIC => - // Apply changes to a topic's dynamic configuration. - info(s"Updating topic ${resource.name()} with new configuration : " + - toLoggableProps(resource, props).mkString(",")) - dynamicConfigHandlers(ConfigType.Topic). - processConfigChanges(resource.name(), props) + try { + // Apply changes to a topic's dynamic configuration. + info(s"Updating topic ${resource.name()} with new configuration : " + + toLoggableProps(resource, props).mkString(",")) + dynamicConfigHandlers(ConfigType.Topic). + processConfigChanges(resource.name(), props) + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating topic " + + s"${resource.name()} with new configuration: ${toLoggableProps(resource, props).mkString(",")} " + + s"in ${deltaName}", t) + } case BROKER => if (resource.name().isEmpty) { - // Apply changes to "cluster configs" (also known as default BROKER configs). - // These are stored in KRaft with an empty name field. - info("Updating cluster configuration : " + - toLoggableProps(resource, props).mkString(",")) - dynamicConfigHandlers(ConfigType.Broker). - processConfigChanges(ConfigEntityName.Default, props) + try { + // Apply changes to "cluster configs" (also known as default BROKER configs). + // These are stored in KRaft with an empty name field. + info("Updating cluster configuration : " + + toLoggableProps(resource, props).mkString(",")) + dynamicConfigHandlers(ConfigType.Broker). + processConfigChanges(ConfigEntityName.Default, props) + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating " + + s"cluster with new configuration: ${toLoggableProps(resource, props).mkString(",")} " + + s"in ${deltaName}", t) + } } else if (resource.name() == brokerId.toString) { - // Apply changes to this broker's dynamic configuration. - info(s"Updating broker $brokerId with new configuration : " + - toLoggableProps(resource, props).mkString(",")) - dynamicConfigHandlers(ConfigType.Broker). - processConfigChanges(resource.name(), props) - // When applying a per broker config (not a cluster config), we also - // reload any associated file. For example, if the ssl.keystore is still - // set to /tmp/foo, we still want to reload /tmp/foo in case its contents - // have changed. This doesn't apply to topic configs or cluster configs. - reloadUpdatedFilesWithoutConfigChange(props) + try { + // Apply changes to this broker's dynamic configuration. + info(s"Updating broker $brokerId with new configuration : " + + toLoggableProps(resource, props).mkString(",")) + dynamicConfigHandlers(ConfigType.Broker). + processConfigChanges(resource.name(), props) + // When applying a per broker config (not a cluster config), we also + // reload any associated file. For example, if the ssl.keystore is still + // set to /tmp/foo, we still want to reload /tmp/foo in case its contents + // have changed. This doesn't apply to topic configs or cluster configs. + reloadUpdatedFilesWithoutConfigChange(props) + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating " + + s"broker with new configuration: ${toLoggableProps(resource, props).mkString(",")} " + + s"in ${deltaName}", t) + } } case _ => // nothing to do } } } - // Apply client quotas delta. - Option(delta.clientQuotasDelta()).foreach { clientQuotasDelta => - clientQuotaMetadataManager.update(clientQuotasDelta) + try { + // Apply client quotas delta. + Option(delta.clientQuotasDelta()).foreach { clientQuotasDelta => + clientQuotaMetadataManager.update(clientQuotasDelta) + } + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating client " + + s"quotas in ${deltaName}", t) } // Apply changes to ACLs. This needs to be handled carefully because while we are @@ -235,20 +283,30 @@ class BrokerMetadataPublisher(conf: KafkaConfig, Option(delta.aclsDelta()).foreach( aclsDelta => _authorizer match { case Some(authorizer: ClusterMetadataAuthorizer) => if (aclsDelta.isSnapshotDelta) { - // If the delta resulted from a snapshot load, we want to apply the new changes - // all at once using ClusterMetadataAuthorizer#loadSnapshot. If this is the - // first snapshot load, it will also complete the futures returned by - // Authorizer#start (which we wait for before processing RPCs). - authorizer.loadSnapshot(newImage.acls().acls()) + try { + // If the delta resulted from a snapshot load, we want to apply the new changes + // all at once using ClusterMetadataAuthorizer#loadSnapshot. If this is the + // first snapshot load, it will also complete the futures returned by + // Authorizer#start (which we wait for before processing RPCs). + authorizer.loadSnapshot(newImage.acls().acls()) + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error loading " + + s"authorizer snapshot in ${deltaName}", t) + } } else { - // Because the changes map is a LinkedHashMap, the deltas will be returned in - // the order they were performed. - aclsDelta.changes().entrySet().forEach(e => - if (e.getValue.isPresent) { - authorizer.addAcl(e.getKey, e.getValue.get()) - } else { - authorizer.removeAcl(e.getKey) - }) + try { + // Because the changes map is a LinkedHashMap, the deltas will be returned in + // the order they were performed. + aclsDelta.changes().entrySet().forEach(e => + if (e.getValue.isPresent) { + authorizer.addAcl(e.getKey, e.getValue.get()) + } else { + authorizer.removeAcl(e.getKey) + }) + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error loading " + + s"authorizer changes in ${deltaName}", t) + } } case _ => // No ClusterMetadataAuthorizer is configured. There is nothing to do. }) @@ -258,8 +316,8 @@ class BrokerMetadataPublisher(conf: KafkaConfig, } publishedOffsetAtomic.set(newImage.highestOffsetAndEpoch().offset) } catch { - case t: Throwable => error(s"Error publishing broker metadata at $highestOffsetAndEpoch", t) - throw t + case t: Throwable => metadataPublishingFaultHandler.handleFault("Uncaught exception while " + + s"publishing broker metadata from ${deltaName}", t) } finally { _firstPublish = false } @@ -282,7 +340,7 @@ class BrokerMetadataPublisher(conf: KafkaConfig, * @param resignation function to call on resignation; the first parameter is the partition id; * the second parameter is the leader epoch */ - private def updateCoordinator( + def updateCoordinator( image: MetadataImage, delta: MetadataDelta, topicName: String, @@ -317,38 +375,60 @@ class BrokerMetadataPublisher(conf: KafkaConfig, } private def initializeManagers(): Unit = { - // Start log manager, which will perform (potentially lengthy) - // recovery-from-unclean-shutdown if required. - logManager.startup(metadataCache.getAllTopics()) - - // Make the LogCleaner available for reconfiguration. We can't do this prior to this - // point because LogManager#startup creates the LogCleaner object, if - // log.cleaner.enable is true. TODO: improve this (see KAFKA-13610) - Option(logManager.cleaner).foreach(conf.dynamicConfig.addBrokerReconfigurable) - - // Start the replica manager. - replicaManager.startup() - - // Start the group coordinator. - groupCoordinator.startup(() => metadataCache.numPartitions( - Topic.GROUP_METADATA_TOPIC_NAME).getOrElse(conf.offsetsTopicPartitions)) - - // Start the transaction coordinator. - txnCoordinator.startup(() => metadataCache.numPartitions( - Topic.TRANSACTION_STATE_TOPIC_NAME).getOrElse(conf.transactionTopicPartitions)) + try { + // Start log manager, which will perform (potentially lengthy) + // recovery-from-unclean-shutdown if required. + logManager.startup(metadataCache.getAllTopics()) + + // Make the LogCleaner available for reconfiguration. We can't do this prior to this + // point because LogManager#startup creates the LogCleaner object, if + // log.cleaner.enable is true. TODO: improve this (see KAFKA-13610) + Option(logManager.cleaner).foreach(conf.dynamicConfig.addBrokerReconfigurable) + } catch { + case t: Throwable => fatalFaultHandler.handleFault("Error starting LogManager", t) + } + try { + // Start the replica manager. + replicaManager.startup() + } catch { + case t: Throwable => fatalFaultHandler.handleFault("Error starting ReplicaManager", t) + } + try { + // Start the group coordinator. + groupCoordinator.startup(() => metadataCache.numPartitions( + Topic.GROUP_METADATA_TOPIC_NAME).getOrElse(conf.offsetsTopicPartitions)) + } catch { + case t: Throwable => fatalFaultHandler.handleFault("Error starting GroupCoordinator", t) + } + try { + // Start the transaction coordinator. + txnCoordinator.startup(() => metadataCache.numPartitions( + Topic.TRANSACTION_STATE_TOPIC_NAME).getOrElse(conf.transactionTopicPartitions)) + } catch { + case t: Throwable => fatalFaultHandler.handleFault("Error starting TransactionCoordinator", t) + } } private def finishInitializingReplicaManager(newImage: MetadataImage): Unit = { - // Delete log directories which we're not supposed to have, according to the - // latest metadata. This is only necessary to do when we're first starting up. If - // we have to load a snapshot later, these topics will appear in deletedTopicIds. - val strayPartitions = findStrayPartitions(brokerId, newImage.topics, logManager.allLogs) - if (strayPartitions.nonEmpty) { - replicaManager.deleteStrayReplicas(strayPartitions) + try { + // Delete log directories which we're not supposed to have, according to the + // latest metadata. This is only necessary to do when we're first starting up. If + // we have to load a snapshot later, these topics will appear in deletedTopicIds. + val strayPartitions = findStrayPartitions(brokerId, newImage.topics, logManager.allLogs) + if (strayPartitions.nonEmpty) { + replicaManager.deleteStrayReplicas(strayPartitions) + } + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error deleting stray " + + "partitions during startup", t) } - - // Make sure that the high water mark checkpoint thread is running for the replica - // manager. - replicaManager.startHighWatermarkCheckPointThread() - } + try { + // Make sure that the high water mark checkpoint thread is running for the replica + // manager. + replicaManager.startHighWatermarkCheckPointThread() + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error starting high " + + "watermark checkpoint thread during startup", t) + } +} } diff --git a/core/src/main/scala/kafka/server/metadata/BrokerServerMetrics.scala b/core/src/main/scala/kafka/server/metadata/BrokerServerMetrics.scala index 0db6f0071c486..3e68ae85f9232 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerServerMetrics.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerServerMetrics.scala @@ -28,6 +28,8 @@ final class BrokerServerMetrics private (metrics: Metrics) extends AutoCloseable val lastAppliedRecordOffset: AtomicLong = new AtomicLong(0) val lastAppliedRecordTimestamp: AtomicLong = new AtomicLong(0) + val metadataLoadErrorCount: AtomicLong = new AtomicLong(0) + val metadataApplyErrorCount: AtomicLong = new AtomicLong(0) val lastAppliedRecordOffsetName = metrics.metricName( "last-applied-record-offset", @@ -47,6 +49,18 @@ final class BrokerServerMetrics private (metrics: Metrics) extends AutoCloseable "The difference between now and the timestamp of the last record from the cluster metadata partition that was applied by the broker" ) + val metadataLoadErrorCountName = metrics.metricName( + "metadata-load-error-count", + metricGroupName, + "The number of errors encountered by the BrokerMetadataListener while loading the metadata log and generating a new MetadataDelta based on it." + ) + + val metadataApplyErrorCountName = metrics.metricName( + "metadata-apply-error-count", + metricGroupName, + "The number of errors encountered by the BrokerMetadataPublisher while applying a new MetadataImage based on the latest MetadataDelta." + ) + addMetric(metrics, lastAppliedRecordOffsetName) { _ => lastAppliedRecordOffset.get } @@ -59,11 +73,21 @@ final class BrokerServerMetrics private (metrics: Metrics) extends AutoCloseable now - lastAppliedRecordTimestamp.get } + addMetric(metrics, metadataLoadErrorCountName) { _ => + metadataLoadErrorCount.get + } + + addMetric(metrics, metadataApplyErrorCountName) { _ => + metadataApplyErrorCount.get + } + override def close(): Unit = { List( lastAppliedRecordOffsetName, lastAppliedRecordTimestampName, - lastAppliedRecordLagMsName + lastAppliedRecordLagMsName, + metadataLoadErrorCountName, + metadataApplyErrorCountName ).foreach(metrics.removeMetric) } } diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java index 42120324f5f83..ecee13c498241 100644 --- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java +++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java @@ -24,6 +24,7 @@ import kafka.server.KafkaConfig$; import kafka.server.KafkaRaftServer; import kafka.server.MetaProperties; +import kafka.server.metadata.BrokerServerMetrics$; import kafka.tools.StorageTool; import kafka.utils.Logging; import org.apache.kafka.clients.CommonClientConfigs; @@ -36,6 +37,7 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.controller.BootstrapMetadata; import org.apache.kafka.controller.Controller; +import org.apache.kafka.controller.MockControllerMetrics; import org.apache.kafka.metadata.MetadataRecordSerde; import org.apache.kafka.raft.RaftConfig; import org.apache.kafka.server.common.ApiMessageAndVersion; @@ -128,6 +130,11 @@ public Builder setConfigProp(String key, String value) { return this; } + public Builder setMetadataFaultHandler(MockFaultHandler metadataFaultHandler) { + this.metadataFaultHandler = metadataFaultHandler; + return this; + } + public KafkaClusterTestKit build() throws Exception { Map controllers = new HashMap<>(); Map brokers = new HashMap<>(); @@ -189,6 +196,7 @@ metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftS raftManager, Time.SYSTEM, new Metrics(), + new MockControllerMetrics(), Option.apply(threadNamePrefix), connectFutureManager.future, KafkaRaftServer.configSchema(), @@ -245,15 +253,20 @@ metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftS Time.SYSTEM, new Metrics(), Option.apply(threadNamePrefix), connectFutureManager.future); raftManagers.put(node.id(), raftManager); } + Metrics metrics = new Metrics(); BrokerServer broker = new BrokerServer( config, nodes.brokerProperties(node.id()), raftManager, Time.SYSTEM, - new Metrics(), + metrics, + BrokerServerMetrics$.MODULE$.apply(metrics), Option.apply(threadNamePrefix), JavaConverters.asScalaBuffer(Collections.emptyList()).toSeq(), - connectFutureManager.future + connectFutureManager.future, + fatalFaultHandler, + metadataFaultHandler, + metadataFaultHandler ); brokers.put(node.id(), broker); } diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala index 9894df9c5f747..c4ca966f9abcb 100755 --- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala +++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala @@ -24,6 +24,7 @@ import java.util.{Collections, Properties} import java.util.concurrent.CompletableFuture import javax.security.auth.login.Configuration import kafka.raft.KafkaRaftManager +import kafka.server.metadata.BrokerServerMetrics import kafka.tools.StorageTool import kafka.utils.{CoreUtils, Logging, TestInfoUtils, TestUtils} import kafka.zk.{AdminZkClient, EmbeddedZookeeper, KafkaZkClient} @@ -32,11 +33,12 @@ import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.security.JaasUtils import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.{Exit, Time} -import org.apache.kafka.controller.BootstrapMetadata +import org.apache.kafka.controller.{BootstrapMetadata, QuorumControllerMetrics} import org.apache.kafka.metadata.MetadataRecordSerde import org.apache.kafka.raft.RaftConfig.{AddressSpec, InetAddressSpec} import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} -import org.apache.kafka.server.fault.MockFaultHandler +import org.apache.kafka.server.fault.{FaultHandler, MockFaultHandler} +import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.zookeeper.client.ZKClientConfig import org.apache.zookeeper.{WatchedEvent, Watcher, ZooKeeper} import org.junit.jupiter.api.Assertions._ @@ -82,26 +84,34 @@ class ZooKeeperQuorumImplementation( } } -class KRaftQuorumImplementation(val raftManager: KafkaRaftManager[ApiMessageAndVersion], - val controllerServer: ControllerServer, - val metadataDir: File, - val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]], - val clusterId: String, - val log: Logging) extends QuorumImplementation { +class KRaftQuorumImplementation( + val raftManager: KafkaRaftManager[ApiMessageAndVersion], + val controllerServer: ControllerServer, + val metadataDir: File, + val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]], + val clusterId: String, + val log: Logging, + val faultHandler: FaultHandler +) extends QuorumImplementation { override def createBroker( config: KafkaConfig, time: Time, startup: Boolean, threadNamePrefix: Option[String], ): KafkaBroker = { + val metrics = new Metrics() val broker = new BrokerServer(config = config, metaProps = new MetaProperties(clusterId, config.nodeId), raftManager = raftManager, time = time, - metrics = new Metrics(), + metrics = metrics, + brokerMetrics = BrokerServerMetrics(metrics), threadNamePrefix = Some("Broker%02d_".format(config.nodeId)), initialOfflineDirs = Seq(), - controllerQuorumVotersFuture = controllerQuorumVotersFuture) + controllerQuorumVotersFuture = controllerQuorumVotersFuture, + fatalFaultHandler = faultHandler, + metadataLoadingFaultHandler = faultHandler, + metadataPublishingFaultHandler = faultHandler) if (startup) broker.startup() broker } @@ -306,6 +316,7 @@ abstract class QuorumTestHarness extends Logging { raftManager = raftManager, time = Time.SYSTEM, metrics = controllerMetrics, + controllerMetrics = new QuorumControllerMetrics(KafkaYammerMetrics.defaultRegistry(), Time.SYSTEM), threadNamePrefix = Option(threadNamePrefix), controllerQuorumVotersFuture = controllerQuorumVotersFuture, configSchema = KafkaRaftServer.configSchema, @@ -336,7 +347,8 @@ abstract class QuorumTestHarness extends Logging { metadataDir, controllerQuorumVotersFuture, metaProperties.clusterId, - this) + this, + faultHandler) } private def newZooKeeperQuorum(): ZooKeeperQuorumImplementation = { diff --git a/core/src/test/scala/kafka/server/metadata/BrokerServerMetricsTest.scala b/core/src/test/scala/kafka/server/metadata/BrokerServerMetricsTest.scala index df114ef59e58a..ea2b439c166ae 100644 --- a/core/src/test/scala/kafka/server/metadata/BrokerServerMetricsTest.scala +++ b/core/src/test/scala/kafka/server/metadata/BrokerServerMetricsTest.scala @@ -37,12 +37,14 @@ final class BrokerServerMetricsTest { val expectedMetrics = Set( new MetricName("last-applied-record-offset", expectedGroup, "", Collections.emptyMap()), new MetricName("last-applied-record-timestamp", expectedGroup, "", Collections.emptyMap()), - new MetricName("last-applied-record-lag-ms", expectedGroup, "", Collections.emptyMap()) + new MetricName("last-applied-record-lag-ms", expectedGroup, "", Collections.emptyMap()), + new MetricName("metadata-load-error-count", expectedGroup, "", Collections.emptyMap()), + new MetricName("metadata-apply-error-count", expectedGroup, "", Collections.emptyMap()) ) TestUtils.resource(BrokerServerMetrics(metrics)) { brokerMetrics => val metricsMap = metrics.metrics().asScala.filter{ case (name, _) => name.group == expectedGroup } - assertEquals(3, metricsMap.size) + assertEquals(expectedMetrics.size, metricsMap.size) metricsMap.foreach { case (name, metric) => assertTrue(expectedMetrics.contains(name)) } @@ -85,4 +87,36 @@ final class BrokerServerMetricsTest { assertEquals(time.milliseconds - timestamp, lagMetric.metricValue.asInstanceOf[Long]) } } + + @Test + def testMetadataLoadErrorCount(): Unit = { + val time = new MockTime() + val metrics = new Metrics(time) + TestUtils.resource(BrokerServerMetrics(metrics)) { brokerMetrics => + val metadataLoadErrorCountMetric = metrics.metrics().get(brokerMetrics.metadataLoadErrorCountName) + + assertEquals(0L, metadataLoadErrorCountMetric.metricValue.asInstanceOf[Long]) + + // Update metric value and check + val errorCount = 100 + brokerMetrics.metadataLoadErrorCount.set(errorCount) + assertEquals(errorCount, metadataLoadErrorCountMetric.metricValue.asInstanceOf[Long]) + } + } + + @Test + def testMetadataApplyErrorCount(): Unit = { + val time = new MockTime() + val metrics = new Metrics(time) + TestUtils.resource(BrokerServerMetrics(metrics)) { brokerMetrics => + val metadataApplyErrorCountMetric = metrics.metrics().get(brokerMetrics.metadataApplyErrorCountName) + + assertEquals(0L, metadataApplyErrorCountMetric.metricValue.asInstanceOf[Long]) + + // Update metric value and check + val errorCount = 100 + brokerMetrics.metadataApplyErrorCount.set(errorCount) + assertEquals(errorCount, metadataApplyErrorCountMetric.metricValue.asInstanceOf[Long]) + } + } } diff --git a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala index b21fe877f2082..29de3c0f24288 100644 --- a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala +++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala @@ -233,16 +233,21 @@ class MetricsTest extends KafkaServerTestHarness with Logging { @ValueSource(strings = Array("kraft")) def testKRaftControllerMetrics(quorum: String): Unit = { val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics - - assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=ActiveControllerCount"), 1) - assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=OfflinePartitionsCount"), 1) - assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=PreferredReplicaImbalanceCount"), 1) - assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=GlobalTopicCount"), 1) - assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=GlobalPartitionCount"), 1) - assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=LastCommittedRecordOffset"), 1) - assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=LastAppliedRecordOffset"), 1) - assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=LastAppliedRecordTimestamp"), 1) - assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=LastAppliedRecordLagMs"), 1) + Set( + "kafka.controller:type=KafkaController,name=ActiveControllerCount", + "kafka.controller:type=KafkaController,name=GlobalPartitionCount", + "kafka.controller:type=KafkaController,name=GlobalTopicCount", + "kafka.controller:type=KafkaController,name=LastAppliedRecordLagMs", + "kafka.controller:type=KafkaController,name=LastAppliedRecordOffset", + "kafka.controller:type=KafkaController,name=LastAppliedRecordTimestamp", + "kafka.controller:type=KafkaController,name=LastCommittedRecordOffset", + "kafka.controller:type=KafkaController,name=MetadataErrorCount", + "kafka.controller:type=KafkaController,name=OfflinePartitionsCount", + "kafka.controller:type=KafkaController,name=PreferredReplicaImbalanceCount", + ).foreach(expected => { + assertEquals(1, metrics.keySet.asScala.count(_.getMBeanName.equals(expected)), + s"Unable to find ${expected}") + }) } /** diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala index 6de448f280294..6c8c2599d296a 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala @@ -27,12 +27,20 @@ import org.apache.kafka.common.{Endpoint, Uuid} import org.apache.kafka.image.{MetadataDelta, MetadataImage} import org.apache.kafka.metadata.{BrokerRegistration, RecordTestUtils, VersionRange} import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} +import org.apache.kafka.server.fault.MockFaultHandler import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} -import org.junit.jupiter.api.Test +import org.junit.jupiter.api.{AfterEach, Test} import scala.jdk.CollectionConverters._ class BrokerMetadataListenerTest { + private val metadataLoadingFaultHandler = new MockFaultHandler("metadata loading") + + @AfterEach + def verifyNoFaults(): Unit = { + metadataLoadingFaultHandler.maybeRethrowFirstException() + } + private def newBrokerMetadataListener( metrics: BrokerServerMetrics = BrokerServerMetrics(new Metrics()), snapshotter: Option[MetadataSnapshotter] = None, @@ -44,7 +52,8 @@ class BrokerMetadataListenerTest { threadNamePrefix = None, maxBytesBetweenSnapshots = maxBytesBetweenSnapshots, snapshotter = snapshotter, - brokerMetrics = metrics + brokerMetrics = metrics, + metadataLoadingFaultHandler = metadataLoadingFaultHandler ) } @@ -77,6 +86,8 @@ class BrokerMetadataListenerTest { assertEquals(100L, listener.highestMetadataOffset) assertEquals(0L, metrics.lastAppliedRecordOffset.get) assertEquals(0L, metrics.lastAppliedRecordTimestamp.get) + assertEquals(0L, metrics.metadataLoadErrorCount.get) + assertEquals(0L, metrics.metadataApplyErrorCount.get) val fencedTimestamp = 500L val fencedLastOffset = 200L @@ -110,6 +121,8 @@ class BrokerMetadataListenerTest { assertEquals(fencedLastOffset, metrics.lastAppliedRecordOffset.get) assertEquals(fencedTimestamp, metrics.lastAppliedRecordTimestamp.get) + assertEquals(0L, metrics.metadataLoadErrorCount.get) + assertEquals(0L, metrics.metadataApplyErrorCount.get) } finally { listener.close() } diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala index 6742530ef5144..652b8b3a0c29f 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala @@ -17,17 +17,16 @@ package unit.kafka.server.metadata -import java.util.Collections.{singleton, singletonMap} +import java.util.Collections.{singleton, singletonList, singletonMap} import java.util.Properties import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} - import kafka.log.UnifiedLog -import kafka.server.KafkaConfig +import kafka.server.{BrokerServer, KafkaConfig} import kafka.server.metadata.BrokerMetadataPublisher import kafka.testkit.{KafkaClusterTestKit, TestKitNodes} import kafka.utils.TestUtils import org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET -import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, ConfigEntry} +import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, ConfigEntry, NewTopic} import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.config.ConfigResource.Type.BROKER import org.apache.kafka.common.utils.Exit @@ -35,10 +34,12 @@ import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.image.{MetadataImageTest, TopicImage, TopicsImage} import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.PartitionRegistration -import org.junit.jupiter.api.Assertions.assertEquals +import org.apache.kafka.server.fault.{FaultHandler, MockFaultHandler} +import org.junit.jupiter.api.Assertions.{assertEquals, assertNotNull, assertTrue} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.ArgumentMatchers.any import org.mockito.Mockito +import org.mockito.Mockito.doThrow import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer @@ -176,6 +177,25 @@ class BrokerMetadataPublisherTest { new TopicsImage(idsMap.asJava, namesMap.asJava) } + private def newMockPublisher( + broker: BrokerServer, + errorHandler: FaultHandler = new MockFaultHandler("publisher") + ): BrokerMetadataPublisher = { + Mockito.spy(new BrokerMetadataPublisher( + conf = broker.config, + metadataCache = broker.metadataCache, + logManager = broker.logManager, + replicaManager = broker.replicaManager, + groupCoordinator = broker.groupCoordinator, + txnCoordinator = broker.transactionCoordinator, + clientQuotaMetadataManager = broker.clientQuotaMetadataManager, + dynamicConfigHandlers = broker.dynamicConfigHandlers.toMap, + _authorizer = Option.empty, + errorHandler, + errorHandler + )) + } + @Test def testReloadUpdatedFilesWithoutConfigChange(): Unit = { val cluster = new KafkaClusterTestKit.Builder( @@ -187,17 +207,7 @@ class BrokerMetadataPublisherTest { cluster.startup() cluster.waitForReadyBrokers() val broker = cluster.brokers().values().iterator().next() - val publisher = Mockito.spy(new BrokerMetadataPublisher( - conf = broker.config, - metadataCache = broker.metadataCache, - logManager = broker.logManager, - replicaManager = broker.replicaManager, - groupCoordinator = broker.groupCoordinator, - txnCoordinator = broker.transactionCoordinator, - clientQuotaMetadataManager = broker.clientQuotaMetadataManager, - dynamicConfigHandlers = broker.dynamicConfigHandlers.toMap, - _authorizer = Option.empty - )) + val publisher = newMockPublisher(broker) val numTimesReloadCalled = new AtomicInteger(0) Mockito.when(publisher.reloadUpdatedFilesWithoutConfigChange(any[Properties]())). thenAnswer(new Answer[Unit]() { @@ -227,4 +237,39 @@ class BrokerMetadataPublisherTest { cluster.close() } } + + @Test + def testExceptionInUpdateCoordinator(): Unit = { + val errorHandler = new MockFaultHandler("publisher") + val cluster = new KafkaClusterTestKit.Builder( + new TestKitNodes.Builder(). + setNumBrokerNodes(1). + setNumControllerNodes(1).build()). + setMetadataFaultHandler(errorHandler).build() + try { + cluster.format() + cluster.startup() + cluster.waitForReadyBrokers() + val broker = cluster.brokers().values().iterator().next() + TestUtils.retry(60000) { + assertNotNull(broker.metadataPublisher) + } + val publisher = Mockito.spy(broker.metadataPublisher) + doThrow(new RuntimeException("injected failure")).when(publisher).updateCoordinator(any(), any(), any(), any(), any()) + broker.metadataListener.alterPublisher(publisher).get() + val admin = Admin.create(cluster.clientProperties()) + try { + admin.createTopics(singletonList(new NewTopic("foo", 1, 1.toShort))).all().get() + } finally { + admin.close() + } + TestUtils.retry(60000) { + assertTrue(Option(errorHandler.firstException()). + flatMap(e => Option(e.getMessage())).getOrElse("(none)").contains("injected failure")) + } + } finally { + errorHandler.setIgnore(true) + cluster.close() + } + } } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ControllerMetrics.java b/metadata/src/main/java/org/apache/kafka/controller/ControllerMetrics.java index 6b470664d6e9a..ff243aebfcb69 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ControllerMetrics.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ControllerMetrics.java @@ -51,6 +51,10 @@ public interface ControllerMetrics extends AutoCloseable { int preferredReplicaImbalanceCount(); + void incrementMetadataErrorCount(); + + int metadataErrorCount(); + void setLastAppliedRecordOffset(long offset); long lastAppliedRecordOffset(); diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index a4cc1d92cba4b..ef87248f13420 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -765,7 +765,7 @@ public void run() throws Exception { "%d of %d record(s) in the batch following last writeOffset %d.", message.message().getClass().getSimpleName(), i, result.records().size(), writeOffset); - fatalFaultHandler.handleFault(failureMessage, e); + throw fatalFaultHandler.handleFault(failureMessage, e); } i++; } @@ -889,7 +889,7 @@ public void handleCommit(BatchReader reader) { "controller, which was %d of %d record(s) in the batch with baseOffset %d.", message.message().getClass().getSimpleName(), i, messages.size(), batch.baseOffset()); - metadataFaultHandler.handleFault(failureMessage, e); + throw metadataFaultHandler.handleFault(failureMessage, e); } i++; } @@ -910,7 +910,7 @@ public void handleSnapshot(SnapshotReader reader) { appendRaftEvent(String.format("handleSnapshot[snapshotId=%s]", reader.snapshotId()), () -> { try { if (isActiveController()) { - fatalFaultHandler.handleFault(String.format("Asked to load snapshot " + + throw fatalFaultHandler.handleFault(String.format("Asked to load snapshot " + "(%s) when it is the active controller (%d)", reader.snapshotId(), curClaimEpoch)); } @@ -945,7 +945,7 @@ public void handleSnapshot(SnapshotReader reader) { "%d record(s) in the batch with baseOffset %d.", message.message().getClass().getSimpleName(), reader.snapshotId(), i, messages.size(), batch.baseOffset()); - metadataFaultHandler.handleFault(failureMessage, e); + throw metadataFaultHandler.handleFault(failureMessage, e); } i++; } diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumControllerMetrics.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumControllerMetrics.java index 5abf0d9770694..b96a687b0f34e 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumControllerMetrics.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumControllerMetrics.java @@ -26,6 +26,7 @@ import java.util.Arrays; import java.util.Objects; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; public final class QuorumControllerMetrics implements ControllerMetrics { @@ -47,6 +48,8 @@ public final class QuorumControllerMetrics implements ControllerMetrics { "KafkaController", "OfflinePartitionsCount"); private final static MetricName PREFERRED_REPLICA_IMBALANCE_COUNT = getMetricName( "KafkaController", "PreferredReplicaImbalanceCount"); + private final static MetricName METADATA_ERROR_COUNT = getMetricName( + "KafkaController", "MetadataErrorCount"); private final static MetricName LAST_APPLIED_RECORD_OFFSET = getMetricName( "KafkaController", "LastAppliedRecordOffset"); private final static MetricName LAST_COMMITTED_RECORD_OFFSET = getMetricName( @@ -64,6 +67,7 @@ public final class QuorumControllerMetrics implements ControllerMetrics { private volatile int globalPartitionCount; private volatile int offlinePartitionCount; private volatile int preferredReplicaImbalanceCount; + private volatile AtomicInteger metadataErrorCount; private final AtomicLong lastAppliedRecordOffset = new AtomicLong(0); private final AtomicLong lastCommittedRecordOffset = new AtomicLong(0); private final AtomicLong lastAppliedRecordTimestamp = new AtomicLong(0); @@ -74,6 +78,7 @@ public final class QuorumControllerMetrics implements ControllerMetrics { private final Gauge globalTopicCountGauge; private final Gauge offlinePartitionCountGauge; private final Gauge preferredReplicaImbalanceCountGauge; + private final Gauge metadataErrorCountGauge; private final Gauge lastAppliedRecordOffsetGauge; private final Gauge lastCommittedRecordOffsetGauge; private final Gauge lastAppliedRecordTimestampGauge; @@ -93,6 +98,7 @@ public QuorumControllerMetrics( this.globalPartitionCount = 0; this.offlinePartitionCount = 0; this.preferredReplicaImbalanceCount = 0; + this.metadataErrorCount = new AtomicInteger(0); this.activeControllerCount = registry.newGauge(ACTIVE_CONTROLLER_COUNT, new Gauge() { @Override public Integer value() { @@ -137,6 +143,12 @@ public Integer value() { return preferredReplicaImbalanceCount; } }); + this.metadataErrorCountGauge = registry.newGauge(METADATA_ERROR_COUNT, new Gauge() { + @Override + public Integer value() { + return metadataErrorCount.get(); + } + }); lastAppliedRecordOffsetGauge = registry.newGauge(LAST_APPLIED_RECORD_OFFSET, new Gauge() { @Override public Long value() { @@ -242,6 +254,15 @@ public int preferredReplicaImbalanceCount() { return this.preferredReplicaImbalanceCount; } + @Override + public void incrementMetadataErrorCount() { + this.metadataErrorCount.getAndIncrement(); + } + + @Override + public int metadataErrorCount() { + return this.metadataErrorCount.get(); + } @Override public void setLastAppliedRecordOffset(long offset) { lastAppliedRecordOffset.set(offset); @@ -276,12 +297,15 @@ public long lastAppliedRecordTimestamp() { public void close() { Arrays.asList( ACTIVE_CONTROLLER_COUNT, + FENCED_BROKER_COUNT, + ACTIVE_BROKER_COUNT, EVENT_QUEUE_TIME_MS, EVENT_QUEUE_PROCESSING_TIME_MS, GLOBAL_TOPIC_COUNT, GLOBAL_PARTITION_COUNT, OFFLINE_PARTITION_COUNT, PREFERRED_REPLICA_IMBALANCE_COUNT, + METADATA_ERROR_COUNT, LAST_APPLIED_RECORD_OFFSET, LAST_COMMITTED_RECORD_OFFSET, LAST_APPLIED_RECORD_TIMESTAMP, diff --git a/metadata/src/main/java/org/apache/kafka/metadata/fault/MetadataFaultException.java b/metadata/src/main/java/org/apache/kafka/metadata/fault/MetadataFaultException.java deleted file mode 100644 index c57ce46fb359f..0000000000000 --- a/metadata/src/main/java/org/apache/kafka/metadata/fault/MetadataFaultException.java +++ /dev/null @@ -1,32 +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.metadata.fault; - - -/** - * A fault that we encountered while we replayed cluster metadata. - */ -public class MetadataFaultException extends RuntimeException { - public MetadataFaultException(String message, Throwable cause) { - super(message, cause); - } - - public MetadataFaultException(String message) { - super(message); - } -} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/fault/MetadataFaultHandler.java b/metadata/src/main/java/org/apache/kafka/metadata/fault/MetadataFaultHandler.java deleted file mode 100644 index e9f71b80e675e..0000000000000 --- a/metadata/src/main/java/org/apache/kafka/metadata/fault/MetadataFaultHandler.java +++ /dev/null @@ -1,36 +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.metadata.fault; - -import org.apache.kafka.server.fault.FaultHandler; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - -/** - * Handles faults in Kafka metadata management. - */ -public class MetadataFaultHandler implements FaultHandler { - private static final Logger log = LoggerFactory.getLogger(MetadataFaultHandler.class); - - @Override - public void handleFault(String failureMessage, Throwable cause) { - FaultHandler.logFailureMessage(log, failureMessage, cause); - throw new MetadataFaultException("Encountered metadata fault: " + failureMessage, cause); - } -} diff --git a/metadata/src/test/java/org/apache/kafka/controller/MockControllerMetrics.java b/metadata/src/test/java/org/apache/kafka/controller/MockControllerMetrics.java index 5991fcc34f31d..ca13d90ddeae1 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/MockControllerMetrics.java +++ b/metadata/src/test/java/org/apache/kafka/controller/MockControllerMetrics.java @@ -17,6 +17,8 @@ package org.apache.kafka.controller; +import java.util.concurrent.atomic.AtomicInteger; + public final class MockControllerMetrics implements ControllerMetrics { private volatile boolean active = false; private volatile int fencedBrokers = 0; @@ -25,6 +27,7 @@ public final class MockControllerMetrics implements ControllerMetrics { private volatile int partitions = 0; private volatile int offlinePartitions = 0; private volatile int preferredReplicaImbalances = 0; + private volatile AtomicInteger metadataErrors = new AtomicInteger(0); private volatile long lastAppliedRecordOffset = 0; private volatile long lastCommittedRecordOffset = 0; private volatile long lastAppliedRecordTimestamp = 0; @@ -111,6 +114,16 @@ public int preferredReplicaImbalanceCount() { return this.preferredReplicaImbalances; } + @Override + public void incrementMetadataErrorCount() { + this.metadataErrors.getAndIncrement(); + } + + @Override + public int metadataErrorCount() { + return this.metadataErrors.get(); + } + @Override public void setLastAppliedRecordOffset(long offset) { lastAppliedRecordOffset = offset; diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerMetricsTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerMetricsTest.java index 2ab999559435c..400b860197e53 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerMetricsTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerMetricsTest.java @@ -36,10 +36,13 @@ public void testKafkaControllerMetricNames() { String expectedType = "KafkaController"; Set expectedMetricNames = Utils.mkSet( "ActiveControllerCount", + "FencedBrokerCount", + "ActiveBrokerCount", "GlobalTopicCount", "GlobalPartitionCount", "OfflinePartitionsCount", "PreferredReplicaImbalanceCount", + "MetadataErrorCount", "LastAppliedRecordLagMs", "LastAppliedRecordOffset", "LastAppliedRecordTimestamp", @@ -125,6 +128,25 @@ public void testLastAppliedRecordMetrics() { } } + @Test + public void testMetadataErrorCount() { + MetricsRegistry registry = new MetricsRegistry(); + MockTime time = new MockTime(); + try { + try (QuorumControllerMetrics quorumControllerMetrics = new QuorumControllerMetrics(registry, time)) { + @SuppressWarnings("unchecked") + Gauge metadataErrorCount = (Gauge) registry + .allMetrics() + .get(metricName("KafkaController", "MetadataErrorCount")); + assertEquals(0, metadataErrorCount.value()); + quorumControllerMetrics.incrementMetadataErrorCount(); + assertEquals(1, metadataErrorCount.value()); + } + } finally { + registry.shutdown(); + } + } + private static void assertMetricsCreatedAndRemovedUponClose(String expectedType, Set expectedMetricNames) { MetricsRegistry registry = new MetricsRegistry(); MockTime time = new MockTime(); @@ -151,10 +173,18 @@ private static MetricName metricName(String type, String name) { } private static void assertMetricsCreated(MetricsRegistry registry, Set expectedMetricNames, String expectedType) { + assertEquals(registry.allMetrics().keySet().stream() + .filter(k -> k.getType() == expectedType).count(), + expectedMetricNames.size()); expectedMetricNames.forEach(expectedName -> { MetricName expectMetricName = metricName(expectedType, expectedName); assertTrue(registry.allMetrics().containsKey(expectMetricName), "Missing metric: " + expectMetricName); }); + registry.allMetrics().forEach((actualMetricName, actualMetric) -> { + if (actualMetricName.getType() == expectedType) { + assertTrue(expectedMetricNames.contains(actualMetricName.getName()), "Unexpected metric: " + actualMetricName); + } + }); } private static void assertMetricsRemoved(MetricsRegistry registry, Set expectedMetricNames, String expectedType) { diff --git a/server-common/src/main/java/org/apache/kafka/server/fault/FaultHandler.java b/server-common/src/main/java/org/apache/kafka/server/fault/FaultHandler.java index 4c03eacc32f3e..5efc145ea9443 100644 --- a/server-common/src/main/java/org/apache/kafka/server/fault/FaultHandler.java +++ b/server-common/src/main/java/org/apache/kafka/server/fault/FaultHandler.java @@ -17,8 +17,6 @@ package org.apache.kafka.server.fault; -import org.slf4j.Logger; - /** * Handle a server fault. @@ -28,9 +26,11 @@ public interface FaultHandler { * Handle a fault. * * @param failureMessage The failure message to log. + * + * @return The fault exception. */ - default void handleFault(String failureMessage) { - handleFault(failureMessage, null); + default RuntimeException handleFault(String failureMessage) { + return handleFault(failureMessage, null); } /** @@ -38,21 +38,8 @@ default void handleFault(String failureMessage) { * * @param failureMessage The failure message to log. * @param cause The exception that caused the problem, or null. - */ - void handleFault(String failureMessage, Throwable cause); - - /** - * Log a failure message about a fault. * - * @param log The log4j logger. - * @param failureMessage The failure message. - * @param cause The exception which caused the failure, or null. + * @return The fault exception. */ - static void logFailureMessage(Logger log, String failureMessage, Throwable cause) { - if (cause == null) { - log.error("Encountered fatal fault: {}", failureMessage); - } else { - log.error("Encountered fatal fault: {}", failureMessage, cause); - } - } + RuntimeException handleFault(String failureMessage, Throwable cause); } diff --git a/server-common/src/test/java/org/apache/kafka/server/fault/MockFaultHandlerException.java b/server-common/src/main/java/org/apache/kafka/server/fault/FaultHandlerException.java similarity index 83% rename from server-common/src/test/java/org/apache/kafka/server/fault/MockFaultHandlerException.java rename to server-common/src/main/java/org/apache/kafka/server/fault/FaultHandlerException.java index ef9b11bdeb53e..ec3b7dc4b0c19 100644 --- a/server-common/src/test/java/org/apache/kafka/server/fault/MockFaultHandlerException.java +++ b/server-common/src/main/java/org/apache/kafka/server/fault/FaultHandlerException.java @@ -19,10 +19,10 @@ /** - * An exception thrown by MockFaultHandler. + * An exception thrown by a fault handler. */ -public class MockFaultHandlerException extends RuntimeException { - public MockFaultHandlerException(String failureMessage, Throwable cause) { +public class FaultHandlerException extends RuntimeException { + public FaultHandlerException(String failureMessage, Throwable cause) { super(failureMessage, cause); // If a cause exception was provided, set our the stack trace its stack trace. This is // useful in junit tests where a limited number of stack frames are printed, and usually @@ -32,7 +32,7 @@ public MockFaultHandlerException(String failureMessage, Throwable cause) { } } - public MockFaultHandlerException(String failureMessage) { + public FaultHandlerException(String failureMessage) { this(failureMessage, null); } } diff --git a/server-common/src/main/java/org/apache/kafka/server/fault/LoggingFaultHandler.java b/server-common/src/main/java/org/apache/kafka/server/fault/LoggingFaultHandler.java new file mode 100644 index 0000000000000..9242cef4eb9dc --- /dev/null +++ b/server-common/src/main/java/org/apache/kafka/server/fault/LoggingFaultHandler.java @@ -0,0 +1,54 @@ +/* + * 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.server.fault; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * A fault handler which logs an error message and executes a runnable. + */ +public class LoggingFaultHandler implements FaultHandler { + private static final Logger log = LoggerFactory.getLogger(LoggingFaultHandler.class); + private final String type; + private final Runnable action; + + public LoggingFaultHandler( + String type, + Runnable action + ) { + this.type = type; + this.action = action; + } + + @Override + public RuntimeException handleFault(String failureMessage, Throwable cause) { + if (cause == null) { + log.error("Encountered {} fault: {}", type, failureMessage); + } else { + log.error("Encountered {} fault: {}", type, failureMessage, cause); + } + try { + action.run(); + } catch (Throwable e) { + log.error("Failed to run LoggingFaultHandler action.", e); + } + return new FaultHandlerException(failureMessage, cause); + } +} diff --git a/server-common/src/main/java/org/apache/kafka/server/fault/ProcessExitingFaultHandler.java b/server-common/src/main/java/org/apache/kafka/server/fault/ProcessExitingFaultHandler.java index e3b9f25a3bea8..b7c0d241a2ad3 100644 --- a/server-common/src/main/java/org/apache/kafka/server/fault/ProcessExitingFaultHandler.java +++ b/server-common/src/main/java/org/apache/kafka/server/fault/ProcessExitingFaultHandler.java @@ -30,8 +30,13 @@ public class ProcessExitingFaultHandler implements FaultHandler { private static final Logger log = LoggerFactory.getLogger(ProcessExitingFaultHandler.class); @Override - public void handleFault(String failureMessage, Throwable cause) { - FaultHandler.logFailureMessage(log, failureMessage, cause); + public RuntimeException handleFault(String failureMessage, Throwable cause) { + if (cause == null) { + log.error("Encountered fatal fault: {}", failureMessage); + } else { + log.error("Encountered fatal fault: {}", failureMessage, cause); + } Exit.exit(1); + return null; } } diff --git a/server-common/src/test/java/org/apache/kafka/server/fault/LoggingFaultHandlerTest.java b/server-common/src/test/java/org/apache/kafka/server/fault/LoggingFaultHandlerTest.java new file mode 100644 index 0000000000000..1a11098a21b47 --- /dev/null +++ b/server-common/src/test/java/org/apache/kafka/server/fault/LoggingFaultHandlerTest.java @@ -0,0 +1,57 @@ +/* + * 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.server.fault; + +import org.junit.jupiter.api.Test; + +import java.util.concurrent.atomic.AtomicInteger; + +import static org.junit.jupiter.api.Assertions.assertEquals; + + +/** + * Tests LoggingFaultHandler + */ +public class LoggingFaultHandlerTest { + /** + * Test handling faults with and without exceptions. + */ + @Test + public void testHandleFault() { + AtomicInteger counter = new AtomicInteger(0); + LoggingFaultHandler handler = new LoggingFaultHandler("test", () -> { + counter.incrementAndGet(); + }); + handler.handleFault("uh oh"); + assertEquals(1, counter.get()); + handler.handleFault("uh oh", new RuntimeException("yikes")); + assertEquals(2, counter.get()); + } + + /** + * Test handling an exception in the action callback. + */ + @Test + public void testHandleExceptionInAction() { + LoggingFaultHandler handler = new LoggingFaultHandler("test", () -> { + throw new RuntimeException("action failed"); + }); + handler.handleFault("uh oh"); // should not throw + handler.handleFault("uh oh", new RuntimeException("yikes")); // should not throw + } +} diff --git a/server-common/src/test/java/org/apache/kafka/server/fault/MockFaultHandler.java b/server-common/src/test/java/org/apache/kafka/server/fault/MockFaultHandler.java index 39b3ed078471d..e49f2bdc6c25b 100644 --- a/server-common/src/test/java/org/apache/kafka/server/fault/MockFaultHandler.java +++ b/server-common/src/test/java/org/apache/kafka/server/fault/MockFaultHandler.java @@ -29,7 +29,7 @@ public class MockFaultHandler implements FaultHandler { private static final Logger log = LoggerFactory.getLogger(MockFaultHandler.class); private final String name; - private MockFaultHandlerException firstException = null; + private FaultHandlerException firstException = null; private boolean ignore = false; public MockFaultHandler(String name) { @@ -37,16 +37,20 @@ public MockFaultHandler(String name) { } @Override - public synchronized void handleFault(String failureMessage, Throwable cause) { - FaultHandler.logFailureMessage(log, failureMessage, cause); - MockFaultHandlerException e = (cause == null) ? - new MockFaultHandlerException(name + ": " + failureMessage) : - new MockFaultHandlerException(name + ": " + failureMessage + + public synchronized RuntimeException handleFault(String failureMessage, Throwable cause) { + if (cause == null) { + log.error("Encountered {} fault: {}", name, failureMessage); + } else { + log.error("Encountered {} fault: {}", name, failureMessage, cause); + } + FaultHandlerException e = (cause == null) ? + new FaultHandlerException(name + ": " + failureMessage) : + new FaultHandlerException(name + ": " + failureMessage + ": " + cause.getMessage(), cause); if (firstException == null) { firstException = e; } - throw e; + return firstException; } public synchronized void maybeRethrowFirstException() { @@ -55,7 +59,7 @@ public synchronized void maybeRethrowFirstException() { } } - public synchronized MockFaultHandlerException firstException() { + public synchronized FaultHandlerException firstException() { return firstException; } From af40e11978495d043957efb6fa849a664cb8fe26 Mon Sep 17 00:00:00 2001 From: Justine Olshan Date: Wed, 10 Aug 2022 01:25:35 -0700 Subject: [PATCH 037/175] KAFKA-14140: Ensure an offline or in-controlled-shutdown replica is not eligible to join ISR in ZK mode (#12487) This patch prevents offline or in-controller-shutdown replicas from being added back to the ISR and therefore to become leaders in ZK mode. This is an extra line of defense to ensure that it never happens. This is a continuation of the work done in KIP-841. Reviewers: David Mao , Jason Gustafson , Jun Rao , David Jacot --- .../main/scala/kafka/cluster/Partition.scala | 9 +- .../kafka/controller/KafkaController.scala | 18 ++- .../unit/kafka/cluster/PartitionTest.scala | 37 ++++-- .../ControllerIntegrationTest.scala | 113 +++++++++++++----- 4 files changed, 137 insertions(+), 40 deletions(-) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 538c51f9035c7..1eab4c4669a01 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -26,7 +26,7 @@ import kafka.log._ import kafka.metrics.KafkaMetricsGroup import kafka.server._ import kafka.server.checkpoints.OffsetCheckpoints -import kafka.server.metadata.KRaftMetadataCache +import kafka.server.metadata.{KRaftMetadataCache, ZkMetadataCache} import kafka.utils.CoreUtils.{inReadLock, inWriteLock} import kafka.utils._ import kafka.zookeeper.ZooKeeperClientException @@ -881,11 +881,16 @@ class Partition(val topicPartition: TopicPartition, private def isReplicaIsrEligible(followerReplicaId: Int): Boolean = { metadataCache match { // In KRaft mode, only replicas which are not fenced nor in controlled shutdown are - // allowed to join the ISR. This does not apply to ZK mode. + // allowed to join the ISR. case kRaftMetadataCache: KRaftMetadataCache => !kRaftMetadataCache.isBrokerFenced(followerReplicaId) && !kRaftMetadataCache.isBrokerShuttingDown(followerReplicaId) + // In ZK mode, we just ensure the broker is alive. Although we do not check for shutting down brokers here, + // the controller will block them from being added to ISR. + case zkMetadataCache: ZkMetadataCache => + zkMetadataCache.hasAliveBroker(followerReplicaId) + case _ => true } } diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index d179bcd6ca475..0154d9cbe54ab 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -2364,7 +2364,23 @@ class KafkaController(val config: KafkaConfig, ) None } else { - Some(tp -> newLeaderAndIsr) + // Pull out replicas being added to ISR and verify they are all online. + // If a replica is not online, reject the update as specified in KIP-841. + val ineligibleReplicas = newLeaderAndIsr.isr.toSet -- controllerContext.liveBrokerIds + if (ineligibleReplicas.nonEmpty) { + info(s"Rejecting AlterPartition request from node $brokerId for $tp because " + + s"it specified ineligible replicas $ineligibleReplicas in the new ISR ${newLeaderAndIsr.isr}." + ) + + if (alterPartitionRequestVersion > 1) { + partitionResponses(tp) = Left(Errors.INELIGIBLE_REPLICA) + } else { + partitionResponses(tp) = Left(Errors.OPERATION_NOT_ATTEMPTED) + } + None + } else { + Some(tp -> newLeaderAndIsr) + } } case None => diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index 5038219579440..948abc6c3b87f 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -46,7 +46,7 @@ import java.nio.ByteBuffer import java.util.Optional import java.util.concurrent.{CountDownLatch, Semaphore} import kafka.server.epoch.LeaderEpochFileCache -import kafka.server.metadata.KRaftMetadataCache +import kafka.server.metadata.{KRaftMetadataCache, ZkMetadataCache} import org.apache.kafka.clients.ClientResponse import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.replica.ClientMetadata @@ -55,6 +55,8 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0 import org.apache.kafka.server.metrics.KafkaYammerMetrics +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import scala.compat.java8.OptionConverters._ import scala.jdk.CollectionConverters._ @@ -1375,8 +1377,11 @@ class PartitionTest extends AbstractPartitionTest { assertEquals(alterPartitionListener.failures.get, 1) } - @Test - def testIsrNotExpandedIfReplicaIsFenced(): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testIsrNotExpandedIfReplicaIsFencedOrShutdown(quorum: String): Unit = { + val kraft = quorum == "kraft" + val log = logManager.getOrCreateLog(topicPartition, topicId = None) seedLogData(log, numRecords = 10, leaderEpoch = 4) @@ -1386,7 +1391,19 @@ class PartitionTest extends AbstractPartitionTest { val replicas = List(brokerId, remoteBrokerId) val isr = Set(brokerId) - val metadataCache = mock(classOf[KRaftMetadataCache]) + val metadataCache: MetadataCache = if (kraft) mock(classOf[KRaftMetadataCache]) else mock(classOf[ZkMetadataCache]) + + // Mark the remote broker as eligible or ineligible in the metadata cache of the leader. + // When using kraft, we can make the broker ineligible by fencing it. + // In ZK mode, we must mark the broker as alive for it to be eligible. + def markRemoteReplicaEligible(eligible: Boolean): Unit = { + if (kraft) { + when(metadataCache.asInstanceOf[KRaftMetadataCache].isBrokerFenced(remoteBrokerId)).thenReturn(!eligible) + } else { + when(metadataCache.hasAliveBroker(remoteBrokerId)).thenReturn(eligible) + } + } + val partition = new Partition( topicPartition, replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, @@ -1414,6 +1431,8 @@ class PartitionTest extends AbstractPartitionTest { assertEquals(isr, partition.partitionState.isr) assertEquals(isr, partition.partitionState.maximalIsr) + markRemoteReplicaEligible(true) + // Fetch to let the follower catch up to the log end offset and // to check if an expansion is possible. fetchFollower(partition, replicaId = remoteBrokerId, fetchOffset = log.logEndOffset) @@ -1430,7 +1449,7 @@ class PartitionTest extends AbstractPartitionTest { assertEquals(replicas.toSet, partition.partitionState.maximalIsr) assertEquals(1, alterPartitionManager.isrUpdates.size) - // Controller rejects the expansion because the broker is fenced. + // Controller rejects the expansion because the broker is fenced or offline. alterPartitionManager.failIsrUpdate(Errors.INELIGIBLE_REPLICA) // The leader reverts back to the previous ISR. @@ -1439,8 +1458,8 @@ class PartitionTest extends AbstractPartitionTest { assertFalse(partition.partitionState.isInflight) assertEquals(0, alterPartitionManager.isrUpdates.size) - // The leader eventually learns about the fenced broker. - when(metadataCache.isBrokerFenced(remoteBrokerId)).thenReturn(true) + // The leader eventually learns about the fenced or offline broker. + markRemoteReplicaEligible(false) // The follower fetches again. fetchFollower(partition, replicaId = remoteBrokerId, fetchOffset = log.logEndOffset) @@ -1451,8 +1470,8 @@ class PartitionTest extends AbstractPartitionTest { assertFalse(partition.partitionState.isInflight) assertEquals(0, alterPartitionManager.isrUpdates.size) - // The broker is eventually unfenced. - when(metadataCache.isBrokerFenced(remoteBrokerId)).thenReturn(false) + // The broker is eventually unfenced or brought back online. + markRemoteReplicaEligible(true) // The follower fetches again. fetchFollower(partition, replicaId = remoteBrokerId, fetchOffset = log.logEndOffset) diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala index 7618889437139..0c8d000656a40 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala @@ -26,11 +26,11 @@ import kafka.server.{KafkaConfig, KafkaServer, QuorumTestHarness} import kafka.utils.{LogCaptureAppender, TestUtils} import kafka.zk.{FeatureZNodeStatus, _} import org.apache.kafka.common.errors.{ControllerMovedException, StaleBrokerEpochException} -import org.apache.kafka.common.message.AlterPartitionRequestData -import org.apache.kafka.common.message.AlterPartitionResponseData +import org.apache.kafka.common.message.{AlterPartitionRequestData, AlterPartitionResponseData} import org.apache.kafka.common.metrics.KafkaMetric import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource import org.apache.kafka.common.{ElectionType, TopicPartition, Uuid} import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.server.common.MetadataVersion @@ -40,8 +40,7 @@ import org.apache.log4j.Level import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals, assertTrue} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.Arguments -import org.junit.jupiter.params.provider.MethodSource +import org.junit.jupiter.params.provider.{Arguments, MethodSource} import org.mockito.Mockito.{doAnswer, spy, verify} import org.mockito.invocation.InvocationOnMock @@ -904,12 +903,7 @@ class ControllerIntegrationTest extends QuorumTestHarness { ).asJava) ).asJava) - val future = new CompletableFuture[AlterPartitionResponseData]() - controller.eventManager.put(AlterPartitionReceived( - alterPartitionRequest, - alterPartitionVersion, - future.complete - )) + val future = alterPartitionFuture(alterPartitionRequest, alterPartitionVersion) val expectedAlterPartitionResponse = new AlterPartitionResponseData() .setTopics(Seq(new AlterPartitionResponseData.TopicData() @@ -968,12 +962,7 @@ class ControllerIntegrationTest extends QuorumTestHarness { ).asJava) ).asJava) - val future = new CompletableFuture[AlterPartitionResponseData]() - controller.eventManager.put(AlterPartitionReceived( - alterPartitionRequest, - ApiKeys.ALTER_PARTITION.latestVersion, - future.complete - )) + val future = alterPartitionFuture(alterPartitionRequest, ApiKeys.ALTER_PARTITION.latestVersion) val expectedAlterPartitionResponse = new AlterPartitionResponseData() .setTopics(Seq(new AlterPartitionResponseData.TopicData() @@ -1024,12 +1013,7 @@ class ControllerIntegrationTest extends QuorumTestHarness { ).asJava) ).asJava) - val future = new CompletableFuture[AlterPartitionResponseData]() - controller.eventManager.put(AlterPartitionReceived( - alterPartitionRequest, - AlterPartitionRequestData.HIGHEST_SUPPORTED_VERSION, - future.complete - )) + val future = alterPartitionFuture(alterPartitionRequest, AlterPartitionRequestData.HIGHEST_SUPPORTED_VERSION) // When re-sending an ISR update, we should not get and error or any ISR changes val expectedAlterPartitionResponse = new AlterPartitionResponseData() @@ -1056,6 +1040,73 @@ class ControllerIntegrationTest extends QuorumTestHarness { sendAndVerifyAlterPartitionResponse(newPartitionEpoch) } + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.ALTER_PARTITION) + def testShutdownBrokerNotAddedToIsr(alterPartitionVersion: Short): Unit = { + servers = makeServers(2) + val controllerId = TestUtils.waitUntilControllerElected(zkClient) + val otherBroker = servers.find(_.config.brokerId != controllerId).get + val brokerId = otherBroker.config.brokerId + val tp = new TopicPartition("t", 0) + val assignment = Map(tp.partition -> Seq(controllerId, brokerId)) + val fullIsr = List(controllerId, brokerId) + TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) + + // Shut down follower. + servers(brokerId).shutdown() + servers(brokerId).awaitShutdown() + + val controller = getController().kafkaController + val leaderIsrAndControllerEpochMap = controller.controllerContext.partitionsLeadershipInfo + val leaderAndIsr = leaderIsrAndControllerEpochMap(tp).leaderAndIsr + val topicId = controller.controllerContext.topicIds(tp.topic) + val controllerEpoch = controller.controllerContext.liveBrokerIdAndEpochs(controllerId) + + // We expect only the controller (online broker) to be in ISR + assertEquals(List(controllerId), leaderAndIsr.isr) + + val requestTopic = new AlterPartitionRequestData.TopicData() + .setPartitions(Seq(new AlterPartitionRequestData.PartitionData() + .setPartitionIndex(tp.partition) + .setLeaderEpoch(leaderAndIsr.leaderEpoch) + .setPartitionEpoch(leaderAndIsr.partitionEpoch) + .setNewIsr(fullIsr.map(Int.box).asJava) + .setLeaderRecoveryState(leaderAndIsr.leaderRecoveryState.value)).asJava) + if (alterPartitionVersion > 1) requestTopic.setTopicId(topicId) else requestTopic.setTopicName(tp.topic) + + // Try to update ISR to contain the offline broker. + val alterPartitionRequest = new AlterPartitionRequestData() + .setBrokerId(controllerId) + .setBrokerEpoch(controllerEpoch) + .setTopics(Seq(requestTopic).asJava) + + val future = alterPartitionFuture(alterPartitionRequest, alterPartitionVersion) + + val expectedError = if (alterPartitionVersion > 1) Errors.INELIGIBLE_REPLICA else Errors.OPERATION_NOT_ATTEMPTED + val expectedResponseTopic = new AlterPartitionResponseData.TopicData() + .setPartitions(Seq(new AlterPartitionResponseData.PartitionData() + .setPartitionIndex(tp.partition) + .setErrorCode(expectedError.code()) + .setLeaderRecoveryState(leaderAndIsr.leaderRecoveryState.value) + ).asJava) + if (alterPartitionVersion > 1) expectedResponseTopic.setTopicId(topicId) else expectedResponseTopic.setTopicName(tp.topic) + + // We expect an ineligble replica error response for the partition. + val expectedAlterPartitionResponse = new AlterPartitionResponseData() + .setTopics(Seq(expectedResponseTopic).asJava) + + val newLeaderIsrAndControllerEpochMap = controller.controllerContext.partitionsLeadershipInfo + val newLeaderAndIsr = newLeaderIsrAndControllerEpochMap(tp).leaderAndIsr + assertEquals(expectedAlterPartitionResponse, future.get(10, TimeUnit.SECONDS)) + assertEquals(List(controllerId), newLeaderAndIsr.isr) + + // Bring replica back online. + servers(brokerId).startup() + + // Wait for broker to rejoin ISR. + TestUtils.waitUntilTrue(() => fullIsr == zkClient.getTopicPartitionState(tp).get.leaderAndIsr.isr, "Replica did not rejoin ISR.") + } + @Test def testAlterPartitionErrors(): Unit = { servers = makeServers(2) @@ -1338,12 +1389,7 @@ class ControllerIntegrationTest extends QuorumTestHarness { .setNewIsr(isr.toList.map(Int.box).asJava) .setLeaderRecoveryState(leaderRecoveryState)).asJava)).asJava) - val future = new CompletableFuture[AlterPartitionResponseData]() - getController().kafkaController.eventManager.put(AlterPartitionReceived( - alterPartitionRequest, - if (topicIdOpt.isDefined) AlterPartitionRequestData.HIGHEST_SUPPORTED_VERSION else 1, - future.complete - )) + val future = alterPartitionFuture(alterPartitionRequest, if (topicIdOpt.isDefined) AlterPartitionRequestData.HIGHEST_SUPPORTED_VERSION else 1) val expectedAlterPartitionResponse = if (topLevelError != Errors.NONE) { new AlterPartitionResponseData().setErrorCode(topLevelError.code) @@ -1818,4 +1864,15 @@ class ControllerIntegrationTest extends QuorumTestHarness { servers.filter(s => s.config.brokerId == controllerId).head } + private def alterPartitionFuture(alterPartitionRequest: AlterPartitionRequestData, + alterPartitionVersion: Short): CompletableFuture[AlterPartitionResponseData] = { + val future = new CompletableFuture[AlterPartitionResponseData]() + getController().kafkaController.eventManager.put(AlterPartitionReceived( + alterPartitionRequest, + alterPartitionVersion, + future.complete + )) + future + } + } From 42a6078cf3566aadbc1b3aaa85ad4e9375e24509 Mon Sep 17 00:00:00 2001 From: Divij Vaidya Date: Wed, 10 Aug 2022 15:25:05 +0200 Subject: [PATCH 038/175] Fix the rate window size calculation for edge cases (#12184) ## Problem Implementation of connection creation rate quotas in Kafka is dependent on two configurations: [quota.window.num](https://kafka.apache.org/documentation.html#brokerconfigs_quota.window.num) AND [quota.window.size.seconds](https://kafka.apache.org/documentation.html#brokerconfigs_quota.window.size.seconds) The minimum possible values of these configuration is 1 as per the documentation. However, when we set 1 as the configuration value, we can hit a situation where rate is calculated as NaN (and hence, leads to exceptions). This specific scenario occurs when an event is recorded at the start of a sample window. ## Solution This patch fixes this edge case by ensuring that the windowSize over which Rate is calculated is at least 1ms (even if it is calculated at the start of the sample window). ## Test Added a unit test which fails before the patch and passes after the patch Reviewers: Ismael Juma , David Mao --- .../kafka/common/metrics/stats/Rate.java | 5 +- .../kafka/common/metrics/MetricsTest.java | 7 +- .../kafka/common/metrics/stats/RateTest.java | 67 +++++++++++++++++++ 3 files changed, 75 insertions(+), 4 deletions(-) create mode 100644 clients/src/test/java/org/apache/kafka/common/metrics/stats/RateTest.java diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java index c6b8574186a88..09b7c05c8f283 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java @@ -91,7 +91,10 @@ public long windowSize(MetricConfig config, long now) { if (numFullWindows < minFullWindows) totalElapsedTimeMs += (minFullWindows - numFullWindows) * config.timeWindowMs(); - return totalElapsedTimeMs; + // If window size is being calculated at the exact beginning of the window with no prior samples, the window size + // will result in a value of 0. Calculation of rate over a window is size 0 is undefined, hence, we assume the + // minimum window size to be at least 1ms. + return Math.max(totalElapsedTimeMs, 1); } @Override diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java index 3dd114d9fd4b1..bc1fc5d9e5624 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java @@ -45,6 +45,7 @@ import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.internals.MetricsUtils; import org.apache.kafka.common.metrics.stats.Avg; import org.apache.kafka.common.metrics.stats.CumulativeSum; import org.apache.kafka.common.metrics.stats.Max; @@ -607,15 +608,15 @@ public void testRateWindowing() throws Exception { // Sleep for half the window. time.sleep(cfg.timeWindowMs() / 2); - // prior to any time passing - double elapsedSecs = (cfg.timeWindowMs() * (cfg.samples() - 1) + cfg.timeWindowMs() / 2) / 1000.0; + // prior to any time passing, elapsedSecs = sampleWindowSize * (total samples - half of final sample) + double elapsedSecs = MetricsUtils.convert(cfg.timeWindowMs(), TimeUnit.SECONDS) * (cfg.samples() - 0.5); KafkaMetric rateMetric = metrics.metrics().get(rateMetricName); KafkaMetric countRateMetric = metrics.metrics().get(countRateMetricName); assertEquals(sum / elapsedSecs, (Double) rateMetric.metricValue(), EPS, "Rate(0...2) = 2.666"); assertEquals(count / elapsedSecs, (Double) countRateMetric.metricValue(), EPS, "Count rate(0...2) = 0.02666"); assertEquals(elapsedSecs, - ((Rate) rateMetric.measurable()).windowSize(cfg, time.milliseconds()) / 1000, EPS, "Elapsed Time = 75 seconds"); + MetricsUtils.convert(((Rate) rateMetric.measurable()).windowSize(cfg, time.milliseconds()), TimeUnit.SECONDS), EPS, "Elapsed Time = 75 seconds"); assertEquals(sum, (Double) totalMetric.metricValue(), EPS); assertEquals(count, (Double) countTotalMetric.metricValue(), EPS); diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/stats/RateTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/stats/RateTest.java new file mode 100644 index 0000000000000..04c5ca1292f98 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/metrics/stats/RateTest.java @@ -0,0 +1,67 @@ +/* + * 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.common.metrics.stats; + +import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.metrics.internals.MetricsUtils; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; + +import java.util.concurrent.TimeUnit; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; + +public class RateTest { + private static final double EPS = 0.000001; + private Rate r; + private Time timeClock; + + @BeforeEach + public void setup() { + r = new Rate(); + timeClock = new MockTime(); + } + + // Tests the scenario where the recording and measurement is done before the window for first sample finishes + // with no prior samples retained. + @ParameterizedTest + @CsvSource({"1,1", "1,11", "11,1", "11,11"}) + public void testRateWithNoPriorAvailableSamples(int numSample, int sampleWindowSizeSec) { + final MetricConfig config = new MetricConfig().samples(numSample).timeWindow(sampleWindowSizeSec, TimeUnit.SECONDS); + final double sampleValue = 50.0; + // record at beginning of the window + r.record(config, sampleValue, timeClock.milliseconds()); + // forward time till almost the end of window + final long measurementTime = TimeUnit.SECONDS.toMillis(sampleWindowSizeSec) - 1; + timeClock.sleep(measurementTime); + // calculate rate at almost the end of window + final double observedRate = r.measure(config, timeClock.milliseconds()); + assertFalse(Double.isNaN(observedRate)); + + // In a scenario where sufficient number of samples is not available yet, the rate calculation algorithm assumes + // presence of N-1 (where N = numSample) prior samples with sample values of 0. Hence, the window size for rate + // calculation accounts for N-1 prior samples + final int dummyPriorSamplesAssumedByAlgorithm = numSample - 1; + final double windowSize = MetricsUtils.convert(measurementTime, TimeUnit.SECONDS) + (dummyPriorSamplesAssumedByAlgorithm * sampleWindowSizeSec); + double expectedRatePerSec = sampleValue / windowSize; + assertEquals(expectedRatePerSec, observedRate, EPS); + } +} From 44229581ed8994416c41ac6584c150564185f0da Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Thu, 11 Aug 2022 14:08:37 -0700 Subject: [PATCH 039/175] KAFKA-13986; Brokers should include node.id in fetches to metadata quorum (#12498) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Currently we do not set the replicaId in fetches from brokers to the metadata quorum. It is useful to do so since that allows us to debug replication using the `DescribeQuorum` API. Reviewers: dengziming , José Armando García Sancio --- core/src/main/scala/kafka/raft/RaftManager.scala | 8 +------- .../src/test/java/kafka/test/ClusterInstance.java | 13 +++++++++++++ .../test/junit/RaftClusterInvocationContext.java | 15 +++++++++++++++ .../test/junit/ZkClusterInvocationContext.java | 13 +++++++++++++ .../scala/unit/kafka/raft/RaftManagerTest.scala | 8 ++++---- .../kafka/server/DescribeQuorumRequestTest.scala | 9 +++++---- 6 files changed, 51 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index cbb9f7b89bf8e..a44d9d8fe014b 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -24,7 +24,6 @@ import java.util.concurrent.CompletableFuture import kafka.log.UnifiedLog import kafka.raft.KafkaRaftManager.RaftIoThread import kafka.server.{KafkaConfig, MetaProperties} -import kafka.server.KafkaRaftServer.ControllerRole import kafka.utils.timer.SystemTimer import kafka.utils.{KafkaScheduler, Logging, ShutdownableThread} import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, NetworkClient} @@ -181,12 +180,7 @@ class KafkaRaftManager[T]( val expirationTimer = new SystemTimer("raft-expiration-executor") val expirationService = new TimingWheelExpirationService(expirationTimer) val quorumStateStore = new FileBasedStateStore(new File(dataDir, "quorum-state")) - - val nodeId = if (config.processRoles.contains(ControllerRole)) { - OptionalInt.of(config.nodeId) - } else { - OptionalInt.empty() - } + val nodeId = OptionalInt.of(config.nodeId) val client = new KafkaRaftClient( recordSerde, diff --git a/core/src/test/java/kafka/test/ClusterInstance.java b/core/src/test/java/kafka/test/ClusterInstance.java index a7052857c364f..9058508fa94f7 100644 --- a/core/src/test/java/kafka/test/ClusterInstance.java +++ b/core/src/test/java/kafka/test/ClusterInstance.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.Optional; import java.util.Properties; +import java.util.Set; public interface ClusterInstance { @@ -50,6 +51,18 @@ default boolean isKRaftTest() { */ ClusterConfig config(); + /** + * Return the set of all controller IDs configured for this test. For kraft, this + * will return only the nodes which have the "controller" role enabled in `process.roles`. + * For zookeeper, this will return all broker IDs since they are all eligible controllers. + */ + Set controllerIds(); + + /** + * Return the set of all broker IDs configured for this test. + */ + Set brokerIds(); + /** * The listener for this cluster as configured by {@link ClusterTest} or by {@link ClusterConfig}. If * unspecified by those sources, this will return the listener for the default security protocol PLAINTEXT diff --git a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java index cef71042d3f92..5cd3ec3e24619 100644 --- a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java @@ -42,6 +42,7 @@ import java.util.Map; import java.util.Optional; import java.util.Properties; +import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; @@ -192,6 +193,20 @@ public ClusterConfig config() { return clusterConfig; } + @Override + public Set controllerIds() { + return controllers() + .map(controllerServer -> controllerServer.config().nodeId()) + .collect(Collectors.toSet()); + } + + @Override + public Set brokerIds() { + return brokers() + .map(brokerServer -> brokerServer.config().nodeId()) + .collect(Collectors.toSet()); + } + @Override public KafkaClusterTestKit getUnderlying() { return clusterReference.get(); diff --git a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java index d8375b012796b..18a85e2d7bf66 100644 --- a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java @@ -45,6 +45,7 @@ import java.util.Map; import java.util.Optional; import java.util.Properties; +import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; @@ -257,6 +258,18 @@ public ClusterConfig config() { return config; } + @Override + public Set controllerIds() { + return brokerIds(); + } + + @Override + public Set brokerIds() { + return servers() + .map(brokerServer -> brokerServer.config().nodeId()) + .collect(Collectors.toSet()); + } + @Override public IntegrationTestHarness getUnderlying() { return clusterReference.get(); diff --git a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala index f8fac503d6ec1..9d7a93db94c74 100644 --- a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala +++ b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala @@ -82,23 +82,23 @@ class RaftManagerTest { } @Test - def testSentinelNodeIdIfBrokerRoleOnly(): Unit = { + def testNodeIdPresentIfBrokerRoleOnly(): Unit = { val raftManager = instantiateRaftManagerWithConfigs(new TopicPartition("__raft_id_test", 0), "broker", "1") - assertFalse(raftManager.client.nodeId.isPresent) + assertEquals(1, raftManager.client.nodeId.getAsInt) raftManager.shutdown() } @Test def testNodeIdPresentIfControllerRoleOnly(): Unit = { val raftManager = instantiateRaftManagerWithConfigs(new TopicPartition("__raft_id_test", 0), "controller", "1") - assertTrue(raftManager.client.nodeId.getAsInt == 1) + assertEquals(1, raftManager.client.nodeId.getAsInt) raftManager.shutdown() } @Test def testNodeIdPresentIfColocated(): Unit = { val raftManager = instantiateRaftManagerWithConfigs(new TopicPartition("__raft_id_test", 0), "controller,broker", "1") - assertTrue(raftManager.client.nodeId.getAsInt == 1) + assertEquals(1, raftManager.client.nodeId.getAsInt) raftManager.shutdown() } diff --git a/core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala index f8da00f10e8f5..b53004b2eaffd 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala @@ -80,11 +80,12 @@ class DescribeQuorumRequestTest(cluster: ClusterInstance) { assertTrue(leaderState.logEndOffset > 0) val voterData = partitionData.currentVoters.asScala + assertEquals(cluster.controllerIds().asScala, voterData.map(_.replicaId).toSet); + val observerData = partitionData.observers.asScala - assertEquals(1, voterData.size) - assertEquals(0, observerData.size) - voterData.foreach { state => - assertTrue(0 < state.replicaId) + assertEquals(cluster.brokerIds().asScala, observerData.map(_.replicaId).toSet); + + (voterData ++ observerData).foreach { state => assertTrue(0 < state.logEndOffset) assertEquals(-1, state.lastFetchTimestamp) assertEquals(-1, state.lastCaughtUpTimestamp) From 36ff5d5c98f2649ec2cc5ab302375cd41298a727 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Thu, 11 Aug 2022 16:43:12 -0700 Subject: [PATCH 040/175] KAFKA-14154; Return NOT_CONTROLLER from AlterPartition if leader is ahead of controller (#12506) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit It is possible for the leader to send an `AlterPartition` request to a zombie controller which includes either a partition or leader epoch which is larger than what is found in the controller context. Prior to https://github.com/apache/kafka/pull/12032, the controller handled this in the following way: 1. If the `LeaderAndIsr` state exactly matches the current state on the controller excluding the partition epoch, then the `AlterPartition` request is considered successful and no error is returned. The risk with this handling is that this may cause the leader to incorrectly assume that the state had been successfully updated. Since the controller's state is stale, there is no way to know what the latest ISR state is. 2. Otherwise, the controller will attempt to update the state in zookeeper with the leader/partition epochs from the `AlterPartition` request. This operation would fail if the controller's epoch was not still current in Zookeeper and the result would be a `NOT_CONTROLLER` error. Following https://github.com/apache/kafka/pull/12032, the controller's validation is stricter. If the partition epoch is larger than expected, then the controller will return `INVALID_UPDATE_VERSION` without attempting the operation. Similarly, if the leader epoch is larger than expected, the controller will return `FENCED_LEADER_EPOCH`. The problem with this new handling is that the leader treats the errors from the controller as authoritative. For example, if it sees the `FENCED_LEADER_EPOCH` error, then it will not retry the request and will simply wait until the next leader epoch arrives. The ISR state gets suck in a pending state, which can lead to persistent URPs until the leader epoch gets bumped. In this patch, we want to fix the issues with this handling, but we don't want to restore the buggy idempotent check. The approach is straightforward. If the controller sees a partition/leader epoch which is larger than what it has in the controller context, then it assumes that has become a zombie and returns `NOT_CONTROLLER` to the leader. This will cause the leader to attempt to reset the controller from its local metadata cache and retry the `AlterPartition` request. Reviewers: David Jacot , José Armando García Sancio --- .../kafka/controller/KafkaController.scala | 9 ++++++- .../ControllerIntegrationTest.scala | 27 +++++++++++++++---- 2 files changed, 30 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 0154d9cbe54ab..999bcb818e91c 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -2336,7 +2336,14 @@ class KafkaController(val config: KafkaConfig, controllerContext.partitionLeadershipInfo(tp) match { case Some(leaderIsrAndControllerEpoch) => val currentLeaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr - if (newLeaderAndIsr.leaderEpoch != currentLeaderAndIsr.leaderEpoch) { + if (newLeaderAndIsr.partitionEpoch > currentLeaderAndIsr.partitionEpoch + || newLeaderAndIsr.leaderEpoch > currentLeaderAndIsr.leaderEpoch) { + // If the partition leader has a higher partition/leader epoch, then it is likely + // that this node is no longer the active controller. We return NOT_CONTROLLER in + // this case to give the leader an opportunity to find the new controller. + partitionResponses(tp) = Left(Errors.NOT_CONTROLLER) + None + } else if (newLeaderAndIsr.leaderEpoch != currentLeaderAndIsr.leaderEpoch) { partitionResponses(tp) = Left(Errors.FENCED_LEADER_EPOCH) None } else if (newLeaderAndIsr.equalsAllowStalePartitionEpoch(currentLeaderAndIsr)) { diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala index 0c8d000656a40..532ff1a946e9e 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala @@ -1184,7 +1184,7 @@ class ControllerIntegrationTest extends QuorumTestHarness { ) assertAlterPartition( - partitionError = Errors.INVALID_UPDATE_VERSION, + partitionError = Errors.NOT_CONTROLLER, partitionEpoch = partitionEpoch + 1 ) @@ -1194,7 +1194,7 @@ class ControllerIntegrationTest extends QuorumTestHarness { ) assertAlterPartition( - partitionError = Errors.FENCED_LEADER_EPOCH, + partitionError = Errors.NOT_CONTROLLER, leaderEpoch = leaderEpoch + 1 ) @@ -1218,6 +1218,12 @@ class ControllerIntegrationTest extends QuorumTestHarness { partitionEpoch = partitionEpoch - 1 ) + assertAlterPartition( + partitionError = Errors.NOT_CONTROLLER, + leaderRecoveryState = LeaderRecoveryState.RECOVERING.value, + partitionEpoch = partitionEpoch + 1 + ) + assertAlterPartition( partitionError = Errors.FENCED_LEADER_EPOCH, leaderRecoveryState = LeaderRecoveryState.RECOVERING.value, @@ -1225,7 +1231,7 @@ class ControllerIntegrationTest extends QuorumTestHarness { ) assertAlterPartition( - partitionError = Errors.FENCED_LEADER_EPOCH, + partitionError = Errors.NOT_CONTROLLER, leaderRecoveryState = LeaderRecoveryState.RECOVERING.value, leaderEpoch = leaderEpoch + 1 ) @@ -1324,13 +1330,18 @@ class ControllerIntegrationTest extends QuorumTestHarness { partitionEpoch = partitionEpoch - 1 ) + assertAlterPartition( + partitionError = Errors.NOT_CONTROLLER, + partitionEpoch = partitionEpoch + 1 + ) + assertAlterPartition( partitionError = Errors.FENCED_LEADER_EPOCH, leaderEpoch = leaderEpoch - 1 ) assertAlterPartition( - partitionError = Errors.FENCED_LEADER_EPOCH, + partitionError = Errors.NOT_CONTROLLER, leaderEpoch = leaderEpoch + 1 ) @@ -1348,6 +1359,12 @@ class ControllerIntegrationTest extends QuorumTestHarness { leaderRecoveryState = LeaderRecoveryState.RECOVERING.value ) + assertAlterPartition( + partitionError = Errors.NOT_CONTROLLER, + partitionEpoch = partitionEpoch + 1, + leaderRecoveryState = LeaderRecoveryState.RECOVERING.value + ) + assertAlterPartition( partitionError = Errors.FENCED_LEADER_EPOCH, leaderEpoch = leaderEpoch - 1, @@ -1355,7 +1372,7 @@ class ControllerIntegrationTest extends QuorumTestHarness { ) assertAlterPartition( - partitionError = Errors.FENCED_LEADER_EPOCH, + partitionError = Errors.NOT_CONTROLLER, leaderEpoch = leaderEpoch + 1, leaderRecoveryState = LeaderRecoveryState.RECOVERING.value ) From 5cfd248f8e4be267ced8fbf200db45ec041a93e9 Mon Sep 17 00:00:00 2001 From: dengziming Date: Sat, 13 Aug 2022 00:06:24 +0800 Subject: [PATCH 041/175] KAFKA-13959: Controller should unfence Broker with busy metadata log (#12274) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The reason for KAFKA-13959 is a little complex, the two keys to this problem are: KafkaRaftClient.MAX_FETCH_WAIT_MS==MetadataMaxIdleIntervalMs == 500ms. We rely on fetchPurgatory to complete a FetchRequest, in details, if FetchRequest.fetchOffset >= log.endOffset, we will wait for 500ms to send a FetchResponse. The follower needs to send one more FetchRequest to get the HW. Here are the event sequences: 1. When starting the leader(active controller) LEO=m+1(m is the offset of the last record), leader HW=m(because we need more than half of the voters to reach m+1) 2. Follower (standby controller) and observer (broker) send FetchRequest(fetchOffset=m) 2.1. leader receives FetchRequest, set leader HW=m and waits 500ms before send FetchResponse 2.2. leader send FetchResponse(HW=m) 3.3 broker receive FetchResponse(HW=m), set metadataOffset=m. 3. Leader append NoOpRecord, LEO=m+2. leader HW=m 4. Looping 1-4 If we change MAX_FETCH_WAIT_MS=200 (less than half of MetadataMaxIdleIntervalMs), this problem can be solved temporarily. We plan to improve this problem in 2 ways, firstly, in this PR, we change the controller to unfence a broker when the broker's high-watermark has reached the broker registration record for that broker. Secondly, we will propagate the HWM to the replicas as quickly as possible in KAFKA-14145. Reviewers: Luke Chen , José Armando García Sancio --- .../main/scala/kafka/server/KafkaConfig.scala | 2 +- .../controller/BrokerHeartbeatManager.java | 26 +++++++-------- .../controller/ClusterControlManager.java | 25 ++++++++++++-- .../kafka/controller/QuorumController.java | 27 +++++++++++---- .../controller/ReplicationControlManager.java | 6 ++-- .../controller/ClusterControlManagerTest.java | 19 ++++++----- .../ProducerIdControlManagerTest.java | 2 +- .../kafka/metadata/RecordTestUtils.java | 33 +++++++++++-------- .../kafka/raft/internals/FuturePurgatory.java | 4 +-- 9 files changed, 92 insertions(+), 52 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 4e253047ee604..860056f9a3e4b 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -81,7 +81,7 @@ object Defaults { val BrokerHeartbeatIntervalMs = 2000 val BrokerSessionTimeoutMs = 9000 val MetadataSnapshotMaxNewRecordBytes = 20 * 1024 * 1024 - val MetadataMaxIdleIntervalMs = 5000 + val MetadataMaxIdleIntervalMs = 500 /** KRaft mode configs */ val EmptyNodeId: Int = -1 diff --git a/metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java b/metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java index f31df917d7679..428f1c5833ea4 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java @@ -511,17 +511,17 @@ BrokerControlState currentBrokerState(BrokerHeartbeatState broker) { /** * Calculate the next broker state for a broker that just sent a heartbeat request. * - * @param brokerId The broker id. - * @param request The incoming heartbeat request. - * @param lastCommittedOffset The last committed offset of the quorum controller. - * @param hasLeaderships A callback which evaluates to true if the broker leads - * at least one partition. + * @param brokerId The broker id. + * @param request The incoming heartbeat request. + * @param registerBrokerRecordOffset The offset of the broker's {@link org.apache.kafka.common.metadata.RegisterBrokerRecord}. + * @param hasLeaderships A callback which evaluates to true if the broker leads + * at least one partition. * - * @return The current and next broker states. + * @return The current and next broker states. */ BrokerControlStates calculateNextBrokerState(int brokerId, BrokerHeartbeatRequestData request, - long lastCommittedOffset, + long registerBrokerRecordOffset, Supplier hasLeaderships) { BrokerHeartbeatState broker = brokers.getOrDefault(brokerId, new BrokerHeartbeatState(brokerId)); @@ -533,17 +533,17 @@ BrokerControlStates calculateNextBrokerState(int brokerId, "shutdown.", brokerId); return new BrokerControlStates(currentState, SHUTDOWN_NOW); } else if (!request.wantFence()) { - if (request.currentMetadataOffset() >= lastCommittedOffset) { + if (request.currentMetadataOffset() >= registerBrokerRecordOffset) { log.info("The request from broker {} to unfence has been granted " + - "because it has caught up with the last committed metadata " + - "offset {}.", brokerId, lastCommittedOffset); + "because it has caught up with the offset of it's register " + + "broker record {}.", brokerId, registerBrokerRecordOffset); return new BrokerControlStates(currentState, UNFENCED); } else { if (log.isDebugEnabled()) { log.debug("The request from broker {} to unfence cannot yet " + - "be granted because it has not caught up with the last " + - "committed metadata offset {}. It is still at offset {}.", - brokerId, lastCommittedOffset, request.currentMetadataOffset()); + "be granted because it has not caught up with the offset of " + + "it's register broker record {}. It is still at offset {}.", + brokerId, registerBrokerRecordOffset, request.currentMetadataOffset()); } return new BrokerControlStates(currentState, FENCED); } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java index 235f077cfff7a..d30f43242179f 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java @@ -60,6 +60,7 @@ import java.util.Map.Entry; import java.util.NoSuchElementException; import java.util.Optional; +import java.util.OptionalLong; import java.util.Random; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -217,6 +218,14 @@ boolean check() { */ private final TimelineHashMap brokerRegistrations; + /** + * Save the offset of each broker registration record, we will only unfence a + * broker when its high watermark has reached its broker registration record, + * this is not necessarily the exact offset of each broker registration record + * but should not be smaller than it. + */ + private final TimelineHashMap registerBrokerRecordOffsets; + /** * A reference to the controller's metrics registry. */ @@ -255,6 +264,7 @@ private ClusterControlManager( this.sessionTimeoutNs = sessionTimeoutNs; this.replicaPlacer = replicaPlacer; this.brokerRegistrations = new TimelineHashMap<>(snapshotRegistry, 0); + this.registerBrokerRecordOffsets = new TimelineHashMap<>(snapshotRegistry, 0); this.heartbeatManager = null; this.readyBrokersFuture = Optional.empty(); this.controllerMetrics = metrics; @@ -366,7 +376,15 @@ public ControllerResult registerBroker( return ControllerResult.atomicOf(records, new BrokerRegistrationReply(brokerEpoch)); } - public void replay(RegisterBrokerRecord record) { + public OptionalLong registerBrokerRecordOffset(int brokerId) { + if (registerBrokerRecordOffsets.containsKey(brokerId)) { + return OptionalLong.of(registerBrokerRecordOffsets.get(brokerId)); + } + return OptionalLong.empty(); + } + + public void replay(RegisterBrokerRecord record, long offset) { + registerBrokerRecordOffsets.put(record.brokerId(), offset); int brokerId = record.brokerId(); List listeners = new ArrayList<>(); for (BrokerEndpoint endpoint : record.endPoints()) { @@ -401,14 +419,15 @@ public void replay(RegisterBrokerRecord record) { } public void replay(UnregisterBrokerRecord record) { + registerBrokerRecordOffsets.remove(record.brokerId()); int brokerId = record.brokerId(); BrokerRegistration registration = brokerRegistrations.get(brokerId); if (registration == null) { throw new RuntimeException(String.format("Unable to replay %s: no broker " + - "registration found for that id", record.toString())); + "registration found for that id", record)); } else if (registration.epoch() != record.brokerEpoch()) { throw new RuntimeException(String.format("Unable to replay %s: no broker " + - "registration with that epoch found", record.toString())); + "registration with that epoch found", record)); } else { if (heartbeatManager != null) heartbeatManager.remove(brokerId); brokerRegistrations.remove(brokerId); diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index ef87248f13420..3fee25841ba74 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -27,6 +27,7 @@ import org.apache.kafka.common.errors.BrokerIdNotRegisteredException; import org.apache.kafka.common.errors.InvalidRequestException; import org.apache.kafka.common.errors.NotControllerException; +import org.apache.kafka.common.errors.StaleBrokerEpochException; import org.apache.kafka.common.errors.UnknownServerException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.message.AllocateProducerIdsRequestData; @@ -759,7 +760,7 @@ public void run() throws Exception { int i = 1; for (ApiMessageAndVersion message : result.records()) { try { - replay(message.message(), Optional.empty()); + replay(message.message(), Optional.empty(), writeOffset + result.records().size()); } catch (Throwable e) { String failureMessage = String.format("Unable to apply %s record, which was " + "%d of %d record(s) in the batch following last writeOffset %d.", @@ -883,7 +884,7 @@ public void handleCommit(BatchReader reader) { int i = 1; for (ApiMessageAndVersion message : messages) { try { - replay(message.message(), Optional.empty()); + replay(message.message(), Optional.empty(), offset); } catch (Throwable e) { String failureMessage = String.format("Unable to apply %s record on standby " + "controller, which was %d of %d record(s) in the batch with baseOffset %d.", @@ -938,7 +939,7 @@ public void handleSnapshot(SnapshotReader reader) { int i = 1; for (ApiMessageAndVersion message : messages) { try { - replay(message.message(), Optional.of(reader.snapshotId())); + replay(message.message(), Optional.of(reader.snapshotId()), reader.lastContainedLogOffset()); } catch (Throwable e) { String failureMessage = String.format("Unable to apply %s record " + "from snapshot %s on standby controller, which was %d of " + @@ -1305,12 +1306,19 @@ private void handleFeatureControlChange() { } } - @SuppressWarnings("unchecked") - private void replay(ApiMessage message, Optional snapshotId) { + /** + * Apply the metadata record to its corresponding in-memory state(s) + * + * @param message The metadata record + * @param snapshotId The snapshotId if this record is from a snapshot + * @param batchLastOffset The offset of the last record in the log batch, or the lastContainedLogOffset + * if this record is from a snapshot, this is used along with RegisterBrokerRecord + */ + private void replay(ApiMessage message, Optional snapshotId, long batchLastOffset) { MetadataRecordType type = MetadataRecordType.fromId(message.apiKey()); switch (type) { case REGISTER_BROKER_RECORD: - clusterControl.replay((RegisterBrokerRecord) message); + clusterControl.replay((RegisterBrokerRecord) message, batchLastOffset); break; case UNREGISTER_BROKER_RECORD: clusterControl.replay((UnregisterBrokerRecord) message); @@ -1874,8 +1882,13 @@ public CompletableFuture processBrokerHeartbeat( @Override public ControllerResult generateRecordsAndResult() { + OptionalLong offsetForRegisterBrokerRecord = clusterControl.registerBrokerRecordOffset(brokerId); + if (!offsetForRegisterBrokerRecord.isPresent()) { + throw new StaleBrokerEpochException( + String.format("Receive a heartbeat from broker %d before registration", brokerId)); + } ControllerResult result = replicationControl. - processBrokerHeartbeat(request, lastCommittedOffset); + processBrokerHeartbeat(request, offsetForRegisterBrokerRecord.getAsLong()); inControlledShutdown = result.response().inControlledShutdown(); rescheduleMaybeFenceStaleBrokers(); return result; diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java index bf3a679d2cef0..4ffb339967c2f 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java @@ -1355,13 +1355,13 @@ ApiError electLeader(String topic, int partitionId, ElectionType electionType, } ControllerResult processBrokerHeartbeat( - BrokerHeartbeatRequestData request, long lastCommittedOffset) { + BrokerHeartbeatRequestData request, long registerBrokerRecordOffset) { int brokerId = request.brokerId(); long brokerEpoch = request.brokerEpoch(); clusterControl.checkBrokerEpoch(brokerId, brokerEpoch); BrokerHeartbeatManager heartbeatManager = clusterControl.heartbeatManager(); BrokerControlStates states = heartbeatManager.calculateNextBrokerState(brokerId, - request, lastCommittedOffset, () -> brokersToIsrs.hasLeaderships(brokerId)); + request, registerBrokerRecordOffset, () -> brokersToIsrs.hasLeaderships(brokerId)); List records = new ArrayList<>(); if (states.current() != states.next()) { switch (states.next()) { @@ -1382,7 +1382,7 @@ ControllerResult processBrokerHeartbeat( heartbeatManager.touch(brokerId, states.next().fenced(), request.currentMetadataOffset()); - boolean isCaughtUp = request.currentMetadataOffset() >= lastCommittedOffset; + boolean isCaughtUp = request.currentMetadataOffset() >= registerBrokerRecordOffset; BrokerHeartbeatReply reply = new BrokerHeartbeatReply(isCaughtUp, states.next().fenced(), states.next().inControlledShutdown(), diff --git a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java index 97d6c88377277..e47def81e6d5d 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java @@ -96,7 +96,7 @@ public void testReplay(MetadataVersion metadataVersion) { setPort((short) 9092). setName("PLAINTEXT"). setHost("example.com")); - clusterControl.replay(brokerRecord); + clusterControl.replay(brokerRecord, 100L); clusterControl.checkBrokerEpoch(1, 100); assertThrows(StaleBrokerEpochException.class, () -> clusterControl.checkBrokerEpoch(1, 101)); @@ -165,19 +165,20 @@ public void testReplayRegisterBrokerRecord() { setPort((short) 9092). setName("PLAINTEXT"). setHost("example.com")); - clusterControl.replay(brokerRecord); + clusterControl.replay(brokerRecord, 100L); assertFalse(clusterControl.unfenced(0)); assertTrue(clusterControl.inControlledShutdown(0)); brokerRecord.setInControlledShutdown(false); - clusterControl.replay(brokerRecord); + clusterControl.replay(brokerRecord, 100L); assertFalse(clusterControl.unfenced(0)); assertFalse(clusterControl.inControlledShutdown(0)); + assertEquals(100L, clusterControl.registerBrokerRecordOffset(brokerRecord.brokerId()).getAsLong()); brokerRecord.setFenced(false); - clusterControl.replay(brokerRecord); + clusterControl.replay(brokerRecord, 100L); assertTrue(clusterControl.unfenced(0)); assertFalse(clusterControl.inControlledShutdown(0)); @@ -217,7 +218,7 @@ public void testReplayBrokerRegistrationChangeRecord() { setPort((short) 9092). setName("PLAINTEXT"). setHost("example.com")); - clusterControl.replay(brokerRecord); + clusterControl.replay(brokerRecord, 100L); assertTrue(clusterControl.unfenced(0)); assertFalse(clusterControl.inControlledShutdown(0)); @@ -341,17 +342,19 @@ public void testUnregister() throws Exception { setFeatureControlManager(featureControl). build(); clusterControl.activate(); - clusterControl.replay(brokerRecord); + clusterControl.replay(brokerRecord, 100L); assertEquals(new BrokerRegistration(1, 100, Uuid.fromString("fPZv1VBsRFmnlRvmGcOW9w"), Collections.singletonMap("PLAINTEXT", new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "example.com", 9092)), Collections.emptyMap(), Optional.of("arack"), true, false), clusterControl.brokerRegistrations().get(1)); + assertEquals(100L, clusterControl.registerBrokerRecordOffset(brokerRecord.brokerId()).getAsLong()); UnregisterBrokerRecord unregisterRecord = new UnregisterBrokerRecord(). setBrokerId(1). setBrokerEpoch(100); clusterControl.replay(unregisterRecord); assertFalse(clusterControl.brokerRegistrations().containsKey(1)); + assertFalse(clusterControl.registerBrokerRecordOffset(brokerRecord.brokerId()).isPresent()); } @ParameterizedTest @@ -382,7 +385,7 @@ public void testPlaceReplicas(int numUsableBrokers) throws Exception { setPort((short) 9092). setName("PLAINTEXT"). setHost("example.com")); - clusterControl.replay(brokerRecord); + clusterControl.replay(brokerRecord, 100L); UnfenceBrokerRecord unfenceRecord = new UnfenceBrokerRecord().setId(i).setEpoch(100); clusterControl.replay(unfenceRecord); @@ -442,7 +445,7 @@ public void testIterator(MetadataVersion metadataVersion) throws Exception { setPort((short) 9092 + i). setName("PLAINTEXT"). setHost("example.com")); - clusterControl.replay(brokerRecord); + clusterControl.replay(brokerRecord, 100L); } for (int i = 0; i < 2; i++) { UnfenceBrokerRecord unfenceBrokerRecord = diff --git a/metadata/src/test/java/org/apache/kafka/controller/ProducerIdControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ProducerIdControlManagerTest.java index ccdd3a5b2331f..80c5c505ae0eb 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ProducerIdControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ProducerIdControlManagerTest.java @@ -76,7 +76,7 @@ public void setUp() { setPort((short) 9092). setName("PLAINTEXT"). setHost(String.format("broker-%02d.example.org", i))); - clusterControl.replay(brokerRecord); + clusterControl.replay(brokerRecord, 100L); } this.producerIdControlManager = new ProducerIdControlManager(clusterControl, snapshotRegistry); diff --git a/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java b/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java index f5a8da5f8a2f9..c21bdb544789b 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java @@ -57,20 +57,25 @@ public static void replayAll(Object target, for (ApiMessageAndVersion recordAndVersion : recordsAndVersions) { ApiMessage record = recordAndVersion.message(); try { - Method method = target.getClass().getMethod("replay", record.getClass()); - method.invoke(target, record); - } catch (NoSuchMethodException e) { try { - Method method = target.getClass().getMethod("replay", - record.getClass(), - Optional.class); - method.invoke(target, record, Optional.empty()); - } catch (NoSuchMethodException t) { - // ignore - } catch (InvocationTargetException t) { - throw new RuntimeException(t); - } catch (IllegalAccessException t) { - throw new RuntimeException(t); + Method method = target.getClass().getMethod("replay", record.getClass()); + method.invoke(target, record); + } catch (NoSuchMethodException e) { + try { + Method method = target.getClass().getMethod("replay", + record.getClass(), + Optional.class); + method.invoke(target, record, Optional.empty()); + } catch (NoSuchMethodException t) { + try { + Method method = target.getClass().getMethod("replay", + record.getClass(), + long.class); + method.invoke(target, record, 0L); + } catch (NoSuchMethodException i) { + // ignore + } + } } } catch (InvocationTargetException e) { throw new RuntimeException(e); @@ -119,7 +124,7 @@ public static void replayAllBatches(Object target, * @param delta the metadata delta on which to replay the records * @param highestOffset highest offset from the list of record batches * @param highestEpoch highest epoch from the list of record batches - * @param recordsAndVersions list of batches of records + * @param batches list of batches of records */ public static void replayAllBatches( MetadataDelta delta, diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/FuturePurgatory.java b/raft/src/main/java/org/apache/kafka/raft/internals/FuturePurgatory.java index b37fb3a3847c5..e5dceeaa0c30e 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/FuturePurgatory.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/FuturePurgatory.java @@ -56,8 +56,8 @@ public interface FuturePurgatory> { CompletableFuture await(T threshold, long maxWaitTimeMs); /** - * Complete awaiting futures whose associated values are larger than the given threshold value. - * The completion callbacks will be triggered from the calling thread. + * Complete awaiting futures whose threshold value from {@link FuturePurgatory#await} are smaller + * than the given threshold value. The completion callbacks will be triggered from the calling thread. * * @param value the threshold value used to determine which futures can be completed * @param currentTimeMs the current time in milliseconds that will be passed to From a2110e60dfd9008aff657c75c9a9d73b6d1f4030 Mon Sep 17 00:00:00 2001 From: Derek Troy-West Date: Sat, 13 Aug 2022 10:41:44 +1000 Subject: [PATCH 042/175] MINOR: Add note on IDEMPOTENT_WRITE ACL to notable changes (#12260) Update notable changes documentation to mention requiring IDEMPOTENT_WRITE permission when producing messages with default/idempotent configuration and broker version lower than 2.8.0. Reviewers: Ismael Juma , Luke Chen --- docs/upgrade.html | 30 +++++++++++++++++++----------- 1 file changed, 19 insertions(+), 11 deletions(-) diff --git a/docs/upgrade.html b/docs/upgrade.html index 07517fed5feb1..c72afec4ce3cc 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -72,7 +72,10 @@

Upgrading to 3.2.0 from any vers
Notable changes in 3.2.0